From a98ce7d2be69fbcf774fd71f8375b85ada5358ee Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Dec 2022 23:39:16 +0000 Subject: [PATCH 01/78] propagate PROXYv1 forwarded_for address to http handler in composable protocols; some logging added --- src/Server/HTTP/HTTPServerConnection.cpp | 3 +++ src/Server/HTTP/HTTPServerConnection.h | 13 +++++++++++++ src/Server/HTTP/HTTPServerConnectionFactory.cpp | 5 +++++ src/Server/HTTP/HTTPServerConnectionFactory.h | 1 + src/Server/ProxyV1Handler.cpp | 3 +++ src/Server/ProxyV1Handler.h | 4 +++- src/Server/TCPHandler.cpp | 2 ++ 7 files changed, 30 insertions(+), 1 deletion(-) diff --git a/src/Server/HTTP/HTTPServerConnection.cpp b/src/Server/HTTP/HTTPServerConnection.cpp index 926d37a11ee..ad17bc4348a 100644 --- a/src/Server/HTTP/HTTPServerConnection.cpp +++ b/src/Server/HTTP/HTTPServerConnection.cpp @@ -34,6 +34,9 @@ void HTTPServerConnection::run() Poco::Timestamp now; + if (!forwarded_for.empty()) + request.set("X-Forwarded-For", forwarded_for); + if (request.isSecure()) { size_t hsts_max_age = context->getMaxHstsAge(); diff --git a/src/Server/HTTP/HTTPServerConnection.h b/src/Server/HTTP/HTTPServerConnection.h index cce4f44f203..7087f8d5a21 100644 --- a/src/Server/HTTP/HTTPServerConnection.h +++ b/src/Server/HTTP/HTTPServerConnection.h @@ -21,6 +21,18 @@ public: Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory); + HTTPServerConnection( + HTTPContextPtr context_, + TCPServer & tcp_server_, + const Poco::Net::StreamSocket & socket_, + Poco::Net::HTTPServerParams::Ptr params_, + HTTPRequestHandlerFactoryPtr factory_, + const String & forwarded_for_) + : HTTPServerConnection(context_, tcp_server_, socket_, params_, factory_) + { + forwarded_for = forwarded_for_; + } + void run() override; protected: @@ -31,6 +43,7 @@ private: TCPServer & tcp_server; Poco::Net::HTTPServerParams::Ptr params; HTTPRequestHandlerFactoryPtr factory; + String forwarded_for; bool stopped; std::mutex mutex; // guards the |factory| with assumption that creating handlers is not thread-safe. }; diff --git a/src/Server/HTTP/HTTPServerConnectionFactory.cpp b/src/Server/HTTP/HTTPServerConnectionFactory.cpp index 7e4edbbf542..2c9ac0cda2a 100644 --- a/src/Server/HTTP/HTTPServerConnectionFactory.cpp +++ b/src/Server/HTTP/HTTPServerConnectionFactory.cpp @@ -16,4 +16,9 @@ Poco::Net::TCPServerConnection * HTTPServerConnectionFactory::createConnection(c return new HTTPServerConnection(context, tcp_server, socket, params, factory); } +Poco::Net::TCPServerConnection * HTTPServerConnectionFactory::createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server, TCPProtocolStackData & stack_data) +{ + return new HTTPServerConnection(context, tcp_server, socket, params, factory, stack_data.forwarded_for); +} + } diff --git a/src/Server/HTTP/HTTPServerConnectionFactory.h b/src/Server/HTTP/HTTPServerConnectionFactory.h index 03648ce7be7..e18249da4de 100644 --- a/src/Server/HTTP/HTTPServerConnectionFactory.h +++ b/src/Server/HTTP/HTTPServerConnectionFactory.h @@ -15,6 +15,7 @@ public: HTTPServerConnectionFactory(HTTPContextPtr context, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory); Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override; + Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server, TCPProtocolStackData & stack_data) override; private: HTTPContextPtr context; diff --git a/src/Server/ProxyV1Handler.cpp b/src/Server/ProxyV1Handler.cpp index 838a1de1c04..bf02f34c6a3 100644 --- a/src/Server/ProxyV1Handler.cpp +++ b/src/Server/ProxyV1Handler.cpp @@ -60,6 +60,9 @@ void ProxyV1Handler::run() // read port and "\r\n" if (!readWord(5, word, eol) || !eol) throw ParsingException("PROXY protocol violation", ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED); + + if (!stack_data.forwarded_for.empty()) + LOG_TRACE(log, "Forwarded client address from PROXY header: {}", stack_data.forwarded_for); } bool ProxyV1Handler::readWord(int max_len, std::string & word, bool & eol) diff --git a/src/Server/ProxyV1Handler.h b/src/Server/ProxyV1Handler.h index 062cc0e291a..e56f4cd3545 100644 --- a/src/Server/ProxyV1Handler.h +++ b/src/Server/ProxyV1Handler.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -13,7 +14,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), server(server_), conf_name(conf_name_), stack_data(stack_data_) {} + : Poco::Net::TCPServerConnection(socket), log(&Poco::Logger::get("ProxyV1Handler")), server(server_), conf_name(conf_name_), stack_data(stack_data_) {} void run() override; @@ -21,6 +22,7 @@ protected: bool readWord(int max_len, std::string & word, bool & eol); private: + Poco::Logger * log; IServer & server; std::string conf_name; TCPProtocolStackData & stack_data; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 73b91b29f31..a25062dfe36 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -119,6 +119,8 @@ TCPHandler::TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::N , default_database(stack_data.default_database) , server_display_name(std::move(server_display_name_)) { + if (!forwarded_for.empty()) + LOG_TRACE(log, "Forwarded client address: {}", forwarded_for); } TCPHandler::~TCPHandler() From 9092134544a9c153621b72ba08d9a13b5e67a4b1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 20 Dec 2022 17:39:40 +0000 Subject: [PATCH 02/78] auth_use_forwarded_address config setting added, using forwarded address for proxied TCP and HTTP connection, likely bug is fixed in AccessControl, relevant test is appended --- src/Access/AccessControl.cpp | 3 ++- src/Server/HTTPHandler.cpp | 17 ++++++++++++++++- src/Server/TCPHandler.cpp | 18 +++++++++++++++++- .../configs/config.xml | 1 + .../configs/users.xml | 7 +++++++ .../test_composable_protocols/test.py | 17 +++++++++++++++++ 6 files changed, 60 insertions(+), 3 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 063d48b1d70..ed113e8d0b3 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -568,7 +568,8 @@ std::shared_ptr AccessControl::getContextAccess( /// Extract the last entry from comma separated list of X-Forwarded-For addresses. /// Only the last proxy can be trusted (if any). Strings forwarded_addresses; - boost::split(forwarded_addresses, client_info.forwarded_for, boost::is_any_of(",")); + if (!client_info.forwarded_for.empty()) + boost::split(forwarded_addresses, client_info.forwarded_for, boost::is_any_of(",")); if (!forwarded_addresses.empty()) { String & last_forwarded_address = forwarded_addresses.back(); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 2b63524fb79..cd84153d3ee 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -27,6 +27,8 @@ #include #include #include +#include +#include #include #include @@ -41,6 +43,7 @@ #include #include #include +#include #include #include @@ -469,9 +472,21 @@ bool HTTPHandler::authenticateUser( client_info.forwarded_for = request.get("X-Forwarded-For", ""); client_info.quota_key = quota_key; + /// Extract the last entry from comma separated list of forwarded_for addresses. + /// Only the last proxy can be trusted (if any). + Strings forwarded_addresses; + if (!client_info.forwarded_for.empty()) + boost::split(forwarded_addresses, client_info.forwarded_for, boost::is_any_of(",")); + try { - session->authenticate(*request_credentials, request.clientAddress()); + if (!forwarded_addresses.empty() && server.config().getBool("auth_use_forwarded_address", false)) + { + boost::trim(forwarded_addresses.back()); + session->authenticate(*request_credentials, Poco::Net::SocketAddress(forwarded_addresses.back(), request.clientAddress().port())); + } + else + session->authenticate(*request_credentials, request.clientAddress()); } catch (const Authentication::Require & required_credentials) { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a25062dfe36..1c7028b49f5 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -44,6 +45,8 @@ #include #include #include +#include +#include #include #include @@ -1151,7 +1154,20 @@ void TCPHandler::receiveHello() } session = makeSession(); - session->authenticate(user, password, socket().peerAddress()); + auto & client_info = session->getClientInfo(); + + /// Extract the last entry from comma separated list of forwarded_for addresses. + /// Only the last proxy can be trusted (if any). + Strings forwarded_addresses; + if (!client_info.forwarded_for.empty()) + boost::split(forwarded_addresses, client_info.forwarded_for, boost::is_any_of(",")); + if (!forwarded_addresses.empty() && server.config().getBool("auth_use_forwarded_address", false)) + { + boost::trim(forwarded_addresses.back()); + session->authenticate(user, password, Poco::Net::SocketAddress(forwarded_addresses.back(), socket().peerAddress().port())); + } + else + session->authenticate(user, password, socket().peerAddress()); } void TCPHandler::receiveAddendum() diff --git a/tests/integration/test_composable_protocols/configs/config.xml b/tests/integration/test_composable_protocols/configs/config.xml index 35673c3e7e5..f42bff335ef 100644 --- a/tests/integration/test_composable_protocols/configs/config.xml +++ b/tests/integration/test_composable_protocols/configs/config.xml @@ -1,4 +1,5 @@ + true diff --git a/tests/integration/test_composable_protocols/configs/users.xml b/tests/integration/test_composable_protocols/configs/users.xml index da8425b3695..8a837ad2693 100644 --- a/tests/integration/test_composable_protocols/configs/users.xml +++ b/tests/integration/test_composable_protocols/configs/users.xml @@ -12,5 +12,12 @@ default + + + + 123.123.123.123 + + default + diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index d861af929c3..ba7c25f4e08 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -92,3 +92,20 @@ def test_connections(): ) >= 0 ) + + data_user_allowed = "PROXY TCP4 123.123.123.123 255.255.255.255 65535 65535\r\n\0\021ClickHouse client\024\r\253\251\003\0\007user123\0\004\001\0\001\0\0\t0.0.0.0:0\001\tmilovidov\021milovidov-desktop\vClickHouse \024\r\253\251\003\0\001\0\0\0\002\001\025SELECT 'Hello, world'\002\0\247\203\254l\325\\z|\265\254F\275\333\206\342\024\202\024\0\0\0\n\0\0\0\240\01\0\02\377\377\377\377\0\0\0" + assert ( + netcat(server.ip_address, 9100, bytearray(data_user_allowed, "latin-1")).find( + bytearray("Hello, world", "latin-1") + ) + >= 0 + ) + + data_user_restricted = "PROXY TCP4 127.0.0.1 255.255.255.255 65535 65535\r\n\0\021ClickHouse client\024\r\253\251\003\0\007user123\0\004\001\0\001\0\0\t0.0.0.0:0\001\tmilovidov\021milovidov-desktop\vClickHouse \024\r\253\251\003\0\001\0\0\0\002\001\025SELECT 'Hello, world'\002\0\247\203\254l\325\\z|\265\254F\275\333\206\342\024\202\024\0\0\0\n\0\0\0\240\01\0\02\377\377\377\377\0\0\0" + assert ( + netcat(server.ip_address, 9100, bytearray(data_user_restricted, "latin-1")).find( + bytearray("Exception: user123: Authentication failed", "latin-1") + ) + >= 0 + ) + \ No newline at end of file From 82203e629cd8d419d94272983db70d72123e679e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 20 Dec 2022 17:46:17 +0000 Subject: [PATCH 03/78] Automatic style fix --- tests/integration/test_composable_protocols/test.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index ba7c25f4e08..bc87fea5296 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -103,9 +103,8 @@ def test_connections(): data_user_restricted = "PROXY TCP4 127.0.0.1 255.255.255.255 65535 65535\r\n\0\021ClickHouse client\024\r\253\251\003\0\007user123\0\004\001\0\001\0\0\t0.0.0.0:0\001\tmilovidov\021milovidov-desktop\vClickHouse \024\r\253\251\003\0\001\0\0\0\002\001\025SELECT 'Hello, world'\002\0\247\203\254l\325\\z|\265\254F\275\333\206\342\024\202\024\0\0\0\n\0\0\0\240\01\0\02\377\377\377\377\0\0\0" assert ( - netcat(server.ip_address, 9100, bytearray(data_user_restricted, "latin-1")).find( - bytearray("Exception: user123: Authentication failed", "latin-1") - ) + netcat( + server.ip_address, 9100, bytearray(data_user_restricted, "latin-1") + ).find(bytearray("Exception: user123: Authentication failed", "latin-1")) >= 0 ) - \ No newline at end of file From 4ab3e903821e35558442664c571368f354be30fe Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 21:21:30 +0000 Subject: [PATCH 04/78] Validate types in table function arguments/CAST function arguments/JSONAsObject schema inference --- src/DataTypes/DataTypeFixedString.h | 1 + src/Formats/EscapingRuleUtils.cpp | 4 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Formats/SchemaInferenceUtils.cpp | 6 +- src/Functions/CastOverloadResolver.h | 14 ++-- .../parseColumnsListForTableFunction.cpp | 70 ++++++++++++++++++- .../parseColumnsListForTableFunction.h | 27 ++++++- .../Impl/JSONAsStringRowInputFormat.cpp | 14 +++- .../Formats/Impl/JSONAsStringRowInputFormat.h | 2 + .../02513_validate_data_types.reference | 0 .../0_stateless/02513_validate_data_types.sql | 17 +++++ 12 files changed, 139 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02513_validate_data_types.reference create mode 100644 tests/queries/0_stateless/02513_validate_data_types.sql diff --git a/src/DataTypes/DataTypeFixedString.h b/src/DataTypes/DataTypeFixedString.h index 7c089866b23..437e3f9d3f5 100644 --- a/src/DataTypes/DataTypeFixedString.h +++ b/src/DataTypes/DataTypeFixedString.h @@ -5,6 +5,7 @@ #include #define MAX_FIXEDSTRING_SIZE 0xFFFFFF +#define MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS 256 namespace DB diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index df9c711b432..2a600fb9020 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -421,10 +421,10 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo break; case FormatSettings::EscapingRule::JSON: result += fmt::format( - ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, try_infer_objects={}", + ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, allow_object_type={}", settings.json.try_infer_numbers_from_strings, settings.json.read_bools_as_numbers, - settings.json.try_infer_objects); + settings.json.allow_object_type); break; default: break; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 91f2b0958f6..9385a562971 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -103,7 +103,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata; format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8; format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name; - format_settings.json.try_infer_objects = context->getSettingsRef().allow_experimental_object_type; + format_settings.json.allow_object_type = context->getSettingsRef().allow_experimental_object_type; format_settings.null_as_default = settings.input_format_null_as_default; format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros; format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 250601c3bf0..dcdd44edfeb 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -161,7 +161,7 @@ struct FormatSettings bool try_infer_numbers_from_strings = false; bool validate_types_from_metadata = true; bool validate_utf8 = false; - bool try_infer_objects = false; + bool allow_object_type = false; } json; struct diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 29b530966a0..89c69bcc9f6 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -363,7 +363,7 @@ namespace transformJSONTuplesAndArraysToArrays(data_types, settings, type_indexes, json_info); /// Convert Maps to Objects if needed. - if (settings.json.try_infer_objects) + if (settings.json.allow_object_type) transformMapsAndObjectsToObjects(data_types, type_indexes); if (settings.json.read_objects_as_strings) @@ -713,7 +713,7 @@ namespace { if constexpr (is_json) { - if (settings.json.try_infer_objects) + if (settings.json.allow_object_type) return std::make_shared("json", true); } /// Empty Map is Map(Nothing, Nothing) @@ -732,7 +732,7 @@ namespace transformInferredTypesIfNeededImpl(value_types, settings, json_info); if (!checkIfTypesAreEqual(value_types)) { - if (settings.json.try_infer_objects) + if (settings.json.allow_object_type) return std::make_shared("json", true); if (settings.json.read_objects_as_strings) return makeNullable(std::make_shared()); diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index 6aa3d97ff0a..ad11c802214 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { @@ -32,10 +33,11 @@ public: ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - explicit CastOverloadResolverImpl(std::optional diagnostic_, bool keep_nullable_, bool cast_ipv4_ipv6_default_on_conversion_error_) + explicit CastOverloadResolverImpl(std::optional diagnostic_, bool keep_nullable_, bool cast_ipv4_ipv6_default_on_conversion_error_, const DataTypeValidationSettings & data_type_validation_settings_) : diagnostic(std::move(diagnostic_)) , keep_nullable(keep_nullable_) , cast_ipv4_ipv6_default_on_conversion_error(cast_ipv4_ipv6_default_on_conversion_error_) + , data_type_validation_settings(data_type_validation_settings_) { } @@ -44,15 +46,15 @@ public: const auto & settings_ref = context->getSettingsRef(); if constexpr (internal) - return createImpl({}, false /*keep_nullable*/, settings_ref.cast_ipv4_ipv6_default_on_conversion_error); + return createImpl({}, false /*keep_nullable*/, settings_ref.cast_ipv4_ipv6_default_on_conversion_error, DataTypeValidationSettings(settings_ref)); - return createImpl({}, settings_ref.cast_keep_nullable, settings_ref.cast_ipv4_ipv6_default_on_conversion_error); + return createImpl({}, settings_ref.cast_keep_nullable, settings_ref.cast_ipv4_ipv6_default_on_conversion_error, DataTypeValidationSettings(settings_ref)); } - static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false, bool cast_ipv4_ipv6_default_on_conversion_error = false) + static FunctionOverloadResolverPtr createImpl(std::optional diagnostic = {}, bool keep_nullable = false, bool cast_ipv4_ipv6_default_on_conversion_error = false, const DataTypeValidationSettings & data_type_validation_settings = {}) { assert(!internal || !keep_nullable); - return std::make_unique(std::move(diagnostic), keep_nullable, cast_ipv4_ipv6_default_on_conversion_error); + return std::make_unique(std::move(diagnostic), keep_nullable, cast_ipv4_ipv6_default_on_conversion_error, data_type_validation_settings); } protected: @@ -83,6 +85,7 @@ protected: ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue()); + validateDataType(type, data_type_validation_settings); if constexpr (cast_type == CastType::accurateOrNull) return makeNullable(type); @@ -104,6 +107,7 @@ private: std::optional diagnostic; bool keep_nullable; bool cast_ipv4_ipv6_default_on_conversion_error; + DataTypeValidationSettings data_type_validation_settings; }; diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index ed270c2118b..29466514a04 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -4,6 +4,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -12,9 +15,65 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY; + extern const int ILLEGAL_COLUMN; + } -ColumnsDescription parseColumnsListFromString(const std::string & structure, ContextPtr context) +void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings & settings) +{ + if (!settings.allow_suspicious_low_cardinality_types) + { + if (const auto * lc_type = typeid_cast(type.get())) + { + if (!isStringOrFixedString(*removeNullable(lc_type->getDictionaryType()))) + throw Exception( + ErrorCodes::SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY, + "Creating columns of type {} is prohibited by default due to expected negative impact on performance. " + "It can be enabled with the \"allow_suspicious_low_cardinality_types\" setting.", + lc_type->getName()); + } + } + + if (!settings.allow_experimental_geo_types) + { + const auto & type_name = type->getName(); + if (type_name == "MultiPolygon" || type_name == "Polygon" || type_name == "Ring" || type_name == "Point") + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Cannot create column with type '{}' because experimental geo types are not allowed. Set setting " + "allow_experimental_geo_types = 1 in order to allow it", type_name); + } + } + + if (!settings.allow_experimental_object_type) + { + if (type->hasDynamicSubcolumns()) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Cannot create column with type '{}' because experimental Object type is not allowed. " + "Set setting allow_experimental_object_type = 1 in order to allow it", type->getName()); + } + } + + if (!settings.allow_suspicious_fixed_string_types) + { + auto basic_type = removeLowCardinality(removeNullable(type)); + if (const auto * fixed_string = typeid_cast(basic_type.get())) + { + if (fixed_string->getN() > MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Cannot create column with type '{}' because fixed string with size > {} is suspicious. " + "Set setting allow_suspicious_fixed_string_types = 1 in order to allow it", + type->getName(), MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS); + } + } +} + +ColumnsDescription parseColumnsListFromString(const std::string & structure, const ContextPtr & context) { ParserColumnDeclarationList parser(true, true); const Settings & settings = context->getSettingsRef(); @@ -25,10 +84,13 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, Con if (!columns_list) throw Exception("Could not cast AST to ASTExpressionList", ErrorCodes::LOGICAL_ERROR); - return InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); + auto columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); + for (const auto & [name, type] : columns.getAll()) + validateDataType(type, DataTypeValidationSettings(context->getSettingsRef())); + return columns; } -bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, ContextPtr context) +bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, const ContextPtr & context) { ParserColumnDeclarationList parser(true, true); const Settings & settings = context->getSettingsRef(); @@ -47,6 +109,8 @@ bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescrip try { columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); + for (const auto & [name, type] : columns.getAll()) + validateDataType(type, DataTypeValidationSettings(context->getSettingsRef())); return true; } catch (...) diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index e82a32f3d23..26968ff4290 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -9,9 +10,29 @@ namespace DB class Context; -/// Parses a common argument for table functions such as table structure given in string -ColumnsDescription parseColumnsListFromString(const std::string & structure, ContextPtr context); +struct DataTypeValidationSettings +{ + DataTypeValidationSettings() = default; -bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, ContextPtr context); + explicit DataTypeValidationSettings(const Settings & settings) + : allow_suspicious_low_cardinality_types(settings.allow_suspicious_low_cardinality_types) + , allow_experimental_geo_types(settings.allow_experimental_geo_types) + , allow_experimental_object_type(settings.allow_experimental_object_type) + , allow_suspicious_fixed_string_types(settings.allow_suspicious_fixed_string_types) + { + } + + bool allow_suspicious_low_cardinality_types = false; + bool allow_experimental_geo_types = false; + bool allow_experimental_object_type = false; + bool allow_suspicious_fixed_string_types = false; +}; + +void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings & settings); + +/// Parses a common argument for table functions such as table structure given in string +ColumnsDescription parseColumnsListFromString(const std::string & structure, const ContextPtr & context); + +bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, const ContextPtr & context); } diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index d369eedceea..dc4d12d7b18 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -12,6 +12,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int INCORRECT_DATA; + extern const int ILLEGAL_COLUMN; } JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) @@ -207,6 +208,15 @@ void JSONAsObjectRowInputFormat::readJSONObject(IColumn & column) serializations[0]->deserializeTextJSON(column, *buf, format_settings); } +JSONAsObjectExternalSchemaReader::JSONAsObjectExternalSchemaReader(const FormatSettings & settings) +{ + if (!settings.json.allow_object_type) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Cannot infer the data structure in JSONAsObject format because experimental Object type is not allowed. Set setting " + "allow_experimental_object_type = 1 in order to allow it"); +} + void registerInputFormatJSONAsString(FormatFactory & factory) { factory.registerInputFormat("JSONAsString", []( @@ -261,9 +271,9 @@ void registerFileSegmentationEngineJSONAsObject(FormatFactory & factory) void registerJSONAsObjectSchemaReader(FormatFactory & factory) { - factory.registerExternalSchemaReader("JSONAsObject", [](const FormatSettings &) + factory.registerExternalSchemaReader("JSONAsObject", [](const FormatSettings & settings) { - return std::make_shared(); + return std::make_shared(settings); }); } diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h index 438107e73e6..3f2d1998139 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h @@ -77,6 +77,8 @@ public: class JSONAsObjectExternalSchemaReader : public IExternalSchemaReader { public: + JSONAsObjectExternalSchemaReader(const FormatSettings & settings); + NamesAndTypesList readSchema() override { return {{"json", std::make_shared("json", false)}}; diff --git a/tests/queries/0_stateless/02513_validate_data_types.reference b/tests/queries/0_stateless/02513_validate_data_types.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02513_validate_data_types.sql b/tests/queries/0_stateless/02513_validate_data_types.sql new file mode 100644 index 00000000000..e4514c79e8d --- /dev/null +++ b/tests/queries/0_stateless/02513_validate_data_types.sql @@ -0,0 +1,17 @@ +set allow_experimental_object_type=0; +select CAST('{"x" : 1}', 'JSON'); -- {serverError ILLEGAL_COLUMN} +desc file(nonexist.json, JSONAsObject); -- {serverError ILLEGAL_COLUMN} +desc file(nonexist.json, JSONEachRow, 'x JSON'); -- {serverError ILLEGAL_COLUMN} + +set allow_experimental_geo_types=0; +select CAST([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]], 'Polygon'); -- {serverError ILLEGAL_COLUMN} +desc file(nonexist.json, JSONEachRow, 'pg Polygon'); -- {serverError ILLEGAL_COLUMN} + +set allow_suspicious_low_cardinality_types=0; +select CAST(1000000, 'LowCardinality(UInt64)'); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} +desc file(nonexist.json, JSONEachRow, 'lc LowCardinality(UInt64)'); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} + +set allow_suspicious_fixed_string_types=0; +select CAST('', 'FixedString(1000)'); -- {serverError ILLEGAL_COLUMN} +desc file(nonexist.json, JSONEachRow, 'fs FixedString(1000)'); -- {serverError ILLEGAL_COLUMN} + From f4b2266db0cc391211ec0391cd277923d4718340 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 21:22:26 +0000 Subject: [PATCH 05/78] Fix style --- .../parseColumnsListForTableFunction.cpp | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index 29466514a04..909efd35324 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -60,16 +60,17 @@ void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings if (!settings.allow_suspicious_fixed_string_types) { - auto basic_type = removeLowCardinality(removeNullable(type)); - if (const auto * fixed_string = typeid_cast(basic_type.get())) - { - if (fixed_string->getN() > MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Cannot create column with type '{}' because fixed string with size > {} is suspicious. " - "Set setting allow_suspicious_fixed_string_types = 1 in order to allow it", - type->getName(), MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS); - } + auto basic_type = removeLowCardinality(removeNullable(type)); + if (const auto * fixed_string = typeid_cast(basic_type.get())) + { + if (fixed_string->getN() > MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Cannot create column with type '{}' because fixed string with size > {} is suspicious. " + "Set setting allow_suspicious_fixed_string_types = 1 in order to allow it", + type->getName(), + MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS); + } } } From ba86424922d150d1cab21789e83cbf4b7b638003 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 21:23:12 +0000 Subject: [PATCH 06/78] Better --- src/Interpreters/parseColumnsListForTableFunction.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index 909efd35324..c44259a3ccc 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -86,8 +86,9 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, con throw Exception("Could not cast AST to ASTExpressionList", ErrorCodes::LOGICAL_ERROR); auto columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); + auto validation_settings = DataTypeValidationSettings(context->getSettingsRef()); for (const auto & [name, type] : columns.getAll()) - validateDataType(type, DataTypeValidationSettings(context->getSettingsRef())); + validateDataType(type, validation_settings); return columns; } @@ -110,8 +111,9 @@ bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescrip try { columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); + auto validation_settings = DataTypeValidationSettings(context->getSettingsRef()); for (const auto & [name, type] : columns.getAll()) - validateDataType(type, DataTypeValidationSettings(context->getSettingsRef())); + validateDataType(type, validation_settings); return true; } catch (...) From dfe8e3e02eeb503c65e5199953f2beedf70f6756 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 21:32:22 +0000 Subject: [PATCH 07/78] Remove duplicate define --- src/Interpreters/InterpreterCreateQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 50536b66185..43e933d4400 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -76,8 +76,6 @@ #include -#define MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS 256 - namespace DB { From 41c63152cfb443d09344b70d48d98293f490d352 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 23:47:56 +0000 Subject: [PATCH 08/78] Don't check internal CAST --- src/Functions/CastOverloadResolver.h | 2 +- src/Interpreters/parseColumnsListForTableFunction.h | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index ad11c802214..0bd0bad0d14 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -46,7 +46,7 @@ public: const auto & settings_ref = context->getSettingsRef(); if constexpr (internal) - return createImpl({}, false /*keep_nullable*/, settings_ref.cast_ipv4_ipv6_default_on_conversion_error, DataTypeValidationSettings(settings_ref)); + return createImpl({}, false /*keep_nullable*/, settings_ref.cast_ipv4_ipv6_default_on_conversion_error); return createImpl({}, settings_ref.cast_keep_nullable, settings_ref.cast_ipv4_ipv6_default_on_conversion_error, DataTypeValidationSettings(settings_ref)); } diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index 26968ff4290..97923bcad77 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -22,10 +22,10 @@ struct DataTypeValidationSettings { } - bool allow_suspicious_low_cardinality_types = false; - bool allow_experimental_geo_types = false; - bool allow_experimental_object_type = false; - bool allow_suspicious_fixed_string_types = false; + bool allow_suspicious_low_cardinality_types = true; + bool allow_experimental_geo_types = true; + bool allow_experimental_object_type = true; + bool allow_suspicious_fixed_string_types = true; }; void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings & settings); From 0c74b6c066d22d82791f63bc4515a6d9b20d02df Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 22 Dec 2022 17:40:06 +0000 Subject: [PATCH 09/78] Fix tests --- .../0_stateless/02049_lowcardinality_shortcircuit_crash.sql | 2 ++ tests/queries/0_stateless/02119_sumcount.sql | 3 +++ tests/queries/0_stateless/02513_validate_data_types.sql | 2 ++ 3 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql index 2a837380250..84d64d3166e 100644 --- a/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql +++ b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql @@ -1,4 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/30231 +SET allow_suspicious_low_cardinality_types=1; + SELECT * FROM ( SELECT number, diff --git a/tests/queries/0_stateless/02119_sumcount.sql b/tests/queries/0_stateless/02119_sumcount.sql index 22cb8b657da..86625996f44 100644 --- a/tests/queries/0_stateless/02119_sumcount.sql +++ b/tests/queries/0_stateless/02119_sumcount.sql @@ -22,6 +22,9 @@ SELECT toTypeName(sumCount(v)), sumCount(v) FROM ) ORDER BY v ); + +SET allow_suspicious_low_cardinality_types=1; + SELECT toTypeName(sumCount(v)), sumCount(v) FROM ( SELECT v FROM diff --git a/tests/queries/0_stateless/02513_validate_data_types.sql b/tests/queries/0_stateless/02513_validate_data_types.sql index e4514c79e8d..1235d00ba79 100644 --- a/tests/queries/0_stateless/02513_validate_data_types.sql +++ b/tests/queries/0_stateless/02513_validate_data_types.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + set allow_experimental_object_type=0; select CAST('{"x" : 1}', 'JSON'); -- {serverError ILLEGAL_COLUMN} desc file(nonexist.json, JSONAsObject); -- {serverError ILLEGAL_COLUMN} From 3d62bc2635d84105824b8455ee9876f0f3de6f24 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 23 Dec 2022 15:09:24 +0100 Subject: [PATCH 10/78] =?UTF-8?q?Allow=20macro=20substitution=20in=20endpo?= =?UTF-8?q?int=20of=20disks=20=C2=A0=20Implementation:=20*=20Updated=20to?= =?UTF-8?q?=20substitute=20macros=20in=20endpoint=20for=20all=20disks=20(s?= =?UTF-8?q?3,=20hfs,=20web).=20Testing:=20*=20Added=20an=20integration=20t?= =?UTF-8?q?est=20test/integration/test=5Fendpoint=5Fmacro=5Fsubstitution.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- src/Coordination/KeeperDispatcher.cpp | 8 +- src/Coordination/KeeperDispatcher.h | 6 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 10 ++- src/Coordination/KeeperSnapshotManagerS3.h | 10 ++- src/Coordination/TinyContext.cpp | 14 +++- .../ObjectStorages/HDFS/registerDiskHDFS.cpp | 5 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 7 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 5 +- .../ObjectStorages/S3/registerDiskS3.cpp | 5 +- .../Web/registerDiskWebServer.cpp | 7 +- src/Interpreters/Context.cpp | 4 +- .../__init__.py | 0 .../configs/config.xml | 44 +++++++++++ .../configs/macros.xml | 11 +++ .../configs/storage.xml | 22 ++++++ .../configs/users.xml | 22 ++++++ .../test_endpoint_macro_substitution/test.py | 73 +++++++++++++++++++ 17 files changed, 230 insertions(+), 23 deletions(-) create mode 100644 tests/integration/test_endpoint_macro_substitution/__init__.py create mode 100644 tests/integration/test_endpoint_macro_substitution/configs/config.xml create mode 100644 tests/integration/test_endpoint_macro_substitution/configs/macros.xml create mode 100644 tests/integration/test_endpoint_macro_substitution/configs/storage.xml create mode 100644 tests/integration/test_endpoint_macro_substitution/configs/users.xml create mode 100644 tests/integration/test_endpoint_macro_substitution/test.py diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index f6973b30fb2..92aeac425d5 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -296,7 +296,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ return true; } -void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper, bool start_async) +void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper, bool start_async, const MultiVersion::Version & macros) { LOG_DEBUG(log, "Initializing storage dispatcher"); @@ -307,7 +307,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); - snapshot_s3.startup(config); + snapshot_s3.startup(config, macros); server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, snapshot_s3); @@ -687,7 +687,7 @@ bool KeeperDispatcher::isServerActive() const return checkInit() && hasLeader() && !server->isRecovering(); } -void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros) { auto diff = server->getConfigurationDiff(config); if (diff.empty()) @@ -704,7 +704,7 @@ void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfigurati throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue"); } - snapshot_s3.updateS3Configuration(config); + snapshot_s3.updateS3Configuration(config, macros); } void KeeperDispatcher::updateKeeperStatLatency(uint64_t process_time_ms) diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 632e5e65e5f..2b4cd0c2b49 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -15,6 +15,8 @@ #include #include #include +#include +#include namespace DB { @@ -109,7 +111,7 @@ public: /// Initialization from config. /// standalone_keeper -- we are standalone keeper application (not inside clickhouse server) - void initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper, bool start_async); + void initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper, bool start_async, const MultiVersion::Version & macros); void startServer(); @@ -124,7 +126,7 @@ public: /// Registered in ConfigReloader callback. Add new configuration changes to /// update_configuration_queue. Keeper Dispatcher apply them asynchronously. - void updateConfiguration(const Poco::Util::AbstractConfiguration & config); + void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); /// Shutdown internal keeper parts (server, state machine, log storage, etc) void shutdown(); diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 361265e382a..5fac71ccbbc 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -47,7 +48,7 @@ KeeperSnapshotManagerS3::KeeperSnapshotManagerS3() , uuid(UUIDHelpers::generateV4()) {} -void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractConfiguration & config) +void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros) { try { @@ -64,7 +65,8 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo auto auth_settings = S3::AuthSettings::loadFromConfig(config_prefix, config); - auto endpoint = config.getString(config_prefix + ".endpoint"); + String endpoint = config.getString(config_prefix + ".endpoint"); + endpoint = macros->expand(endpoint); auto new_uri = S3::URI{endpoint}; { @@ -261,9 +263,9 @@ void KeeperSnapshotManagerS3::uploadSnapshot(const std::string & path, bool asyn uploadSnapshotImpl(path); } -void KeeperSnapshotManagerS3::startup(const Poco::Util::AbstractConfiguration & config) +void KeeperSnapshotManagerS3::startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros) { - updateS3Configuration(config); + updateS3Configuration(config, macros); snapshot_s3_thread = ThreadFromGlobalPool([this] { snapshotS3Thread(); }); } diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index 5b62d114aae..592217fbe66 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -3,6 +3,8 @@ #include "config.h" #include +#include +#include #if USE_AWS_S3 #include @@ -21,10 +23,10 @@ class KeeperSnapshotManagerS3 public: KeeperSnapshotManagerS3(); - void updateS3Configuration(const Poco::Util::AbstractConfiguration & config); + void updateS3Configuration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void uploadSnapshot(const std::string & path, bool async_upload = true); - void startup(const Poco::Util::AbstractConfiguration & config); + void startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macro); void shutdown(); private: using SnapshotS3Queue = ConcurrentBoundedQueue; @@ -56,10 +58,10 @@ class KeeperSnapshotManagerS3 public: KeeperSnapshotManagerS3() = default; - void updateS3Configuration(const Poco::Util::AbstractConfiguration &) {} + void updateS3Configuration(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} void uploadSnapshot(const std::string &, [[maybe_unused]] bool async_upload = true) {} - void startup(const Poco::Util::AbstractConfiguration &) {} + void startup(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} void shutdown() {} }; diff --git a/src/Coordination/TinyContext.cpp b/src/Coordination/TinyContext.cpp index 967e6b23d70..47b0a48dcda 100644 --- a/src/Coordination/TinyContext.cpp +++ b/src/Coordination/TinyContext.cpp @@ -36,7 +36,12 @@ void TinyContext::initializeKeeperDispatcher([[maybe_unused]] bool start_async) if (config_ref.has("keeper_server")) { keeper_dispatcher = std::make_shared(); - keeper_dispatcher->initialize(config_ref, true, start_async); + + MultiVersion::Version macros; + + if (config_ref.has("macros")) + macros = std::make_unique(config_ref, "macros", &Poco::Logger::get("TinyContext")); + keeper_dispatcher->initialize(config_ref, true, start_async, macros); } } @@ -71,7 +76,12 @@ void TinyContext::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::A if (!keeper_dispatcher) return; - keeper_dispatcher->updateConfiguration(config_); + MultiVersion::Version macros; + + if (config_.has("macros")) + macros = std::make_unique(config_, "macros", &Poco::Logger::get("TinyContext")); + + keeper_dispatcher->updateConfiguration(config_, macros); } } diff --git a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp index 7bec0ee5a6c..4a42e1ad9e2 100644 --- a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp +++ b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -23,7 +24,9 @@ void registerDiskHDFS(DiskFactory & factory, bool global_skip_access_check) ContextPtr context, const DisksMap & /*map*/) -> DiskPtr { - String uri{config.getString(config_prefix + ".endpoint")}; + String endpoint = config.getString(config_prefix + ".endpoint"); + endpoint = context->getMacros()->expand(endpoint); + String uri{endpoint}; checkHDFSURL(uri); if (uri.back() != '/') diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index d655fd37458..e549b5f8dd0 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -1,5 +1,7 @@ #include #include +#include + #if USE_AWS_S3 @@ -31,6 +33,7 @@ #include #include #include +#include namespace ProfileEvents @@ -634,10 +637,12 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( { auto new_s3_settings = getSettings(config, config_prefix, context); auto new_client = getClient(config, config_prefix, context, *new_s3_settings); + String endpoint = config.getString(config_prefix + ".endpoint"); + endpoint = context->getMacros()->expand(endpoint); return std::make_unique( std::move(new_client), std::move(new_s3_settings), version_id, s3_capabilities, new_namespace, - config.getString(config_prefix + ".endpoint")); + endpoint); } } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index ba3828c312c..aaf6e4cf885 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -22,6 +22,7 @@ #include #include #include +#include namespace DB { @@ -122,7 +123,9 @@ std::unique_ptr getClient( settings.request_settings.get_request_throttler, settings.request_settings.put_request_throttler); - S3::URI uri(config.getString(config_prefix + ".endpoint")); + String endpoint = config.getString(config_prefix + ".endpoint"); + endpoint = context->getMacros()->expand(endpoint); + S3::URI uri(endpoint); if (uri.key.back() != '/') throw Exception("S3 path must ends with '/', but '" + uri.key + "' doesn't.", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 533a925aa1b..b2a0b4234ab 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -23,6 +23,7 @@ #include #include +#include namespace DB @@ -104,7 +105,9 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) ContextPtr context, const DisksMap & /*map*/) -> DiskPtr { - S3::URI uri(config.getString(config_prefix + ".endpoint")); + String endpoint = config.getString(config_prefix + ".endpoint"); + endpoint = context->getMacros()->expand(endpoint); + S3::URI uri(endpoint); if (uri.key.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "No key in S3 uri: {}", uri.uri.toString()); diff --git a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp index 253d32ceb14..33480ca3f0a 100644 --- a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp +++ b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp @@ -5,6 +5,9 @@ #include #include #include +#include +#include + namespace DB { @@ -23,7 +26,9 @@ void registerDiskWebServer(DiskFactory & factory, bool global_skip_access_check) ContextPtr context, const DisksMap & /*map*/) -> DiskPtr { - String uri{config.getString(config_prefix + ".endpoint")}; + String endpoint = config.getString(config_prefix + ".endpoint"); + endpoint = context->getMacros()->expand(endpoint); + String uri{endpoint}; bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); if (!uri.ends_with('/')) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9e56aba9c0a..d5916fc7295 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2291,7 +2291,7 @@ void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) cons } shared->keeper_dispatcher = std::make_shared(); - shared->keeper_dispatcher->initialize(config, is_standalone_app, start_async); + shared->keeper_dispatcher->initialize(config, is_standalone_app, start_async, getMacros()); } #endif } @@ -2333,7 +2333,7 @@ void Context::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::Abstr if (!shared->keeper_dispatcher) return; - shared->keeper_dispatcher->updateConfiguration(config); + shared->keeper_dispatcher->updateConfiguration(config, getMacros()); #endif } diff --git a/tests/integration/test_endpoint_macro_substitution/__init__.py b/tests/integration/test_endpoint_macro_substitution/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_endpoint_macro_substitution/configs/config.xml b/tests/integration/test_endpoint_macro_substitution/configs/config.xml new file mode 100644 index 00000000000..74e190c8880 --- /dev/null +++ b/tests/integration/test_endpoint_macro_substitution/configs/config.xml @@ -0,0 +1,44 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + + + + + + s3 + http://minio1:9001/root/{endpoint_substitution}/ + minio + minio123 + + + encrypted + disk_s3 + 1234567812345678 + + + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + ./clickhouse/ + users.xml + diff --git a/tests/integration/test_endpoint_macro_substitution/configs/macros.xml b/tests/integration/test_endpoint_macro_substitution/configs/macros.xml new file mode 100644 index 00000000000..adead5fd2e4 --- /dev/null +++ b/tests/integration/test_endpoint_macro_substitution/configs/macros.xml @@ -0,0 +1,11 @@ + + + Hello, world! + test_shard_localhost + s1 + r1 + data + /clickhouse/tables/{database}/{shard}/ + table_{table} + + diff --git a/tests/integration/test_endpoint_macro_substitution/configs/storage.xml b/tests/integration/test_endpoint_macro_substitution/configs/storage.xml new file mode 100644 index 00000000000..f657f9ed932 --- /dev/null +++ b/tests/integration/test_endpoint_macro_substitution/configs/storage.xml @@ -0,0 +1,22 @@ + + + + + + s3 + http://minio1:9001/root/{endpoint_substitution}/ + minio + minio123 + + + hdfs + hdfs://hdfs1:9000/ + + + encrypted + disk_s3 + 1234567812345678 + + + + diff --git a/tests/integration/test_endpoint_macro_substitution/configs/users.xml b/tests/integration/test_endpoint_macro_substitution/configs/users.xml new file mode 100644 index 00000000000..4555a2ed494 --- /dev/null +++ b/tests/integration/test_endpoint_macro_substitution/configs/users.xml @@ -0,0 +1,22 @@ + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_endpoint_macro_substitution/test.py b/tests/integration/test_endpoint_macro_substitution/test.py new file mode 100644 index 00000000000..cf492658a2d --- /dev/null +++ b/tests/integration/test_endpoint_macro_substitution/test.py @@ -0,0 +1,73 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +disk_types = { + "default": "local", + "disk_s3": "s3", + "disk_hdfs": "hdfs", + "disk_encrypted": "s3", +} + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", + main_configs=["configs/storage.xml", "configs/macros.xml"], + with_minio=True, + with_hdfs=True, + ) + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_different_types(cluster): + node = cluster.instances["node"] + response = TSV.toMat(node.query("SELECT * FROM system.disks FORMAT TSVWithNames")) + + assert len(response) > len(disk_types) # at least one extra line for header + + name_col_ix = response[0].index("name") + type_col_ix = response[0].index("type") + encrypted_col_ix = response[0].index("is_encrypted") + + for fields in response[1:]: # skip header + assert len(fields) >= 7 + assert ( + disk_types.get(fields[name_col_ix], "UNKNOWN") == fields[type_col_ix] + ), f"Wrong type ({fields[type_col_ix]}) for disk {fields[name_col_ix]}!" + if "encrypted" in fields[name_col_ix]: + assert ( + fields[encrypted_col_ix] == "1" + ), f"{fields[name_col_ix]} expected to be encrypted!" + else: + assert ( + fields[encrypted_col_ix] == "0" + ), f"{fields[name_col_ix]} expected to be non-encrypted!" + + +def test_select_by_type(cluster): + node = cluster.instances["node"] + for name, disk_type in list(disk_types.items()): + if disk_type != "s3": + assert ( + node.query( + "SELECT name FROM system.disks WHERE type='" + disk_type + "'" + ) + == name + "\n" + ) + else: + assert ( + node.query( + "SELECT name FROM system.disks WHERE type='" + + disk_type + + "' ORDER BY name" + ) + == "disk_encrypted\ndisk_s3\n" + ) From 84362461fbe8b48f21225e76022f66b799af8103 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 23 Dec 2022 16:07:56 +0100 Subject: [PATCH 11/78] Updated test to use macro substitution for hdfs disk - Allow macro substitution in endpoint --- .../configs/config.xml | 44 ------------------- .../configs/macros.xml | 1 + .../configs/storage.xml | 2 +- .../test_endpoint_macro_substitution/test.py | 4 ++ 4 files changed, 6 insertions(+), 45 deletions(-) delete mode 100644 tests/integration/test_endpoint_macro_substitution/configs/config.xml diff --git a/tests/integration/test_endpoint_macro_substitution/configs/config.xml b/tests/integration/test_endpoint_macro_substitution/configs/config.xml deleted file mode 100644 index 74e190c8880..00000000000 --- a/tests/integration/test_endpoint_macro_substitution/configs/config.xml +++ /dev/null @@ -1,44 +0,0 @@ - - - trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - 1000M - 10 - - - - - - - - s3 - http://minio1:9001/root/{endpoint_substitution}/ - minio - minio123 - - - encrypted - disk_s3 - 1234567812345678 - - - - - 9000 - 127.0.0.1 - - - - true - none - - AcceptCertificateHandler - - - - - 500 - ./clickhouse/ - users.xml - diff --git a/tests/integration/test_endpoint_macro_substitution/configs/macros.xml b/tests/integration/test_endpoint_macro_substitution/configs/macros.xml index adead5fd2e4..ee21a24c3ba 100644 --- a/tests/integration/test_endpoint_macro_substitution/configs/macros.xml +++ b/tests/integration/test_endpoint_macro_substitution/configs/macros.xml @@ -5,6 +5,7 @@ s1 r1 data + clickhouse /clickhouse/tables/{database}/{shard}/ table_{table} diff --git a/tests/integration/test_endpoint_macro_substitution/configs/storage.xml b/tests/integration/test_endpoint_macro_substitution/configs/storage.xml index f657f9ed932..403ce8e52ae 100644 --- a/tests/integration/test_endpoint_macro_substitution/configs/storage.xml +++ b/tests/integration/test_endpoint_macro_substitution/configs/storage.xml @@ -10,7 +10,7 @@ hdfs - hdfs://hdfs1:9000/ + hdfs://hdfs1:9000/{hdfs_endpoint_substitution}/ encrypted diff --git a/tests/integration/test_endpoint_macro_substitution/test.py b/tests/integration/test_endpoint_macro_substitution/test.py index cf492658a2d..ba880a5095d 100644 --- a/tests/integration/test_endpoint_macro_substitution/test.py +++ b/tests/integration/test_endpoint_macro_substitution/test.py @@ -1,6 +1,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +from pyhdfs import HdfsClient disk_types = { "default": "local", @@ -22,6 +23,9 @@ def cluster(): ) cluster.start() + fs = HdfsClient(hosts=cluster.hdfs_ip) + fs.mkdirs("/clickhouse") + yield cluster finally: cluster.shutdown() From a166dde9c41c9961c8080c2ffcd9f132d0270022 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 23 Dec 2022 18:15:36 +0000 Subject: [PATCH 12/78] add member function getLastForwardedFor to ClientInfo --- src/Access/AccessControl.cpp | 10 +--------- src/Interpreters/ClientInfo.h | 9 +++++++++ src/Server/HTTPHandler.cpp | 12 +++--------- src/Server/TCPHandler.cpp | 11 +++-------- 4 files changed, 16 insertions(+), 26 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 80199683a96..9b4dbad640e 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -695,15 +695,7 @@ std::shared_ptr AccessControl::getContextAccess( /// Extract the last entry from comma separated list of X-Forwarded-For addresses. /// Only the last proxy can be trusted (if any). - Strings forwarded_addresses; - if (!client_info.forwarded_for.empty()) - boost::split(forwarded_addresses, client_info.forwarded_for, boost::is_any_of(",")); - if (!forwarded_addresses.empty()) - { - String & last_forwarded_address = forwarded_addresses.back(); - boost::trim(last_forwarded_address); - params.forwarded_address = last_forwarded_address; - } + params.forwarded_address = client_info.getLastForwardedFor(); return getContextAccess(params); } diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index f7a172b226d..5c5a284d63b 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -101,6 +102,14 @@ public: /// The element can be trusted only if you trust the corresponding proxy. /// NOTE This field can also be reused in future for TCP interface with PROXY v1/v2 protocols. String forwarded_for; + String getLastForwardedFor() const + { + if (forwarded_for.empty()) + return {}; + String last = forwarded_for.substr(forwarded_for.find_last_of(',') + 1); + boost::trim(last); + return last; + } /// Common String quota_key; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index cd84153d3ee..c3dd821a5bf 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -474,17 +474,11 @@ bool HTTPHandler::authenticateUser( /// Extract the last entry from comma separated list of forwarded_for addresses. /// Only the last proxy can be trusted (if any). - Strings forwarded_addresses; - if (!client_info.forwarded_for.empty()) - boost::split(forwarded_addresses, client_info.forwarded_for, boost::is_any_of(",")); - + String forwarded_address = client_info.getLastForwardedFor(); try { - if (!forwarded_addresses.empty() && server.config().getBool("auth_use_forwarded_address", false)) - { - boost::trim(forwarded_addresses.back()); - session->authenticate(*request_credentials, Poco::Net::SocketAddress(forwarded_addresses.back(), request.clientAddress().port())); - } + if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false)) + session->authenticate(*request_credentials, Poco::Net::SocketAddress(forwarded_address, request.clientAddress().port())); else session->authenticate(*request_credentials, request.clientAddress()); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c9d57a8e3be..c643cd6fa00 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1165,14 +1165,9 @@ void TCPHandler::receiveHello() /// Extract the last entry from comma separated list of forwarded_for addresses. /// Only the last proxy can be trusted (if any). - Strings forwarded_addresses; - if (!client_info.forwarded_for.empty()) - boost::split(forwarded_addresses, client_info.forwarded_for, boost::is_any_of(",")); - if (!forwarded_addresses.empty() && server.config().getBool("auth_use_forwarded_address", false)) - { - boost::trim(forwarded_addresses.back()); - session->authenticate(user, password, Poco::Net::SocketAddress(forwarded_addresses.back(), socket().peerAddress().port())); - } + String forwarded_address = client_info.getLastForwardedFor(); + if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false)) + session->authenticate(user, password, Poco::Net::SocketAddress(forwarded_address, socket().peerAddress().port())); else session->authenticate(user, password, socket().peerAddress()); } From ce533af16209d5ca42759b6e180d0c9e7011f194 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 23 Dec 2022 19:29:51 +0000 Subject: [PATCH 13/78] remove unnecessary includes --- src/Access/AccessControl.cpp | 2 -- src/Server/HTTPHandler.cpp | 2 -- src/Server/TCPHandler.cpp | 2 -- 3 files changed, 6 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 9b4dbad640e..4b1e7051a9b 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -25,8 +25,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index c3dd821a5bf..7856659c6a7 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -27,8 +27,6 @@ #include #include #include -#include -#include #include #include diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c643cd6fa00..331313fedab 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -46,8 +46,6 @@ #include #include #include -#include -#include #include #include From 8e53b54082aa3ca73ee9337f7f648f6b9da1296b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 21 Nov 2022 22:29:40 +0100 Subject: [PATCH 14/78] Update AWS SDK to version 1.10.36 --- .gitmodules | 36 ++- contrib/CMakeLists.txt | 15 +- contrib/aws | 2 +- contrib/aws-c-auth | 1 + contrib/aws-c-cal | 1 + contrib/aws-c-common | 2 +- contrib/aws-c-compression | 1 + contrib/aws-c-event-stream | 2 +- contrib/aws-c-http | 1 + contrib/aws-c-io | 1 + contrib/aws-c-mqtt | 1 + contrib/aws-c-s3 | 1 + contrib/aws-c-sdkutils | 1 + contrib/aws-checksums | 2 +- contrib/aws-cmake/AwsFeatureTests.cmake | 114 +++++++ contrib/aws-cmake/AwsSIMD.cmake | 74 +++++ contrib/aws-cmake/AwsThreadAffinity.cmake | 50 +++ contrib/aws-cmake/AwsThreadName.cmake | 61 ++++ contrib/aws-cmake/CMakeLists.txt | 376 ++++++++++++++++++++++ contrib/aws-crt-cpp | 1 + contrib/aws-s2n-tls | 1 + contrib/aws-s3-cmake/CMakeLists.txt | 122 ------- 22 files changed, 736 insertions(+), 130 deletions(-) create mode 160000 contrib/aws-c-auth create mode 160000 contrib/aws-c-cal create mode 160000 contrib/aws-c-compression create mode 160000 contrib/aws-c-http create mode 160000 contrib/aws-c-io create mode 160000 contrib/aws-c-mqtt create mode 160000 contrib/aws-c-s3 create mode 160000 contrib/aws-c-sdkutils create mode 100644 contrib/aws-cmake/AwsFeatureTests.cmake create mode 100644 contrib/aws-cmake/AwsSIMD.cmake create mode 100644 contrib/aws-cmake/AwsThreadAffinity.cmake create mode 100644 contrib/aws-cmake/AwsThreadName.cmake create mode 100644 contrib/aws-cmake/CMakeLists.txt create mode 160000 contrib/aws-crt-cpp create mode 160000 contrib/aws-s2n-tls delete mode 100644 contrib/aws-s3-cmake/CMakeLists.txt diff --git a/.gitmodules b/.gitmodules index 0805b6d5492..d35741362b0 100644 --- a/.gitmodules +++ b/.gitmodules @@ -104,13 +104,13 @@ url = https://github.com/ClickHouse/aws-sdk-cpp.git [submodule "aws-c-event-stream"] path = contrib/aws-c-event-stream - url = https://github.com/ClickHouse/aws-c-event-stream.git + url = https://github.com/awslabs/aws-c-event-stream.git [submodule "aws-c-common"] path = contrib/aws-c-common - url = https://github.com/ClickHouse/aws-c-common.git + url = https://github.com/awslabs/aws-c-common.git [submodule "aws-checksums"] path = contrib/aws-checksums - url = https://github.com/ClickHouse/aws-checksums.git + url = https://github.com/awslabs/aws-checksums.git [submodule "contrib/curl"] path = contrib/curl url = https://github.com/curl/curl.git @@ -294,3 +294,33 @@ [submodule "contrib/libdivide"] path = contrib/libdivide url = https://github.com/ridiculousfish/libdivide.git +[submodule "contrib/aws-crt-cpp"] + path = contrib/aws-crt-cpp + url = https://github.com/awslabs/aws-crt-cpp.git +[submodule "contrib/aws-c-io"] + path = contrib/aws-c-io + url = https://github.com/awslabs/aws-c-io.git +[submodule "contrib/aws-c-mqtt"] + path = contrib/aws-c-mqtt + url = https://github.com/awslabs/aws-c-mqtt.git +[submodule "contrib/aws-c-auth"] + path = contrib/aws-c-auth + url = https://github.com/awslabs/aws-c-auth.git +[submodule "contrib/aws-c-cal"] + path = contrib/aws-c-cal + url = https://github.com/awslabs/aws-c-cal.git +[submodule "contrib/aws-c-sdkutils"] + path = contrib/aws-c-sdkutils + url = https://github.com/awslabs/aws-c-sdkutils.git +[submodule "contrib/aws-c-http"] + path = contrib/aws-c-http + url = https://github.com/awslabs/aws-c-http.git +[submodule "contrib/aws-c-s3"] + path = contrib/aws-c-s3 + url = https://github.com/awslabs/aws-c-s3.git +[submodule "contrib/aws-c-compression"] + path = contrib/aws-c-compression + url = https://github.com/awslabs/aws-c-compression.git +[submodule "contrib/aws-s2n-tls"] + path = contrib/aws-s2n-tls + url = https://github.com/aws/s2n-tls.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 6f80059498e..13c4722e149 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -115,12 +115,25 @@ endif() add_contrib (llvm-project-cmake llvm-project) add_contrib (libfuzzer-cmake llvm-project) add_contrib (libxml2-cmake libxml2) -add_contrib (aws-s3-cmake + +add_contrib (aws-cmake aws + aws-c-auth + aws-c-cal aws-c-common + aws-c-compression aws-c-event-stream + aws-c-http + aws-c-io + aws-c-mqtt + aws-c-s3 + aws-c-sdkutils + aws-s2n-tls aws-checksums + aws-crt-cpp + aws-cmake ) + add_contrib (base64-cmake base64) add_contrib (simdjson-cmake simdjson) add_contrib (rapidjson-cmake rapidjson) diff --git a/contrib/aws b/contrib/aws index 00b03604543..2b711c84575 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 00b03604543367d7e310cb0993973fdcb723ea79 +Subproject commit 2b711c845751a93695ad86433de8bc24ecc4746a diff --git a/contrib/aws-c-auth b/contrib/aws-c-auth new file mode 160000 index 00000000000..30df6c407e2 --- /dev/null +++ b/contrib/aws-c-auth @@ -0,0 +1 @@ +Subproject commit 30df6c407e2df43bd244e2c34c9b4a4b87372bfb diff --git a/contrib/aws-c-cal b/contrib/aws-c-cal new file mode 160000 index 00000000000..ac4216b78d5 --- /dev/null +++ b/contrib/aws-c-cal @@ -0,0 +1 @@ +Subproject commit ac4216b78d5323b5b8ce95a3dd4a8fc0f95e2d33 diff --git a/contrib/aws-c-common b/contrib/aws-c-common index 736a82d1697..d6a6a6057af 160000 --- a/contrib/aws-c-common +++ b/contrib/aws-c-common @@ -1 +1 @@ -Subproject commit 736a82d1697c108b04a277e66438a7f4e19b6857 +Subproject commit d6a6a6057afd8024cf2790a50de4a9818014cec6 diff --git a/contrib/aws-c-compression b/contrib/aws-c-compression new file mode 160000 index 00000000000..b517b7decd0 --- /dev/null +++ b/contrib/aws-c-compression @@ -0,0 +1 @@ +Subproject commit b517b7decd0dac30be2162f5186c250221c53aff diff --git a/contrib/aws-c-event-stream b/contrib/aws-c-event-stream index 3bc33662f9c..39bfa94a14b 160000 --- a/contrib/aws-c-event-stream +++ b/contrib/aws-c-event-stream @@ -1 +1 @@ -Subproject commit 3bc33662f9ccff4f4cbcf9509cc78c26e022fde0 +Subproject commit 39bfa94a14b7126bf0c1330286ef8db452d87e66 diff --git a/contrib/aws-c-http b/contrib/aws-c-http new file mode 160000 index 00000000000..2c5a2a7d555 --- /dev/null +++ b/contrib/aws-c-http @@ -0,0 +1 @@ +Subproject commit 2c5a2a7d5556600b9782ffa6c9d7e09964df1abc diff --git a/contrib/aws-c-io b/contrib/aws-c-io new file mode 160000 index 00000000000..d353ffde788 --- /dev/null +++ b/contrib/aws-c-io @@ -0,0 +1 @@ +Subproject commit d353ffde788b70f39e7da889784262e4d4eb5101 diff --git a/contrib/aws-c-mqtt b/contrib/aws-c-mqtt new file mode 160000 index 00000000000..882c689561a --- /dev/null +++ b/contrib/aws-c-mqtt @@ -0,0 +1 @@ +Subproject commit 882c689561a3db1466330ccfe3b63637e0a575d3 diff --git a/contrib/aws-c-s3 b/contrib/aws-c-s3 new file mode 160000 index 00000000000..a41255ece72 --- /dev/null +++ b/contrib/aws-c-s3 @@ -0,0 +1 @@ +Subproject commit a41255ece72a7c887bba7f9d998ca3e14f4c8a1b diff --git a/contrib/aws-c-sdkutils b/contrib/aws-c-sdkutils new file mode 160000 index 00000000000..25bf5cf225f --- /dev/null +++ b/contrib/aws-c-sdkutils @@ -0,0 +1 @@ +Subproject commit 25bf5cf225f977c3accc6a05a0a7a181ef2a4a30 diff --git a/contrib/aws-checksums b/contrib/aws-checksums index 519d6d90938..48e7c0e0147 160000 --- a/contrib/aws-checksums +++ b/contrib/aws-checksums @@ -1 +1 @@ -Subproject commit 519d6d9093819b6cf89ffff589a27ef8f83d0f65 +Subproject commit 48e7c0e01479232f225c8044d76c84e74192889d diff --git a/contrib/aws-cmake/AwsFeatureTests.cmake b/contrib/aws-cmake/AwsFeatureTests.cmake new file mode 100644 index 00000000000..54727e08d6b --- /dev/null +++ b/contrib/aws-cmake/AwsFeatureTests.cmake @@ -0,0 +1,114 @@ +# Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. +# SPDX-License-Identifier: Apache-2.0. + +include(CheckCSourceRuns) + +option(USE_CPU_EXTENSIONS "Whenever possible, use functions optimized for CPUs with specific extensions (ex: SSE, AVX)." ON) + +# In the current (11/2/21) state of mingw64, the packaged gcc is not capable of emitting properly aligned avx2 instructions under certain circumstances. +# This leads to crashes for windows builds using mingw64 when invoking the avx2-enabled versions of certain functions. Until we can find a better +# work-around, disable avx2 (and all other extensions) in mingw builds. +# +# https://gcc.gnu.org/bugzilla/show_bug.cgi?id=54412 +# +if (MINGW) + message(STATUS "MINGW detected! Disabling avx2 and other CPU extensions") + set(USE_CPU_EXTENSIONS OFF) +endif() + +if(NOT CMAKE_CROSSCOMPILING) + check_c_source_runs(" + #include + bool foo(int a, int b, int *c) { + return __builtin_mul_overflow(a, b, c); + } + + int main() { + int out; + if (foo(1, 2, &out)) { + return 0; + } + + return 0; + }" AWS_HAVE_GCC_OVERFLOW_MATH_EXTENSIONS) + + if (USE_CPU_EXTENSIONS) + check_c_source_runs(" + int main() { + int foo = 42; + _mulx_u32(1, 2, &foo); + return foo != 2; + }" AWS_HAVE_MSVC_MULX) + endif() + +endif() + +check_c_source_compiles(" + #include + #if WINAPI_FAMILY_PARTITION(WINAPI_PARTITION_DESKTOP) + int main() { + return 0; + } + #else + it's not windows desktop + #endif +" AWS_HAVE_WINAPI_DESKTOP) + +check_c_source_compiles(" + int main() { +#if !(defined(__x86_64__) || defined(__i386__) || defined(_M_X64) || defined(_M_IX86)) +# error \"not intel\" +#endif + return 0; + } +" AWS_ARCH_INTEL) + +check_c_source_compiles(" + int main() { +#if !(defined(__aarch64__) || defined(_M_ARM64)) +# error \"not arm64\" +#endif + return 0; + } +" AWS_ARCH_ARM64) + +check_c_source_compiles(" + int main() { +#if !(defined(__arm__) || defined(_M_ARM)) +# error \"not arm\" +#endif + return 0; + } +" AWS_ARCH_ARM32) + +check_c_source_compiles(" +int main() { + int foo = 42, bar = 24; + __asm__ __volatile__(\"\":\"=r\"(foo):\"r\"(bar):\"memory\"); +}" AWS_HAVE_GCC_INLINE_ASM) + +check_c_source_compiles(" +#include +int main() { +#ifdef __linux__ + getauxval(AT_HWCAP); + getauxval(AT_HWCAP2); +#endif + return 0; +}" AWS_HAVE_AUXV) + +string(REGEX MATCH "^(aarch64|arm)" ARM_CPU "${CMAKE_SYSTEM_PROCESSOR}") +if(NOT LEGACY_COMPILER_SUPPORT OR ARM_CPU) + check_c_source_compiles(" + #include + int main() { + backtrace(NULL, 0); + return 0; + }" AWS_HAVE_EXECINFO) +endif() + +check_c_source_compiles(" +#include +int main() { + return 1; +}" AWS_HAVE_LINUX_IF_LINK_H) diff --git a/contrib/aws-cmake/AwsSIMD.cmake b/contrib/aws-cmake/AwsSIMD.cmake new file mode 100644 index 00000000000..bd6f4064e78 --- /dev/null +++ b/contrib/aws-cmake/AwsSIMD.cmake @@ -0,0 +1,74 @@ +# Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. +# SPDX-License-Identifier: Apache-2.0. + +include(CheckCCompilerFlag) +include(CheckIncludeFile) + +if (USE_CPU_EXTENSIONS) + if (MSVC) + check_c_compiler_flag("/arch:AVX2" HAVE_M_AVX2_FLAG) + if (HAVE_M_AVX2_FLAG) + set(AVX2_CFLAGS "/arch:AVX2") + endif() + else() + check_c_compiler_flag(-mavx2 HAVE_M_AVX2_FLAG) + if (HAVE_M_AVX2_FLAG) + set(AVX2_CFLAGS "-mavx -mavx2") + endif() + endif() + + + cmake_push_check_state() + set(CMAKE_REQUIRED_FLAGS "${CMAKE_REQUIRED_FLAGS} ${AVX2_CFLAGS}") + + check_c_source_compiles(" + #include + #include + #include + + int main() { + __m256i vec; + memset(&vec, 0, sizeof(vec)); + + _mm256_shuffle_epi8(vec, vec); + _mm256_set_epi32(1,2,3,4,5,6,7,8); + _mm256_permutevar8x32_epi32(vec, vec); + + return 0; + }" HAVE_AVX2_INTRINSICS) + + check_c_source_compiles(" + #include + #include + + int main() { + __m256i vec; + memset(&vec, 0, sizeof(vec)); + return (int)_mm256_extract_epi64(vec, 2); + }" HAVE_MM256_EXTRACT_EPI64) + + cmake_pop_check_state() +endif() # USE_CPU_EXTENSIONS + +macro(simd_add_definition_if target definition) + if(${definition}) + target_compile_definitions(${target} PRIVATE -D${definition}) + endif(${definition}) +endmacro(simd_add_definition_if) + +# Configure private preprocessor definitions for SIMD-related features +# Does not set any processor feature codegen flags +function(simd_add_definitions target) + simd_add_definition_if(${target} HAVE_AVX2_INTRINSICS) + simd_add_definition_if(${target} HAVE_MM256_EXTRACT_EPI64) +endfunction(simd_add_definitions) + +# Adds source files only if AVX2 is supported. These files will be built with +# avx2 intrinsics enabled. +# Usage: simd_add_source_avx2(target file1.c file2.c ...) +function(simd_add_source_avx2 target) + foreach(file ${ARGN}) + target_sources(${target} PRIVATE ${file}) + set_source_files_properties(${file} PROPERTIES COMPILE_FLAGS "${AVX2_CFLAGS}") + endforeach() +endfunction(simd_add_source_avx2) diff --git a/contrib/aws-cmake/AwsThreadAffinity.cmake b/contrib/aws-cmake/AwsThreadAffinity.cmake new file mode 100644 index 00000000000..9e53481272c --- /dev/null +++ b/contrib/aws-cmake/AwsThreadAffinity.cmake @@ -0,0 +1,50 @@ +# Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. +# SPDX-License-Identifier: Apache-2.0. + +include(CheckSymbolExists) + +# Check if the platform supports setting thread affinity +# (important for hitting full NIC entitlement on NUMA architectures) +function(aws_set_thread_affinity_method target) + + # Non-POSIX, Android, and Apple platforms do not support thread affinity. + if (NOT UNIX OR ANDROID OR APPLE) + target_compile_definitions(${target} PRIVATE + -DAWS_AFFINITY_METHOD=AWS_AFFINITY_METHOD_NONE) + return() + endif() + + cmake_push_check_state() + list(APPEND CMAKE_REQUIRED_DEFINITIONS -D_GNU_SOURCE) + list(APPEND CMAKE_REQUIRED_LIBRARIES pthread) + + set(headers "pthread.h") + # BSDs put nonportable pthread declarations in a separate header. + if(CMAKE_SYSTEM_NAME MATCHES BSD) + set(headers "${headers};pthread_np.h") + endif() + + # Using pthread attrs is the preferred method, but is glibc-specific. + check_symbol_exists(pthread_attr_setaffinity_np "${headers}" USE_PTHREAD_ATTR_SETAFFINITY) + if (USE_PTHREAD_ATTR_SETAFFINITY) + target_compile_definitions(${target} PRIVATE + -DAWS_AFFINITY_METHOD=AWS_AFFINITY_METHOD_PTHREAD_ATTR) + return() + endif() + + # This method is still nonportable, but is supported by musl and BSDs. + check_symbol_exists(pthread_setaffinity_np "${headers}" USE_PTHREAD_SETAFFINITY) + if (USE_PTHREAD_SETAFFINITY) + target_compile_definitions(${target} PRIVATE + -DAWS_AFFINITY_METHOD=AWS_AFFINITY_METHOD_PTHREAD) + return() + endif() + + # If we got here, we expected thread affinity support but didn't find it. + # We still build with degraded NUMA performance, but show a warning. + message(WARNING "No supported method for setting thread affinity") + target_compile_definitions(${target} PRIVATE + -DAWS_AFFINITY_METHOD=AWS_AFFINITY_METHOD_NONE) + + cmake_pop_check_state() +endfunction() diff --git a/contrib/aws-cmake/AwsThreadName.cmake b/contrib/aws-cmake/AwsThreadName.cmake new file mode 100644 index 00000000000..a67416b4f83 --- /dev/null +++ b/contrib/aws-cmake/AwsThreadName.cmake @@ -0,0 +1,61 @@ +# Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. +# SPDX-License-Identifier: Apache-2.0. + +include(CheckSymbolExists) + +# Check how the platform supports setting thread name +function(aws_set_thread_name_method target) + + if (WINDOWS) + # On Windows we do a runtime check, instead of compile-time check + return() + elseif (APPLE) + # All Apple platforms we support have the same function, so no need for compile-time check. + return() + endif() + + cmake_push_check_state() + list(APPEND CMAKE_REQUIRED_DEFINITIONS -D_GNU_SOURCE) + list(APPEND CMAKE_REQUIRED_LIBRARIES pthread) + + # The start of the test program + set(c_source_start " + #define _GNU_SOURCE + #include + + #if defined(__FreeBSD__) || defined(__NETBSD__) + #include + #endif + + int main() { + pthread_t thread_id; + ") + + # The end of the test program + set(c_source_end "}") + + # pthread_setname_np() usually takes 2 args + check_c_source_compiles(" + ${c_source_start} + pthread_setname_np(thread_id, \"asdf\"); + ${c_source_end}" + PTHREAD_SETNAME_TAKES_2ARGS) + if (PTHREAD_SETNAME_TAKES_2ARGS) + target_compile_definitions(${target} PRIVATE -DAWS_PTHREAD_SETNAME_TAKES_2ARGS) + return() + endif() + + # But on NetBSD it takes 3! + check_c_source_compiles(" + ${c_source_start} + pthread_setname_np(thread_id, \"asdf\", NULL); + ${c_source_end} + " PTHREAD_SETNAME_TAKES_3ARGS) + if (PTHREAD_SETNAME_TAKES_3ARGS) + target_compile_definitions(${target} PRIVATE -DAWS_PTHREAD_SETNAME_TAKES_3ARGS) + return() + endif() + + # And on many older/weirder platforms it's just not supported + cmake_pop_check_state() +endfunction() diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt new file mode 100644 index 00000000000..52533cd6483 --- /dev/null +++ b/contrib/aws-cmake/CMakeLists.txt @@ -0,0 +1,376 @@ +set(ENABLE_AWS_S3_DEFAULT OFF) + +if(ENABLE_LIBRARIES AND (OS_LINUX OR OS_DARWIN) AND TARGET OpenSSL::Crypto) + set(ENABLE_AWS_S3_DEFAULT ON) +endif() + +option(ENABLE_AWS_S3 "Enable AWS S3" ${ENABLE_AWS_S3_DEFAULT}) + +if(ENABLE_AWS_S3) + if(NOT TARGET OpenSSL::Crypto) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use AWS SDK without OpenSSL") + elseif(NOT (OS_LINUX OR OS_DARWIN)) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use AWS SDK with platform ${CMAKE_SYSTEM_NAME}") + endif() +endif() + +if(NOT ENABLE_AWS_S3) + message(STATUS "Not using AWS S3") + return() +endif() + + +# Utilities. +include("${ClickHouse_SOURCE_DIR}/contrib/aws-cmake/AwsFeatureTests.cmake") +include("${ClickHouse_SOURCE_DIR}/contrib/aws-cmake/AwsThreadAffinity.cmake") +include("${ClickHouse_SOURCE_DIR}/contrib/aws-cmake/AwsThreadName.cmake") +include("${ClickHouse_SOURCE_DIR}/contrib/aws-cmake/AwsSIMD.cmake") + + +# Gather sources and options. +set(AWS_SOURCES) +set(AWS_PUBLIC_INCLUDES) +set(AWS_PRIVATE_INCLUDES) +set(AWS_PUBLIC_COMPILE_DEFS) +set(AWS_PRIVATE_COMPILE_DEFS) +set(AWS_PRIVATE_LIBS) + +if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") + list(APPEND AWS_PRIVATE_COMPILE_DEFS "-DDEBUG_BUILD") +endif() + +set(ENABLE_OPENSSL_ENCRYPTION ON) +if (ENABLE_OPENSSL_ENCRYPTION) + list(APPEND AWS_PRIVATE_COMPILE_DEFS "-DENABLE_OPENSSL_ENCRYPTION") +endif() + +set(USE_S2N ON) +if (USE_S2N) + list(APPEND AWS_PRIVATE_COMPILE_DEFS "-DUSE_S2N") +endif() + + +# Directories. +SET(AWS_SDK_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws") +SET(AWS_SDK_CORE_DIR "${AWS_SDK_DIR}/aws-cpp-sdk-core") +SET(AWS_SDK_S3_DIR "${AWS_SDK_DIR}/aws-cpp-sdk-s3") + +SET(AWS_AUTH_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-auth") +SET(AWS_CAL_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-cal") +SET(AWS_CHECKSUMS_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-checksums") +SET(AWS_COMMON_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-common") +SET(AWS_COMPRESSION_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-compression") +SET(AWS_CRT_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-crt-cpp") +SET(AWS_EVENT_STREAM_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-event-stream") +SET(AWS_HTTP_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-http") +SET(AWS_IO_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-io") +SET(AWS_MQTT_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-mqtt") +SET(AWS_S2N_TLS_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-s2n-tls") +SET(AWS_S3_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-s3") +SET(AWS_SDKUTILS_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-sdkutils") + + +# aws-cpp-sdk-core +file(GLOB AWS_SDK_CORE_SRC + "${AWS_SDK_CORE_DIR}/source/*.cpp" + "${AWS_SDK_CORE_DIR}/source/auth/*.cpp" + "${AWS_SDK_CORE_DIR}/source/auth/bearer-token-provider/*.cpp" + "${AWS_SDK_CORE_DIR}/source/auth/signer/*.cpp" + "${AWS_SDK_CORE_DIR}/source/auth/signer-provider/*.cpp" + "${AWS_SDK_CORE_DIR}/source/client/*.cpp" + "${AWS_SDK_CORE_DIR}/source/config/*.cpp" + "${AWS_SDK_CORE_DIR}/source/config/defaults/*.cpp" + "${AWS_SDK_CORE_DIR}/source/endpoint/*.cpp" + "${AWS_SDK_CORE_DIR}/source/endpoint/internal/*.cpp" + "${AWS_SDK_CORE_DIR}/source/external/cjson/*.cpp" + "${AWS_SDK_CORE_DIR}/source/external/tinyxml2/*.cpp" + "${AWS_SDK_CORE_DIR}/source/http/*.cpp" + "${AWS_SDK_CORE_DIR}/source/http/standard/*.cpp" + "${AWS_SDK_CORE_DIR}/source/internal/*.cpp" + "${AWS_SDK_CORE_DIR}/source/monitoring/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/base64/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/crypto/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/crypto/openssl/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/crypto/factory/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/event/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/json/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/logging/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/memory/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/memory/stl/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/stream/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/threading/*.cpp" + "${AWS_SDK_CORE_DIR}/source/utils/xml/*.cpp" +) + +if(OS_LINUX OR OS_DARWIN) + file(GLOB AWS_SDK_CORE_NET_SRC "${AWS_SDK_CORE_DIR}/source/net/linux-shared/*.cpp") + file(GLOB AWS_SDK_CORE_PLATFORM_SRC "${AWS_SDK_CORE_DIR}/source/platform/linux-shared/*.cpp") +else() + file(GLOB AWS_SDK_CORE_NET_SRC "${AWS_SDK_CORE_DIR}/source/net/*.cpp") + set(AWS_SDK_CORE_PLATFORM_SRC) +endif() + +OPTION(USE_AWS_MEMORY_MANAGEMENT "Aws memory management" OFF) +configure_file("${AWS_SDK_CORE_DIR}/include/aws/core/SDKConfig.h.in" + "${CMAKE_CURRENT_BINARY_DIR}/include/aws/core/SDKConfig.h" @ONLY) + +list(APPEND AWS_PUBLIC_COMPILE_DEFS "-DAWS_SDK_VERSION_MAJOR=1") +list(APPEND AWS_PUBLIC_COMPILE_DEFS "-DAWS_SDK_VERSION_MINOR=10") +list(APPEND AWS_PUBLIC_COMPILE_DEFS "-DAWS_SDK_VERSION_PATCH=36") + +list(APPEND AWS_SOURCES ${AWS_SDK_CORE_SRC} ${AWS_SDK_CORE_NET_SRC} ${AWS_SDK_CORE_PLATFORM_SRC}) + +list(APPEND AWS_PUBLIC_INCLUDES + "${AWS_SDK_CORE_DIR}/include/" + "${CMAKE_CURRENT_BINARY_DIR}/include" +) + + +# aws-cpp-sdk-s3 +file(GLOB AWS_SDK_S3_SRC + "${AWS_SDK_S3_DIR}/source/*.cpp" + "${AWS_SDK_S3_DIR}/source/model/*.cpp" +) + +list(APPEND AWS_SOURCES ${AWS_SDK_S3_SRC}) +list(APPEND AWS_PUBLIC_INCLUDES "${AWS_SDK_S3_DIR}/include/") + + +# aws-c-auth +file(GLOB AWS_AUTH_SRC + "${AWS_AUTH_DIR}/source/*.c" +) + +list(APPEND AWS_SOURCES ${AWS_AUTH_SRC}) +list(APPEND AWS_PUBLIC_INCLUDES "${AWS_AUTH_DIR}/include/") + + +# aws-c-cal +file(GLOB AWS_CAL_SRC + "${AWS_CAL_DIR}/source/*.c" +) + +if (ENABLE_OPENSSL_ENCRYPTION) + file(GLOB AWS_CAL_OS_SRC + "${AWS_CAL_DIR}/source/unix/*.c" + ) + list(APPEND AWS_PRIVATE_LIBS OpenSSL::Crypto) +endif() + +list(APPEND AWS_SOURCES ${AWS_CAL_SRC} ${AWS_CAL_OS_SRC}) +list(APPEND AWS_PRIVATE_INCLUDES "${AWS_CAL_DIR}/include/") + + +# aws-c-event-stream +file(GLOB AWS_EVENT_STREAM_SRC + "${AWS_EVENT_STREAM_DIR}/source/*.c" +) + +list(APPEND AWS_SOURCES ${AWS_EVENT_STREAM_SRC}) +list(APPEND AWS_PRIVATE_INCLUDES "${AWS_EVENT_STREAM_DIR}/include/") + + +# aws-c-common +file(GLOB AWS_COMMON_SRC + "${AWS_COMMON_DIR}/source/*.c" + "${AWS_COMMON_DIR}/source/external/*.c" + "${AWS_COMMON_DIR}/source/posix/*.c" +) + +file(GLOB AWS_COMMON_ARCH_SRC + "${AWS_COMMON_DIR}/source/arch/generic/*.c" +) + +if (AWS_ARCH_INTEL) + file(GLOB AWS_COMMON_ARCH_SRC + "${AWS_COMMON_DIR}/source/arch/intel/cpuid.c" + "${AWS_COMMON_DIR}/source/arch/intel/asm/*.c" + ) +elseif (AWS_ARCH_ARM64 OR AWS_ARCH_ARM32) + if (AWS_HAVE_AUXV) + file(GLOB AWS_COMMON_ARCH_SRC + "${AWS_COMMON_DIR}/source/arch/arm/asm/*.c" + ) + endif() +endif() + +set(AWS_COMMON_AVX2_SRC) +if (HAVE_AVX2_INTRINSICS) + list(APPEND AWS_PRIVATE_COMPILE_DEFS "-DUSE_SIMD_ENCODING") + set(AWS_COMMON_AVX2_SRC "${AWS_COMMON_DIR}/source/arch/intel/encoding_avx2.c") + set_source_files_properties(${AWS_COMMON_AVX2_SRC} PROPERTIES COMPILE_FLAGS "${AVX2_CFLAGS}") +endif() + +configure_file("${AWS_COMMON_DIR}/include/aws/common/config.h.in" + "${CMAKE_CURRENT_BINARY_DIR}/include/aws/common/config.h" @ONLY) + +list(APPEND AWS_SOURCES ${AWS_COMMON_SRC} ${AWS_COMMON_ARCH_SRC} ${AWS_COMMON_AVX2_SRC}) + +list(APPEND AWS_PUBLIC_INCLUDES + "${AWS_COMMON_DIR}/include/" + "${CMAKE_CURRENT_BINARY_DIR}/include" +) + + +# aws-checksums +file(GLOB AWS_CHECKSUMS_SRC + "${AWS_CHECKSUMS_DIR}/source/*.c" + "${AWS_CHECKSUMS_DIR}/source/intel/*.c" + "${AWS_CHECKSUMS_DIR}/source/intel/asm/*.c" + "${AWS_CHECKSUMS_DIR}/source/arm/*.c" +) + +if(AWS_ARCH_INTEL AND AWS_HAVE_GCC_INLINE_ASM) + file(GLOB AWS_CHECKSUMS_ARCH_SRC + "${AWS_CHECKSUMS_DIR}/source/intel/asm/*.c" + ) +endif() + +if (AWS_ARCH_ARM64) + file(GLOB AWS_CHECKSUMS_ARCH_SRC + "${AWS_CHECKSUMS_DIR}/source/arm/*.c" + ) + set_source_files_properties("${AWS_CHECKSUMS_DIR}/source/arm/crc32c_arm.c" PROPERTIES COMPILE_FLAGS -march=armv8-a+crc) +elseif (AWS_ARCH_ARM32) + if (AWS_ARM32_CRC) + file(GLOB AWS_CHECKSUMS_ARCH_SRC + "${AWS_CHECKSUMS_DIR}/source/arm/*.c" + "${AWS_CHECKSUMS_DIR}/source/arm/asm/*.c" + ) + set_source_files_properties(source/arm/crc32c_arm.c PROPERTIES COMPILE_FLAGS -march=armv8-a+crc) + endif() +endif() + +list(APPEND AWS_SOURCES ${AWS_CHECKSUMS_SRC} ${AWS_CHECKSUMS_ARCH_SRC}) +list(APPEND AWS_PRIVATE_INCLUDES "${AWS_CHECKSUMS_DIR}/include/") + + +# aws-c-io +file(GLOB AWS_IO_SRC + "${AWS_IO_DIR}/source/*.c" +) + +if (OS_LINUX) + file(GLOB AWS_IO_OS_SRC + "${AWS_IO_DIR}/source/linux/*.c" + "${AWS_IO_DIR}/source/posix/*.c" + ) +elseif (OS_DARWIN) + file(GLOB AWS_IO_OS_SRC + "${AWS_IO_DIR}/source/bsd/*.c" + "${AWS_IO_DIR}/source/posix/*.c" + ) +endif() + +set(AWS_IO_TLS_SRC) +if (USE_S2N) + file(GLOB AWS_IO_TLS_SRC + "${AWS_IO_DIR}/source/s2n/*.c" + ) +endif() + +list(APPEND AWS_SOURCES ${AWS_IO_SRC} ${AWS_IO_OS_SRC} ${AWS_IO_TLS_SRC}) +list(APPEND AWS_PUBLIC_INCLUDES "${AWS_IO_DIR}/include/") + + +# aws-s2n-tls +if (USE_S2N) + file(GLOB AWS_S2N_TLS_SRC + "${AWS_S2N_TLS_DIR}/crypto/*.c" + "${AWS_S2N_TLS_DIR}/error/*.c" + "${AWS_S2N_TLS_DIR}/stuffer/*.c" + "${AWS_S2N_TLS_DIR}/pq-crypto/*.c" + "${AWS_S2N_TLS_DIR}/pq-crypto/kyber_r3/*.c" + "${AWS_S2N_TLS_DIR}/tls/*.c" + "${AWS_S2N_TLS_DIR}/tls/extensions/*.c" + "${AWS_S2N_TLS_DIR}/utils/*.c" + ) + + list(APPEND AWS_SOURCES ${AWS_S2N_TLS_SRC}) + + list(APPEND AWS_PRIVATE_INCLUDES + "${AWS_S2N_TLS_DIR}/" + "${AWS_S2N_TLS_DIR}/api/" + ) +endif() + + +# aws-crt-cpp +file(GLOB AWS_CRT_SRC + "${AWS_CRT_DIR}/source/*.cpp" + "${AWS_CRT_DIR}/source/auth/*.cpp" + "${AWS_CRT_DIR}/source/crypto/*.cpp" + "${AWS_CRT_DIR}/source/endpoints/*.cpp" + "${AWS_CRT_DIR}/source/external/*.cpp" + "${AWS_CRT_DIR}/source/http/*.cpp" + "${AWS_CRT_DIR}/source/io/*.cpp" +) + +list(APPEND AWS_SOURCES ${AWS_CRT_SRC}) +list(APPEND AWS_PUBLIC_INCLUDES "${AWS_CRT_DIR}/include/") + + +# aws-c-mqtt +file(GLOB AWS_MQTT_SRC + "${AWS_MQTT_DIR}/source/*.c" +) + +list(APPEND AWS_SOURCES ${AWS_MQTT_SRC}) +list(APPEND AWS_PUBLIC_INCLUDES "${AWS_MQTT_DIR}/include/") + + +# aws-c-http +file(GLOB AWS_HTTP_SRC + "${AWS_HTTP_DIR}/source/*.c" +) + +list(APPEND AWS_SOURCES ${AWS_HTTP_SRC}) +list(APPEND AWS_PRIVATE_INCLUDES "${AWS_HTTP_DIR}/include/") + + +# aws-c-compression +file(GLOB AWS_COMPRESSION_SRC + "${AWS_COMPRESSION_DIR}/source/*.c" +) + +list(APPEND AWS_SOURCES ${AWS_COMPRESSION_SRC}) +list(APPEND AWS_PRIVATE_INCLUDES "${AWS_COMPRESSION_DIR}/include/") + + +# aws-c-s3 +file(GLOB AWS_S3_SRC + "${AWS_S3_DIR}/source/*.c" +) + +list(APPEND AWS_SOURCES ${AWS_S3_SRC}) +list(APPEND AWS_PRIVATE_INCLUDES "${AWS_S3_DIR}/include/") + + +# aws-c-sdkutils +file(GLOB AWS_SDKUTILS_SRC + "${AWS_SDKUTILS_DIR}/source/*.c" +) + +list(APPEND AWS_SOURCES ${AWS_SDKUTILS_SRC}) +list(APPEND AWS_PUBLIC_INCLUDES "${AWS_SDKUTILS_DIR}/include/") + + +# Add library. +add_library(_aws ${AWS_SOURCES}) + +target_include_directories(_aws SYSTEM BEFORE PUBLIC ${AWS_PUBLIC_INCLUDES}) +target_include_directories(_aws SYSTEM BEFORE PRIVATE ${AWS_PRIVATE_INCLUDES}) +target_compile_definitions(_aws PUBLIC ${AWS_PUBLIC_COMPILE_DEFS}) +target_compile_definitions(_aws PRIVATE ${AWS_PRIVATE_COMPILE_DEFS}) +target_link_libraries(_aws PRIVATE ${AWS_PRIVATE_LIBS}) + +aws_set_thread_affinity_method(_aws) +aws_set_thread_name_method(_aws) + +# The library is large - avoid bloat. +if (OMIT_HEAVY_DEBUG_SYMBOLS) + target_compile_options (_aws PRIVATE -g0) +endif() + +add_library(ch_contrib::aws_s3 ALIAS _aws) diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp new file mode 160000 index 00000000000..0a9e0ad7ab0 --- /dev/null +++ b/contrib/aws-crt-cpp @@ -0,0 +1 @@ +Subproject commit 0a9e0ad7ab07113c65b4846ece3a386407c9c0d3 diff --git a/contrib/aws-s2n-tls b/contrib/aws-s2n-tls new file mode 160000 index 00000000000..15d534e8a9c --- /dev/null +++ b/contrib/aws-s2n-tls @@ -0,0 +1 @@ +Subproject commit 15d534e8a9ca1eda6bacee514e37d08b4f38a526 diff --git a/contrib/aws-s3-cmake/CMakeLists.txt b/contrib/aws-s3-cmake/CMakeLists.txt deleted file mode 100644 index eabed601722..00000000000 --- a/contrib/aws-s3-cmake/CMakeLists.txt +++ /dev/null @@ -1,122 +0,0 @@ -if(NOT OS_FREEBSD) - option(ENABLE_S3 "Enable S3" ${ENABLE_LIBRARIES}) -elseif(ENABLE_S3) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use S3 on FreeBSD") -endif() - -if(NOT ENABLE_S3) - message(STATUS "Not using S3") - return() -endif() - -SET(AWS_S3_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3") -SET(AWS_CORE_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-core") -SET(AWS_CHECKSUMS_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-checksums") -SET(AWS_COMMON_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-common") -SET(AWS_EVENT_STREAM_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-event-stream") - -OPTION(USE_AWS_MEMORY_MANAGEMENT "Aws memory management" OFF) -configure_file("${AWS_CORE_LIBRARY_DIR}/include/aws/core/SDKConfig.h.in" - "${CMAKE_CURRENT_BINARY_DIR}/include/aws/core/SDKConfig.h" @ONLY) - -configure_file("${AWS_COMMON_LIBRARY_DIR}/include/aws/common/config.h.in" - "${CMAKE_CURRENT_BINARY_DIR}/include/aws/common/config.h" @ONLY) - - -file(GLOB AWS_CORE_SOURCES - "${AWS_CORE_LIBRARY_DIR}/source/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/auth/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/client/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/http/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/http/standard/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/config/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/external/cjson/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/external/tinyxml2/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/internal/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/monitoring/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/net/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/linux-shared/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/platform/linux-shared/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/base64/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/event/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/crypto/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/crypto/openssl/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/crypto/factory/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/json/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/logging/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/memory/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/memory/stl/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/stream/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/threading/*.cpp" - "${AWS_CORE_LIBRARY_DIR}/source/utils/xml/*.cpp" -) - -file(GLOB AWS_S3_SOURCES - "${AWS_S3_LIBRARY_DIR}/source/*.cpp" -) - -file(GLOB AWS_S3_MODEL_SOURCES - "${AWS_S3_LIBRARY_DIR}/source/model/*.cpp" -) - -file(GLOB AWS_EVENT_STREAM_SOURCES - "${AWS_EVENT_STREAM_LIBRARY_DIR}/source/*.c" -) - -file(GLOB AWS_COMMON_SOURCES - "${AWS_COMMON_LIBRARY_DIR}/source/*.c" - "${AWS_COMMON_LIBRARY_DIR}/source/posix/*.c" -) - -file(GLOB AWS_CHECKSUMS_SOURCES - "${AWS_CHECKSUMS_LIBRARY_DIR}/source/*.c" - "${AWS_CHECKSUMS_LIBRARY_DIR}/source/intel/*.c" - "${AWS_CHECKSUMS_LIBRARY_DIR}/source/arm/*.c" -) - -file(GLOB S3_UNIFIED_SRC - ${AWS_EVENT_STREAM_SOURCES} - ${AWS_COMMON_SOURCES} - ${AWS_S3_SOURCES} - ${AWS_S3_MODEL_SOURCES} - ${AWS_CORE_SOURCES} -) - -set(S3_INCLUDES - "${AWS_COMMON_LIBRARY_DIR}/include/" - "${AWS_EVENT_STREAM_LIBRARY_DIR}/include/" - "${AWS_S3_LIBRARY_DIR}/include/" - "${AWS_CORE_LIBRARY_DIR}/include/" - "${CMAKE_CURRENT_BINARY_DIR}/include/" -) - -add_library(_aws_s3_checksums ${AWS_CHECKSUMS_SOURCES}) -target_include_directories(_aws_s3_checksums SYSTEM PUBLIC "${AWS_CHECKSUMS_LIBRARY_DIR}/include/") -if(CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") - target_compile_definitions(_aws_s3_checksums PRIVATE "-DDEBUG_BUILD") -endif() -set_target_properties(_aws_s3_checksums PROPERTIES LINKER_LANGUAGE C) -set_property(TARGET _aws_s3_checksums PROPERTY C_STANDARD 99) - -add_library(_aws_s3 ${S3_UNIFIED_SRC}) - -target_compile_definitions(_aws_s3 PUBLIC "AWS_SDK_VERSION_MAJOR=1") -target_compile_definitions(_aws_s3 PUBLIC "AWS_SDK_VERSION_MINOR=7") -target_compile_definitions(_aws_s3 PUBLIC "AWS_SDK_VERSION_PATCH=231") -target_include_directories(_aws_s3 SYSTEM BEFORE PUBLIC ${S3_INCLUDES}) - -if (TARGET OpenSSL::SSL) - target_compile_definitions(_aws_s3 PUBLIC -DENABLE_OPENSSL_ENCRYPTION) - target_link_libraries(_aws_s3 PRIVATE OpenSSL::Crypto OpenSSL::SSL) -endif() - -target_link_libraries(_aws_s3 PRIVATE _aws_s3_checksums) - -# The library is large - avoid bloat. -if (OMIT_HEAVY_DEBUG_SYMBOLS) - target_compile_options (_aws_s3 PRIVATE -g0) - target_compile_options (_aws_s3_checksums PRIVATE -g0) -endif() - -add_library(ch_contrib::aws_s3 ALIAS _aws_s3) From 665813e1270f1195828218cf1f8b0055b1502d92 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 23 Dec 2022 13:08:01 +0100 Subject: [PATCH 15/78] Remove subsubmodules contrib/aws/crt and contrib/aws-crt-cpp/crt because they were moved to the contrib folder. --- .gitmodules | 2 +- contrib/aws | 2 +- contrib/aws-crt-cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index d35741362b0..fe305128232 100644 --- a/.gitmodules +++ b/.gitmodules @@ -296,7 +296,7 @@ url = https://github.com/ridiculousfish/libdivide.git [submodule "contrib/aws-crt-cpp"] path = contrib/aws-crt-cpp - url = https://github.com/awslabs/aws-crt-cpp.git + url = https://github.com/ClickHouse/aws-crt-cpp.git [submodule "contrib/aws-c-io"] path = contrib/aws-c-io url = https://github.com/awslabs/aws-c-io.git diff --git a/contrib/aws b/contrib/aws index 2b711c84575..94a16294727 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 2b711c845751a93695ad86433de8bc24ecc4746a +Subproject commit 94a1629472778e3b5c5c08866eaf95ea77cbc336 diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp index 0a9e0ad7ab0..3c053a549f3 160000 --- a/contrib/aws-crt-cpp +++ b/contrib/aws-crt-cpp @@ -1 +1 @@ -Subproject commit 0a9e0ad7ab07113c65b4846ece3a386407c9c0d3 +Subproject commit 3c053a549f38b5d36c502079339882f22fe08beb From 8efb1ae7bbc487c5d0b0263fcb86fe887b67bf10 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 23 Dec 2022 12:51:40 +0100 Subject: [PATCH 16/78] Fix support for boringssl. --- .gitmodules | 2 +- contrib/aws-c-cal | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index fe305128232..01290970962 100644 --- a/.gitmodules +++ b/.gitmodules @@ -308,7 +308,7 @@ url = https://github.com/awslabs/aws-c-auth.git [submodule "contrib/aws-c-cal"] path = contrib/aws-c-cal - url = https://github.com/awslabs/aws-c-cal.git + url = https://github.com/ClickHouse/aws-c-cal.git [submodule "contrib/aws-c-sdkutils"] path = contrib/aws-c-sdkutils url = https://github.com/awslabs/aws-c-sdkutils.git diff --git a/contrib/aws-c-cal b/contrib/aws-c-cal index ac4216b78d5..26c2f862024 160000 --- a/contrib/aws-c-cal +++ b/contrib/aws-c-cal @@ -1 +1 @@ -Subproject commit ac4216b78d5323b5b8ce95a3dd4a8fc0f95e2d33 +Subproject commit 26c2f8620246d04ab8a84ac5455586aa5debbee4 From f8f598e298d5b1fbf35d229191c57d60b6399718 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 23 Dec 2022 12:51:51 +0100 Subject: [PATCH 17/78] Correctly handle that dlopen(NULL) returns NULL in ClickHouse. --- contrib/aws-c-cal | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws-c-cal b/contrib/aws-c-cal index 26c2f862024..85dd7664b78 160000 --- a/contrib/aws-c-cal +++ b/contrib/aws-c-cal @@ -1 +1 @@ -Subproject commit 26c2f8620246d04ab8a84ac5455586aa5debbee4 +Subproject commit 85dd7664b786a389c6fb1a6f031ab4bb2282133d From 1eb0d5db160569c2066a847dc8d6c130759a8f04 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 22 Dec 2022 02:49:08 +0100 Subject: [PATCH 18/78] Disable loading EC2-metadata. --- contrib/aws | 2 +- src/IO/S3Common.cpp | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/contrib/aws b/contrib/aws index 94a16294727..e15b0695c5b 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 94a1629472778e3b5c5c08866eaf95ea77cbc336 +Subproject commit e15b0695c5b3c0f8dc15c89b5e7eb73b2747fdec diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index c03f7f07310..45714ddfeba 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -677,6 +677,12 @@ namespace S3 ClientFactory::ClientFactory() { aws_options = Aws::SDKOptions{}; + + /// We don't load EC2 metadata. + /// This is needed to allow to subclass `ClientConfiguration` so that any possible SDK client will use extended configuration + /// without circular dependencies `Client` -> `ClientConfiguration` -> `Client`. + setenv("AWS_EC2_METADATA_DISABLED", "true", 1/*override*/); // NOLINT(concurrency-mt-unsafe): this is safe if not called concurrently + Aws::InitAPI(aws_options); Aws::Utils::Logging::InitializeAWSLogging(std::make_shared(false)); Aws::Http::SetHttpClientFactory(std::make_shared()); From e23c42d042c335df622aa32533b69be4eedb6c3e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 22 Dec 2022 02:50:38 +0100 Subject: [PATCH 19/78] Fix checking for embedded error in PocoHTTPClient. --- contrib/aws | 2 +- src/IO/S3/PocoHTTPClient.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/aws b/contrib/aws index e15b0695c5b..6b469eeff97 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit e15b0695c5b3c0f8dc15c89b5e7eb73b2747fdec +Subproject commit 6b469eeff9782698bb9a2607b9668d905922d7a7 diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 33dd3250c9f..6153842520b 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -177,7 +177,7 @@ namespace bool checkRequestCanReturn2xxAndErrorInBody(Aws::Http::HttpRequest & request) { auto query_params = request.GetQueryStringParameters(); - if (request.HasHeader("z-amz-copy-source")) + if (request.HasHeader("x-amz-copy-source")) { /// CopyObject https://docs.aws.amazon.com/AmazonS3/latest/API/API_CopyObject.html if (query_params.empty()) From fea3a6280ca7a7c7ed620d7712330ad994daf732 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 23 Dec 2022 12:26:10 +0100 Subject: [PATCH 20/78] Fix race condition in cJSON_Parse.* calls. --- .gitmodules | 2 +- contrib/aws-c-common | 2 +- contrib/aws-crt-cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index 01290970962..cd2a60bfe0a 100644 --- a/.gitmodules +++ b/.gitmodules @@ -107,7 +107,7 @@ url = https://github.com/awslabs/aws-c-event-stream.git [submodule "aws-c-common"] path = contrib/aws-c-common - url = https://github.com/awslabs/aws-c-common.git + url = https://github.com/ClickHouse/aws-c-common.git [submodule "aws-checksums"] path = contrib/aws-checksums url = https://github.com/awslabs/aws-checksums.git diff --git a/contrib/aws-c-common b/contrib/aws-c-common index d6a6a6057af..324fd1d973c 160000 --- a/contrib/aws-c-common +++ b/contrib/aws-c-common @@ -1 +1 @@ -Subproject commit d6a6a6057afd8024cf2790a50de4a9818014cec6 +Subproject commit 324fd1d973ccb25c813aa747bf1759cfde5121c5 diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp index 3c053a549f3..ec0bea288f4 160000 --- a/contrib/aws-crt-cpp +++ b/contrib/aws-crt-cpp @@ -1 +1 @@ -Subproject commit 3c053a549f38b5d36c502079339882f22fe08beb +Subproject commit ec0bea288f451d884c0d80d534bc5c66241c39a4 From dffeb2a54ae23e676d049baa27d3bf06d2d04c0b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 23 Dec 2022 10:48:47 +0100 Subject: [PATCH 21/78] Disable memory sanitizer for events from epoll --- .gitmodules | 2 +- contrib/aws-c-io | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index cd2a60bfe0a..406e8a7e11e 100644 --- a/.gitmodules +++ b/.gitmodules @@ -299,7 +299,7 @@ url = https://github.com/ClickHouse/aws-crt-cpp.git [submodule "contrib/aws-c-io"] path = contrib/aws-c-io - url = https://github.com/awslabs/aws-c-io.git + url = https://github.com/ClickHouse/aws-c-io.git [submodule "contrib/aws-c-mqtt"] path = contrib/aws-c-mqtt url = https://github.com/awslabs/aws-c-mqtt.git diff --git a/contrib/aws-c-io b/contrib/aws-c-io index d353ffde788..5d32c453560 160000 --- a/contrib/aws-c-io +++ b/contrib/aws-c-io @@ -1 +1 @@ -Subproject commit d353ffde788b70f39e7da889784262e4d4eb5101 +Subproject commit 5d32c453560d0823df521a686bf7fbacde7f9be3 From c7e1aa2ea0c32ecdf46a4c4a2295b6ca2eeb4530 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 22 Dec 2022 02:52:24 +0100 Subject: [PATCH 22/78] Fix finding pthread_atfork() while linking ppc64le build. --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index 0f41651860f..f0081b2649b 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 0f41651860fa4a530ecd68b93a15b8fd77397adf +Subproject commit f0081b2649b94837855f3bc7d05ef326b100bad8 From 3f03fca469cc8e2a450cea0cb4df6bbfa590557c Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 27 Dec 2022 13:59:00 -0500 Subject: [PATCH 23/78] add note for Cloud users --- .../mergetree-family/replication.md | 41 +++++++++++++++---- 1 file changed, 33 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 4867140789f..76a92f2b722 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -6,6 +6,17 @@ sidebar_label: Data Replication # Data Replication +:::note +In ClickHouse Cloud replication is managed for you. Please create your tables without adding arguments. For example, in the text below you would replace: +``` +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{shard}/table_name', '{replica}', ver) +``` +with: +``` +ENGINE = ReplicatedReplacingMergeTree +``` +::: + Replication is only supported for tables in the MergeTree family: - ReplicatedMergeTree @@ -85,15 +96,15 @@ Example of setting the addresses of the auxiliary ZooKeeper cluster: ``` -To store table metadata in an auxiliary ZooKeeper cluster instead of default ZooKeeper cluster, we can use the SQL to create table with -ReplicatedMergeTree engine as follow: +To store table metadata in an auxiliary ZooKeeper cluster instead of the default ZooKeeper cluster, we can use SQL to create the table with +ReplicatedMergeTree engine as follows: ``` CREATE TABLE table_name ( ... ) ENGINE = ReplicatedMergeTree('zookeeper_name_configured_in_auxiliary_zookeepers:path', 'replica_name') ... ``` You can specify any existing ZooKeeper cluster and the system will use a directory on it for its own data (the directory is specified when creating a replicatable table). -If ZooKeeper isn’t set in the config file, you can’t create replicated tables, and any existing replicated tables will be read-only. +If ZooKeeper is not set in the config file, you can’t create replicated tables, and any existing replicated tables will be read-only. ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](/docs/en/operations/settings/settings.md/#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](/docs/en/operations/settings/settings.md/#settings-fallback_to_stale_replicas_for_distributed_queries). @@ -119,8 +130,23 @@ The system monitors data synchronicity on replicas and is able to recover after ## Creating Replicated Tables {#creating-replicated-tables} +:::note +In ClickHouse Cloud replication is managed for you. Please create your tables without adding arguments. For example, in the text below you would replace: +``` +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{shard}/table_name', '{replica}', ver) +``` +with: +``` +ENGINE = ReplicatedReplacingMergeTree +``` +::: + The `Replicated` prefix is added to the table engine name. For example:`ReplicatedMergeTree`. +:::tip +Adding `Replicated` is optional in ClickHouse Cloud, as all of the tables are replicated. +::: + ### Replicated\*MergeTree parameters #### zoo_path @@ -144,7 +170,7 @@ CREATE TABLE table_name CounterID UInt32, UserID UInt32, ver UInt16 -) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}', ver) +) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{shard}/table_name', '{replica}', ver) PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID); @@ -160,7 +186,7 @@ CREATE TABLE table_name EventDate DateTime, CounterID UInt32, UserID UInt32 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192); +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/table_name', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192); ``` @@ -171,7 +197,6 @@ Example: ``` xml - 05 02 example05-02-1 @@ -182,12 +207,12 @@ In this case, the path consists of the following parts: `/clickhouse/tables/` is the common prefix. We recommend using exactly this one. -`{layer}-{shard}` is the shard identifier. In this example it consists of two parts, since the example cluster uses bi-level sharding. For most tasks, you can leave just the {shard} substitution, which will be expanded to the shard identifier. +`{shard}` will be expanded to the shard identifier. `table_name` is the name of the node for the table in ClickHouse Keeper. It is a good idea to make it the same as the table name. It is defined explicitly, because in contrast to the table name, it does not change after a RENAME query. *HINT*: you could add a database name in front of `table_name` as well. E.g. `db_name.table_name` -The two built-in substitutions `{database}` and `{table}` can be used, they expand into the table name and the database name respectively (unless these macros are defined in the `macros` section). So the zookeeper path can be specified as `'/clickhouse/tables/{layer}-{shard}/{database}/{table}'`. +The two built-in substitutions `{database}` and `{table}` can be used, they expand into the table name and the database name respectively (unless these macros are defined in the `macros` section). So the zookeeper path can be specified as `'/clickhouse/tables/{shard}/{database}/{table}'`. Be careful with table renames when using these built-in substitutions. The path in ClickHouse Keeper cannot be changed, and when the table is renamed, the macros will expand into a different path, the table will refer to a path that does not exist in ClickHouse Keeper, and will go into read-only mode. The replica name identifies different replicas of the same table. You can use the server name for this, as in the example. The name only needs to be unique within each shard. From 2891d14da1f894bbef4641bff65acc6ce2185e20 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 27 Dec 2022 19:34:09 +0000 Subject: [PATCH 24/78] Fix tests --- .../0_stateless/01543_avro_deserialization_with_lc.sh | 2 +- tests/queries/0_stateless/01825_type_json_multiple_files.sh | 6 +++--- tests/queries/0_stateless/01825_type_json_order_by.sql | 1 + .../queries/0_stateless/02166_arrow_dictionary_inference.sh | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh index df43fd273e8..6f48456f71b 100755 --- a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh +++ b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh @@ -23,6 +23,6 @@ $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_01543" $CLICKHOUSE_CLIENT --query "SELECT number % 2 ? number: NULL as x from numbers(10) FORMAT Avro" > $USER_FILES_PATH/test_01543.avro -$CLICKHOUSE_CLIENT --query "SELECT * FROM file('test_01543.avro', 'Avro', 'x LowCardinality(Nullable(UInt64))')" +$CLICKHOUSE_CLIENT --query "SELECT * FROM file('test_01543.avro', 'Avro', 'x LowCardinality(Nullable(UInt64))')" --allow_suspicious_low_cardinality_types 1 rm $USER_FILES_PATH/test_01543.avro diff --git a/tests/queries/0_stateless/01825_type_json_multiple_files.sh b/tests/queries/0_stateless/01825_type_json_multiple_files.sh index 9120568bb17..91e7ffb5edf 100755 --- a/tests/queries/0_stateless/01825_type_json_multiple_files.sh +++ b/tests/queries/0_stateless/01825_type_json_multiple_files.sh @@ -17,7 +17,7 @@ done ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_files" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_files (file String, data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON')" +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON')" --allow_experimental_object_type 1 ${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1" @@ -26,7 +26,7 @@ ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE IF EXISTS t_json_files" ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \ SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON') \ - ORDER BY _file LIMIT 3" --max_threads 1 --min_insert_block_size_rows 1 --max_insert_block_size 1 --max_block_size 1 + ORDER BY _file LIMIT 3" --max_threads 1 --min_insert_block_size_rows 1 --max_insert_block_size 1 --max_block_size 1 --allow_experimental_object_type 1 ${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1" @@ -35,7 +35,7 @@ ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE IF EXISTS t_json_files" ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \ SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON') \ - WHERE _file IN ('01825_file_1.json', '01825_file_3.json')" + WHERE _file IN ('01825_file_1.json', '01825_file_3.json')" --allow_experimental_object_type 1 ${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1" diff --git a/tests/queries/0_stateless/01825_type_json_order_by.sql b/tests/queries/0_stateless/01825_type_json_order_by.sql index 1d5768fe0b4..98b2a6105d9 100644 --- a/tests/queries/0_stateless/01825_type_json_order_by.sql +++ b/tests/queries/0_stateless/01825_type_json_order_by.sql @@ -1,5 +1,6 @@ -- Tags: no-fasttest +SET allow_experimental_object_type = 1; SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON); SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON), dummy; SELECT materialize('{"k":"v"}'::JSON) SETTINGS extremes = 1; diff --git a/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh b/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh index 7d313b571d9..04888f3a1f3 100755 --- a/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh +++ b/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "insert into table function file('arrow.dict', 'Arrow', 'x LowCardinality(UInt64)') select number from numbers(10) settings output_format_arrow_low_cardinality_as_dictionary=1, engine_file_truncate_on_insert=1" +$CLICKHOUSE_CLIENT -q "insert into table function file('arrow.dict', 'Arrow', 'x LowCardinality(UInt64)') select number from numbers(10) settings output_format_arrow_low_cardinality_as_dictionary=1, engine_file_truncate_on_insert=1, allow_suspicious_low_cardinality_types=1" $CLICKHOUSE_CLIENT -q "desc file('arrow.dict', 'Arrow')" From 5d1886994f3d919eb34845709112c925428dfa49 Mon Sep 17 00:00:00 2001 From: Bo Lu Date: Wed, 28 Dec 2022 15:45:05 +1100 Subject: [PATCH 25/78] fix table ttl doc example --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- docs/zh/engines/table-engines/mergetree-family/mergetree.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 7614a09c018..061b3996da6 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -621,7 +621,7 @@ CREATE TABLE example_table ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY d -TTL d + INTERVAL 1 MONTH [DELETE], +TTL d + INTERVAL 1 MONTH DELETE, d + INTERVAL 1 WEEK TO VOLUME 'aaa', d + INTERVAL 2 WEEK TO DISK 'bbb'; ``` diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index f024d5f1985..8740bdd76ec 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -523,7 +523,7 @@ CREATE TABLE example_table ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY d -TTL d + INTERVAL 1 MONTH [DELETE], +TTL d + INTERVAL 1 MONTH DELETE, d + INTERVAL 1 WEEK TO VOLUME 'aaa', d + INTERVAL 2 WEEK TO DISK 'bbb'; ``` diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md index 45a27e16b5b..aed9d678a83 100644 --- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md @@ -479,7 +479,7 @@ CREATE TABLE example_table ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY d -TTL d + INTERVAL 1 MONTH [DELETE], +TTL d + INTERVAL 1 MONTH DELETE, d + INTERVAL 1 WEEK TO VOLUME 'aaa', d + INTERVAL 2 WEEK TO DISK 'bbb'; ``` From 9014be89681c4509485e7e809fdda27ccef817d6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 29 Dec 2022 12:25:33 +0100 Subject: [PATCH 26/78] Fix disable loading EC2 metadata by aws-sdk-core. --- contrib/aws | 2 +- src/IO/S3Common.cpp | 6 ------ 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/contrib/aws b/contrib/aws index 6b469eeff97..4a12641211d 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 6b469eeff9782698bb9a2607b9668d905922d7a7 +Subproject commit 4a12641211d4dbc8e2fdb2dd0f1eea0927db9252 diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 45714ddfeba..c03f7f07310 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -677,12 +677,6 @@ namespace S3 ClientFactory::ClientFactory() { aws_options = Aws::SDKOptions{}; - - /// We don't load EC2 metadata. - /// This is needed to allow to subclass `ClientConfiguration` so that any possible SDK client will use extended configuration - /// without circular dependencies `Client` -> `ClientConfiguration` -> `Client`. - setenv("AWS_EC2_METADATA_DISABLED", "true", 1/*override*/); // NOLINT(concurrency-mt-unsafe): this is safe if not called concurrently - Aws::InitAPI(aws_options); Aws::Utils::Logging::InitializeAWSLogging(std::make_shared(false)); Aws::Http::SetHttpClientFactory(std::make_shared()); From 8cc3b2e26c2d8c7fe04b5b0401a9b49e4e84e262 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 29 Dec 2022 16:10:06 +0100 Subject: [PATCH 27/78] Fixed integration flaky test issue by specifying hdfs client in each test- Allow macro substitution in endpoint --- tests/integration/test_endpoint_macro_substitution/test.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration/test_endpoint_macro_substitution/test.py b/tests/integration/test_endpoint_macro_substitution/test.py index ba880a5095d..6ef58ba3da2 100644 --- a/tests/integration/test_endpoint_macro_substitution/test.py +++ b/tests/integration/test_endpoint_macro_substitution/test.py @@ -33,6 +33,8 @@ def cluster(): def test_different_types(cluster): node = cluster.instances["node"] + fs = HdfsClient(hosts=cluster.hdfs_ip) + response = TSV.toMat(node.query("SELECT * FROM system.disks FORMAT TSVWithNames")) assert len(response) > len(disk_types) # at least one extra line for header @@ -58,6 +60,8 @@ def test_different_types(cluster): def test_select_by_type(cluster): node = cluster.instances["node"] + fs = HdfsClient(hosts=cluster.hdfs_ip) + for name, disk_type in list(disk_types.items()): if disk_type != "s3": assert ( From c09d1bc758d428663e7ce7b16d3d8de05aada7d6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 29 Dec 2022 15:16:50 +0000 Subject: [PATCH 28/78] Automatic style fix --- tests/integration/test_endpoint_macro_substitution/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_endpoint_macro_substitution/test.py b/tests/integration/test_endpoint_macro_substitution/test.py index 6ef58ba3da2..42a8ddbda84 100644 --- a/tests/integration/test_endpoint_macro_substitution/test.py +++ b/tests/integration/test_endpoint_macro_substitution/test.py @@ -61,7 +61,7 @@ def test_different_types(cluster): def test_select_by_type(cluster): node = cluster.instances["node"] fs = HdfsClient(hosts=cluster.hdfs_ip) - + for name, disk_type in list(disk_types.items()): if disk_type != "s3": assert ( From 91dea53dbee958f673ad269befcff0653d0451bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 30 Dec 2022 12:03:43 +0100 Subject: [PATCH 29/78] Perf test: Log the time spent waiting for file sync --- docker/test/performance-comparison/perf.py | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index cb23372d31f..76e4c5ad8c1 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -297,6 +297,7 @@ if not args.use_existing_tables: # Let's sync the data to avoid writeback affects performance os.system("sync") +reportStageEnd("sync") # By default, test all queries. queries_to_run = range(0, len(test_queries)) From 4753ac937fc6cf0896165a1b32e6a5db3b0750a4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 30 Dec 2022 20:20:24 +0100 Subject: [PATCH 30/78] Enabling override of endpoint used to connect for EC2 Metadata Service. This commit applies changes https://github.com/aws/aws-sdk-cpp/commit/742b371 and https://github.com/aws/aws-sdk-cpp/commit/34e73e9 to our fork of Aws::Internal::EC2MetadataClient. --- src/IO/S3Common.cpp | 49 +++++++++++++++++++++++++++++++++++++++------ 1 file changed, 43 insertions(+), 6 deletions(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index c03f7f07310..a1a4267496f 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -149,13 +149,12 @@ class AWSEC2MetadataClient : public Aws::Internal::AWSHttpResourceClient static constexpr char EC2_IMDS_TOKEN_TTL_DEFAULT_VALUE[] = "21600"; static constexpr char EC2_IMDS_TOKEN_TTL_HEADER[] = "x-aws-ec2-metadata-token-ttl-seconds"; - static constexpr char EC2_DEFAULT_METADATA_ENDPOINT[] = "http://169.254.169.254"; - public: /// See EC2MetadataClient. - explicit AWSEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration) + explicit AWSEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration, const char * endpoint_) : Aws::Internal::AWSHttpResourceClient(client_configuration) + , endpoint(endpoint_) , logger(&Poco::Logger::get("AWSEC2InstanceProfileConfigLoader")) { } @@ -180,7 +179,7 @@ public: { std::lock_guard locker(token_mutex); - LOG_TRACE(logger, "Getting default credentials for EC2 instance."); + LOG_TRACE(logger, "Getting default credentials for ec2 instance from {}", endpoint); auto result = GetResourceWithAWSWebServiceResult(endpoint.c_str(), EC2_SECURITY_CREDENTIALS_RESOURCE, nullptr); credentials_string = result.GetPayload(); if (result.GetResponseCode() == Aws::Http::HttpResponseCode::UNAUTHORIZED) @@ -286,12 +285,50 @@ public: } private: - const Aws::String endpoint = EC2_DEFAULT_METADATA_ENDPOINT; + const Aws::String endpoint; mutable std::recursive_mutex token_mutex; mutable Aws::String token; Poco::Logger * logger; }; +std::shared_ptr InitEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration) +{ + Aws::String ec2_metadata_service_endpoint = Aws::Environment::GetEnv("AWS_EC2_METADATA_SERVICE_ENDPOINT"); + auto * logger = &Poco::Logger::get("AWSEC2InstanceProfileConfigLoader"); + if (ec2_metadata_service_endpoint.empty()) + { + Aws::String ec2_metadata_service_endpoint_mode = Aws::Environment::GetEnv("AWS_EC2_METADATA_SERVICE_ENDPOINT_MODE"); + if (ec2_metadata_service_endpoint_mode.length() == 0) + { + ec2_metadata_service_endpoint = "http://169.254.169.254"; //default to IPv4 default endpoint + } + else + { + if (ec2_metadata_service_endpoint_mode.length() == 4) + { + if (Aws::Utils::StringUtils::CaselessCompare(ec2_metadata_service_endpoint_mode.c_str(), "ipv4")) + { + ec2_metadata_service_endpoint = "http://169.254.169.254"; //default to IPv4 default endpoint + } + else if (Aws::Utils::StringUtils::CaselessCompare(ec2_metadata_service_endpoint_mode.c_str(), "ipv6")) + { + ec2_metadata_service_endpoint = "http://[fd00:ec2::254]"; + } + else + { + LOG_ERROR(logger, "AWS_EC2_METADATA_SERVICE_ENDPOINT_MODE can only be set to ipv4 or ipv6, received: {}", ec2_metadata_service_endpoint_mode); + } + } + else + { + LOG_ERROR(logger, "AWS_EC2_METADATA_SERVICE_ENDPOINT_MODE can only be set to ipv4 or ipv6, received: {}", ec2_metadata_service_endpoint_mode); + } + } + } + LOG_INFO(logger, "Using IMDS endpoint: {}", ec2_metadata_service_endpoint); + return std::make_shared(client_configuration, ec2_metadata_service_endpoint.c_str()); +} + class AWSEC2InstanceProfileConfigLoader : public Aws::Config::AWSProfileConfigLoader { public: @@ -646,7 +683,7 @@ public: aws_client_configuration.retryStrategy = std::make_shared(1, 1000); - auto ec2_metadata_client = std::make_shared(aws_client_configuration); + auto ec2_metadata_client = InitEC2MetadataClient(aws_client_configuration); auto config_loader = std::make_shared(ec2_metadata_client, !use_insecure_imds_request); AddProvider(std::make_shared(config_loader)); From 426e2a9da108678fc24b2f265c2a099448f093d3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 31 Dec 2022 02:21:14 +0100 Subject: [PATCH 31/78] Add test for getting credentials from AWS EC2 metadata. --- .../test_s3_ec2_metadata/__init__.py | 0 .../configs/use_environment_credentials.xml | 5 + .../request_response_server.py | 36 +++++++ .../integration/test_s3_ec2_metadata/test.py | 94 +++++++++++++++++++ 4 files changed, 135 insertions(+) create mode 100644 tests/integration/test_s3_ec2_metadata/__init__.py create mode 100644 tests/integration/test_s3_ec2_metadata/configs/use_environment_credentials.xml create mode 100644 tests/integration/test_s3_ec2_metadata/ec2_metadata_server/request_response_server.py create mode 100644 tests/integration/test_s3_ec2_metadata/test.py diff --git a/tests/integration/test_s3_ec2_metadata/__init__.py b/tests/integration/test_s3_ec2_metadata/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_s3_ec2_metadata/configs/use_environment_credentials.xml b/tests/integration/test_s3_ec2_metadata/configs/use_environment_credentials.xml new file mode 100644 index 00000000000..db901ec0823 --- /dev/null +++ b/tests/integration/test_s3_ec2_metadata/configs/use_environment_credentials.xml @@ -0,0 +1,5 @@ + + + 1 + + diff --git a/tests/integration/test_s3_ec2_metadata/ec2_metadata_server/request_response_server.py b/tests/integration/test_s3_ec2_metadata/ec2_metadata_server/request_response_server.py new file mode 100644 index 00000000000..f347866be58 --- /dev/null +++ b/tests/integration/test_s3_ec2_metadata/ec2_metadata_server/request_response_server.py @@ -0,0 +1,36 @@ +import http.server +import sys + + +class RequestHandler(http.server.BaseHTTPRequestHandler): + def get_response(self): + if self.path == "/": + return "OK" + elif self.path == "/latest/meta-data/iam/security-credentials": + return "myrole" + elif self.path == "/latest/meta-data/iam/security-credentials/myrole": + return '{ "Code" : "Success", "Type" : "AWS-HMAC", "AccessKeyId" : "minio", "SecretAccessKey" : "minio123" }' + else: + return None + + def do_HEAD(self): + response = self.get_response() + if response: + self.send_response(200) + self.send_header("Content-Type", "text/plain") + self.send_header("Content-Length", len(response.encode())) + self.end_headers() + else: + self.send_response(404) + self.send_header("Content-Type", "text/plain") + self.end_headers() + + def do_GET(self): + self.do_HEAD() + response = self.get_response() + if response: + self.wfile.write(response.encode()) + + +httpd = http.server.HTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) +httpd.serve_forever() diff --git a/tests/integration/test_s3_ec2_metadata/test.py b/tests/integration/test_s3_ec2_metadata/test.py new file mode 100644 index 00000000000..982656df009 --- /dev/null +++ b/tests/integration/test_s3_ec2_metadata/test.py @@ -0,0 +1,94 @@ +import pytest +from helpers.cluster import ClickHouseCluster +import logging +import os +import time + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +EC2_METADATA_SERVER_HOSTNAME = "resolver" +EC2_METADATA_SERVER_PORT = 8080 + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + with_minio=True, + main_configs=["configs/use_environment_credentials.xml"], + env_variables={ + "AWS_EC2_METADATA_SERVICE_ENDPOINT": f"{EC2_METADATA_SERVER_HOSTNAME}:{EC2_METADATA_SERVER_PORT}", + }, +) + + +def start_ec2_metadata_server(): + logging.info("Starting EC2 metadata server") + container_id = cluster.get_container_id("resolver") + + cluster.copy_file_to_container( + container_id, + os.path.join(SCRIPT_DIR, "ec2_metadata_server/request_response_server.py"), + "request_response_server.py", + ) + + cluster.exec_in_container( + container_id, + ["python", "request_response_server.py", str(EC2_METADATA_SERVER_PORT)], + detach=True, + ) + + # Wait for the server to start. + num_attempts = 100 + for attempt in range(num_attempts): + ping_response = cluster.exec_in_container( + container_id, + ["curl", "-s", f"http://localhost:{EC2_METADATA_SERVER_PORT}/"], + nothrow=True, + ) + if ping_response != "OK": + if attempt == num_attempts - 1: + assert ping_response == "OK", 'Expected "OK", but got "{}"'.format( + ping_response + ) + else: + time.sleep(1) + else: + logging.debug( + f"request_response_server.py answered {ping_response} on attempt {attempt}" + ) + break + + logging.info("EC2 metadata server started") + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + start_ec2_metadata_server() + yield + finally: + cluster.shutdown() + + +def test_credentials_from_ec2_metadata(): + node.query( + f"INSERT INTO FUNCTION s3('http://{cluster.minio_host}:{cluster.minio_port}/{cluster.minio_bucket}/test1.jsonl') SELECT * FROM numbers(100)" + ) + + assert ( + "100" + == node.query( + f"SELECT count() FROM s3('http://{cluster.minio_host}:{cluster.minio_port}/{cluster.minio_bucket}/test1.jsonl')" + ).strip() + ) + + expected_logs = [ + "Getting default credentials for ec2 instance from resolver:8080", + "Calling EC2MetadataService resource, /latest/meta-data/iam/security-credentials returned credential string myrole", + "Calling EC2MetadataService resource /latest/meta-data/iam/security-credentials/myrole", + "Successfully pulled credentials from EC2MetadataService with access key", + ] + + for expected_msg in expected_logs: + node.contains_in_log("AWSEC2InstanceProfileConfigLoader: " + expected_msg) From 6e946cc75779980b65804c70898beaee3d2e621e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 1 Jan 2023 23:22:26 +0100 Subject: [PATCH 32/78] Less OOM in stress test --- docker/test/stress/run.sh | 19 +++++++++++++----- programs/server/Server.cpp | 40 ++++++++++++++++++++++++++++++++++++++ programs/server/config.xml | 4 ++++ src/Daemon/BaseDaemon.cpp | 29 --------------------------- 4 files changed, 58 insertions(+), 34 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 41245013a4a..dc2b184dc5a 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -53,6 +53,7 @@ function configure() local total_mem total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB total_mem=$(( total_mem*1024 )) # bytes + # Set maximum memory usage as half of total memory (less chance of OOM). # # But not via max_server_memory_usage but via max_memory_usage_for_user, @@ -65,16 +66,17 @@ function configure() # max_server_memory_usage will be hard limit, and queries that should be # executed regardless memory limits will use max_memory_usage_for_user=0, # instead of relying on max_untracked_memory - local max_server_mem - max_server_mem=$((total_mem*75/100)) # 75% - echo "Setting max_server_memory_usage=$max_server_mem" + + max_server_memory_usage_to_ram_ratio=0.5 + echo "Setting max_server_memory_usage_to_ram_ratio to ${max_server_memory_usage_to_ram_ratio}" cat > /etc/clickhouse-server/config.d/max_server_memory_usage.xml < - ${max_server_mem} + ${max_server_memory_usage_to_ram_ratio} EOL + local max_users_mem - max_users_mem=$((total_mem*50/100)) # 50% + max_users_mem=$((total_mem*30/100)) # 30% echo "Setting max_memory_usage_for_user=$max_users_mem" cat > /etc/clickhouse-server/users.d/max_memory_usage_for_user.xml < @@ -97,6 +99,13 @@ EOL --> $PWD +EOL + + # Let OOM killer terminate other processes before clickhouse-server: + cat > /etc/clickhouse-server/config.d/oom_score.xml < + -1000 + EOL # Analyzer is not yet ready for testing diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b43c9a2a021..1388d96063f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -420,6 +420,31 @@ void Server::createServer( } } + +namespace +{ + +void setOOMScore(int value, Poco::Logger * log) +{ + try + { + std::string value_string = std::to_string(value); + DB::WriteBufferFromFile buf("/proc/self/oom_score_adj"); + buf.write(value_string.c_str(), value_string.size()); + buf.next(); + buf.close(); + } + catch (const Poco::Exception & e) + { + LOG_WARNING(log, "Failed to adjust OOM score: '{}'.", e.displayText()); + return; + } + LOG_INFO(log, "Set OOM score adjustment to {}", value); +} + +} + + void Server::uninitialize() { logger().information("shutting down"); @@ -881,6 +906,21 @@ try } } } + + int default_oom_score = 0; + +#if !defined(NDEBUG) + /// In debug version on Linux, increase oom score so that clickhouse is killed + /// first, instead of some service. Use a carefully chosen random score of 555: + /// the maximum is 1000, and chromium uses 300 for its tab processes. Ignore + /// whatever errors that occur, because it's just a debugging aid and we don't + /// care if it breaks. + default_oom_score = 555; +#endif + + int oom_score = config().getInt("oom_score", default_oom_score); + if (oom_score) + setOOMScore(oom_score, log); #endif global_context->setRemoteHostFilter(config()); diff --git a/programs/server/config.xml b/programs/server/config.xml index 0cbc3d9339e..5f4a9fb983f 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1464,4 +1464,8 @@ I don't recommend to change this setting. false --> + + diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 5a08c8ffcd0..d4e76a0cad8 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -602,34 +602,6 @@ void BaseDaemon::closeFDs() } } -namespace -{ -/// In debug version on Linux, increase oom score so that clickhouse is killed -/// first, instead of some service. Use a carefully chosen random score of 555: -/// the maximum is 1000, and chromium uses 300 for its tab processes. Ignore -/// whatever errors that occur, because it's just a debugging aid and we don't -/// care if it breaks. -#if defined(OS_LINUX) && !defined(NDEBUG) -void debugIncreaseOOMScore() -{ - const std::string new_score = "555"; - try - { - DB::WriteBufferFromFile buf("/proc/self/oom_score_adj"); - buf.write(new_score.c_str(), new_score.size()); - buf.close(); - } - catch (const Poco::Exception & e) - { - LOG_WARNING(&Poco::Logger::root(), "Failed to adjust OOM score: '{}'.", e.displayText()); - return; - } - LOG_INFO(&Poco::Logger::root(), "Set OOM score adjustment to {}", new_score); -} -#else -void debugIncreaseOOMScore() {} -#endif -} void BaseDaemon::initialize(Application & self) { @@ -796,7 +768,6 @@ void BaseDaemon::initialize(Application & self) initializeTerminationAndSignalProcessing(); logRevision(); - debugIncreaseOOMScore(); for (const auto & key : DB::getMultipleKeysFromConfig(config(), "", "graphite")) { From f7f5aaf35ddb15f324a3993a7330f6fa104c2233 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 2 Jan 2023 01:20:27 +0100 Subject: [PATCH 33/78] What if I will remove two lines of code? --- docker/packager/binary/build.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 436fcbe921c..a3f9f796ef0 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -107,8 +107,6 @@ fi mv ./programs/clickhouse* /output [ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds -find . -name '*.so' -print -exec mv '{}' /output \; -find . -name '*.so.*' -print -exec mv '{}' /output \; prepare_combined_output () { local OUTPUT From 47ae8c5c791aa29e48e8fb03b433c0bce5c48f85 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 2 Jan 2023 02:06:11 +0100 Subject: [PATCH 34/78] Remove more lines --- .github/workflows/master.yml | 31 ---- .github/workflows/pull_request.yml | 31 ---- docker/images.json | 5 - docker/packager/binary/build.sh | 2 +- docker/test/split_build_smoke_test/Dockerfile | 9 - .../process_split_build_smoke_test_result.py | 64 -------- docker/test/split_build_smoke_test/run.sh | 22 --- docs/en/development/continuous-integration.md | 11 -- docs/zh/development/continuous-integration.md | 9 - programs/CMakeLists.txt | 6 - programs/clickhouse-split-helper | 12 -- tests/ci/ci_config.py | 3 - tests/ci/split_build_smoke_check.py | 154 ------------------ tests/ci/tests/docker_images.json | 10 -- tests/clickhouse-test | 7 +- 15 files changed, 4 insertions(+), 372 deletions(-) delete mode 100644 docker/test/split_build_smoke_test/Dockerfile delete mode 100755 docker/test/split_build_smoke_test/process_split_build_smoke_test_result.py delete mode 100755 docker/test/split_build_smoke_test/run.sh delete mode 100755 programs/clickhouse-split-helper delete mode 100644 tests/ci/split_build_smoke_check.py diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index dc81755ec7b..89af99d7175 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -141,37 +141,6 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" - SharedBuildSmokeTest: - needs: [BuilderDebShared] - runs-on: [self-hosted, style-checker] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/split_build_check - REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse - REPORTS_PATH=${{runner.temp}}/reports_dir - EOF - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - - name: Download json reports - uses: actions/download-artifact@v3 - with: - path: ${{ env.REPORTS_PATH }} - - name: Shared build check - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 split_build_smoke_check.py - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" ######################################################################################### #################################### ORDINARY BUILDS #################################### ######################################################################################### diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index efb7d50dd28..a0bac29f166 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -203,37 +203,6 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" - SharedBuildSmokeTest: - needs: [BuilderDebShared] - runs-on: [self-hosted, style-checker] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/split_build_check - REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse - REPORTS_PATH=${{runner.temp}}/reports_dir - EOF - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - - name: Download json reports - uses: actions/download-artifact@v3 - with: - path: ${{ env.REPORTS_PATH }} - - name: Shared build check - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 split_build_smoke_check.py - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" ######################################################################################### #################################### ORDINARY BUILDS #################################### ######################################################################################### diff --git a/docker/images.json b/docker/images.json index 8339205b52f..bc8e9924955 100644 --- a/docker/images.json +++ b/docker/images.json @@ -2,7 +2,6 @@ "docker/packager/binary": { "name": "clickhouse/binary-builder", "dependent": [ - "docker/test/split_build_smoke_test", "docker/test/codebrowser" ] }, @@ -55,10 +54,6 @@ "name": "clickhouse/stress-test", "dependent": [] }, - "docker/test/split_build_smoke_test": { - "name": "clickhouse/split-build-smoke-test", - "dependent": [] - }, "docker/test/codebrowser": { "name": "clickhouse/codebrowser", "dependent": [] diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index a3f9f796ef0..912366fb4ff 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -163,7 +163,7 @@ then ) fi -# May be set for split build or for performance test. +# May be set for performance test. if [ "" != "$COMBINED_OUTPUT" ] then prepare_combined_output /output diff --git a/docker/test/split_build_smoke_test/Dockerfile b/docker/test/split_build_smoke_test/Dockerfile deleted file mode 100644 index 5f84eb42216..00000000000 --- a/docker/test/split_build_smoke_test/Dockerfile +++ /dev/null @@ -1,9 +0,0 @@ -# rebuild in #33610 -# docker build -t clickhouse/split-build-smoke-test . -ARG FROM_TAG=latest -FROM clickhouse/binary-builder:$FROM_TAG - -COPY run.sh /run.sh -COPY process_split_build_smoke_test_result.py / - -CMD /run.sh diff --git a/docker/test/split_build_smoke_test/process_split_build_smoke_test_result.py b/docker/test/split_build_smoke_test/process_split_build_smoke_test_result.py deleted file mode 100755 index b5bc82e6818..00000000000 --- a/docker/test/split_build_smoke_test/process_split_build_smoke_test_result.py +++ /dev/null @@ -1,64 +0,0 @@ -#!/usr/bin/env python3 - -import os -import logging -import argparse -import csv - -RESULT_LOG_NAME = "run.log" - - -def process_result(result_folder): - - status = "success" - description = "Server started and responded" - summary = [("Smoke test", "OK")] - with open(os.path.join(result_folder, RESULT_LOG_NAME), "r") as run_log: - lines = run_log.read().split("\n") - if not lines or lines[0].strip() != "OK": - status = "failure" - logging.info("Lines is not ok: %s", str("\n".join(lines))) - summary = [("Smoke test", "FAIL")] - description = "Server failed to respond, see result in logs" - - result_logs = [] - server_log_path = os.path.join(result_folder, "clickhouse-server.log") - stderr_log_path = os.path.join(result_folder, "stderr.log") - client_stderr_log_path = os.path.join(result_folder, "clientstderr.log") - - if os.path.exists(server_log_path): - result_logs.append(server_log_path) - - if os.path.exists(stderr_log_path): - result_logs.append(stderr_log_path) - - if os.path.exists(client_stderr_log_path): - result_logs.append(client_stderr_log_path) - - return status, description, summary, result_logs - - -def write_results(results_file, status_file, results, status): - with open(results_file, "w") as f: - out = csv.writer(f, delimiter="\t") - out.writerows(results) - with open(status_file, "w") as f: - out = csv.writer(f, delimiter="\t") - out.writerow(status) - - -if __name__ == "__main__": - logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s") - parser = argparse.ArgumentParser( - description="ClickHouse script for parsing results of split build smoke test" - ) - parser.add_argument("--in-results-dir", default="/test_output/") - parser.add_argument("--out-results-file", default="/test_output/test_results.tsv") - parser.add_argument("--out-status-file", default="/test_output/check_status.tsv") - args = parser.parse_args() - - state, description, test_results, logs = process_result(args.in_results_dir) - logging.info("Result parsed") - status = (state, description) - write_results(args.out_results_file, args.out_status_file, test_results, status) - logging.info("Result written") diff --git a/docker/test/split_build_smoke_test/run.sh b/docker/test/split_build_smoke_test/run.sh deleted file mode 100755 index b565d7a481e..00000000000 --- a/docker/test/split_build_smoke_test/run.sh +++ /dev/null @@ -1,22 +0,0 @@ -#!/bin/bash - -set -x - -install_and_run_server() { - mkdir /unpacked - tar -xzf /package_folder/shared_build.tgz -C /unpacked --strip 1 - LD_LIBRARY_PATH=/unpacked /unpacked/clickhouse-server --config /unpacked/config/config.xml >/test_output/stderr.log 2>&1 & -} - -run_client() { - for i in {1..100}; do - sleep 1 - LD_LIBRARY_PATH=/unpacked /unpacked/clickhouse-client --query "select 'OK'" > /test_output/run.log 2> /test_output/clientstderr.log && break - [[ $i == 100 ]] && echo 'FAIL' - done -} - -install_and_run_server -run_client -mv /var/log/clickhouse-server/clickhouse-server.log /test_output/clickhouse-server.log -/process_split_build_smoke_test_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 677fb81efdd..68dc1b57044 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -118,7 +118,6 @@ Builds ClickHouse in various configurations for use in further steps. You have t - **Compiler**: `gcc-9` or `clang-10` (or `clang-10-xx` for other architectures e.g. `clang-10-freebsd`). - **Build type**: `Debug` or `RelWithDebInfo` (cmake). - **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). -- **Split** `splitted` is a [split build](../development/build.md#split-build) - **Status**: `success` or `fail` - **Build log**: link to the building and files copying log, useful when build failed. - **Build time**. @@ -169,16 +168,6 @@ concurrency-related errors. If it fails: of error. -## Split Build Smoke Test - -Checks that the server build in [split build](../development/developer-instruction.md#split-build) -configuration can start and run simple queries. If it fails: - - * Fix other test errors first; - * Build the server in [split build](../development/developer-instruction.md#split-build) configuration - locally and check whether it can start and run `select 1`. - - ## Compatibility Check Checks that `clickhouse` binary runs on distributions with old libc versions. If it fails, ask a maintainer for help. diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index 67b1eb228f1..de9a6a7807a 100644 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -85,7 +85,6 @@ git push - **Build type**: `Debug` or `RelWithDebInfo` (cmake). - **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). - **Bundled**: `bundled` 构建使用来自 `contrib` 库, 而 `unbundled` 构建使用系统库. -- **Splitted**: `splitted` is a [split build](https://clickhouse.com/docs/en/development/build/#split-build) - **Status**: `成功` 或 `失败` - **Build log**: 链接到构建和文件复制日志, 当构建失败时很有用. - **Build time**. @@ -123,14 +122,6 @@ git push of error. ``` -## 冒烟测试 {#split-build-smoke-test} -检查[拆分构建](./build.md#split-build)配置中的服务器构建是否可以启动并运行简单查询.如果失败: -``` -* Fix other test errors first; -* Build the server in [split build](./build.md#split-build) configuration - locally and check whether it can start and run `select 1`. -``` - ## 兼容性检查 {#compatibility-check} 检查`clickhouse`二进制文件是否可以在带有旧libc版本的发行版上运行.如果失败, 请向维护人员寻求帮助. diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 5b6c5b26633..b74ec8bf9a6 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -13,12 +13,6 @@ option (ENABLE_CLICKHOUSE_SERVER "Server mode (main mode)" ${ENABLE_CLICKHOUSE_A option (ENABLE_CLICKHOUSE_CLIENT "Client mode (interactive tui/shell that connects to the server)" ${ENABLE_CLICKHOUSE_ALL}) -# Don't create self-extracting clickhouse for split build -if (ENABLE_CLICKHOUSE_SELF_EXTRACTING AND SPLIT_SHARED_LIBRARIES) - message (STATUS "Self-extracting on split build is not supported") - unset (ENABLE_CLICKHOUSE_SELF_EXTRACTING CACHE) -endif () - # https://clickhouse.com/docs/en/operations/utilities/clickhouse-local/ option (ENABLE_CLICKHOUSE_LOCAL "Local files fast processing mode" ${ENABLE_CLICKHOUSE_ALL}) diff --git a/programs/clickhouse-split-helper b/programs/clickhouse-split-helper deleted file mode 100755 index 14a86f76097..00000000000 --- a/programs/clickhouse-split-helper +++ /dev/null @@ -1,12 +0,0 @@ -#!/bin/sh - -# Helper for split build mode. -# Allows to run commands like -# clickhouse client -# clickhouse server -# ... - -set -e -CMD=$1 -shift -clickhouse-$CMD $* diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 753da25f300..7ace7255a65 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -308,9 +308,6 @@ CI_CONFIG = { "Compatibility check": { "required_build": "package_release", }, - "Split build smoke test": { - "required_build": "binary_shared", - }, "Unit tests (release-clang)": { "required_build": "binary_release", }, diff --git a/tests/ci/split_build_smoke_check.py b/tests/ci/split_build_smoke_check.py deleted file mode 100644 index c6bf1051c87..00000000000 --- a/tests/ci/split_build_smoke_check.py +++ /dev/null @@ -1,154 +0,0 @@ -#!/usr/bin/env python3 - -import os -import logging -import subprocess -import sys - -from github import Github - -from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH -from s3_helper import S3Helper -from get_robot_token import get_best_robot_token -from pr_info import PRInfo -from build_download_helper import download_shared_build -from upload_result_helper import upload_results -from docker_pull_helper import get_image_with_version -from commit_status_helper import post_commit_status -from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse -from stopwatch import Stopwatch -from rerun_helper import RerunHelper - - -DOCKER_IMAGE = "clickhouse/split-build-smoke-test" -DOWNLOAD_RETRIES_COUNT = 5 -RESULT_LOG_NAME = "run.log" -CHECK_NAME = "Split build smoke test" - - -def process_result(result_folder, server_log_folder): - status = "success" - description = "Server started and responded" - summary = [("Smoke test", "OK")] - with open(os.path.join(result_folder, RESULT_LOG_NAME), "r") as run_log: - lines = run_log.read().split("\n") - if not lines or lines[0].strip() != "OK": - status = "failure" - logging.info("Lines is not ok: %s", str("\n".join(lines))) - summary = [("Smoke test", "FAIL")] - description = "Server failed to respond, see result in logs" - - result_logs = [] - server_log_path = os.path.join(server_log_folder, "clickhouse-server.log") - stderr_log_path = os.path.join(result_folder, "stderr.log") - client_stderr_log_path = os.path.join(result_folder, "clientstderr.log") - run_log_path = os.path.join(result_folder, RESULT_LOG_NAME) - - for path in [ - server_log_path, - stderr_log_path, - client_stderr_log_path, - run_log_path, - ]: - if os.path.exists(path): - result_logs.append(path) - - return status, description, summary, result_logs - - -def get_run_command(build_path, result_folder, server_log_folder, docker_image): - return ( - f"docker run --network=host --volume={build_path}:/package_folder" - f" --volume={server_log_folder}:/var/log/clickhouse-server" - f" --volume={result_folder}:/test_output" - f" {docker_image} >{result_folder}/{RESULT_LOG_NAME}" - ) - - -if __name__ == "__main__": - logging.basicConfig(level=logging.INFO) - - stopwatch = Stopwatch() - - temp_path = TEMP_PATH - repo_path = REPO_COPY - reports_path = REPORTS_PATH - - pr_info = PRInfo() - - gh = Github(get_best_robot_token(), per_page=100) - - rerun_helper = RerunHelper(gh, pr_info, CHECK_NAME) - if rerun_helper.is_already_finished_by_status(): - logging.info("Check is already finished according to github status, exiting") - sys.exit(0) - - for root, _, files in os.walk(reports_path): - for f in files: - if f == "changed_images.json": - images_path = os.path.join(root, "changed_images.json") - break - - docker_image = get_image_with_version(reports_path, DOCKER_IMAGE) - - packages_path = os.path.join(temp_path, "packages") - if not os.path.exists(packages_path): - os.makedirs(packages_path) - - download_shared_build(CHECK_NAME, reports_path, packages_path) - - server_log_path = os.path.join(temp_path, "server_log") - if not os.path.exists(server_log_path): - os.makedirs(server_log_path) - - result_path = os.path.join(temp_path, "result_path") - if not os.path.exists(result_path): - os.makedirs(result_path) - - run_command = get_run_command( - packages_path, result_path, server_log_path, docker_image - ) - - logging.info("Going to run command %s", run_command) - with subprocess.Popen(run_command, shell=True) as process: - retcode = process.wait() - if retcode == 0: - logging.info("Run successfully") - else: - logging.info("Run failed") - - subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) - print("Result path", os.listdir(result_path)) - print("Server log path", os.listdir(server_log_path)) - - state, description, test_results, additional_logs = process_result( - result_path, server_log_path - ) - - ch_helper = ClickHouseHelper() - s3_helper = S3Helper() - report_url = upload_results( - s3_helper, - pr_info.number, - pr_info.sha, - test_results, - additional_logs, - CHECK_NAME, - ) - print(f"::notice ::Report url: {report_url}") - post_commit_status(gh, pr_info.sha, CHECK_NAME, description, state, report_url) - - prepared_events = prepare_tests_results_for_clickhouse( - pr_info, - test_results, - state, - stopwatch.duration_seconds, - stopwatch.start_time_str, - report_url, - CHECK_NAME, - ) - - ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) - - if state == "error": - sys.exit(1) diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images.json index ca5c516bccb..8c13c760168 100644 --- a/tests/ci/tests/docker_images.json +++ b/tests/ci/tests/docker_images.json @@ -6,8 +6,6 @@ "docker/packager/binary": { "name": "clickhouse/binary-builder", "dependent": [ - "docker/test/split_build_smoke_test", - "docker/test/pvs", "docker/test/codebrowser" ] }, @@ -31,10 +29,6 @@ "name": "clickhouse/performance-comparison", "dependent": [] }, - "docker/test/pvs": { - "name": "clickhouse/pvs-test", - "dependent": [] - }, "docker/test/util": { "name": "clickhouse/test-util", "dependent": [ @@ -63,10 +57,6 @@ "name": "clickhouse/stress-test", "dependent": [] }, - "docker/test/split_build_smoke_test": { - "name": "clickhouse/split-build-smoke-test", - "dependent": [] - }, "docker/test/codebrowser": { "name": "clickhouse/codebrowser", "dependent": [] diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 13669981daa..2709ad1eecf 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1820,8 +1820,7 @@ def main(args): global restarted_tests if not check_server_started(args): - msg = "Server is not responding. Cannot execute 'SELECT 1' query. \ - If you are using split build, you have to specify -c option." + msg = "Server is not responding. Cannot execute 'SELECT 1' query." if args.hung_check: print(msg) pid = get_server_pid() @@ -2006,14 +2005,14 @@ if __name__ == "__main__": "-b", "--binary", default="clickhouse", - help="Path to clickhouse (if monolithic build, clickhouse-server otherwise) " + help="Path to clickhouse" "binary or name of binary in PATH", ) parser.add_argument( "-c", "--client", - help="Path to clickhouse-client (if split build, useless otherwise) binary of " + help="Path to clickhouse-client, this option is useless" "name of binary in PATH", ) From b859cc2ddcf70e78caa54c56422841f74f26f72c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 2 Jan 2023 03:27:21 +0100 Subject: [PATCH 35/78] Fix build --- programs/server/Server.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1388d96063f..a4ddd8553c6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -421,6 +421,7 @@ void Server::createServer( } +#if defined(OS_LINUX) namespace { @@ -443,6 +444,7 @@ void setOOMScore(int value, Poco::Logger * log) } } +#endif void Server::uninitialize() From 3a1dd045ddccaacad3ee0813e146a6e2d149d9c0 Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 2 Jan 2023 09:27:37 +0000 Subject: [PATCH 36/78] Fix output_format_pretty_row_numbers does not preserve the counter across the blocks --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 2 +- .../Impl/PrettyCompactBlockOutputFormat.cpp | 2 +- .../Impl/PrettySpaceBlockOutputFormat.cpp | 2 +- ...output_format_pretty_row_numbers.reference | 48 +++++++++++++++++++ ...01509_output_format_pretty_row_numbers.sql | 18 +++++++ 5 files changed, 69 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index faf4681340f..a5ae1d50d14 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -295,7 +295,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind if (format_settings.pretty.output_format_pretty_row_numbers) { // Write row number; - auto row_num_string = std::to_string(i + 1) + ". "; + auto row_num_string = std::to_string(i + 1 + total_rows) + ". "; for (size_t j = 0; j < row_number_width - row_num_string.size(); ++j) { writeCString(" ", out); diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index cc26a064b54..2ba9ec725e2 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -144,7 +144,7 @@ void PrettyCompactBlockOutputFormat::writeRow( if (format_settings.pretty.output_format_pretty_row_numbers) { // Write row number; - auto row_num_string = std::to_string(row_num + 1) + ". "; + auto row_num_string = std::to_string(row_num + 1 + total_rows) + ". "; for (size_t i = 0; i < row_number_width - row_num_string.size(); ++i) { writeCString(" ", out); diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 4afb380dec9..46d1872412c 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -73,7 +73,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port if (format_settings.pretty.output_format_pretty_row_numbers) { // Write row number; - auto row_num_string = std::to_string(row + 1) + ". "; + auto row_num_string = std::to_string(row + 1 + total_rows) + ". "; for (size_t i = 0; i < row_number_width - row_num_string.size(); ++i) { writeCString(" ", out); diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference index 9010c371fdf..76fbeb4986a 100644 --- a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference @@ -210,3 +210,51 @@ 8. 7 9. 8 10. 9 + ┏━━━┓ + ┃ a ┃ + ┡━━━┩ +1. │ 1 │ + └───┘ + ┏━━━┓ + ┃ a ┃ + ┡━━━┩ +2. │ 2 │ + └───┘ + ┌─a─┐ +1. │ 1 │ + └───┘ + ┌─a─┐ +2. │ 2 │ + └───┘ + ┌─a─┐ +1. │ 1 │ +2. │ 2 │ + └───┘ + ┏━━━┓ + ┃ a ┃ + ┡━━━┩ +1. │ 1 │ + └───┘ + ┏━━━┓ + ┃ a ┃ + ┡━━━┩ +2. │ 2 │ + └───┘ + ┌─a─┐ +1. │ 1 │ + └───┘ + ┌─a─┐ +2. │ 2 │ + └───┘ + a + +1. 1 + a + +2. 2 + a + +1. 1 + a + +2. 2 diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql index 3c0eeaf0e53..fe60c85f89e 100644 --- a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql @@ -13,3 +13,21 @@ SELECT * FROM numbers(10) FORMAT PrettyNoEscapes; SELECT * FROM numbers(10) FORMAT PrettyCompactNoEscapes; SELECT * FROM numbers(10) FORMAT PrettySpaceNoEscapes; SELECT * FROM numbers(10) FORMAT PrettySpace; + +SET max_block_size=1; + +SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT Pretty; +SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT PrettyCompact; +SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT PrettyCompactMonoBlock; +SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT PrettyNoEscapes; +SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT PrettyCompactNoEscapes; +SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT PrettySpace; +SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT PrettySpaceNoEscapes; + +-- SELECT * FROM numbers(10) ORDER BY number FORMAT Pretty; +-- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyCompact; +-- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyCompactMonoBlock; +-- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyNoEscapes; +-- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyCompactNoEscapes; +-- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettySpace +-- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettySpaceNoEscapes; From 2f73bf48b8558ab24805d2ce104c36c61effbbe5 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 2 Jan 2023 11:43:19 +0100 Subject: [PATCH 37/78] Added config to integration test test_endpoint_macro_substitution - Allow macro substitution in endpoint --- .../configs/config.xml | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 tests/integration/test_endpoint_macro_substitution/configs/config.xml diff --git a/tests/integration/test_endpoint_macro_substitution/configs/config.xml b/tests/integration/test_endpoint_macro_substitution/configs/config.xml new file mode 100644 index 00000000000..d4a2a9cf367 --- /dev/null +++ b/tests/integration/test_endpoint_macro_substitution/configs/config.xml @@ -0,0 +1,26 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + ./clickhouse/ + users.xml + From f779fe3c0a42a71cf8bf2c80c1eb4c6a49c68ac9 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 29 Dec 2022 14:24:23 +0000 Subject: [PATCH 38/78] Improve some integration tests --- docker/test/integration/runner/Dockerfile | 1 + tests/integration/runner | 12 +++- tests/integration/test_grpc_protocol/test.py | 7 +- .../test_grpc_protocol_ssl/test.py | 3 + .../integration/test_odbc_interaction/test.py | 69 +++++++++++++++---- 5 files changed, 76 insertions(+), 16 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 84c04dd03ec..ccfd63c8ed0 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -83,6 +83,7 @@ RUN python3 -m pip install \ pytest \ pytest-order==1.0.0 \ pytest-timeout \ + pytest-random \ pytest-xdist \ pytest-repeat \ pytz \ diff --git a/tests/integration/runner b/tests/integration/runner index f4f853e00ad..6a05d383089 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -242,6 +242,10 @@ if __name__ == "__main__": "-n", "--parallel", action="store", dest="parallel", help="Parallelism" ) + parser.add_argument( + "--no-random", action="store", dest="no_random", help="Disable tests order randomization" + ) + parser.add_argument( "-t", "--tests_list", @@ -294,6 +298,11 @@ if __name__ == "__main__": parallel_args += "--dist=loadfile" parallel_args += " -n {}".format(args.parallel) + rand_args = "" + if not args.no_random: + rand_args += f"--random-seed={os.getpid()}" + + net = "" if args.network: net = "--net={}".format(args.network) @@ -383,7 +392,7 @@ if __name__ == "__main__": {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 \ -e XTABLES_LOCKFILE=/run/host/xtables.lock \ -e PYTHONUNBUFFERED=1 \ - {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} -vvv' {img} {command}".format( + {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} {rand} -vvv' {img} {command}".format( net=net, tty=tty, bin=args.binary, @@ -395,6 +404,7 @@ if __name__ == "__main__": env_tags=env_tags, env_cleanup=env_cleanup, parallel=parallel_args, + rand=rand_args, opts=" ".join(args.pytest_args).replace("'", "\\'"), tests_list=" ".join(args.tests_list), dockerd_internal_volume=dockerd_internal_volume, diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index a3f2650eac7..f4884d309b1 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -37,7 +37,12 @@ import clickhouse_grpc_pb2_grpc config_dir = os.path.join(SCRIPT_DIR, "./configs") cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=["configs/grpc_config.xml"]) +node = cluster.add_instance( + "node", + main_configs=["configs/grpc_config.xml"], + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + # second_deadlock_stack -- just ordinary option we use everywhere, don't want to overwrite it + env_variables={"TSAN_OPTIONS": "report_atomic_races=0 second_deadlock_stack=1"},) main_channel = None diff --git a/tests/integration/test_grpc_protocol_ssl/test.py b/tests/integration/test_grpc_protocol_ssl/test.py index 80599126dcf..3c28fb666c8 100644 --- a/tests/integration/test_grpc_protocol_ssl/test.py +++ b/tests/integration/test_grpc_protocol_ssl/test.py @@ -42,6 +42,9 @@ node = cluster.add_instance( "configs/server-cert.pem", "configs/ca-cert.pem", ], + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + # second_deadlock_stack -- just ordinary option we use everywhere, don't want to overwrite it + env_variables={"TSAN_OPTIONS": "report_atomic_races=0 second_deadlock_stack=1"}, ) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 743e4ecd68a..faa48c00c44 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -4,7 +4,6 @@ import psycopg2 import pymysql.cursors import pytest import logging -import os.path from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry @@ -80,9 +79,11 @@ def create_mysql_db(conn, name): def create_mysql_table(conn, table_name): with conn.cursor() as cursor: - cursor.execute(drop_table_sql_template.format(table_name)) cursor.execute(create_table_sql_template.format(table_name)) +def drop_mysql_table(conn, table_name): + with conn.cursor() as cursor: + cursor.execute(drop_table_sql_template.format(table_name)) def get_postgres_conn(started_cluster): conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format( @@ -267,7 +268,9 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nulla # just to be sure :) assert node1.query("select 1") == "1\n" + node1.query(f"DROP TABLE {table_name}") conn.close() + drop_mysql_table(conn, table_name) def test_mysql_insert(started_cluster): @@ -309,6 +312,9 @@ def test_mysql_insert(started_cluster): == "3\tinsert\t33\t333\t3333\n4\tTEST\t44\t444\t\\N\n" ) + node1.query("DROP TABLE mysql_insert") + drop_mysql_table(conn, table_name) + def test_sqlite_simple_select_function_works(started_cluster): skip_test_msan(node1) @@ -367,6 +373,12 @@ def test_sqlite_simple_select_function_works(started_cluster): == "1\t1\n" ) + node1.exec_in_container( + ["sqlite3", sqlite_db, "DELETE FROM t1;"], + privileged=True, + user="root", + ) + def test_sqlite_table_function(started_cluster): skip_test_msan(node1) @@ -392,6 +404,12 @@ def test_sqlite_table_function(started_cluster): assert node1.query("select x, y from odbc_tf") == "1\t2\n" assert node1.query("select z, x, y from odbc_tf") == "3\t1\t2\n" assert node1.query("select count(), sum(x) from odbc_tf group by x") == "1\t1\n" + node1.query("DROP TABLE odbc_tf") + node1.exec_in_container( + ["sqlite3", sqlite_db, "DELETE FROM tf1;"], + privileged=True, + user="root", + ) def test_sqlite_simple_select_storage_works(started_cluster): @@ -418,6 +436,13 @@ def test_sqlite_simple_select_storage_works(started_cluster): assert node1.query("select x, y from SqliteODBC") == "1\t2\n" assert node1.query("select z, x, y from SqliteODBC") == "3\t1\t2\n" assert node1.query("select count(), sum(x) from SqliteODBC group by x") == "1\t1\n" + node1.query("DROP TABLE SqliteODBC") + + node1.exec_in_container( + ["sqlite3", sqlite_db, "DELETE FROM t4;"], + privileged=True, + user="root", + ) def test_sqlite_odbc_hashed_dictionary(started_cluster): @@ -496,6 +521,12 @@ def test_sqlite_odbc_hashed_dictionary(started_cluster): node1, "select dictGetUInt8('sqlite3_odbc_hashed', 'Z', toUInt64(200))", "7" ) + node1.exec_in_container( + ["sqlite3", sqlite_db, "DELETE FROM t2;"], + privileged=True, + user="root", + ) + def test_sqlite_odbc_cached_dictionary(started_cluster): skip_test_msan(node1) @@ -537,13 +568,19 @@ def test_sqlite_odbc_cached_dictionary(started_cluster): node1, "select dictGetUInt8('sqlite3_odbc_cached', 'Z', toUInt64(1))", "12" ) + node1.exec_in_container( + ["sqlite3", sqlite_db, "DELETE FROM t3;"], + privileged=True, + user="root", + ) + + node1.query("SYSTEM RELOAD DICTIONARIES") def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): skip_test_msan(node1) conn = get_postgres_conn(started_cluster) cursor = conn.cursor() - cursor.execute("truncate table clickhouse.test_table") cursor.execute( "insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')" ) @@ -562,6 +599,7 @@ def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))", "world", ) + cursor.execute("truncate table clickhouse.test_table") def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): @@ -569,7 +607,6 @@ def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): conn = get_postgres_conn(started_cluster) cursor = conn.cursor() - cursor.execute("truncate table clickhouse.test_table") cursor.execute("insert into clickhouse.test_table values(3, 3, 'xxx')") for i in range(100): try: @@ -582,13 +619,13 @@ def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(3))", "xxx", ) + cursor.execute("truncate table clickhouse.test_table") def test_postgres_insert(started_cluster): skip_test_msan(node1) conn = get_postgres_conn(started_cluster) - conn.cursor().execute("truncate table clickhouse.test_table") # Also test with Servername containing '.' and '-' symbols (defined in # postgres .yml file). This is needed to check parsing, validation and @@ -615,6 +652,8 @@ def test_postgres_insert(started_cluster): ) == "55\t10\n" ) + node1.query("DROP TABLE pg_insert") + conn.cursor().execute("truncate table clickhouse.test_table") def test_bridge_dies_with_parent(started_cluster): @@ -675,7 +714,7 @@ def test_odbc_postgres_date_data_type(started_cluster): conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute( - "CREATE TABLE IF NOT EXISTS clickhouse.test_date (id integer, column1 integer, column2 date)" + "CREATE TABLE clickhouse.test_date (id integer, column1 integer, column2 date)" ) cursor.execute("INSERT INTO clickhouse.test_date VALUES (1, 1, '2020-12-01')") @@ -692,8 +731,8 @@ def test_odbc_postgres_date_data_type(started_cluster): expected = "1\t1\t2020-12-01\n2\t2\t2020-12-02\n3\t3\t2020-12-03\n" result = node1.query("SELECT * FROM test_date") assert result == expected - cursor.execute("DROP TABLE IF EXISTS clickhouse.test_date") - node1.query("DROP TABLE IF EXISTS test_date") + cursor.execute("DROP TABLE clickhouse.test_date") + node1.query("DROP TABLE test_date") def test_odbc_postgres_conversions(started_cluster): @@ -703,7 +742,7 @@ def test_odbc_postgres_conversions(started_cluster): cursor = conn.cursor() cursor.execute( - """CREATE TABLE IF NOT EXISTS clickhouse.test_types ( + """CREATE TABLE clickhouse.test_types ( a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, h timestamp)""" ) @@ -729,7 +768,7 @@ def test_odbc_postgres_conversions(started_cluster): cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types") cursor.execute( - """CREATE TABLE IF NOT EXISTS clickhouse.test_types (column1 Timestamp, column2 Numeric)""" + """CREATE TABLE clickhouse.test_types (column1 Timestamp, column2 Numeric)""" ) node1.query( @@ -747,8 +786,8 @@ def test_odbc_postgres_conversions(started_cluster): "SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)" ) result = node1.query("SELECT * FROM test_types") - logging.debug(result) - cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types") + cursor.execute("DROP TABLE clickhouse.test_types") + node1.query("DROP TABLE test_types") assert result == expected @@ -776,6 +815,7 @@ def test_odbc_cyrillic_with_varchar(started_cluster): """ SELECT name FROM odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_cyrillic') """ ) assert result == "A-nice-word\nКрасивенько\n" + node1.query("DROP TABLE test_cyrillic") def test_many_connections(started_cluster): @@ -784,7 +824,6 @@ def test_many_connections(started_cluster): conn = get_postgres_conn(started_cluster) cursor = conn.cursor() - cursor.execute("DROP TABLE IF EXISTS clickhouse.test_pg_table") cursor.execute("CREATE TABLE clickhouse.test_pg_table (key integer, value integer)") node1.query( @@ -802,6 +841,7 @@ def test_many_connections(started_cluster): query += "SELECT key FROM {t})" assert node1.query(query.format(t="test_pg_table")) == "250\n" + cursor.execute("DROP TABLE clickhouse.test_pg_table") def test_concurrent_queries(started_cluster): @@ -901,7 +941,6 @@ def test_odbc_long_text(started_cluster): conn = get_postgres_conn(started_cluster) cursor = conn.cursor() - cursor.execute("drop table if exists clickhouse.test_long_text") cursor.execute("create table clickhouse.test_long_text(flen int, field1 text)") # sample test from issue 9363 @@ -929,3 +968,5 @@ def test_odbc_long_text(started_cluster): ) result = node1.query("select field1 from test_long_text where flen=400000;") assert result.strip() == long_text + node1.query("DROP TABLE test_long_text") + cursor.execute("drop table clickhouse.test_long_text") From 6ec3bf40b13034b93def69d233f004949ffcaa87 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 29 Dec 2022 19:07:14 +0000 Subject: [PATCH 39/78] black --- tests/integration/test_grpc_protocol/test.py | 3 ++- tests/integration/test_grpc_protocol_ssl/test.py | 2 +- tests/integration/test_odbc_interaction/test.py | 3 +++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index f4884d309b1..f870fd64fea 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -42,7 +42,8 @@ node = cluster.add_instance( main_configs=["configs/grpc_config.xml"], # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 # second_deadlock_stack -- just ordinary option we use everywhere, don't want to overwrite it - env_variables={"TSAN_OPTIONS": "report_atomic_races=0 second_deadlock_stack=1"},) + env_variables={"TSAN_OPTIONS": "report_atomic_races=0 second_deadlock_stack=1"}, +) main_channel = None diff --git a/tests/integration/test_grpc_protocol_ssl/test.py b/tests/integration/test_grpc_protocol_ssl/test.py index 3c28fb666c8..2c2a7f6c61e 100644 --- a/tests/integration/test_grpc_protocol_ssl/test.py +++ b/tests/integration/test_grpc_protocol_ssl/test.py @@ -42,7 +42,7 @@ node = cluster.add_instance( "configs/server-cert.pem", "configs/ca-cert.pem", ], - # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 # second_deadlock_stack -- just ordinary option we use everywhere, don't want to overwrite it env_variables={"TSAN_OPTIONS": "report_atomic_races=0 second_deadlock_stack=1"}, ) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index faa48c00c44..23a031359c2 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -81,10 +81,12 @@ def create_mysql_table(conn, table_name): with conn.cursor() as cursor: cursor.execute(create_table_sql_template.format(table_name)) + def drop_mysql_table(conn, table_name): with conn.cursor() as cursor: cursor.execute(drop_table_sql_template.format(table_name)) + def get_postgres_conn(started_cluster): conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format( started_cluster.postgres_ip, started_cluster.postgres_port @@ -576,6 +578,7 @@ def test_sqlite_odbc_cached_dictionary(started_cluster): node1.query("SYSTEM RELOAD DICTIONARIES") + def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): skip_test_msan(node1) From 54e2df724f562316ba5eca08640a2cb4d5d4bc5e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 30 Dec 2022 09:46:59 +0000 Subject: [PATCH 40/78] fix test --- tests/integration/test_odbc_interaction/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 23a031359c2..befcabb8adc 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -271,8 +271,8 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nulla assert node1.query("select 1") == "1\n" node1.query(f"DROP TABLE {table_name}") - conn.close() drop_mysql_table(conn, table_name) + conn.close() def test_mysql_insert(started_cluster): From 7b487dd92361a0fef607ffa982784914f443df32 Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 2 Jan 2023 12:37:43 +0000 Subject: [PATCH 41/78] fix --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 9 +- ...output_format_pretty_row_numbers.reference | 232 ++++++++++++++++++ ...01509_output_format_pretty_row_numbers.sql | 14 +- 3 files changed, 246 insertions(+), 9 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index a5ae1d50d14..442be88db7c 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; } @@ -37,8 +38,8 @@ void PrettyBlockOutputFormat::calculateWidths( { size_t num_rows = std::min(chunk.getNumRows(), format_settings.pretty.max_rows); - /// len(num_rows) + len(". ") - row_number_width = static_cast(std::floor(std::log10(num_rows))) + 3; + /// len(num_rows + total_rows) + len(". ") + row_number_width = static_cast(std::floor(std::log10(num_rows + total_rows))) + 3; size_t num_columns = chunk.getNumColumns(); const auto & columns = chunk.getColumns(); @@ -296,6 +297,10 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind { // Write row number; auto row_num_string = std::to_string(i + 1 + total_rows) + ". "; + size_t black_width = row_number_width - row_num_string.size(); + if (black_width > row_number_width) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "blak width overflow {}, {}, {}", black_width, row_num_string.size(), row_number_width); for (size_t j = 0; j < row_number_width - row_num_string.size(); ++j) { writeCString(" ", out); diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference index 76fbeb4986a..a270447ffc9 100644 --- a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference @@ -258,3 +258,235 @@ a 2. 2 + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +1. │ 0 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +2. │ 1 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +3. │ 2 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +4. │ 3 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +5. │ 4 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +6. │ 5 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +7. │ 6 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +8. │ 7 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +9. │ 8 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +10. │ 9 │ + └────────┘ + ┌─number─┐ +1. │ 0 │ + └────────┘ + ┌─number─┐ +2. │ 1 │ + └────────┘ + ┌─number─┐ +3. │ 2 │ + └────────┘ + ┌─number─┐ +4. │ 3 │ + └────────┘ + ┌─number─┐ +5. │ 4 │ + └────────┘ + ┌─number─┐ +6. │ 5 │ + └────────┘ + ┌─number─┐ +7. │ 6 │ + └────────┘ + ┌─number─┐ +8. │ 7 │ + └────────┘ + ┌─number─┐ +9. │ 8 │ + └────────┘ + ┌─number─┐ +10. │ 9 │ + └────────┘ + ┌─number─┐ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +1. │ 0 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +2. │ 1 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +3. │ 2 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +4. │ 3 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +5. │ 4 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +6. │ 5 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +7. │ 6 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +8. │ 7 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +9. │ 8 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ +10. │ 9 │ + └────────┘ + ┌─number─┐ +1. │ 0 │ + └────────┘ + ┌─number─┐ +2. │ 1 │ + └────────┘ + ┌─number─┐ +3. │ 2 │ + └────────┘ + ┌─number─┐ +4. │ 3 │ + └────────┘ + ┌─number─┐ +5. │ 4 │ + └────────┘ + ┌─number─┐ +6. │ 5 │ + └────────┘ + ┌─number─┐ +7. │ 6 │ + └────────┘ + ┌─number─┐ +8. │ 7 │ + └────────┘ + ┌─number─┐ +9. │ 8 │ + └────────┘ + ┌─number─┐ +10. │ 9 │ + └────────┘ + number + +1. 0 + number + +2. 1 + number + +3. 2 + number + +4. 3 + number + +5. 4 + number + +6. 5 + number + +7. 6 + number + +8. 7 + number + +9. 8 + number + +10. 9 + number + +1. 0 + number + +2. 1 + number + +3. 2 + number + +4. 3 + number + +5. 4 + number + +6. 5 + number + +7. 6 + number + +8. 7 + number + +9. 8 + number + +10. 9 diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql index fe60c85f89e..f8ec0be74d7 100644 --- a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql @@ -24,10 +24,10 @@ SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT PrettyCo SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT PrettySpace; SELECT * FROM (SELECT 1 AS a UNION ALL SELECT 2 as a) ORDER BY a FORMAT PrettySpaceNoEscapes; --- SELECT * FROM numbers(10) ORDER BY number FORMAT Pretty; --- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyCompact; --- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyCompactMonoBlock; --- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyNoEscapes; --- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyCompactNoEscapes; --- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettySpace --- SELECT * FROM numbers(10) ORDER BY number FORMAT PrettySpaceNoEscapes; +SELECT * FROM numbers(10) ORDER BY number FORMAT Pretty; +SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyCompact; +SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyCompactMonoBlock; +SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyNoEscapes; +SELECT * FROM numbers(10) ORDER BY number FORMAT PrettyCompactNoEscapes; +SELECT * FROM numbers(10) ORDER BY number FORMAT PrettySpace; +SELECT * FROM numbers(10) ORDER BY number FORMAT PrettySpaceNoEscapes; From 7780fae9db175abd5d05633109e97e6d4edc4843 Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 2 Jan 2023 12:38:51 +0000 Subject: [PATCH 42/78] fix --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 442be88db7c..cefe3ee4a98 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -14,12 +14,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - PrettyBlockOutputFormat::PrettyBlockOutputFormat( WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_, bool mono_block_) : IOutputFormat(header_, out_), format_settings(format_settings_), serializations(header_.getSerializations()), mono_block(mono_block_) @@ -297,10 +291,6 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind { // Write row number; auto row_num_string = std::to_string(i + 1 + total_rows) + ". "; - size_t black_width = row_number_width - row_num_string.size(); - if (black_width > row_number_width) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "blak width overflow {}, {}, {}", black_width, row_num_string.size(), row_number_width); for (size_t j = 0; j < row_number_width - row_num_string.size(); ++j) { writeCString(" ", out); From 5592d30798745d069012c6c705eab56faf6f93dd Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 2 Jan 2023 14:03:51 +0100 Subject: [PATCH 43/78] Updated to skip access check for hdfs , as done for other tests - Allow macro substitution in endpoint --- .../test_endpoint_macro_substitution/configs/storage.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_endpoint_macro_substitution/configs/storage.xml b/tests/integration/test_endpoint_macro_substitution/configs/storage.xml index 403ce8e52ae..4c6320b2e48 100644 --- a/tests/integration/test_endpoint_macro_substitution/configs/storage.xml +++ b/tests/integration/test_endpoint_macro_substitution/configs/storage.xml @@ -11,6 +11,8 @@ hdfs hdfs://hdfs1:9000/{hdfs_endpoint_substitution}/ + + true encrypted From ec5e8ee9cfc58a667b3b87ac40bdd61df122801e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 2 Jan 2023 13:31:19 +0000 Subject: [PATCH 44/78] Done --- .pylintrc | 4 + tests/integration/test_library_bridge/test.py | 62 +--------- .../test_library_bridge/test_exiled.py | 97 ++++++++++++++++ .../integration/test_odbc_interaction/test.py | 55 +-------- .../test_odbc_interaction/test_exiled.py | 106 ++++++++++++++++++ 5 files changed, 213 insertions(+), 111 deletions(-) create mode 100644 tests/integration/test_library_bridge/test_exiled.py create mode 100644 tests/integration/test_odbc_interaction/test_exiled.py diff --git a/.pylintrc b/.pylintrc index b672cbfdfad..3fd991a53b8 100644 --- a/.pylintrc +++ b/.pylintrc @@ -12,6 +12,10 @@ max-statements=200 [FORMAT] ignore-long-lines = (# )??$ +[LOGGING] +# Not to warn about lazy formatting in logging functions +logging-format-style=fstr + [MESSAGES CONTROL] disable = missing-docstring, too-few-public-methods, diff --git a/tests/integration/test_library_bridge/test.py b/tests/integration/test_library_bridge/test.py index e50c89f9567..a4dca545d44 100644 --- a/tests/integration/test_library_bridge/test.py +++ b/tests/integration/test_library_bridge/test.py @@ -16,9 +16,9 @@ instance = cluster.add_instance( ) -def create_dict_simple(): - instance.query("DROP DICTIONARY IF EXISTS lib_dict_c") - instance.query( +def create_dict_simple(ch_instance): + ch_instance.query("DROP DICTIONARY IF EXISTS lib_dict_c") + ch_instance.query( """ CREATE DICTIONARY lib_dict_c (key UInt64, value1 UInt64, value2 UInt64, value3 UInt64) PRIMARY KEY key SOURCE(library(PATH '/etc/clickhouse-server/config.d/dictionaries_lib/dict_lib.so')) @@ -242,7 +242,7 @@ def test_recover_after_bridge_crash(ch_cluster): if instance.is_built_with_memory_sanitizer(): pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") - create_dict_simple() + create_dict_simple(instance) result = instance.query("""select dictGet(lib_dict_c, 'value1', toUInt64(0));""") assert result.strip() == "100" @@ -269,7 +269,7 @@ def test_server_restart_bridge_might_be_stil_alive(ch_cluster): if instance.is_built_with_memory_sanitizer(): pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") - create_dict_simple() + create_dict_simple(instance) result = instance.query("""select dictGet(lib_dict_c, 'value1', toUInt64(1));""") assert result.strip() == "101" @@ -290,58 +290,6 @@ def test_server_restart_bridge_might_be_stil_alive(ch_cluster): instance.query("DROP DICTIONARY lib_dict_c") -def test_bridge_dies_with_parent(ch_cluster): - if instance.is_built_with_memory_sanitizer(): - pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") - if instance.is_built_with_address_sanitizer(): - pytest.skip( - "Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge" - ) - - create_dict_simple() - result = instance.query("""select dictGet(lib_dict_c, 'value1', toUInt64(1));""") - assert result.strip() == "101" - - clickhouse_pid = instance.get_process_pid("clickhouse server") - bridge_pid = instance.get_process_pid("library-bridge") - assert clickhouse_pid is not None - assert bridge_pid is not None - - try: - instance.exec_in_container( - ["kill", str(clickhouse_pid)], privileged=True, user="root" - ) - except: - pass - - for i in range(30): - time.sleep(1) - clickhouse_pid = instance.get_process_pid("clickhouse server") - if clickhouse_pid is None: - break - - for i in range(30): - time.sleep(1) - bridge_pid = instance.get_process_pid("library-bridge") - if bridge_pid is None: - break - - if bridge_pid: - out = instance.exec_in_container( - ["gdb", "-p", str(bridge_pid), "--ex", "thread apply all bt", "--ex", "q"], - privileged=True, - user="root", - ) - logging.debug(f"Bridge is running, gdb output:\n{out}") - - try: - assert clickhouse_pid is None - assert bridge_pid is None - finally: - instance.start_clickhouse(20) - instance.query("DROP DICTIONARY lib_dict_c") - - def test_path_validation(ch_cluster): if instance.is_built_with_memory_sanitizer(): pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") diff --git a/tests/integration/test_library_bridge/test_exiled.py b/tests/integration/test_library_bridge/test_exiled.py new file mode 100644 index 00000000000..1a7ead8d50b --- /dev/null +++ b/tests/integration/test_library_bridge/test_exiled.py @@ -0,0 +1,97 @@ +import os +import os.path as p +import pytest +import time +import logging + +from helpers.cluster import ClickHouseCluster, run_and_check +from test_library_bridge.test import create_dict_simple + +cluster = ClickHouseCluster(__file__) + +instance = cluster.add_instance( + "instance", + dictionaries=["configs/dictionaries/dict1.xml"], + main_configs=["configs/config.d/config.xml"], + stay_alive=True, +) + +@pytest.fixture(scope="module") +def ch_cluster(): + try: + cluster.start() + instance.query("CREATE DATABASE test") + + instance.copy_file_to_container( + os.path.join( + os.path.dirname(os.path.realpath(__file__)), "configs/dict_lib.cpp" + ), + "/etc/clickhouse-server/config.d/dictionaries_lib/dict_lib.cpp", + ) + + instance.query("SYSTEM RELOAD CONFIG") + + instance.exec_in_container( + [ + "bash", + "-c", + "/usr/bin/g++ -shared -o /etc/clickhouse-server/config.d/dictionaries_lib/dict_lib.so -fPIC /etc/clickhouse-server/config.d/dictionaries_lib/dict_lib.cpp", + ], + user="root", + ) + yield cluster + + finally: + cluster.shutdown() + + +def test_bridge_dies_with_parent(ch_cluster): + if instance.is_built_with_memory_sanitizer(): + pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") + if instance.is_built_with_address_sanitizer(): + pytest.skip( + "Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge" + ) + + create_dict_simple(instance) + result = instance.query("""select dictGet(lib_dict_c, 'value1', toUInt64(1));""") + assert result.strip() == "101" + + clickhouse_pid = instance.get_process_pid("clickhouse server") + bridge_pid = instance.get_process_pid("library-bridge") + assert clickhouse_pid is not None + assert bridge_pid is not None + + try: + instance.exec_in_container( + ["kill", str(clickhouse_pid)], privileged=True, user="root" + ) + except: + pass + + for i in range(30): + time.sleep(1) + clickhouse_pid = instance.get_process_pid("clickhouse server") + if clickhouse_pid is None: + break + + for i in range(30): + time.sleep(1) + bridge_pid = instance.get_process_pid("library-bridge") + if bridge_pid is None: + break + + if bridge_pid: + out = instance.exec_in_container( + ["gdb", "-p", str(bridge_pid), "--ex", "thread apply all bt", "--ex", "q"], + privileged=True, + user="root", + ) + logging.debug(f"Bridge is running, gdb output:\n{out}") + + try: + assert clickhouse_pid is None + assert bridge_pid is None + finally: + instance.start_clickhouse(20) + instance.query("DROP DICTIONARY lib_dict_c") diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 743e4ecd68a..e2cf4bdab8a 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -22,8 +22,7 @@ node1 = cluster.add_instance( "configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml", "configs/dictionaries/sqlite3_odbc_cached_dictionary.xml", "configs/dictionaries/postgres_odbc_hashed_dictionary.xml", - ], - stay_alive=True, + ] ) @@ -617,58 +616,6 @@ def test_postgres_insert(started_cluster): ) -def test_bridge_dies_with_parent(started_cluster): - skip_test_msan(node1) - - if node1.is_built_with_address_sanitizer(): - # TODO: Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge in this test and - # that's linked somehow with that we have replaced getauxval() in glibc-compatibility. - # The leak sanitizer calls getauxval() for its own purposes, and our replaced version doesn't seem to be equivalent in that case. - pytest.skip( - "Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge" - ) - - node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))") - - clickhouse_pid = node1.get_process_pid("clickhouse server") - bridge_pid = node1.get_process_pid("odbc-bridge") - assert clickhouse_pid is not None - assert bridge_pid is not None - - try: - node1.exec_in_container( - ["kill", str(clickhouse_pid)], privileged=True, user="root" - ) - except: - pass - - for i in range(30): - time.sleep(1) - clickhouse_pid = node1.get_process_pid("clickhouse server") - if clickhouse_pid is None: - break - - for i in range(30): - time.sleep(1) # just for sure, that odbc-bridge caught signal - bridge_pid = node1.get_process_pid("odbc-bridge") - if bridge_pid is None: - break - - if bridge_pid: - out = node1.exec_in_container( - ["gdb", "-p", str(bridge_pid), "--ex", "thread apply all bt", "--ex", "q"], - privileged=True, - user="root", - ) - logging.debug(f"Bridge is running, gdb output:\n{out}") - - try: - assert clickhouse_pid is None - assert bridge_pid is None - finally: - node1.start_clickhouse(20) - - def test_odbc_postgres_date_data_type(started_cluster): skip_test_msan(node1) diff --git a/tests/integration/test_odbc_interaction/test_exiled.py b/tests/integration/test_odbc_interaction/test_exiled.py new file mode 100644 index 00000000000..b0174a5b8de --- /dev/null +++ b/tests/integration/test_odbc_interaction/test_exiled.py @@ -0,0 +1,106 @@ +import time +import logging +import pytest + +from helpers.cluster import ClickHouseCluster, assert_eq_with_retry +from test_odbc_interaction.test import create_mysql_db, create_mysql_table, get_mysql_conn, skip_test_msan + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + with_odbc_drivers=True, + main_configs=["configs/openssl.xml", "configs/odbc_logging.xml"], + stay_alive=True, + dictionaries=["configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + sqlite_db = node1.odbc_drivers["SQLite3"]["Database"] + logging.debug(f"sqlite data received: {sqlite_db}") + node1.exec_in_container( + [ + "sqlite3", + sqlite_db, + "CREATE TABLE t2(id INTEGER PRIMARY KEY ASC, X INTEGER, Y, Z);", + ], + privileged=True, + user="root", + ) + + node1.exec_in_container( + ["sqlite3", sqlite_db, "INSERT INTO t2 values(1, 1, 2, 3);"], + privileged=True, + user="root", + ) + + node1.query("SYSTEM RELOAD DICTIONARY sqlite3_odbc_hashed") + + yield cluster + except Exception as ex: + logging.exception(ex) + raise ex + finally: + cluster.shutdown() + + +# This test kills ClickHouse server and ODBC bridge and in worst scenario +# may cause group test crashes. Thus, this test is executed in a separate "module" +# with separate environment. +def test_bridge_dies_with_parent(started_cluster): + skip_test_msan(node1) + + if node1.is_built_with_address_sanitizer(): + # TODO: Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge in this test and + # that's linked somehow with that we have replaced getauxval() in glibc-compatibility. + # The leak sanitizer calls getauxval() for its own purposes, and our replaced version doesn't seem to be equivalent in that case. + pytest.skip( + "Leak sanitizer falsely reports about a leak of 16 bytes in clickhouse-odbc-bridge" + ) + + assert_eq_with_retry( + node1, "select dictGetUInt8('sqlite3_odbc_hashed', 'Z', toUInt64(1))", "3" + ) + + clickhouse_pid = node1.get_process_pid("clickhouse server") + bridge_pid = node1.get_process_pid("odbc-bridge") + assert clickhouse_pid is not None + assert bridge_pid is not None + + try: + node1.exec_in_container( + ["kill", str(clickhouse_pid)], privileged=True, user="root" + ) + except: + pass + + for _ in range(30): + time.sleep(1) + clickhouse_pid = node1.get_process_pid("clickhouse server") + if clickhouse_pid is None: + break + + for _ in range(30): + time.sleep(1) # just for sure, that odbc-bridge caught signal + bridge_pid = node1.get_process_pid("odbc-bridge") + if bridge_pid is None: + break + + if bridge_pid: + out = node1.exec_in_container( + ["gdb", "-p", str(bridge_pid), "--ex", "thread apply all bt", "--ex", "q"], + privileged=True, + user="root", + ) + logging.debug(f"Bridge is running, gdb output:\n{out}") + + try: + assert clickhouse_pid is None + assert bridge_pid is None + finally: + node1.start_clickhouse(20) From 675fe5cd912b975df35ebbc3fc3cb9566d31ae5d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 2 Jan 2023 13:38:18 +0000 Subject: [PATCH 45/78] Automatic style fix --- tests/integration/test_library_bridge/test_exiled.py | 1 + tests/integration/test_odbc_interaction/test.py | 2 +- tests/integration/test_odbc_interaction/test_exiled.py | 7 ++++++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_library_bridge/test_exiled.py b/tests/integration/test_library_bridge/test_exiled.py index 1a7ead8d50b..56be1bec20a 100644 --- a/tests/integration/test_library_bridge/test_exiled.py +++ b/tests/integration/test_library_bridge/test_exiled.py @@ -16,6 +16,7 @@ instance = cluster.add_instance( stay_alive=True, ) + @pytest.fixture(scope="module") def ch_cluster(): try: diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index e2cf4bdab8a..f7c8f79fcec 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -22,7 +22,7 @@ node1 = cluster.add_instance( "configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml", "configs/dictionaries/sqlite3_odbc_cached_dictionary.xml", "configs/dictionaries/postgres_odbc_hashed_dictionary.xml", - ] + ], ) diff --git a/tests/integration/test_odbc_interaction/test_exiled.py b/tests/integration/test_odbc_interaction/test_exiled.py index b0174a5b8de..bdc819b8221 100644 --- a/tests/integration/test_odbc_interaction/test_exiled.py +++ b/tests/integration/test_odbc_interaction/test_exiled.py @@ -3,7 +3,12 @@ import logging import pytest from helpers.cluster import ClickHouseCluster, assert_eq_with_retry -from test_odbc_interaction.test import create_mysql_db, create_mysql_table, get_mysql_conn, skip_test_msan +from test_odbc_interaction.test import ( + create_mysql_db, + create_mysql_table, + get_mysql_conn, + skip_test_msan, +) cluster = ClickHouseCluster(__file__) From 5d4611bca47a7726e3771f54b9e38d40189cfe1c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 2 Jan 2023 14:45:51 +0100 Subject: [PATCH 46/78] Update .pylintrc --- .pylintrc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.pylintrc b/.pylintrc index 3fd991a53b8..1ebb0c86dd7 100644 --- a/.pylintrc +++ b/.pylintrc @@ -14,7 +14,7 @@ ignore-long-lines = (# )??$ [LOGGING] # Not to warn about lazy formatting in logging functions -logging-format-style=fstr +logging-format-style=new [MESSAGES CONTROL] disable = missing-docstring, From 21dfc9c15696519425b7c8e1c425e546f606c375 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 2 Jan 2023 14:54:52 +0100 Subject: [PATCH 47/78] Update .pylintrc --- .pylintrc | 4 ---- 1 file changed, 4 deletions(-) diff --git a/.pylintrc b/.pylintrc index 1ebb0c86dd7..b672cbfdfad 100644 --- a/.pylintrc +++ b/.pylintrc @@ -12,10 +12,6 @@ max-statements=200 [FORMAT] ignore-long-lines = (# )??$ -[LOGGING] -# Not to warn about lazy formatting in logging functions -logging-format-style=new - [MESSAGES CONTROL] disable = missing-docstring, too-few-public-methods, From a67878a03eef86398bb3767414bf9770acd09a06 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 2 Jan 2023 20:18:12 +0100 Subject: [PATCH 48/78] Fix workflows --- .github/workflows/master.yml | 43 ----------------------------- .github/workflows/pull_request.yml | 44 ------------------------------ 2 files changed, 87 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 89af99d7175..cd4ee482702 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -477,47 +477,6 @@ jobs: ########################################################################################## ##################################### SPECIAL BUILDS ##################################### ########################################################################################## - BuilderDebShared: - needs: [DockerHubPush] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_shared - EOF - - name: Download changed images - uses: actions/download-artifact@v3 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - submodules: true - - name: Build - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v3 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderBinClangTidy: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -937,7 +896,6 @@ jobs: - BuilderBinAmd64Compat - BuilderBinAarch64V80Compat - BuilderBinClangTidy - - BuilderDebShared runs-on: [self-hosted, style-checker] if: ${{ success() || failure() }} steps: @@ -3108,7 +3066,6 @@ jobs: - UnitTestsMsan - UnitTestsUBsan - UnitTestsReleaseClang - - SharedBuildSmokeTest - SQLancerTestRelease - SQLancerTestDebug runs-on: [self-hosted, style-checker] diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index a0bac29f166..df650187012 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -539,47 +539,6 @@ jobs: ########################################################################################## ##################################### SPECIAL BUILDS ##################################### ########################################################################################## - BuilderDebShared: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_shared - EOF - - name: Download changed images - uses: actions/download-artifact@v3 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - submodules: true - - name: Build - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v3 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderBinClangTidy: needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] @@ -987,12 +946,10 @@ jobs: - BuilderBinDarwin - BuilderBinDarwinAarch64 - BuilderBinFreeBSD - # - BuilderBinGCC - BuilderBinPPC64 - BuilderBinAmd64Compat - BuilderBinAarch64V80Compat - BuilderBinClangTidy - - BuilderDebShared runs-on: [self-hosted, style-checker] if: ${{ success() || failure() }} steps: @@ -4417,7 +4374,6 @@ jobs: - UnitTestsMsan - UnitTestsUBsan - UnitTestsReleaseClang - - SharedBuildSmokeTest - CompatibilityCheck - IntegrationTestsFlakyCheck - SQLancerTestRelease From d07f0c9e3f7a05b28fd48794e62be2d865d5157e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 2 Jan 2023 23:44:42 +0100 Subject: [PATCH 49/78] Fix GitHub --- docker/packager/binary/build.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 docker/packager/binary/build.sh diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh old mode 100644 new mode 100755 From 8125f3adc0de31bb04a5631958cc44a205a4e955 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 3 Jan 2023 01:17:28 +0100 Subject: [PATCH 50/78] More cases of OOM in Fuzzer --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 65e97c4308b..af479d04ebe 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -267,7 +267,7 @@ quit echo "Lost connection to server. See the logs." > description.txt fi - if grep -F --text 'Sanitizer: out-of-memory' description.txt + if grep -E --text 'Sanitizer: (out-of-memory|failed to allocate)' description.txt then # OOM of sanitizer is not a problem we can handle - treat it as success, but preserve the description. task_exit_code=0 From b7062c182ae27aa911acaf0707f6bea47336c57f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 3 Jan 2023 00:18:29 +0000 Subject: [PATCH 51/78] Fix: sorted DISTINCT with empty string --- src/Common/HashTable/ClearableHashSet.h | 24 +++++++++++++++++++ ...distinct_zero_size_key_bug_44831.reference | 1 + ...02515_distinct_zero_size_key_bug_44831.sql | 1 + 3 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/02515_distinct_zero_size_key_bug_44831.reference create mode 100644 tests/queries/0_stateless/02515_distinct_zero_size_key_bug_44831.sql diff --git a/src/Common/HashTable/ClearableHashSet.h b/src/Common/HashTable/ClearableHashSet.h index 371302dd8a9..292f0e4059a 100644 --- a/src/Common/HashTable/ClearableHashSet.h +++ b/src/Common/HashTable/ClearableHashSet.h @@ -48,6 +48,30 @@ struct ClearableHashTableCell : public BaseCell ClearableHashTableCell(const Key & key_, const State & state) : BaseCell(key_, state), version(state.version) {} }; +using StringRefBaseCell = HashSetCellWithSavedHash, ClearableHashSetState>; + +/// specialization for StringRef to allow zero size key (empty string) +template <> +struct ClearableHashTableCell : public StringRefBaseCell +{ + using State = ClearableHashSetState; + using value_type = typename StringRefBaseCell::value_type; + + UInt32 version; + + bool isZero(const State & state) const { return version != state.version; } + static bool isZero(const StringRef & key_, const State & state_) { return StringRefBaseCell::isZero(key_, state_); } + + /// Set the key value to zero. + void setZero() { version = 0; } + + /// Do I need to store the zero key separately (that is, can a zero key be inserted into the hash table). + static constexpr bool need_zero_value_storage = true; + + ClearableHashTableCell() { } //-V730 /// NOLINT + ClearableHashTableCell(const StringRef & key_, const State & state) : StringRefBaseCell(key_, state), version(state.version) { } +}; + template < typename Key, typename Hash = DefaultHash, diff --git a/tests/queries/0_stateless/02515_distinct_zero_size_key_bug_44831.reference b/tests/queries/0_stateless/02515_distinct_zero_size_key_bug_44831.reference new file mode 100644 index 00000000000..8908480c027 --- /dev/null +++ b/tests/queries/0_stateless/02515_distinct_zero_size_key_bug_44831.reference @@ -0,0 +1 @@ +\N diff --git a/tests/queries/0_stateless/02515_distinct_zero_size_key_bug_44831.sql b/tests/queries/0_stateless/02515_distinct_zero_size_key_bug_44831.sql new file mode 100644 index 00000000000..96072b281db --- /dev/null +++ b/tests/queries/0_stateless/02515_distinct_zero_size_key_bug_44831.sql @@ -0,0 +1 @@ +SELECT DISTINCT NULL, if(number > 0, 't', '') AS res FROM numbers(1) ORDER BY res; From c4431e993140c15aae1e889be691bc2c3fa15f3e Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 3 Jan 2023 02:21:26 +0000 Subject: [PATCH 52/78] Fix MSan build --- src/Dictionaries/RangeHashedDictionary.cpp | 89 +-------------- src/Dictionaries/RangeHashedDictionary.h | 3 + .../registerRangeHashedDictionary.cpp | 101 ++++++++++++++++++ src/Functions/FunctionsHashing.cpp | 10 -- src/Functions/FunctionsHashingSSL.cpp | 25 +++++ 5 files changed, 131 insertions(+), 97 deletions(-) create mode 100644 src/Dictionaries/registerRangeHashedDictionary.cpp create mode 100644 src/Functions/FunctionsHashingSSL.cpp diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 4ebef1c6737..95bcd1076d5 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -15,7 +15,6 @@ #include #include -#include #include @@ -1017,91 +1016,7 @@ Pipe RangeHashedDictionary::read(const Names & column_names return result; } -template -static DictionaryPtr createRangeHashedDictionary(const std::string & full_name, - const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - DictionarySourcePtr source_ptr) -{ - static constexpr auto layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "range_hashed" : "complex_key_range_hashed"; - - if constexpr (dictionary_key_type == DictionaryKeyType::Simple) - { - if (dict_struct.key) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout 'range_hashed'"); - } - else - { - if (dict_struct.id) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for dictionary of layout 'complex_key_range_hashed'"); - } - - if (!dict_struct.range_min || !dict_struct.range_max) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "{}: dictionary of layout '{}' requires .structure.range_min and .structure.range_max", - full_name, - layout_name); - - const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); - const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; - const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - - String dictionary_layout_prefix = config_prefix + ".layout." + layout_name; - const bool convert_null_range_bound_to_open = config.getBool(dictionary_layout_prefix + ".convert_null_range_bound_to_open", true); - String range_lookup_strategy = config.getString(dictionary_layout_prefix + ".range_lookup_strategy", "min"); - RangeHashedDictionaryLookupStrategy lookup_strategy = RangeHashedDictionaryLookupStrategy::min; - - if (range_lookup_strategy == "min") - lookup_strategy = RangeHashedDictionaryLookupStrategy::min; - else if (range_lookup_strategy == "max") - lookup_strategy = RangeHashedDictionaryLookupStrategy::max; - - RangeHashedDictionaryConfiguration configuration - { - .convert_null_range_bound_to_open = convert_null_range_bound_to_open, - .lookup_strategy = lookup_strategy, - .require_nonempty = require_nonempty - }; - - DictionaryPtr result = std::make_unique>( - dict_id, - dict_struct, - std::move(source_ptr), - dict_lifetime, - configuration); - - return result; -} - -void registerDictionaryRangeHashed(DictionaryFactory & factory) -{ - auto create_layout_simple = [=](const std::string & full_name, - const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - DictionarySourcePtr source_ptr, - ContextPtr /* global_context */, - bool /*created_from_ddl*/) -> DictionaryPtr - { - return createRangeHashedDictionary(full_name, dict_struct, config, config_prefix, std::move(source_ptr)); - }; - - factory.registerLayout("range_hashed", create_layout_simple, false); - - auto create_layout_complex = [=](const std::string & full_name, - const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - DictionarySourcePtr source_ptr, - ContextPtr /* context */, - bool /*created_from_ddl*/) -> DictionaryPtr - { - return createRangeHashedDictionary(full_name, dict_struct, config, config_prefix, std::move(source_ptr)); - }; - - factory.registerLayout("complex_key_range_hashed", create_layout_complex, true); -} +template class RangeHashedDictionary; +template class RangeHashedDictionary; } diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 78d62e9d7de..257bfcd528c 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -248,4 +248,7 @@ private: Arena string_arena; }; +extern template class RangeHashedDictionary; +extern template class RangeHashedDictionary; + } diff --git a/src/Dictionaries/registerRangeHashedDictionary.cpp b/src/Dictionaries/registerRangeHashedDictionary.cpp new file mode 100644 index 00000000000..93784a0709e --- /dev/null +++ b/src/Dictionaries/registerRangeHashedDictionary.cpp @@ -0,0 +1,101 @@ +#include "RangeHashedDictionary.h" +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; + extern const int BAD_ARGUMENTS; +} + +template +static DictionaryPtr createRangeHashedDictionary(const std::string & full_name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr) +{ + static constexpr auto layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "range_hashed" : "complex_key_range_hashed"; + + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + { + if (dict_struct.key) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout 'range_hashed'"); + } + else + { + if (dict_struct.id) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for dictionary of layout 'complex_key_range_hashed'"); + } + + if (!dict_struct.range_min || !dict_struct.range_max) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "{}: dictionary of layout '{}' requires .structure.range_min and .structure.range_max", + full_name, + layout_name); + + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); + const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + + String dictionary_layout_prefix = config_prefix + ".layout." + layout_name; + const bool convert_null_range_bound_to_open = config.getBool(dictionary_layout_prefix + ".convert_null_range_bound_to_open", true); + String range_lookup_strategy = config.getString(dictionary_layout_prefix + ".range_lookup_strategy", "min"); + RangeHashedDictionaryLookupStrategy lookup_strategy = RangeHashedDictionaryLookupStrategy::min; + + if (range_lookup_strategy == "min") + lookup_strategy = RangeHashedDictionaryLookupStrategy::min; + else if (range_lookup_strategy == "max") + lookup_strategy = RangeHashedDictionaryLookupStrategy::max; + + RangeHashedDictionaryConfiguration configuration + { + .convert_null_range_bound_to_open = convert_null_range_bound_to_open, + .lookup_strategy = lookup_strategy, + .require_nonempty = require_nonempty + }; + + DictionaryPtr result = std::make_unique>( + dict_id, + dict_struct, + std::move(source_ptr), + dict_lifetime, + configuration); + + return result; +} + +void registerDictionaryRangeHashed(DictionaryFactory & factory) +{ + auto create_layout_simple = [=](const std::string & full_name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr, + ContextPtr /* global_context */, + bool /*created_from_ddl*/) -> DictionaryPtr + { + return createRangeHashedDictionary(full_name, dict_struct, config, config_prefix, std::move(source_ptr)); + }; + + factory.registerLayout("range_hashed", create_layout_simple, false); + + auto create_layout_complex = [=](const std::string & full_name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr, + ContextPtr /* context */, + bool /*created_from_ddl*/) -> DictionaryPtr + { + return createRangeHashedDictionary(full_name, dict_struct, config, config_prefix, std::move(source_ptr)); + }; + + factory.registerLayout("complex_key_range_hashed", create_layout_complex, true); +} + +} diff --git a/src/Functions/FunctionsHashing.cpp b/src/Functions/FunctionsHashing.cpp index 8f616b0be94..c51898b271b 100644 --- a/src/Functions/FunctionsHashing.cpp +++ b/src/Functions/FunctionsHashing.cpp @@ -8,16 +8,6 @@ namespace DB REGISTER_FUNCTION(Hashing) { -#if USE_SSL - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); -#endif factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsHashingSSL.cpp b/src/Functions/FunctionsHashingSSL.cpp new file mode 100644 index 00000000000..4f8c7808c3b --- /dev/null +++ b/src/Functions/FunctionsHashingSSL.cpp @@ -0,0 +1,25 @@ +#include "config.h" + +#if USE_SSL + +#include "FunctionsHashing.h" +#include + + +namespace DB +{ + +REGISTER_FUNCTION(HashingSSL) +{ + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); +} +} + +#endif From 8a5fd7f96d352c28c88cc0c89f8f49c4dd1e8e5d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 3 Jan 2023 11:44:26 +0100 Subject: [PATCH 53/78] Added comments and addressed review comments - Allow macro substitution in endpoint --- src/Coordination/KeeperDispatcher.h | 2 ++ src/Coordination/KeeperSnapshotManagerS3.cpp | 3 +-- src/Coordination/KeeperSnapshotManagerS3.h | 4 +++- src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp | 3 +-- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 3 +-- src/Disks/ObjectStorages/S3/diskSettings.cpp | 3 +-- src/Disks/ObjectStorages/S3/registerDiskS3.cpp | 3 +-- src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp | 3 +-- 8 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 2b4cd0c2b49..ff902d8e036 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -111,6 +111,7 @@ public: /// Initialization from config. /// standalone_keeper -- we are standalone keeper application (not inside clickhouse server) + /// 'macros' are used to substitute macros in endpoint of disks void initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper, bool start_async, const MultiVersion::Version & macros); void startServer(); @@ -126,6 +127,7 @@ public: /// Registered in ConfigReloader callback. Add new configuration changes to /// update_configuration_queue. Keeper Dispatcher apply them asynchronously. + /// 'macros' are used to substitute macros in endpoint of disks void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); /// Shutdown internal keeper parts (server, state machine, log storage, etc) diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 5fac71ccbbc..b793cef4b94 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -65,8 +65,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo auto auth_settings = S3::AuthSettings::loadFromConfig(config_prefix, config); - String endpoint = config.getString(config_prefix + ".endpoint"); - endpoint = macros->expand(endpoint); + String endpoint = macros->expand(config.getString(config_prefix + ".endpoint")); auto new_uri = S3::URI{endpoint}; { diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index 592217fbe66..197f528b192 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -23,10 +23,12 @@ class KeeperSnapshotManagerS3 public: KeeperSnapshotManagerS3(); + /// 'macros' are used to substitute macros in endpoint of disks void updateS3Configuration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void uploadSnapshot(const std::string & path, bool async_upload = true); - void startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macro); + /// 'macros' are used to substitute macros in endpoint of disks + void startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void shutdown(); private: using SnapshotS3Queue = ConcurrentBoundedQueue; diff --git a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp index 43b837c00bc..db134f3dcba 100644 --- a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp +++ b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp @@ -23,8 +23,7 @@ void registerDiskHDFS(DiskFactory & factory, bool global_skip_access_check) ContextPtr context, const DisksMap & /*map*/) -> DiskPtr { - String endpoint = config.getString(config_prefix + ".endpoint"); - endpoint = context->getMacros()->expand(endpoint); + String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); String uri{endpoint}; checkHDFSURL(uri); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index e549b5f8dd0..3c620ca819e 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -637,8 +637,7 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( { auto new_s3_settings = getSettings(config, config_prefix, context); auto new_client = getClient(config, config_prefix, context, *new_s3_settings); - String endpoint = config.getString(config_prefix + ".endpoint"); - endpoint = context->getMacros()->expand(endpoint); + String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); return std::make_unique( std::move(new_client), std::move(new_s3_settings), version_id, s3_capabilities, new_namespace, diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 5bb994b89bd..d5c6b03082f 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -122,8 +122,7 @@ std::unique_ptr getClient( settings.request_settings.get_request_throttler, settings.request_settings.put_request_throttler); - String endpoint = config.getString(config_prefix + ".endpoint"); - endpoint = context->getMacros()->expand(endpoint); + String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); S3::URI uri(endpoint); if (uri.key.back() != '/') throw Exception("S3 path must ends with '/', but '" + uri.key + "' doesn't.", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 70d94fc3020..1c192a0d89c 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -105,8 +105,7 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) ContextPtr context, const DisksMap & /*map*/) -> DiskPtr { - String endpoint = config.getString(config_prefix + ".endpoint"); - endpoint = context->getMacros()->expand(endpoint); + String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); S3::URI uri(endpoint); if (uri.key.empty()) diff --git a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp index 33480ca3f0a..5f3d9dfc1d9 100644 --- a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp +++ b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp @@ -26,8 +26,7 @@ void registerDiskWebServer(DiskFactory & factory, bool global_skip_access_check) ContextPtr context, const DisksMap & /*map*/) -> DiskPtr { - String endpoint = config.getString(config_prefix + ".endpoint"); - endpoint = context->getMacros()->expand(endpoint); + String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); String uri{endpoint}; bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); From 795a677aaa97d5db2c40af03c8307f159bb2fd28 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 3 Jan 2023 15:26:02 +0100 Subject: [PATCH 54/78] Fix non-detected Errors in performance tests message --- tests/ci/performance_comparison_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index 667c80110cd..d0c84d56496 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -236,7 +236,7 @@ if __name__ == "__main__": # TODO: Remove me, always green mode for the first time, unless errors status = "success" - if "errors" in message: + if "errors" in message.lower(): status = "failure" # TODO: Remove until here except Exception: From 6c8dbcc04040b83d8966f6fc76883d84ca828a6d Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 3 Jan 2023 16:27:51 +0100 Subject: [PATCH 55/78] Fix flaky test 01459_manual_write_to_replicas_quorum_detach_attach and several typos --- src/Compression/tests/gtest_compressionCodec.cpp | 2 +- src/Functions/FunctionsMiscellaneous.h | 2 +- src/Functions/PerformanceAdaptors.h | 2 +- src/Functions/rowNumberInAllBlocks.cpp | 2 +- src/Functions/runningAccumulate.cpp | 2 +- src/Functions/runningDifference.h | 4 ++-- .../01459_manual_write_to_replicas_quorum_detach_attach.sh | 7 ++++++- 7 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index 4765aeb45ed..b4c29bf9ce6 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -1203,7 +1203,7 @@ auto DDperformanceTestSequence() + generateSeq(G(SameValueGenerator(42)), 0, times); // best } -// prime numbers in ascending order with some random repitions hit all the cases of Gorilla. +// prime numbers in ascending order with some random repetitions hit all the cases of Gorilla. auto PrimesWithMultiplierGenerator = [](int multiplier = 1) { return [multiplier](auto i) diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index da90889fd4d..d5c92c5d666 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -205,7 +205,7 @@ public: const String & expression_return_name_) : expression_actions(std::move(expression_actions_)) { - /// Check that expression does not contain unusual actions that will break columnss structure. + /// Check that expression does not contain unusual actions that will break columns structure. for (const auto & action : expression_actions->getActions()) if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception("Expression with arrayJoin or other unusual action cannot be captured", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Functions/PerformanceAdaptors.h b/src/Functions/PerformanceAdaptors.h index 5b690d83805..9aed3893fff 100644 --- a/src/Functions/PerformanceAdaptors.h +++ b/src/Functions/PerformanceAdaptors.h @@ -210,7 +210,7 @@ public: throw Exception("There are no available implementations for function " "TODO(dakovalkov): add name", ErrorCodes::NO_SUITABLE_FUNCTION_IMPLEMENTATION); - /// Statistics shouldn't rely on small columnss. + /// Statistics shouldn't rely on small columns. bool considerable = (input_rows_count > 1000); ColumnPtr res; diff --git a/src/Functions/rowNumberInAllBlocks.cpp b/src/Functions/rowNumberInAllBlocks.cpp index 28d61afcdc1..750f8691cbb 100644 --- a/src/Functions/rowNumberInAllBlocks.cpp +++ b/src/Functions/rowNumberInAllBlocks.cpp @@ -10,7 +10,7 @@ namespace DB namespace { -/** Incremental number of row within all columnss passed to this function. */ +/** Incremental number of row within all columns passed to this function. */ class FunctionRowNumberInAllBlocks : public IFunction { private: diff --git a/src/Functions/runningAccumulate.cpp b/src/Functions/runningAccumulate.cpp index 436637fbe56..877ad877f56 100644 --- a/src/Functions/runningAccumulate.cpp +++ b/src/Functions/runningAccumulate.cpp @@ -27,7 +27,7 @@ namespace * Takes state of aggregate function (example runningAccumulate(uniqState(UserID))), * and for each row of columns, return result of aggregate function on merge of states of all previous rows and current row. * - * So, result of function depends on partition of data to columnss and on order of data in columns. + * So, result of function depends on partition of data to columns and on order of data in columns. */ class FunctionRunningAccumulate : public IFunction { diff --git a/src/Functions/runningDifference.h b/src/Functions/runningDifference.h index cf534b30c90..5e58d0d8aaf 100644 --- a/src/Functions/runningDifference.h +++ b/src/Functions/runningDifference.h @@ -38,13 +38,13 @@ struct FunctionRunningDifferenceName }; /** Calculate difference of consecutive values in columns. - * So, result of function depends on partition of data to columnss and on order of data in columns. + * So, result of function depends on partition of data to columns and on order of data in columns. */ template class FunctionRunningDifferenceImpl : public IFunction { private: - /// It is possible to track value from previous columns, to calculate continuously across all columnss. Not implemented. + /// It is possible to track value from previous columns, to calculate continuously across all columns. Not implemented. template static NO_SANITIZE_UNDEFINED void process(const PaddedPODArray & src, PaddedPODArray & dst, const NullMap * null_map) diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh index b97fcece267..01c88336282 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh @@ -42,5 +42,10 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" + # We filter out 'Removing temporary directory' on table DROP because in this test + # we constantly DETACH and ATTACH tables. So some replica can start fetching some part + # and other replica can be DETACHed during fetch. We will get unfinished tmp directory + # which should be removed in background, but it's async operation so the tmp directory can + # left on disk until table DROP. + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" 2>&1 | grep -v 'Removing temporary directory' ||: done From 41cdf42276980a10597493b3df1052141632bb6c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 3 Jan 2023 16:30:36 +0100 Subject: [PATCH 56/78] Upload status files to S3 report for bugfix check --- .github/workflows/pull_request.yml | 2 +- tests/ci/bugfix_validate_check.py | 2 +- tests/ci/functional_test_check.py | 11 +++++++---- tests/ci/integration_test_check.py | 5 +++-- 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index efb7d50dd28..fade183aeaa 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -2639,7 +2639,7 @@ jobs: python3 functional_test_check.py "Stateless $CHECK_NAME" "$KILL_TIMEOUT" \ --validate-bugfix --post-commit-status=file || echo 'ignore exit code' - python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/post_commit_status.tsv" "${TEMP_PATH}/integration/post_commit_status.tsv" + python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/functional_commit_status.tsv" "${TEMP_PATH}/integration/integration_commit_status.tsv" - name: Cleanup if: always() run: | diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index 5050d6c38cb..6bdf3b1f7d2 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -77,7 +77,7 @@ def main(args): pr_info.number, pr_info.sha, test_results, - [], + args.status, check_name_with_group, ) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index a9c692db0d0..e7689a198cd 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -203,6 +203,7 @@ if __name__ == "__main__": temp_path = TEMP_PATH repo_path = REPO_COPY reports_path = REPORTS_PATH + post_commit_path = os.path.join(temp_path, "functional_commit_status.tsv") args = parse_args() check_name = args.check_name @@ -227,7 +228,7 @@ if __name__ == "__main__": if validate_bugfix_check and "pr-bugfix" not in pr_info.labels: if args.post_commit_status == "file": post_commit_status_to_file( - os.path.join(temp_path, "post_commit_status.tsv"), + post_commit_path, f"Skipped (no pr-bugfix in {pr_info.labels})", "success", "null", @@ -264,9 +265,11 @@ if __name__ == "__main__": state=state, ) elif args.post_commit_status == "file": - fpath = os.path.join(temp_path, "post_commit_status.tsv") post_commit_status_to_file( - fpath, description=NO_CHANGES_MSG, state=state, report_url="null" + post_commit_path, + description=NO_CHANGES_MSG, + state=state, + report_url="null", ) sys.exit(0) @@ -348,7 +351,7 @@ if __name__ == "__main__": ) elif args.post_commit_status == "file": post_commit_status_to_file( - os.path.join(temp_path, "post_commit_status.tsv"), + post_commit_path, description, state, report_url, diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 31fae9a578b..86f38a5b8b4 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -148,6 +148,7 @@ if __name__ == "__main__": stopwatch = Stopwatch() temp_path = TEMP_PATH + post_commit_path = os.path.join(temp_path, "integration_commit_status.tsv") repo_path = REPO_COPY reports_path = REPORTS_PATH @@ -180,7 +181,7 @@ if __name__ == "__main__": if validate_bugfix_check and "pr-bugfix" not in pr_info.labels: if args.post_commit_status == "file": post_commit_status_to_file( - os.path.join(temp_path, "post_commit_status.tsv"), + post_commit_path, f"Skipped (no pr-bugfix in {pr_info.labels})", "success", "null", @@ -275,7 +276,7 @@ if __name__ == "__main__": ) elif args.post_commit_status == "file": post_commit_status_to_file( - os.path.join(temp_path, "post_commit_status.tsv"), + post_commit_path, description, state, report_url, From ef55fab71f11c14c27915d76d56ff1e743c8b317 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 3 Jan 2023 16:34:41 +0100 Subject: [PATCH 57/78] Use thiner runners for bugfix checks --- .github/workflows/pull_request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index fade183aeaa..f6ee1d1d917 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -2603,7 +2603,7 @@ jobs: sudo rm -fr "$TEMP_PATH" TestsBugfixCheck: needs: [CheckLabels, StyleCheck] - runs-on: [self-hosted, stress-tester] + runs-on: [self-hosted, func-tester] steps: - name: Set envs run: | From 48fe25aae75d356cab4b1ecc53238b7337300f9f Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 3 Jan 2023 16:50:54 +0100 Subject: [PATCH 58/78] Fix flaky test 02503_insert_storage_snapshot --- .../02503_insert_storage_snapshot.reference | 2 +- .../02503_insert_storage_snapshot.sh | 17 ++++++++++++++++- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02503_insert_storage_snapshot.reference b/tests/queries/0_stateless/02503_insert_storage_snapshot.reference index 4e07416f18a..9f9dfd29dc6 100644 --- a/tests/queries/0_stateless/02503_insert_storage_snapshot.reference +++ b/tests/queries/0_stateless/02503_insert_storage_snapshot.reference @@ -1 +1 @@ -all_1_1_0 1 1 +"all_1_1_0",1,1 diff --git a/tests/queries/0_stateless/02503_insert_storage_snapshot.sh b/tests/queries/0_stateless/02503_insert_storage_snapshot.sh index af2952839df..b494adeb785 100755 --- a/tests/queries/0_stateless/02503_insert_storage_snapshot.sh +++ b/tests/queries/0_stateless/02503_insert_storage_snapshot.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel set -e @@ -13,7 +14,21 @@ $CLICKHOUSE_CLIENT -q "INSERT INTO t_insert_storage_snapshot VALUES (1)" query_id="$CLICKHOUSE_DATABASE-$RANDOM" $CLICKHOUSE_CLIENT --query_id $query_id -q "INSERT INTO t_insert_storage_snapshot SELECT sleep(1) FROM numbers(1000) SETTINGS max_block_size = 1" 2>/dev/null & -$CLICKHOUSE_CLIENT -q "SELECT name, active, refcount FROM system.parts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_insert_storage_snapshot'" +counter=0 retries=60 + +# There can be different background processes which can hold the references to parts +# for a short period of time. To avoid flakyness we check that refcount became 1 at least once during long INSERT query. +# It proves that the INSERT query doesn't hold redundant references to parts. +while [[ $counter -lt $retries ]]; do + query_result=$($CLICKHOUSE_CLIENT -q "SELECT name, active, refcount FROM system.parts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_insert_storage_snapshot' FORMAT CSV") + if [ "$query_result" == '"all_1_1_0",1,1' ]; then + echo "$query_result" + break; + fi + sleep 0.1 + ((++counter)) +done + $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query_id = '$query_id' SYNC" >/dev/null wait From 0785bf6b295b4e60a5ed81f43787185fd203d0bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 3 Jan 2023 17:05:54 +0100 Subject: [PATCH 59/78] Do not throw DB::Exception when folders do not exist --- src/Common/filesystemHelpers.cpp | 6 ++++-- src/Databases/DatabaseOnDisk.cpp | 16 ++++++++++++---- src/Storages/MergeTree/MergeTreeData.cpp | 14 ++------------ 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 43f88dd7faa..aabe7422d14 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -352,7 +352,8 @@ time_t getModificationTime(const std::string & path) struct stat st; if (stat(path.c_str(), &st) == 0) return st.st_mtime; - DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::CANNOT_STAT); + std::error_code m_ec(errno, std::generic_category()); + throw fs::filesystem_error("Cannot check modification time for file", path, m_ec); } time_t getChangeTime(const std::string & path) @@ -360,7 +361,8 @@ time_t getChangeTime(const std::string & path) struct stat st; if (stat(path.c_str(), &st) == 0) return st.st_ctime; - DB::throwFromErrnoWithPath("Cannot check change time for file: " + path, path, DB::ErrorCodes::CANNOT_STAT); + std::error_code m_ec(errno, std::generic_category()); + throw fs::filesystem_error("Cannot check change time for file", path, m_ec); } Poco::Timestamp getModificationTimestamp(const std::string & path) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 197f1a0543b..b1e265c084c 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -539,11 +539,19 @@ String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_name) const { String table_metadata_path = getObjectMetadataPath(object_name); - - if (fs::exists(table_metadata_path)) + try + { return FS::getModificationTime(table_metadata_path); - else - return static_cast(0); + } + catch (const fs::filesystem_error & e) + { + if (e.code() == std::errc::no_such_file_or_directory) + { + return static_cast(0); + } + else + throw; + } } void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 674d235574e..84ce759607c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1595,7 +1595,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) /// (Only files on the first level of nesting are considered). static bool isOldPartDirectory(const DiskPtr & disk, const String & directory_path, time_t threshold) { - if (disk->getLastModified(directory_path).epochTime() >= threshold) + if (!disk->isDirectory(directory_path) || disk->getLastModified(directory_path).epochTime() >= threshold) return false; for (auto it = disk->iterateDirectory(directory_path); it->isValid(); it->next()) @@ -1645,7 +1645,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lif try { - if (disk->isDirectory(it->path()) && isOldPartDirectory(disk, it->path(), deadline)) + if (isOldPartDirectory(disk, it->path(), deadline)) { if (temporary_parts.contains(basename)) { @@ -1673,16 +1673,6 @@ size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lif } } } - /// see getModificationTime() - catch (const ErrnoException & e) - { - if (e.getErrno() == ENOENT) - { - /// If the file is already deleted, do nothing. - } - else - throw; - } catch (const fs::filesystem_error & e) { if (e.code() == std::errc::no_such_file_or_directory) From cfb6feffdee7363e03bc720464aebb939873e702 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 3 Jan 2023 18:35:24 +0000 Subject: [PATCH 60/78] What happens if I remove these 139 lines of code? --- CMakeLists.txt | 16 +------ contrib/boringssl-cmake/CMakeLists.txt | 7 --- contrib/sentry-native-cmake/CMakeLists.txt | 8 +--- docker/packager/packager | 21 ++------- docs/en/development/continuous-integration.md | 1 - docs/ru/development/developer-instruction.md | 6 --- docs/zh/development/continuous-integration.md | 1 - programs/CMakeLists.txt | 8 +--- programs/main.cpp | 4 +- rust/skim/CMakeLists.txt | 10 +---- src/CMakeLists.txt | 45 +++---------------- src/Common/remapExecutable.cpp | 2 +- src/Functions/CMakeLists.txt | 13 ++---- tests/ci/build_check.py | 4 -- tests/ci/build_download_helper.py | 6 --- tests/ci/ci_config.py | 10 ----- 16 files changed, 23 insertions(+), 139 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 66fdaa6a765..6239fe94f1c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -74,17 +74,6 @@ message (STATUS "CMAKE_BUILD_TYPE: ${CMAKE_BUILD_TYPE}") string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) option(USE_STATIC_LIBRARIES "Disable to use shared libraries" ON) -# DEVELOPER ONLY. -# Faster linking if turned on. -option(SPLIT_SHARED_LIBRARIES "Keep all internal libraries as separate .so files" OFF) - -if (USE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) - message(FATAL_ERROR "SPLIT_SHARED_LIBRARIES=1 must not be used together with USE_STATIC_LIBRARIES=1") -endif() - -if (NOT USE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) - set(BUILD_SHARED_LIBS 1 CACHE INTERNAL "") -endif () if (USE_STATIC_LIBRARIES) list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) @@ -171,7 +160,7 @@ option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) option(ENABLE_BENCHMARKS "Build all benchmark programs in 'benchmarks' subdirectories" OFF) -if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND USE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND NOT USE_MUSL) +if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND USE_STATIC_LIBRARIES AND NOT USE_MUSL) # Only for Linux, x86_64 or aarch64. option(GLIBC_COMPATIBILITY "Enable compatibility with older glibc libraries." ON) elseif(GLIBC_COMPATIBILITY) @@ -506,8 +495,7 @@ endif () message (STATUS "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE} ; - USE_STATIC_LIBRARIES=${USE_STATIC_LIBRARIES} - SPLIT_SHARED_LIBRARIES=${SPLIT_SHARED_LIBRARIES}") + USE_STATIC_LIBRARIES=${USE_STATIC_LIBRARIES}") include (GNUInstallDirs) diff --git a/contrib/boringssl-cmake/CMakeLists.txt b/contrib/boringssl-cmake/CMakeLists.txt index 9a1f667f52e..828919476a7 100644 --- a/contrib/boringssl-cmake/CMakeLists.txt +++ b/contrib/boringssl-cmake/CMakeLists.txt @@ -139,13 +139,6 @@ if(NOT OPENSSL_NO_ASM) endif() endif() -if(BUILD_SHARED_LIBS) - add_definitions(-DBORINGSSL_SHARED_LIBRARY) - # Enable position-independent code globally. This is needed because - # some library targets are OBJECT libraries. - set(CMAKE_POSITION_INDEPENDENT_CODE TRUE) -endif() - set( CRYPTO_ios_aarch64_SOURCES diff --git a/contrib/sentry-native-cmake/CMakeLists.txt b/contrib/sentry-native-cmake/CMakeLists.txt index 520fa176b91..377f955f856 100644 --- a/contrib/sentry-native-cmake/CMakeLists.txt +++ b/contrib/sentry-native-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -if (NOT OS_FREEBSD AND NOT SPLIT_SHARED_LIBRARIES AND NOT (OS_DARWIN AND COMPILER_CLANG)) +if (NOT OS_FREEBSD AND NOT (OS_DARWIN AND COMPILER_CLANG)) option (ENABLE_SENTRY "Enable Sentry" ${ENABLE_LIBRARIES}) else() option (ENABLE_SENTRY "Enable Sentry" OFF) @@ -51,11 +51,7 @@ endif() add_library(_sentry ${SRCS}) -if(BUILD_SHARED_LIBS) - target_compile_definitions(_sentry PRIVATE SENTRY_BUILD_SHARED) -else() - target_compile_definitions(_sentry PUBLIC SENTRY_BUILD_STATIC) -endif() +target_compile_definitions(_sentry PUBLIC SENTRY_BUILD_STATIC) target_link_libraries(_sentry PRIVATE ch_contrib::curl pthread) target_include_directories(_sentry PUBLIC "${SRC_DIR}/include" PRIVATE "${SRC_DIR}/src") diff --git a/docker/packager/packager b/docker/packager/packager index 716071fcac6..e097c003cf9 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -100,12 +100,11 @@ def run_docker_image_with_env( subprocess.check_call(cmd, shell=True) -def is_release_build(build_type, package_type, sanitizer, shared_libraries): +def is_release_build(build_type, package_type, sanitizer): return ( build_type == "" and package_type == "deb" and sanitizer == "" - and not shared_libraries ) @@ -116,7 +115,6 @@ def parse_env_variables( package_type, cache, distcc_hosts, - shared_libraries, clang_tidy, version, author, @@ -218,7 +216,7 @@ def parse_env_variables( cmake_flags.append("-DCMAKE_INSTALL_PREFIX=/usr") cmake_flags.append("-DCMAKE_INSTALL_SYSCONFDIR=/etc") cmake_flags.append("-DCMAKE_INSTALL_LOCALSTATEDIR=/var") - if is_release_build(build_type, package_type, sanitizer, shared_libraries): + if is_release_build(build_type, package_type, sanitizer): cmake_flags.append("-DSPLIT_DEBUG_SYMBOLS=ON") result.append("WITH_PERFORMANCE=1") if is_cross_arm: @@ -231,12 +229,10 @@ def parse_env_variables( cmake_flags.append(f"-DCMAKE_C_COMPILER={cc}") cmake_flags.append(f"-DCMAKE_CXX_COMPILER={cxx}") - # Create combined output archive for shared library build and for performance tests. + # Create combined output archive for performance tests. if package_type == "coverity": result.append("COMBINED_OUTPUT=coverity") result.append('COVERITY_TOKEN="$COVERITY_TOKEN"') - elif shared_libraries: - result.append("COMBINED_OUTPUT=shared_build") if sanitizer: result.append(f"SANITIZER={sanitizer}") @@ -285,15 +281,6 @@ def parse_env_variables( result.append("BINARY_OUTPUT=tests") cmake_flags.append("-DENABLE_TESTS=1") - if shared_libraries: - cmake_flags.append("-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1") - # We can't always build utils because it requires too much space, but - # we have to build them at least in some way in CI. The shared library - # build is probably the least heavy disk-wise. - cmake_flags.append("-DENABLE_UTILS=1") - # utils are not included into clickhouse-bundle, so build everything - build_target = "all" - if clang_tidy: cmake_flags.append("-DENABLE_CLANG_TIDY=1") cmake_flags.append("-DENABLE_TESTS=1") @@ -371,7 +358,6 @@ if __name__ == "__main__": default="", ) - parser.add_argument("--shared-libraries", action="store_true") parser.add_argument("--clang-tidy", action="store_true") parser.add_argument("--cache", choices=("ccache", "distcc", ""), default="") parser.add_argument( @@ -424,7 +410,6 @@ if __name__ == "__main__": args.package_type, args.cache, args.distcc_hosts, - args.shared_libraries, args.clang_tidy, args.version, args.author, diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 68dc1b57044..ef3efa75d66 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -129,7 +129,6 @@ Builds ClickHouse in various configurations for use in further steps. You have t - `clickhouse`: Main built binary. - `clickhouse-odbc-bridge` - `unit_tests_dbms`: GoogleTest binary with ClickHouse unit tests. - - `shared_build.tgz`: build with shared libraries. - `performance.tgz`: Special package for performance tests. diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 0713fe42f38..80472178ae2 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -169,12 +169,6 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" cmake -D CMAKE_BUILD_TYPE=Debug .. -В случае использования на разработческой машине старого HDD или SSD, а также при желании использовать меньше места для артефактов сборки можно использовать следующую команду: -```bash -cmake -DUSE_DEBUG_HELPERS=1 -DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 .. -``` -При этом надо учесть, что получаемые в результате сборки исполнимые файлы будут динамически слинкованы с библиотеками, и поэтому фактически станут непереносимыми на другие компьютеры (либо для этого нужно будет предпринять значительно больше усилий по сравнению со статической сборкой). Плюсом же в данном случае является значительно меньшее время сборки (это проявляется не на первой сборке, а на последующих, после внесения изменений в исходный код - тратится меньшее время на линковку по сравнению со статической сборкой) и значительно меньшее использование места на жёстком диске (экономия более, чем в 3 раза по сравнению со статической сборкой). Для целей разработки, когда планируются только отладочные запуски на том же компьютере, где осуществлялась сборка, это может быть наиболее удобным вариантом. - Вы можете изменить вариант сборки, выполнив новую команду в директории build. Запустите ninja для сборки: diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index de9a6a7807a..e0c8b41147a 100644 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -96,7 +96,6 @@ git push - `clickhouse`: Main built binary. - `clickhouse-odbc-bridge` - `unit_tests_dbms`: 带有 ClickHouse 单元测试的 GoogleTest 二进制文件. - - `shared_build.tgz`: 使用共享库构建. - `performance.tgz`: 用于性能测试的特殊包. ## 特殊构建检查 {#special-buildcheck} diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index b74ec8bf9a6..7098bc87f8e 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -167,7 +167,7 @@ else() message(STATUS "ClickHouse keeper-converter mode: OFF") endif() -if(NOT (USE_STATIC_LIBRARIES OR SPLIT_SHARED_LIBRARIES)) +if(NOT USE_STATIC_LIBRARIES) set(CLICKHOUSE_ONE_SHARED ON) endif() @@ -313,12 +313,6 @@ endif() clickhouse_add_executable (clickhouse main.cpp) -if (NOT USE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) - # Shared split (dev) build: In CI, the server is run with custom LD_LIBRARY_PATH. This makes the harmful env check re-execute the - # process in a clean environment but as in CI the containing directory is not included in DT_RUNPATH/DT_RPATH, the server won't come up. - target_compile_definitions(clickhouse PRIVATE DISABLE_HARMFUL_ENV_VAR_CHECK) -endif () - # A library that prevent usage of several functions from libc. if (ARCH_AMD64 AND OS_LINUX AND NOT OS_ANDROID) set (HARMFUL_LIB harmful) diff --git a/programs/main.cpp b/programs/main.cpp index f40bafc7027..389eae92091 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -345,7 +345,7 @@ struct Checker ; -#if !defined(DISABLE_HARMFUL_ENV_VAR_CHECK) && !defined(USE_MUSL) +#if !defined(USE_MUSL) /// NOTE: We will migrate to full static linking or our own dynamic loader to make this code obsolete. void checkHarmfulEnvironmentVariables(char ** argv) { @@ -457,7 +457,7 @@ int main(int argc_, char ** argv_) /// Note: we forbid dlopen in our code. updatePHDRCache(); -#if !defined(DISABLE_HARMFUL_ENV_VAR_CHECK) && !defined(USE_MUSL) +#if !defined(USE_MUSL) checkHarmfulEnvironmentVariables(argv_); #endif diff --git a/rust/skim/CMakeLists.txt b/rust/skim/CMakeLists.txt index e626dd5742e..1e7a43aba7c 100644 --- a/rust/skim/CMakeLists.txt +++ b/rust/skim/CMakeLists.txt @@ -35,15 +35,7 @@ add_custom_command(OUTPUT ${ffi_binding_final_path} DEPENDS cargo-build__ch_rust_skim_rust) add_library(_ch_rust_skim_ffi ${ffi_binding_final_path}) -if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) - # static -else() - if (OS_DARWIN) - target_link_libraries(_ch_rust_skim_ffi PRIVATE -Wl,-undefined,dynamic_lookup) - else() - target_link_libraries(_ch_rust_skim_ffi PRIVATE -Wl,--unresolved-symbols=ignore-all) - endif() -endif() + # cxx bridge compiles such bindings set_target_properties(_ch_rust_skim_ffi PROPERTIES COMPILE_FLAGS "${CXXBRIDGE_CXXFLAGS}") diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 5e8fe368dfa..adcffa1f7b6 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -200,10 +200,6 @@ endif () add_library(clickhouse_common_io ${clickhouse_common_io_headers} ${clickhouse_common_io_sources}) -if (SPLIT_SHARED_LIBRARIES) - target_compile_definitions(clickhouse_common_io PRIVATE SPLIT_SHARED_LIBRARIES) -endif () - add_library (clickhouse_malloc OBJECT Common/malloc.cpp) set_source_files_properties(Common/malloc.cpp PROPERTIES COMPILE_FLAGS "-fno-builtin") @@ -227,18 +223,7 @@ add_subdirectory(Common/Config) set (all_modules) macro(add_object_library name common_path) - if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) - add_headers_and_sources(dbms ${common_path}) - else () - list (APPEND all_modules ${name}) - add_headers_and_sources(${name} ${common_path}) - add_library(${name} SHARED ${${name}_sources} ${${name}_headers}) - if (OS_DARWIN) - target_link_libraries (${name} PRIVATE -Wl,-undefined,dynamic_lookup) - else() - target_link_libraries (${name} PRIVATE -Wl,--unresolved-symbols=ignore-all) - endif() - endif () + add_headers_and_sources(dbms ${common_path}) endmacro() add_object_library(clickhouse_access Access) @@ -297,28 +282,12 @@ if (TARGET ch_contrib::nuraft) add_object_library(clickhouse_coordination Coordination) endif() -if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) - add_library (dbms STATIC ${dbms_headers} ${dbms_sources}) - target_link_libraries (dbms PRIVATE ch_contrib::libdivide) - if (TARGET ch_contrib::jemalloc) - target_link_libraries (dbms PRIVATE ch_contrib::jemalloc) - endif() - set (all_modules dbms) -else() - add_library (dbms SHARED ${dbms_headers} ${dbms_sources}) - target_link_libraries (dbms PUBLIC ${all_modules}) - target_link_libraries (clickhouse_interpreters PRIVATE ch_contrib::libdivide) - if (TARGET ch_contrib::jemalloc) - target_link_libraries (clickhouse_interpreters PRIVATE ch_contrib::jemalloc) - endif() - list (APPEND all_modules dbms) - # force all split libs to be linked - if (OS_DARWIN) - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,-undefined,error") - else() - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-as-needed") - endif() -endif () +add_library (dbms STATIC ${dbms_headers} ${dbms_sources}) +target_link_libraries (dbms PRIVATE ch_contrib::libdivide) +if (TARGET ch_contrib::jemalloc) + target_link_libraries (dbms PRIVATE ch_contrib::jemalloc) +endif() +set (all_modules dbms) macro (dbms_target_include_directories) foreach (module ${all_modules}) diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp index b987a4aac09..206314ea295 100644 --- a/src/Common/remapExecutable.cpp +++ b/src/Common/remapExecutable.cpp @@ -1,6 +1,6 @@ #include "remapExecutable.h" -#if defined(OS_LINUX) && defined(__amd64__) && defined(__SSE2__) && !defined(SANITIZER) && defined(NDEBUG) && !defined(SPLIT_SHARED_LIBRARIES) +#if defined(OS_LINUX) && defined(__amd64__) && defined(__SSE2__) && !defined(SANITIZER) && defined(NDEBUG) #include #include diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 93374f933b7..e9810e918b4 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -112,12 +112,7 @@ endif () target_link_libraries(clickhouse_functions_obj PUBLIC ${PUBLIC_LIBS} PRIVATE ${PRIVATE_LIBS}) -if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) - # Used to forward the linking information to the final binaries such as clickhouse / unit_tests_dbms, - # since such information are lost after we convert to OBJECT target - add_library(clickhouse_functions INTERFACE) - target_link_libraries(clickhouse_functions INTERFACE ${OBJECT_LIBS} ${PUBLIC_LIBS} ${PRIVATE_LIBS}) -else() - add_library(clickhouse_functions SHARED ${OBJECT_LIBS}) - target_link_libraries(clickhouse_functions PUBLIC ${PUBLIC_LIBS} PRIVATE ${PRIVATE_LIBS}) -endif () +# Used to forward the linking information to the final binaries such as clickhouse / unit_tests_dbms, +# since such information are lost after we convert to OBJECT target +add_library(clickhouse_functions INTERFACE) +target_link_libraries(clickhouse_functions INTERFACE ${OBJECT_LIBS} ${PUBLIC_LIBS} ${PRIVATE_LIBS}) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index a718bd53418..603f4619d53 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -38,8 +38,6 @@ BUILD_LOG_NAME = "build_log.log" def _can_export_binaries(build_config: BuildConfig) -> bool: if build_config["package_type"] != "deb": return False - if build_config["libraries"] == "shared": - return False if build_config["sanitizer"] != "": return True if build_config["build_type"] != "": @@ -68,8 +66,6 @@ def get_packager_cmd( cmd += f" --build-type={build_config['build_type']}" if build_config["sanitizer"]: cmd += f" --sanitizer={build_config['sanitizer']}" - if build_config["libraries"] == "shared": - cmd += " --shared-libraries" if build_config["tidy"] == "enable": cmd += " --clang-tidy" diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 1a2fdedefed..746ce13bf72 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -126,12 +126,6 @@ def download_all_deb_packages(check_name, reports_path, result_path): ) -def download_shared_build(check_name, reports_path, result_path): - download_builds_filter( - check_name, reports_path, result_path, lambda x: x.endswith("shared_build.tgz") - ) - - def download_unit_tests(check_name, reports_path, result_path): download_builds_filter( check_name, reports_path, result_path, lambda x: x.endswith("unit_tests_dbms") diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 7ace7255a65..563756de291 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -103,15 +103,6 @@ CI_CONFIG = { "tidy": "enable", "with_coverage": False, }, - "binary_shared": { - "compiler": "clang-15", - "build_type": "", - "sanitizer": "", - "package_type": "binary", - "libraries": "shared", - "tidy": "disable", - "with_coverage": False, - }, "binary_darwin": { "compiler": "clang-15-darwin", "build_type": "", @@ -196,7 +187,6 @@ CI_CONFIG = { ], "ClickHouse special build check": [ "binary_tidy", - "binary_shared", "binary_darwin", "binary_aarch64", "binary_aarch64_v80compat", From 27f5aad49e0796f80db7e98b6aa100433d78d2e9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 3 Jan 2023 18:51:10 +0000 Subject: [PATCH 61/78] What happens if I remove 156 lines of code? --- CMakeLists.txt | 37 +++------ base/base/CMakeLists.txt | 4 - base/glibc-compatibility/CMakeLists.txt | 2 +- cmake/sanitize.cmake | 8 +- contrib/arrow-cmake/CMakeLists.txt | 15 +--- contrib/c-ares-cmake/CMakeLists.txt | 9 +- contrib/libuv-cmake/CMakeLists.txt | 5 -- contrib/llvm-project-cmake/CMakeLists.txt | 2 - programs/CMakeLists.txt | 82 ++----------------- programs/benchmark/CMakeLists.txt | 4 +- programs/local/CMakeLists.txt | 4 +- src/CMakeLists.txt | 6 +- src/Daemon/CMakeLists.txt | 4 - .../System/StorageSystemBuildOptions.cpp.in | 1 - tests/integration/CMakeLists.txt | 2 +- utils/CMakeLists.txt | 6 +- 16 files changed, 35 insertions(+), 156 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 6239fe94f1c..ab976612401 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -73,11 +73,7 @@ message (STATUS "CMAKE_BUILD_TYPE: ${CMAKE_BUILD_TYPE}") string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) -option(USE_STATIC_LIBRARIES "Disable to use shared libraries" ON) - -if (USE_STATIC_LIBRARIES) - list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) -endif () +list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) option (ENABLE_FUZZING "Fuzzy testing using libfuzzer" OFF) @@ -160,7 +156,7 @@ option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) option(ENABLE_BENCHMARKS "Build all benchmark programs in 'benchmarks' subdirectories" OFF) -if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND USE_STATIC_LIBRARIES AND NOT USE_MUSL) +if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND NOT USE_MUSL) # Only for Linux, x86_64 or aarch64. option(GLIBC_COMPATIBILITY "Enable compatibility with older glibc libraries." ON) elseif(GLIBC_COMPATIBILITY) @@ -456,22 +452,13 @@ endif () set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX") -if (USE_STATIC_LIBRARIES) - set (CMAKE_POSITION_INDEPENDENT_CODE OFF) - if (OS_LINUX AND NOT ARCH_AARCH64) - # Slightly more efficient code can be generated - # It's disabled for ARM because otherwise ClickHouse cannot run on Android. - set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie") - set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie -Wl,-no-pie") - endif () -else () - set (CMAKE_POSITION_INDEPENDENT_CODE ON) - # This is required for clang on Arch linux, that uses PIE by default. - # See enable-SSP-and-PIE-by-default.patch [1]. - # - # [1]: https://github.com/archlinux/svntogit-packages/blob/6e681aa860e65ad46a1387081482eb875c2200f2/trunk/enable-SSP-and-PIE-by-default.patch - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie") +set (CMAKE_POSITION_INDEPENDENT_CODE OFF) +if (OS_LINUX AND NOT ARCH_AARCH64) + # Slightly more efficient code can be generated + # It's disabled for ARM because otherwise ClickHouse cannot run on Android. + set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie") + set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie -Wl,-no-pie") endif () if (ENABLE_TESTS) @@ -493,9 +480,7 @@ else () set (CLICKHOUSE_ETC_DIR "${CMAKE_INSTALL_PREFIX}/etc") endif () -message (STATUS - "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE} ; - USE_STATIC_LIBRARIES=${USE_STATIC_LIBRARIES}") +message (STATUS "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE}") include (GNUInstallDirs) @@ -541,7 +526,7 @@ macro (clickhouse_add_executable target) # - _je_zone_register due to JEMALLOC_PRIVATE_NAMESPACE=je_ under OS X. # - but jemalloc-cmake does not run private_namespace.sh # so symbol name should be _zone_register - if (ENABLE_JEMALLOC AND USE_STATIC_LIBRARIES AND OS_DARWIN) + if (ENABLE_JEMALLOC AND OS_DARWIN) set_property(TARGET ${target} APPEND PROPERTY LINK_OPTIONS -u_zone_register) endif() endif() diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 79e62586ad4..5d9f1de8309 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -39,10 +39,6 @@ endif () target_include_directories(common PUBLIC .. "${CMAKE_CURRENT_BINARY_DIR}/..") -if (OS_DARWIN AND NOT USE_STATIC_LIBRARIES) - target_link_libraries(common PUBLIC -Wl,-U,_inside_main) -endif() - target_link_libraries (common PUBLIC ch_contrib::cityhash diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index 7aa9a3b4a61..0539f0c231d 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -37,7 +37,7 @@ if (GLIBC_COMPATIBILITY) target_include_directories(glibc-compatibility PRIVATE libcxxabi ${musl_arch_include_dir}) - if (( NOT USE_STATIC_LIBRARIES AND NOT USE_STATIC_LIBRARIES ) OR ENABLE_OPENSSL_DYNAMIC) + if (ENABLE_OPENSSL_DYNAMIC) target_compile_options(glibc-compatibility PRIVATE -fPIC) endif () diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 3e3bb7ec2b2..cf7f7606618 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -25,7 +25,7 @@ if (SANITIZE) if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${ASAN_FLAGS}") endif() - if (USE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libasan") endif () if (COMPILER_GCC) @@ -50,7 +50,7 @@ if (SANITIZE) if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=memory") endif() - if (USE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libmsan") endif () @@ -71,7 +71,7 @@ if (SANITIZE) if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=thread") endif() - if (USE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libtsan") endif () if (COMPILER_GCC) @@ -103,7 +103,7 @@ if (SANITIZE) if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=undefined") endif() - if (USE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libubsan") endif () if (COMPILER_GCC) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index c289c88ef7b..ae6f270a768 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -78,23 +78,14 @@ set(FLATBUFFERS_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/flatbuffers") set(FLATBUFFERS_INCLUDE_DIR "${FLATBUFFERS_SRC_DIR}/include") # set flatbuffers CMake options -if (USE_STATIC_LIBRARIES) - set(FLATBUFFERS_BUILD_FLATLIB ON CACHE BOOL "Enable the build of the flatbuffers library") - set(FLATBUFFERS_BUILD_SHAREDLIB OFF CACHE BOOL "Disable the build of the flatbuffers shared library") -else () - set(FLATBUFFERS_BUILD_SHAREDLIB ON CACHE BOOL "Enable the build of the flatbuffers shared library") - set(FLATBUFFERS_BUILD_FLATLIB OFF CACHE BOOL "Disable the build of the flatbuffers library") -endif () +set(FLATBUFFERS_BUILD_FLATLIB ON CACHE BOOL "Enable the build of the flatbuffers library") +set(FLATBUFFERS_BUILD_SHAREDLIB OFF CACHE BOOL "Disable the build of the flatbuffers shared library") set(FLATBUFFERS_BUILD_TESTS OFF CACHE BOOL "Skip flatbuffers tests") add_subdirectory(${FLATBUFFERS_SRC_DIR} "${FLATBUFFERS_BINARY_DIR}") add_library(_flatbuffers INTERFACE) -if(USE_STATIC_LIBRARIES) - target_link_libraries(_flatbuffers INTERFACE flatbuffers) -else() - target_link_libraries(_flatbuffers INTERFACE flatbuffers_shared) -endif() +target_link_libraries(_flatbuffers INTERFACE flatbuffers) target_include_directories(_flatbuffers INTERFACE ${FLATBUFFERS_INCLUDE_DIR}) # === hdfs diff --git a/contrib/c-ares-cmake/CMakeLists.txt b/contrib/c-ares-cmake/CMakeLists.txt index 4b1170f9dd1..63e145bec18 100644 --- a/contrib/c-ares-cmake/CMakeLists.txt +++ b/contrib/c-ares-cmake/CMakeLists.txt @@ -63,13 +63,8 @@ SET(SRCS "${LIBRARY_DIR}/src/lib/windows_port.c" ) -if (USE_STATIC_LIBRARIES) - add_library(_c-ares STATIC ${SRCS}) - target_compile_definitions(_c-ares PUBLIC CARES_STATICLIB) -else() - add_library(_c-ares SHARED ${SRCS}) - target_compile_definitions(_c-ares PUBLIC CARES_BUILDING_LIBRARY) -endif() +add_library(_c-ares STATIC ${SRCS}) +target_compile_definitions(_c-ares PUBLIC CARES_STATICLIB) target_compile_definitions(_c-ares PRIVATE HAVE_CONFIG_H=1) diff --git a/contrib/libuv-cmake/CMakeLists.txt b/contrib/libuv-cmake/CMakeLists.txt index ba6bc746c59..7ca2cef2251 100644 --- a/contrib/libuv-cmake/CMakeLists.txt +++ b/contrib/libuv-cmake/CMakeLists.txt @@ -136,11 +136,6 @@ add_library(ch_contrib::uv ALIAS _uv) target_compile_definitions(_uv PRIVATE ${uv_defines}) target_include_directories(_uv SYSTEM PUBLIC ${SOURCE_DIR}/include PRIVATE ${SOURCE_DIR}/src) target_link_libraries(_uv ${uv_libraries}) -if (NOT USE_STATIC_LIBRARIES) - target_compile_definitions(_uv - INTERFACE USING_UV_SHARED=1 - PRIVATE BUILDING_UV_SHARED=1) -endif() if(UNIX) # Now for some gibbering horrors from beyond the stars... diff --git a/contrib/llvm-project-cmake/CMakeLists.txt b/contrib/llvm-project-cmake/CMakeLists.txt index 7af4a23bc9d..8759c16ac3e 100644 --- a/contrib/llvm-project-cmake/CMakeLists.txt +++ b/contrib/llvm-project-cmake/CMakeLists.txt @@ -6,8 +6,6 @@ endif() option (ENABLE_EMBEDDED_COMPILER "Enable support for 'compile_expressions' option for query execution" ${ENABLE_EMBEDDED_COMPILER_DEFAULT}) -# If USE_STATIC_LIBRARIES=0 was passed to CMake, we'll still build LLVM statically to keep complexity minimal. - if (NOT ENABLE_EMBEDDED_COMPILER) message(STATUS "Not using LLVM") return() diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 7098bc87f8e..9e4f66562ca 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -167,10 +167,6 @@ else() message(STATUS "ClickHouse keeper-converter mode: OFF") endif() -if(NOT USE_STATIC_LIBRARIES) - set(CLICKHOUSE_ONE_SHARED ON) -endif() - if (ENABLE_CLICKHOUSE_DISKS) message(STATUS "Clickhouse disks mode: ON") else() @@ -186,11 +182,7 @@ endif() configure_file (config_tools.h.in ${CONFIG_INCLUDE_PATH}/config_tools.h) macro(clickhouse_target_link_split_lib target name) - if(NOT CLICKHOUSE_ONE_SHARED) - target_link_libraries(${target} PRIVATE clickhouse-${name}-lib) - else() - target_link_libraries(${target} PRIVATE clickhouse-lib) - endif() + target_link_libraries(${target} PRIVATE clickhouse-${name}-lib) endmacro() macro(clickhouse_program_add_library name) @@ -202,18 +194,16 @@ macro(clickhouse_program_add_library name) set(CLICKHOUSE_${name_uc}_LINK ${CLICKHOUSE_${name_uc}_LINK} PARENT_SCOPE) set(CLICKHOUSE_${name_uc}_INCLUDE ${CLICKHOUSE_${name_uc}_INCLUDE} PARENT_SCOPE) - if(NOT CLICKHOUSE_ONE_SHARED) - add_library(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_SOURCES}) + add_library(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_SOURCES}) - set(_link ${CLICKHOUSE_${name_uc}_LINK}) # can't use ${} in if() - if(_link) - target_link_libraries(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_LINK}) - endif() + set(_link ${CLICKHOUSE_${name_uc}_LINK}) # can't use ${} in if() + if(_link) + target_link_libraries(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_LINK}) + endif() - set(_include ${CLICKHOUSE_${name_uc}_INCLUDE}) # can't use ${} in if() - if (_include) - target_include_directories(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_INCLUDE}) - endif() + set(_include ${CLICKHOUSE_${name_uc}_INCLUDE}) # can't use ${} in if() + if (_include) + target_include_directories(clickhouse-${name}-lib ${CLICKHOUSE_${name_uc}_INCLUDE}) endif() endmacro() @@ -257,60 +247,6 @@ if (ENABLE_CLICKHOUSE_SELF_EXTRACTING) add_subdirectory (self-extracting) endif () -if (CLICKHOUSE_ONE_SHARED) - add_library(clickhouse-lib SHARED - ${CLICKHOUSE_SERVER_SOURCES} - ${CLICKHOUSE_CLIENT_SOURCES} - ${CLICKHOUSE_LOCAL_SOURCES} - ${CLICKHOUSE_BENCHMARK_SOURCES} - ${CLICKHOUSE_COPIER_SOURCES} - ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} - ${CLICKHOUSE_COMPRESSOR_SOURCES} - ${CLICKHOUSE_FORMAT_SOURCES} - ${CLICKHOUSE_OBFUSCATOR_SOURCES} - ${CLICKHOUSE_GIT_IMPORT_SOURCES} - ${CLICKHOUSE_ODBC_BRIDGE_SOURCES} - ${CLICKHOUSE_KEEPER_SOURCES} - ${CLICKHOUSE_KEEPER_CONVERTER_SOURCES} - ${CLICKHOUSE_STATIC_FILES_DISK_UPLOADER_SOURCES} - ${CLICKHOUSE_SU_SOURCES}) - - target_link_libraries(clickhouse-lib - ${CLICKHOUSE_SERVER_LINK} - ${CLICKHOUSE_CLIENT_LINK} - ${CLICKHOUSE_LOCAL_LINK} - ${CLICKHOUSE_BENCHMARK_LINK} - ${CLICKHOUSE_COPIER_LINK} - ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} - ${CLICKHOUSE_COMPRESSOR_LINK} - ${CLICKHOUSE_FORMAT_LINK} - ${CLICKHOUSE_OBFUSCATOR_LINK} - ${CLICKHOUSE_GIT_IMPORT_LINK} - ${CLICKHOUSE_ODBC_BRIDGE_LINK} - ${CLICKHOUSE_KEEPER_LINK} - ${CLICKHOUSE_KEEPER_CONVERTER_LINK} - ${CLICKHOUSE_STATIC_FILES_DISK_UPLOADER_LINK} - ${CLICKHOUSE_SU_LINK}) - - target_include_directories(clickhouse-lib - ${CLICKHOUSE_SERVER_INCLUDE} - ${CLICKHOUSE_CLIENT_INCLUDE} - ${CLICKHOUSE_LOCAL_INCLUDE} - ${CLICKHOUSE_BENCHMARK_INCLUDE} - ${CLICKHOUSE_COPIER_INCLUDE} - ${CLICKHOUSE_EXTRACT_FROM_CONFIG_INCLUDE} - ${CLICKHOUSE_COMPRESSOR_INCLUDE} - ${CLICKHOUSE_FORMAT_INCLUDE} - ${CLICKHOUSE_OBFUSCATOR_INCLUDE} - ${CLICKHOUSE_GIT_IMPORT_INCLUDE} - ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE} - ${CLICKHOUSE_KEEPER_INCLUDE} - ${CLICKHOUSE_KEEPER_CONVERTER_INCLUDE}) - - set_target_properties(clickhouse-lib PROPERTIES SOVERSION ${VERSION_MAJOR}.${VERSION_MINOR} VERSION ${VERSION_SO} OUTPUT_NAME clickhouse DEBUG_POSTFIX "") - install (TARGETS clickhouse-lib LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} COMPONENT clickhouse) -endif() - clickhouse_add_executable (clickhouse main.cpp) # A library that prevent usage of several functions from libc. diff --git a/programs/benchmark/CMakeLists.txt b/programs/benchmark/CMakeLists.txt index 3fa8deb6bd9..ad211399bb5 100644 --- a/programs/benchmark/CMakeLists.txt +++ b/programs/benchmark/CMakeLists.txt @@ -10,6 +10,4 @@ set (CLICKHOUSE_BENCHMARK_LINK clickhouse_program_add(benchmark) -if(NOT CLICKHOUSE_ONE_SHARED) - target_link_libraries (clickhouse-benchmark-lib PRIVATE clickhouse-client-lib) -endif() +target_link_libraries (clickhouse-benchmark-lib PRIVATE clickhouse-client-lib) diff --git a/programs/local/CMakeLists.txt b/programs/local/CMakeLists.txt index 7366f8685a4..6943af48ab9 100644 --- a/programs/local/CMakeLists.txt +++ b/programs/local/CMakeLists.txt @@ -14,9 +14,7 @@ set (CLICKHOUSE_LOCAL_LINK clickhouse_program_add(local) -if(NOT CLICKHOUSE_ONE_SHARED) - target_link_libraries(clickhouse-local-lib PRIVATE clickhouse-server-lib) -endif() +target_link_libraries(clickhouse-local-lib PRIVATE clickhouse-server-lib) if (TARGET ch_rust::skim) target_link_libraries(clickhouse-local-lib PRIVATE ch_rust::skim) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index adcffa1f7b6..84b75590706 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -11,11 +11,7 @@ if(COMPILER_PIPE) else() set(MAX_COMPILER_MEMORY 1500) endif() -if(USE_STATIC_LIBRARIES) - set(MAX_LINKER_MEMORY 3500) -else() - set(MAX_LINKER_MEMORY 2500) -endif() +set(MAX_LINKER_MEMORY 3500) include(../cmake/limit_jobs.cmake) include (../cmake/version.cmake) diff --git a/src/Daemon/CMakeLists.txt b/src/Daemon/CMakeLists.txt index 316b03dc535..35ea2122dbb 100644 --- a/src/Daemon/CMakeLists.txt +++ b/src/Daemon/CMakeLists.txt @@ -7,10 +7,6 @@ add_library (daemon GitHash.generated.cpp ) -if (OS_DARWIN AND NOT USE_STATIC_LIBRARIES) - target_link_libraries (daemon PUBLIC -Wl,-undefined,dynamic_lookup) -endif() - target_link_libraries (daemon PUBLIC loggers common PRIVATE clickhouse_parsers clickhouse_common_io clickhouse_common_config) if (TARGET ch_contrib::sentry) diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index 47a448900a4..fd4e478004f 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -20,7 +20,6 @@ const char * auto_config_build[] "CXX_FLAGS", "@FULL_CXX_FLAGS_NORMALIZED@", "LINK_FLAGS", "@FULL_EXE_LINKER_FLAGS_NORMALIZED@", "BUILD_COMPILE_DEFINITIONS", "@BUILD_COMPILE_DEFINITIONS@", - "STATIC", "@USE_STATIC_LIBRARIES@", "USE_EMBEDDED_COMPILER", "@USE_EMBEDDED_COMPILER@", "USE_GLIBC_COMPATIBILITY", "@GLIBC_COMPATIBILITY@", "USE_JEMALLOC", "@ENABLE_JEMALLOC@", diff --git a/tests/integration/CMakeLists.txt b/tests/integration/CMakeLists.txt index 1482c068be5..ed12cf5b4a3 100644 --- a/tests/integration/CMakeLists.txt +++ b/tests/integration/CMakeLists.txt @@ -6,7 +6,7 @@ find_program(PYTEST_CMD pytest) find_program(SUDO_CMD sudo) # will mount only one binary to docker container - build with .so cant work -if(USE_STATIC_LIBRARIES AND DOCKER_CMD) +if(DOCKER_CMD) if(INTEGRATION_USE_RUNNER AND SUDO_CMD) add_test(NAME integration-runner WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND ${SUDO_CMD} ${CMAKE_CURRENT_SOURCE_DIR}/runner --binary ${ClickHouse_BINARY_DIR}/programs/clickhouse --configs-dir ${ClickHouse_SOURCE_DIR}/programs/server/) message(STATUS "Using tests in docker with runner SUDO=${SUDO_CMD}; DOCKER=${DOCKER_CMD};") diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 70c32c67063..bb9d4c88fa1 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -2,11 +2,7 @@ if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") endif () -if(USE_STATIC_LIBRARIES) - set(MAX_LINKER_MEMORY 3500) -else() - set(MAX_LINKER_MEMORY 2500) -endif() +set(MAX_LINKER_MEMORY 3500) include(../cmake/limit_jobs.cmake) if (ENABLE_CLICKHOUSE_SELF_EXTRACTING) From 28eb2dbd4c50a2bf9038b0e7b81807048798cff7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 3 Jan 2023 18:53:03 +0000 Subject: [PATCH 62/78] Revert some changes from #42777 --- .../Formats/Impl/CSVRowOutputFormat.cpp | 29 ++----------------- .../Formats/Impl/CSVRowOutputFormat.h | 5 +--- .../JSONCompactEachRowRowOutputFormat.cpp | 21 +------------- .../Impl/JSONCompactEachRowRowOutputFormat.h | 2 -- .../Impl/JSONEachRowRowOutputFormat.cpp | 11 ++++--- ...JSONEachRowWithProgressRowOutputFormat.cpp | 22 +++----------- .../Formats/Impl/MarkdownRowOutputFormat.cpp | 13 +-------- .../Formats/Impl/MarkdownRowOutputFormat.h | 6 +--- .../Formats/Impl/SQLInsertRowOutputFormat.cpp | 19 +++++------- .../Formats/Impl/TSKVRowOutputFormat.cpp | 7 +---- .../Formats/Impl/TSKVRowOutputFormat.h | 1 - .../Impl/TabSeparatedRowOutputFormat.cpp | 29 ++----------------- .../Impl/TabSeparatedRowOutputFormat.h | 5 +--- .../Formats/Impl/XMLRowOutputFormat.cpp | 14 +++------ .../Formats/Impl/XMLRowOutputFormat.h | 1 - 15 files changed, 33 insertions(+), 152 deletions(-) diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index f870ba34b1e..304e877aae9 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -35,16 +35,10 @@ void CSVRowOutputFormat::writePrefix() const auto & sample = getPort(PortKind::Main).getHeader(); if (with_names) - { writeLine(sample.getNames()); - writeRowBetweenDelimiter(); - } if (with_types) - { writeLine(sample.getDataTypeNames()); - writeRowBetweenDelimiter(); - } } @@ -60,38 +54,21 @@ void CSVRowOutputFormat::writeFieldDelimiter() } -void CSVRowOutputFormat::writeRowBetweenDelimiter() +void CSVRowOutputFormat::writeRowEndDelimiter() { if (format_settings.csv.crlf_end_of_line) writeChar('\r', out); writeChar('\n', out); } -void CSVRowOutputFormat::writeSuffix() -{ - /// Write '\n' after data if we had any data. - if (haveWrittenData()) - writeRowBetweenDelimiter(); -} - void CSVRowOutputFormat::writeBeforeTotals() { - writeRowBetweenDelimiter(); + writeChar('\n', out); } void CSVRowOutputFormat::writeBeforeExtremes() { - writeRowBetweenDelimiter(); -} - -void CSVRowOutputFormat::writeAfterTotals() -{ - writeRowBetweenDelimiter(); -} - -void CSVRowOutputFormat::writeAfterExtremes() -{ - writeRowBetweenDelimiter(); + writeChar('\n', out); } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.h b/src/Processors/Formats/Impl/CSVRowOutputFormat.h index d4ccaf3b3e2..131439e6bbc 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.h @@ -33,18 +33,15 @@ public: private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeFieldDelimiter() override; - void writeRowBetweenDelimiter() override; + void writeRowEndDelimiter() override; bool supportTotals() const override { return true; } bool supportExtremes() const override { return true; } void writeBeforeTotals() override; - void writeAfterTotals() override; void writeBeforeExtremes() override; - void writeAfterExtremes() override; void writePrefix() override; - void writeSuffix() override; void writeLine(const std::vector & values); bool with_names; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index a11f1e0da7b..a589558763c 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -52,17 +52,11 @@ void JSONCompactEachRowRowOutputFormat::writeRowStartDelimiter() void JSONCompactEachRowRowOutputFormat::writeRowEndDelimiter() { - writeChar(']', *ostr); -} - -void JSONCompactEachRowRowOutputFormat::writeRowBetweenDelimiter() -{ - writeChar('\n', *ostr); + writeCString("]\n", *ostr); } void JSONCompactEachRowRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) { - writeRowBetweenDelimiter(); size_t columns_size = columns.size(); writeRowStartDelimiter(); for (size_t i = 0; i < columns_size; ++i) @@ -73,7 +67,6 @@ void JSONCompactEachRowRowOutputFormat::writeTotals(const Columns & columns, siz writeField(*columns[i], *serializations[i], row_num); } writeRowEndDelimiter(); - writeRowBetweenDelimiter(); } void JSONCompactEachRowRowOutputFormat::writeLine(const std::vector & values) @@ -96,22 +89,10 @@ void JSONCompactEachRowRowOutputFormat::writePrefix() const auto & header = getPort(PortKind::Main).getHeader(); if (with_names) - { writeLine(JSONUtils::makeNamesValidJSONStrings(header.getNames(), settings, settings.json.validate_utf8)); - writeRowBetweenDelimiter(); - } if (with_types) - { writeLine(JSONUtils::makeNamesValidJSONStrings(header.getDataTypeNames(), settings, settings.json.validate_utf8)); - writeRowBetweenDelimiter(); - } -} - -void JSONCompactEachRowRowOutputFormat::writeSuffix() -{ - if (haveWrittenData()) - writeChar('\n', *ostr); } void JSONCompactEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h index bc5dc8de813..2be39669dd2 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -26,7 +26,6 @@ public: private: void writePrefix() override; - void writeSuffix() override; void writeTotals(const Columns & columns, size_t row_num) override; @@ -34,7 +33,6 @@ private: void writeFieldDelimiter() override; void writeRowStartDelimiter() override; void writeRowEndDelimiter() override; - void writeRowBetweenDelimiter() override; bool supportTotals() const override { return true; } void consumeTotals(Chunk) override; diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 07913c2906c..2e2209bb6e3 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -41,7 +41,10 @@ void JSONEachRowRowOutputFormat::writeRowStartDelimiter() void JSONEachRowRowOutputFormat::writeRowEndDelimiter() { - writeCString("}", *ostr); + if (settings.json.array_of_rows) + writeChar('}', *ostr); + else + writeCString("}\n", *ostr); field_number = 0; } @@ -49,9 +52,7 @@ void JSONEachRowRowOutputFormat::writeRowEndDelimiter() void JSONEachRowRowOutputFormat::writeRowBetweenDelimiter() { if (settings.json.array_of_rows) - writeChar(',', *ostr); - - writeChar('\n', *ostr); + writeCString(",\n", *ostr); } @@ -68,8 +69,6 @@ void JSONEachRowRowOutputFormat::writeSuffix() { if (settings.json.array_of_rows) writeCString("\n]\n", *ostr); - else if (haveWrittenData()) - writeChar('\n', *ostr); } diff --git a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index 988f99c5b7f..449ccb62562 100644 --- a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -10,16 +10,13 @@ namespace DB void JSONEachRowWithProgressRowOutputFormat::writeRowStartDelimiter() { if (has_progress) - { writeProgress(); - writeRowBetweenDelimiter(); - } writeCString("{\"row\":{", *ostr); } void JSONEachRowWithProgressRowOutputFormat::writeRowEndDelimiter() { - writeCString("}}", *ostr); + writeCString("}}\n", *ostr); field_number = 0; } @@ -30,7 +27,7 @@ void JSONEachRowWithProgressRowOutputFormat::onProgress(const Progress & value) WriteBufferFromString buf(progress_line); writeCString("{\"progress\":", buf); progress.writeJSON(buf); - writeCString("}", buf); + writeCString("}\n", buf); buf.finalize(); std::lock_guard lock(progress_lines_mutex); progress_lines.emplace_back(std::move(progress_line)); @@ -40,33 +37,22 @@ void JSONEachRowWithProgressRowOutputFormat::onProgress(const Progress & value) void JSONEachRowWithProgressRowOutputFormat::flush() { if (has_progress) - { - if (haveWrittenData()) - writeRowBetweenDelimiter(); writeProgress(); - } JSONEachRowRowOutputFormat::flush(); } void JSONEachRowWithProgressRowOutputFormat::writeSuffix() { if (has_progress) - { - writeRowBetweenDelimiter(); writeProgress(); - } JSONEachRowRowOutputFormat::writeSuffix(); } void JSONEachRowWithProgressRowOutputFormat::writeProgress() { std::lock_guard lock(progress_lines_mutex); - for (size_t i = 0; i != progress_lines.size(); ++i) - { - if (i != 0) - writeRowBetweenDelimiter(); - writeString(progress_lines[i], *ostr); - } + for (const auto & progress_line : progress_lines) + writeString(progress_line, *ostr); progress_lines.clear(); has_progress = false; } diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp index c5fa4ea890e..ea414171ed6 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp @@ -47,18 +47,7 @@ void MarkdownRowOutputFormat::writeFieldDelimiter() void MarkdownRowOutputFormat::writeRowEndDelimiter() { - writeCString(" |", out); -} - -void MarkdownRowOutputFormat::writeRowBetweenDelimiter() -{ - writeChar('\n', out); -} - -void MarkdownRowOutputFormat::writeSuffix() -{ - if (haveWrittenData()) - writeChar('\n', out); + writeCString(" |\n", out); } void MarkdownRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h index 4edf2a69434..8a4ae1f3b96 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h @@ -21,7 +21,6 @@ private: /// |columnName1|columnName2|...|columnNameN| /// |:-:|:-:|...|:-:| void writePrefix() override; - void writeSuffix() override; /// Write '|' before each row void writeRowStartDelimiter() override; @@ -29,12 +28,9 @@ private: /// Write '|' between values void writeFieldDelimiter() override; - /// Write '|' at the end of each row + /// Write '|\n' at the end of each row void writeRowEndDelimiter() override; - /// Write '\n' after each row - void writeRowBetweenDelimiter() override; - void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; const FormatSettings format_settings; diff --git a/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.cpp b/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.cpp index 891bbb5ff12..6f490c8b085 100644 --- a/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/SQLInsertRowOutputFormat.cpp @@ -65,27 +65,24 @@ void SQLInsertRowOutputFormat::writeRowEndDelimiter() { writeChar(')', out); ++rows_in_line; - if (rows_in_line >= format_settings.sql_insert.max_batch_size) - { - writeChar(';', out); - rows_in_line = 0; - } } void SQLInsertRowOutputFormat::writeRowBetweenDelimiter() { - if (rows_in_line == 0) - writeChar('\n', out); + if (rows_in_line >= format_settings.sql_insert.max_batch_size) + { + writeCString(";\n", out); + rows_in_line = 0; + } else + { writeCString(", ", out); + } } void SQLInsertRowOutputFormat::writeSuffix() { - if (rows_in_line != 0) - writeChar(';', out); - if (haveWrittenData()) - writeChar('\n', out); + writeCString(";\n", out); } void SQLInsertRowOutputFormat::resetFormatterImpl() diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp index f47c03df7b9..808d77f5841 100644 --- a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp @@ -29,14 +29,9 @@ void TSKVRowOutputFormat::writeField(const IColumn & column, const ISerializatio void TSKVRowOutputFormat::writeRowEndDelimiter() -{ - field_number = 0; -} - - -void TSKVRowOutputFormat::writeRowBetweenDelimiter() { writeChar('\n', out); + field_number = 0; } diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h index 2d05c8c05d0..25613dd22d9 100644 --- a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h @@ -21,7 +21,6 @@ public: private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeRowEndDelimiter() override; - void writeRowBetweenDelimiter() override; /// Disable totals and extremes, because they are enabled in TSV. bool supportTotals() const override { return false; } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 1fd6f58ded6..a4a5aea26cb 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -36,16 +36,10 @@ void TabSeparatedRowOutputFormat::writePrefix() const auto & header = getPort(PortKind::Main).getHeader(); if (with_names) - { writeLine(header.getNames()); - writeRowBetweenDelimiter(); - } if (with_types) - { writeLine(header.getDataTypeNames()); - writeRowBetweenDelimiter(); - } } @@ -64,38 +58,21 @@ void TabSeparatedRowOutputFormat::writeFieldDelimiter() } -void TabSeparatedRowOutputFormat::writeRowBetweenDelimiter() +void TabSeparatedRowOutputFormat::writeRowEndDelimiter() { if (format_settings.tsv.crlf_end_of_line) writeChar('\r', out); writeChar('\n', out); } -void TabSeparatedRowOutputFormat::writeSuffix() -{ - /// Output '\n' an the end of data if we had any data. - if (haveWrittenData()) - writeRowBetweenDelimiter(); -} - void TabSeparatedRowOutputFormat::writeBeforeTotals() { - writeRowBetweenDelimiter(); + writeChar('\n', out); } void TabSeparatedRowOutputFormat::writeBeforeExtremes() { - writeRowBetweenDelimiter(); -} - -void TabSeparatedRowOutputFormat::writeAfterTotals() -{ - writeRowBetweenDelimiter(); -} - -void TabSeparatedRowOutputFormat::writeAfterExtremes() -{ - writeRowBetweenDelimiter(); + writeChar('\n', out); } void registerOutputFormatTabSeparated(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index 6d481bdeec4..9facba2052a 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -35,18 +35,15 @@ public: protected: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeFieldDelimiter() override final; - void writeRowBetweenDelimiter() override; + void writeRowEndDelimiter() override; bool supportTotals() const override { return true; } bool supportExtremes() const override { return true; } void writeBeforeTotals() override final; - void writeAfterTotals() override final; void writeBeforeExtremes() override final; - void writeAfterExtremes() override final; void writePrefix() override; - void writeSuffix() override; void writeLine(const std::vector & values); bool with_names; diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index c08b5bc55e6..1d6fb62275c 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -89,20 +89,14 @@ void XMLRowOutputFormat::writeRowStartDelimiter() void XMLRowOutputFormat::writeRowEndDelimiter() { - writeCString("\t\t", *ostr); + writeCString("\t\t\n", *ostr); field_number = 0; ++row_count; } -void XMLRowOutputFormat::writeRowBetweenDelimiter() -{ - writeChar('\n', *ostr); -} - - void XMLRowOutputFormat::writeSuffix() { - writeCString("\n\t\n", *ostr); + writeCString("\t\n", *ostr); } @@ -152,7 +146,7 @@ void XMLRowOutputFormat::writeMaxExtreme(const Columns & columns, size_t row_num void XMLRowOutputFormat::writeAfterExtremes() { - writeCString("\n\t\n", *ostr); + writeCString("\t\n", *ostr); } void XMLRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) @@ -179,7 +173,7 @@ void XMLRowOutputFormat::writeExtremesElement(const char * title, const Columns writeCString("\t\t", *ostr); + writeCString(">\n", *ostr); } diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.h b/src/Processors/Formats/Impl/XMLRowOutputFormat.h index 2f240aa9f81..e25e7129109 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.h +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.h @@ -24,7 +24,6 @@ private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeRowStartDelimiter() override; void writeRowEndDelimiter() override; - void writeRowBetweenDelimiter() override; void writePrefix() override; void writeSuffix() override; void finalizeImpl() override; From 5824741849aee7ac2612081870bd4a1231d355bf Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 3 Jan 2023 19:00:18 +0000 Subject: [PATCH 63/78] Fix tests --- .../0_stateless/00966_live_view_watch_events_http.py | 4 ++-- tests/queries/0_stateless/00967_live_view_watch_http.py | 4 ++-- .../0_stateless/01246_insert_into_watch_live_view.py | 6 +++--- tests/queries/0_stateless/01249_flush_interactive.sh | 4 ++-- .../0_stateless/01553_settings_early_apply.reference | 1 - 5 files changed, 9 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/tests/queries/0_stateless/00966_live_view_watch_events_http.py index 6a7d359656d..1f2ddae23d6 100755 --- a/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ b/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -38,10 +38,10 @@ with client(name="client1>", log=log) as client1: name="client2>", log=log, ) as client2: - client2.expect(".*1") + client2.expect(".*1\n") client1.send("INSERT INTO test.mt VALUES (1),(2),(3)") client1.expect(prompt) - client2.expect(".*2") + client2.expect(".*2\n") finally: client1.send("DROP TABLE test.lv") client1.expect(prompt) diff --git a/tests/queries/0_stateless/00967_live_view_watch_http.py b/tests/queries/0_stateless/00967_live_view_watch_http.py index 48931379e7f..92e192cc7f2 100755 --- a/tests/queries/0_stateless/00967_live_view_watch_http.py +++ b/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -38,10 +38,10 @@ with client(name="client1>", log=log) as client1: name="client2>", log=log, ) as client2: - client2.expect(".*0\t1") + client2.expect(".*0\t1\n") client1.send("INSERT INTO test.mt VALUES (1),(2),(3)") client1.expect(prompt) - client2.expect(".*6\t2") + client2.expect(".*6\t2\n") finally: client1.send("DROP TABLE test.lv") client1.expect(prompt) diff --git a/tests/queries/0_stateless/01246_insert_into_watch_live_view.py b/tests/queries/0_stateless/01246_insert_into_watch_live_view.py index 4e9b799d0e4..02d03abc8c3 100755 --- a/tests/queries/0_stateless/01246_insert_into_watch_live_view.py +++ b/tests/queries/0_stateless/01246_insert_into_watch_live_view.py @@ -53,15 +53,15 @@ with client(name="client1>", log=log) as client1, client( client1.send("INSERT INTO test.sums WATCH test.lv") client1.expect(r"INSERT INTO") - client3.expect("0,1.*") + client3.expect("0,1.*\n") client2.send("INSERT INTO test.mt VALUES (1),(2),(3)") client2.expect(prompt) - client3.expect("6,2.*") + client3.expect("6,2.*\n") client2.send("INSERT INTO test.mt VALUES (4),(5),(6)") client2.expect(prompt) - client3.expect("21,3.*") + client3.expect("21,3.*\n") # send Ctrl-C client3.send("\x03", eol="") diff --git a/tests/queries/0_stateless/01249_flush_interactive.sh b/tests/queries/0_stateless/01249_flush_interactive.sh index 2bb4d175a9e..551e11c8c8d 100755 --- a/tests/queries/0_stateless/01249_flush_interactive.sh +++ b/tests/queries/0_stateless/01249_flush_interactive.sh @@ -16,10 +16,10 @@ function test() { timeout 5 ${CLICKHOUSE_LOCAL} --max_execution_time 10 --query " SELECT DISTINCT number % 5 FROM system.numbers" ||: - echo -e '\n---' + echo -e '---' timeout 5 ${CLICKHOUSE_CURL} -sS --no-buffer "${CLICKHOUSE_URL}&max_execution_time=10" --data-binary " SELECT DISTINCT number % 5 FROM system.numbers" ||: - echo -e '\n---' + echo -e '---' } # The test depends on timeouts. And there is a chance that under high system load the query diff --git a/tests/queries/0_stateless/01553_settings_early_apply.reference b/tests/queries/0_stateless/01553_settings_early_apply.reference index 5047d461af9..a42a7daadaa 100644 --- a/tests/queries/0_stateless/01553_settings_early_apply.reference +++ b/tests/queries/0_stateless/01553_settings_early_apply.reference @@ -31,7 +31,6 @@ UInt64 - 0 From 0330276a0233b42b47d99d0d50597974c1d3acd9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 3 Jan 2023 19:07:29 +0000 Subject: [PATCH 64/78] The "libraries" field is no longer needed --- tests/ci/build_report_check.py | 4 ---- tests/ci/ci_config.py | 17 ----------------- 2 files changed, 21 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 4ece21d5449..1de401cde9c 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -37,7 +37,6 @@ class BuildResult: compiler, build_type, sanitizer, - libraries, status, elapsed_seconds, with_coverage, @@ -45,7 +44,6 @@ class BuildResult: self.compiler = compiler self.build_type = build_type self.sanitizer = sanitizer - self.libraries = libraries self.status = status self.elapsed_seconds = elapsed_seconds self.with_coverage = with_coverage @@ -89,7 +87,6 @@ def get_failed_report( compiler="unknown", build_type="unknown", sanitizer="unknown", - libraries="unknown", status=message, elapsed_seconds=0, with_coverage=False, @@ -105,7 +102,6 @@ def process_report( compiler=build_config["compiler"], build_type=build_config["build_type"], sanitizer=build_config["sanitizer"], - libraries=build_config["libraries"], status="success" if build_report["status"] else "failure", elapsed_seconds=build_report["elapsed_seconds"], with_coverage=False, diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 563756de291..c77acfb679f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -13,7 +13,6 @@ CI_CONFIG = { "sanitizer": "", "package_type": "deb", "static_binary_name": "amd64", - "libraries": "static", "additional_pkgs": True, "tidy": "disable", "with_coverage": False, @@ -23,7 +22,6 @@ CI_CONFIG = { "build_type": "", "sanitizer": "", "package_type": "coverity", - "libraries": "static", "tidy": "disable", "with_coverage": False, "official": False, @@ -34,7 +32,6 @@ CI_CONFIG = { "sanitizer": "", "package_type": "deb", "static_binary_name": "aarch64", - "libraries": "static", "additional_pkgs": True, "tidy": "disable", "with_coverage": False, @@ -44,7 +41,6 @@ CI_CONFIG = { "build_type": "", "sanitizer": "address", "package_type": "deb", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -53,7 +49,6 @@ CI_CONFIG = { "build_type": "", "sanitizer": "undefined", "package_type": "deb", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -62,7 +57,6 @@ CI_CONFIG = { "build_type": "", "sanitizer": "thread", "package_type": "deb", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -71,7 +65,6 @@ CI_CONFIG = { "build_type": "", "sanitizer": "memory", "package_type": "deb", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -80,7 +73,6 @@ CI_CONFIG = { "build_type": "debug", "sanitizer": "", "package_type": "deb", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -89,7 +81,6 @@ CI_CONFIG = { "build_type": "", "sanitizer": "", "package_type": "binary", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -99,7 +90,6 @@ CI_CONFIG = { "sanitizer": "", "package_type": "binary", "static_binary_name": "debug-amd64", - "libraries": "static", "tidy": "enable", "with_coverage": False, }, @@ -109,7 +99,6 @@ CI_CONFIG = { "sanitizer": "", "package_type": "binary", "static_binary_name": "macos", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -118,7 +107,6 @@ CI_CONFIG = { "build_type": "", "sanitizer": "", "package_type": "binary", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -128,7 +116,6 @@ CI_CONFIG = { "sanitizer": "", "package_type": "binary", "static_binary_name": "aarch64v80compat", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -138,7 +125,6 @@ CI_CONFIG = { "sanitizer": "", "package_type": "binary", "static_binary_name": "freebsd", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -148,7 +134,6 @@ CI_CONFIG = { "sanitizer": "", "package_type": "binary", "static_binary_name": "macos-aarch64", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -158,7 +143,6 @@ CI_CONFIG = { "sanitizer": "", "package_type": "binary", "static_binary_name": "powerpc64le", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, @@ -168,7 +152,6 @@ CI_CONFIG = { "sanitizer": "", "package_type": "binary", "static_binary_name": "amd64compat", - "libraries": "static", "tidy": "disable", "with_coverage": False, }, From a2f17d2d9100294acf30a5b43cb81aff026ef10b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 3 Jan 2023 20:35:23 +0100 Subject: [PATCH 65/78] Fix ConcurrentBoundedQueue::emplace() return value in case of finished queue Signed-off-by: Azat Khuzhin --- src/Common/ConcurrentBoundedQueue.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index fa692a4b273..fd4a2d5790b 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -109,8 +109,7 @@ public: template [[nodiscard]] bool emplace(Args &&... args) { - emplaceImpl(std::nullopt /* timeout in milliseconds */, std::forward(args...)); - return true; + return emplaceImpl(std::nullopt /* timeout in milliseconds */, std::forward(args...)); } /// Returns false if queue is finished and empty From fc54ee5fafa0f17ee32911f0b6bd5c3560481b96 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 3 Jan 2023 19:43:25 +0000 Subject: [PATCH 66/78] Add comment about the reason for a separate file --- src/Functions/FunctionsHashingSSL.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionsHashingSSL.cpp b/src/Functions/FunctionsHashingSSL.cpp index 4f8c7808c3b..01a19a3dd6e 100644 --- a/src/Functions/FunctionsHashingSSL.cpp +++ b/src/Functions/FunctionsHashingSSL.cpp @@ -5,6 +5,8 @@ #include "FunctionsHashing.h" #include +/// SSL functions are located in the separate FunctionsHashingSSL.cpp file +/// to lower the compilation time of FunctionsHashing.cpp namespace DB { From 4e261ab230e73568a8799ee96905278636bc686b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 3 Jan 2023 21:16:38 +0100 Subject: [PATCH 67/78] Fix JSONCompactEachRow --- .../Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index a589558763c..0cafc053467 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -57,6 +57,7 @@ void JSONCompactEachRowRowOutputFormat::writeRowEndDelimiter() void JSONCompactEachRowRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) { + writeChar('\n', *ostr); size_t columns_size = columns.size(); writeRowStartDelimiter(); for (size_t i = 0; i < columns_size; ++i) From 9b58c975631b3188a64e045aa6db5ea74279897d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 4 Jan 2023 09:10:16 +0000 Subject: [PATCH 68/78] Store ZK generated data in archive --- .../test_keeper_snapshot_small_distance/test.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/integration/test_keeper_snapshot_small_distance/test.py b/tests/integration/test_keeper_snapshot_small_distance/test.py index a2d1d91cefc..6234a24817c 100644 --- a/tests/integration/test_keeper_snapshot_small_distance/test.py +++ b/tests/integration/test_keeper_snapshot_small_distance/test.py @@ -62,6 +62,13 @@ def clear_clickhouse_data(node): def convert_zookeeper_data(node): + node.exec_in_container( + [ + "bash", + "-c", + "tar -cvzf /var/lib/clickhouse/zk-data.tar.gz /zookeeper/version-2", + ] + ) cmd = "/usr/bin/clickhouse keeper-converter --zookeeper-logs-dir /zookeeper/version-2/ --zookeeper-snapshots-dir /zookeeper/version-2/ --output-dir /var/lib/clickhouse/coordination/snapshots" node.exec_in_container(["bash", "-c", cmd]) return os.path.join( From 58694ff09120d7f5f0e9bf446b5a94da27829e10 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 4 Jan 2023 09:37:56 +0000 Subject: [PATCH 69/78] Fix "AttributeError: 'BuildResult' object has no attribute 'libraries'" in BuilderSpecialReport --- tests/ci/report.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 2904a5519a9..97f6e18122f 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -275,7 +275,6 @@ tr:hover td {{filter: brightness(95%);}} Compiler Build type Sanitizer -Libraries Status Build log Build time @@ -319,8 +318,6 @@ def create_build_html_report( else: row += "none" - row += f"{build_result.libraries}" - if build_result.status: style = _get_status_style(build_result.status) row += f'{build_result.status}' From 23196ceffd43b72618586e2dc69d14d711c56884 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 4 Jan 2023 10:44:15 +0000 Subject: [PATCH 70/78] Upgrade googletest to their lastest main state --- contrib/googletest | 2 +- src/CMakeLists.txt | 6 ------ 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/contrib/googletest b/contrib/googletest index e7e591764ba..71140c3ca7a 160000 --- a/contrib/googletest +++ b/contrib/googletest @@ -1 +1 @@ -Subproject commit e7e591764baba0a0c3c9ad0014430e7a27331d16 +Subproject commit 71140c3ca7a87bb1b5b9c9f1500fea8858cce344 diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 84b75590706..1c44a38e80d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -580,13 +580,7 @@ if (ENABLE_TESTS) # gtest framework has substandard code target_compile_options(unit_tests_dbms PRIVATE - -Wno-zero-as-null-pointer-constant - -Wno-covered-switch-default - -Wno-undef -Wno-sign-compare - -Wno-used-but-marked-unused - -Wno-missing-noreturn - -Wno-gnu-zero-variadic-macro-arguments ) target_link_libraries(unit_tests_dbms PRIVATE From d5e41dbb0b68aa816641f76633d718167b1bc8fd Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Jan 2023 12:29:28 +0100 Subject: [PATCH 71/78] Update src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp --- src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp index 5f3d9dfc1d9..8a54de81815 100644 --- a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp +++ b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp @@ -26,8 +26,7 @@ void registerDiskWebServer(DiskFactory & factory, bool global_skip_access_check) ContextPtr context, const DisksMap & /*map*/) -> DiskPtr { - String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); - String uri{endpoint}; + String uri = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); if (!uri.ends_with('/')) From 890f094829d44af955a31dd8606e33f76b841f51 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 4 Jan 2023 12:04:00 +0000 Subject: [PATCH 72/78] Set retention policy for Kafka topic --- tests/integration/test_storage_kafka/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 2e8df5e142c..9f617369859 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2271,7 +2271,11 @@ def test_kafka_produce_key_timestamp(kafka_cluster): ) topic_name = "insert3" - kafka_create_topic(admin_client, topic_name) + topic_config = { + # default retention, since predefined timestamp_ms is used. + "retention.ms": "-1", + } + kafka_create_topic(admin_client, topic_name, config=topic_config) instance.query( """ From bece361a7994f8a80a2c2ef76ef8e1a830250911 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 4 Jan 2023 07:32:52 -0500 Subject: [PATCH 73/78] add command summary --- docs/en/operations/backup.md | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 061d95c1152..4feb434d762 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -9,6 +9,29 @@ slug: /en/operations/backup - [Backup/restore using an S3 disk](#backuprestore-using-an-s3-disk) - [Alternatives](#alternatives) +## Command summary + +```bash + BACKUP|RESTORE + TABLE [db.]table_name [AS [db.]table_name_in_backup] + [PARTITION[S] partition_expr [,...]] | + DICTIONARY [db.]dictionary_name [AS [db.]name_in_backup] | + DATABASE database_name [AS database_name_in_backup] + [EXCEPT TABLES ...] | + TEMPORARY TABLE table_name [AS table_name_in_backup] | + VIEW view_name [AS view_name_in_backup] + ALL TEMPORARY TABLES [EXCEPT ...] | + ALL DATABASES [EXCEPT ...] } [,...] + [ON CLUSTER 'cluster_name'] + TO|FROM File('/') | Disk('', '/') | S3('/', '', '') + [SETTINGS base_backup = File('/') | Disk(...) | S3('/', '', '')] + +``` + +:::note ALL +`ALL` is only applicable to the `RESTORE` command. +::: + ## Background While [replication](../engines/table-engines/mergetree-family/replication.md) provides protection from hardware failures, it does not protect against human errors: accidental deletion of data, deletion of the wrong table or a table on the wrong cluster, and software bugs that result in incorrect data processing or data corruption. In many cases mistakes like these will affect all replicas. ClickHouse has built-in safeguards to prevent some types of mistakes — for example, by default [you can’t just drop tables with a MergeTree-like engine containing more than 50 Gb of data](server-configuration-parameters/settings.md#max-table-size-to-drop). However, these safeguards do not cover all possible cases and can be circumvented. From e9c7555365e60ecafc8d907c911cc96fbbfae4dc Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 4 Jan 2023 14:49:39 +0100 Subject: [PATCH 74/78] Use TablesDependencyGraph in DatabaseReplicated recovery process (#44697) --- src/Databases/DatabaseReplicated.cpp | 81 ++++---- .../test_replicated_database/test.py | 173 +++++++++++++++++- 2 files changed, 212 insertions(+), 42 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index a152f21ce7b..a909c2e0d41 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1,36 +1,39 @@ #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 namespace DB { @@ -905,31 +908,37 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep for (const auto & id : dropped_tables) DatabaseCatalog::instance().waitTableFinallyDropped(id); - /// FIXME: Use proper dependency calculation instead of just moving MV to the end - using NameToMetadata = std::pair; - std::vector table_name_to_metadata_sorted; - table_name_to_metadata_sorted.reserve(table_name_to_metadata.size()); - std::move(table_name_to_metadata.begin(), table_name_to_metadata.end(), std::back_inserter(table_name_to_metadata_sorted)); - std::sort(table_name_to_metadata_sorted.begin(), table_name_to_metadata_sorted.end(), [](const NameToMetadata & lhs, const NameToMetadata & rhs) -> bool - { - const bool is_materialized_view_lhs = lhs.second.find("MATERIALIZED VIEW") != std::string::npos; - const bool is_materialized_view_rhs = rhs.second.find("MATERIALIZED VIEW") != std::string::npos; - return is_materialized_view_lhs < is_materialized_view_rhs; - }); - for (const auto & name_and_meta : table_name_to_metadata_sorted) + /// Create all needed tables in a proper order + TablesDependencyGraph tables_dependencies("DatabaseReplicated (" + getDatabaseName() + ")"); + for (const auto & [table_name, create_table_query] : table_name_to_metadata) { - if (isTableExist(name_and_meta.first, getContext())) + /// Note that table_name could contain a dot inside (e.g. .inner.1234-1234-1234-1234) + /// And QualifiedTableName::parseFromString doesn't handle this. + auto qualified_name = QualifiedTableName{.database = getDatabaseName(), .table = table_name}; + auto query_ast = parseQueryFromMetadataInZooKeeper(table_name, create_table_query); + tables_dependencies.addDependencies(qualified_name, getDependenciesFromCreateQuery(getContext(), qualified_name, query_ast)); + } + + tables_dependencies.checkNoCyclicDependencies(); + auto tables_to_create = tables_dependencies.getTablesSortedByDependency(); + + for (const auto & table_id : tables_to_create) + { + auto table_name = table_id.getTableName(); + auto create_query_string = table_name_to_metadata[table_name]; + if (isTableExist(table_name, getContext())) { - assert(name_and_meta.second == readMetadataFile(name_and_meta.first)); + assert(create_query_string == readMetadataFile(table_name)); continue; } - auto query_ast = parseQueryFromMetadataInZooKeeper(name_and_meta.first, name_and_meta.second); + auto query_ast = parseQueryFromMetadataInZooKeeper(table_name, create_query_string); LOG_INFO(log, "Executing {}", serializeAST(*query_ast)); auto create_query_context = make_query_context(); InterpreterCreateQuery(query_ast, create_query_context).execute(); } + LOG_INFO(log, "All tables are created successfully"); if (max_log_ptr_at_creation != 0) { diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 1e6a39ee1bd..d3fcc89561a 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -836,6 +836,153 @@ def test_recover_staled_replica(started_cluster): dummy_node.query("DROP DATABASE recover SYNC") +def test_recover_staled_replica_many_mvs(started_cluster): + main_node.query("DROP DATABASE IF EXISTS recover_mvs") + dummy_node.query("DROP DATABASE IF EXISTS recover_mvs") + + main_node.query_with_retry( + "CREATE DATABASE IF NOT EXISTS recover_mvs ENGINE = Replicated('/clickhouse/databases/recover_mvs', 'shard1', 'replica1');" + ) + started_cluster.get_kazoo_client("zoo1").set( + "/clickhouse/databases/recover_mvs/logs_to_keep", b"10" + ) + dummy_node.query_with_retry( + "CREATE DATABASE IF NOT EXISTS recover_mvs ENGINE = Replicated('/clickhouse/databases/recover_mvs', 'shard1', 'replica2');" + ) + + settings = {"distributed_ddl_task_timeout": 0} + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(dummy_node) + dummy_node.query_and_get_error("RENAME TABLE recover_mvs.t1 TO recover_mvs.m1") + + for identifier in ["1", "2", "3", "4"]: + main_node.query( + f"CREATE TABLE recover_mvs.rmt{identifier} (n int) ENGINE=ReplicatedMergeTree ORDER BY n", + settings=settings, + ) + + print("Created tables") + + for identifier in ["1", "2", "3", "4"]: + main_node.query( + f"CREATE TABLE recover_mvs.mv_inner{identifier} (n int) ENGINE=ReplicatedMergeTree ORDER BY n", + settings=settings, + ) + + for identifier in ["1", "2", "3", "4"]: + main_node.query_with_retry( + f"""CREATE MATERIALIZED VIEW recover_mvs.mv{identifier} + TO recover_mvs.mv_inner{identifier} + AS SELECT * FROM recover_mvs.rmt{identifier}""", + settings=settings, + ) + + print("Created MVs") + + for identifier in ["1", "2", "3", "4"]: + main_node.query_with_retry( + f"""CREATE VIEW recover_mvs.view_from_mv{identifier} + AS SELECT * FROM recover_mvs.mv{identifier}""", + settings=settings, + ) + + print("Created Views on top of MVs") + + for identifier in ["1", "2", "3", "4"]: + main_node.query_with_retry( + f"""CREATE MATERIALIZED VIEW recover_mvs.cascade_mv{identifier} + ENGINE=MergeTree() ORDER BY tuple() + POPULATE AS SELECT * FROM recover_mvs.mv_inner{identifier};""", + settings=settings, + ) + + print("Created cascade MVs") + + for identifier in ["1", "2", "3", "4"]: + main_node.query_with_retry( + f"""CREATE VIEW recover_mvs.view_from_cascade_mv{identifier} + AS SELECT * FROM recover_mvs.cascade_mv{identifier}""", + settings=settings, + ) + + print("Created Views on top of cascade MVs") + + for identifier in ["1", "2", "3", "4"]: + main_node.query_with_retry( + f"""CREATE MATERIALIZED VIEW recover_mvs.double_cascade_mv{identifier} + ENGINE=MergeTree() ORDER BY tuple() + POPULATE AS SELECT * FROM recover_mvs.`.inner_id.{get_table_uuid("recover_mvs", f"cascade_mv{identifier}")}`""", + settings=settings, + ) + + print("Created double cascade MVs") + + for identifier in ["1", "2", "3", "4"]: + main_node.query_with_retry( + f"""CREATE VIEW recover_mvs.view_from_double_cascade_mv{identifier} + AS SELECT * FROM recover_mvs.double_cascade_mv{identifier}""", + settings=settings, + ) + + print("Created Views on top of double cascade MVs") + + # This weird table name is actually makes sence because it starts with letter `a` and may break some internal sorting + main_node.query_with_retry( + """ + CREATE VIEW recover_mvs.anime + AS + SELECT n + FROM + ( + SELECT * + FROM + ( + SELECT * + FROM + ( + SELECT * + FROM recover_mvs.mv_inner1 AS q1 + INNER JOIN recover_mvs.mv_inner2 AS q2 ON q1.n = q2.n + ) AS new_table_1 + INNER JOIN recover_mvs.mv_inner3 AS q3 ON new_table_1.n = q3.n + ) AS new_table_2 + INNER JOIN recover_mvs.mv_inner4 AS q4 ON new_table_2.n = q4.n + ) + """, + settings=settings, + ) + + print("Created final boss") + + for identifier in ["1", "2", "3", "4"]: + main_node.query_with_retry( + f"""CREATE DICTIONARY recover_mvs.`11111d{identifier}` (n UInt64) + PRIMARY KEY n + SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'double_cascade_mv{identifier}' DB 'recover_mvs')) + LAYOUT(FLAT()) LIFETIME(1)""", + settings=settings, + ) + + print("Created dictionaries") + + for identifier in ["1", "2", "3", "4"]: + main_node.query_with_retry( + f"""CREATE VIEW recover_mvs.`00000vd{identifier}` + AS SELECT * FROM recover_mvs.`11111d{identifier}`""", + settings=settings, + ) + + print("Created Views on top of dictionaries") + + dummy_node.query("SYSTEM SYNC DATABASE REPLICA recover_mvs") + query = "SELECT name FROM system.tables WHERE database='recover_mvs' ORDER BY name" + assert main_node.query(query) == dummy_node.query(query) + + main_node.query("DROP DATABASE IF EXISTS recover_mvs") + dummy_node.query("DROP DATABASE IF EXISTS recover_mvs") + + def test_startup_without_zk(started_cluster): with PartitionManager() as pm: pm.drop_instance_zk_connections(main_node) @@ -1005,6 +1152,9 @@ def test_force_synchronous_settings(started_cluster): def test_recover_digest_mismatch(started_cluster): + main_node.query("DROP DATABASE IF EXISTS recover_digest_mismatch") + dummy_node.query("DROP DATABASE IF EXISTS recover_digest_mismatch") + main_node.query( "CREATE DATABASE recover_digest_mismatch ENGINE = Replicated('/clickhouse/databases/recover_digest_mismatch', 'shard1', 'replica1');" ) @@ -1014,19 +1164,22 @@ def test_recover_digest_mismatch(started_cluster): create_some_tables("recover_digest_mismatch") + main_node.query("SYSTEM SYNC DATABASE REPLICA recover_digest_mismatch") + dummy_node.query("SYSTEM SYNC DATABASE REPLICA recover_digest_mismatch") + ways_to_corrupt_metadata = [ - f"mv /var/lib/clickhouse/metadata/recover_digest_mismatch/t1.sql /var/lib/clickhouse/metadata/recover_digest_mismatch/m1.sql", - f"sed --follow-symlinks -i 's/Int32/String/' /var/lib/clickhouse/metadata/recover_digest_mismatch/mv1.sql", - f"rm -f /var/lib/clickhouse/metadata/recover_digest_mismatch/d1.sql", + "mv /var/lib/clickhouse/metadata/recover_digest_mismatch/t1.sql /var/lib/clickhouse/metadata/recover_digest_mismatch/m1.sql", + "sed --follow-symlinks -i 's/Int32/String/' /var/lib/clickhouse/metadata/recover_digest_mismatch/mv1.sql", + "rm -f /var/lib/clickhouse/metadata/recover_digest_mismatch/d1.sql", # f"rm -rf /var/lib/clickhouse/metadata/recover_digest_mismatch/", # Directory already exists - f"rm -rf /var/lib/clickhouse/store", + "rm -rf /var/lib/clickhouse/store", ] for command in ways_to_corrupt_metadata: + print(f"Corrupting data using `{command}`") need_remove_is_active_node = "rm -rf" in command dummy_node.stop_clickhouse(kill=not need_remove_is_active_node) dummy_node.exec_in_container(["bash", "-c", command]) - dummy_node.start_clickhouse() query = ( "SELECT name, uuid, create_table_query FROM system.tables WHERE database='recover_digest_mismatch' AND name NOT LIKE '.inner_id.%' " @@ -1034,10 +1187,18 @@ def test_recover_digest_mismatch(started_cluster): ) expected = main_node.query(query) - if "rm -rf" in command: + if need_remove_is_active_node: # NOTE Otherwise it fails to recreate ReplicatedMergeTree table due to "Replica already exists" main_node.query( "SYSTEM DROP REPLICA '2' FROM DATABASE recover_digest_mismatch" ) + # There is a race condition between deleting active node and creating it on server startup + # So we start a server only after we deleted all table replicas from the Keeper + dummy_node.start_clickhouse() assert_eq_with_retry(dummy_node, query, expected) + + main_node.query("DROP DATABASE IF EXISTS recover_digest_mismatch") + dummy_node.query("DROP DATABASE IF EXISTS recover_digest_mismatch") + + print("Everything Okay") From 530f0b36e9c52427fecbd73170630a5d0e12b257 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 4 Jan 2023 09:46:36 -0500 Subject: [PATCH 75/78] add toUUIDOrDefault docs --- .../functions/type-conversion-functions.md | 112 +++++++++--------- .../sql-reference/functions/uuid-functions.md | 64 ++++++++-- 2 files changed, 112 insertions(+), 64 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index bae45de07e9..bb72ab7cfc3 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -8,7 +8,7 @@ sidebar_label: Type Conversion ## Common Issues of Numeric Conversions -When you convert a value from one to another data type, you should remember that in common case, it is an unsafe operation that can lead to a data loss. A data loss can occur if you try to fit value from a larger data type to a smaller data type, or if you convert values between different data types. +When you convert a value from one to another data type, you should remember that if you try to fit a value from a larger data type to a smaller one (for example Int64 to Int32), or convert from one data type to another (for example `String` to `Int`), you could have data loss. Test beforehand. ClickHouse has the [same behavior as C++ programs](https://en.cppreference.com/w/cpp/language/implicit_conversion). @@ -45,7 +45,7 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); Result: -``` text +```response ┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ │ -9223372036854775808 │ 32 │ 16 │ 8 │ └──────────────────────┴─────────────┴───────────────┴─────────────┘ @@ -65,7 +65,7 @@ SELECT toInt64OrZero('123123'), toInt8OrZero('123qwe123'); Result: -``` text +```response ┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ │ 123123 │ 0 │ └─────────────────────────┴───────────────────────────┘ @@ -85,7 +85,7 @@ SELECT toInt64OrNull('123123'), toInt8OrNull('123qwe123'); Result: -``` text +```response ┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ │ 123123 │ ᴺᵁᴸᴸ │ └─────────────────────────┴───────────────────────────┘ @@ -105,7 +105,7 @@ SELECT toInt64OrDefault('123123', cast('-1' as Int64)), toInt8OrDefault('123qwe1 Result: -``` text +```response ┌─toInt64OrDefault('123123', CAST('-1', 'Int64'))─┬─toInt8OrDefault('123qwe123', CAST('-1', 'Int8'))─┐ │ 123123 │ -1 │ └─────────────────────────────────────────────────┴──────────────────────────────────────────────────┘ @@ -144,7 +144,7 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); Result: -``` text +```response ┌───────toUInt64(nan)─┬─toUInt32(-32)─┬─toUInt16('16')─┬─toUInt8(8.8)─┐ │ 9223372036854775808 │ 4294967264 │ 16 │ 8 │ └─────────────────────┴───────────────┴────────────────┴──────────────┘ @@ -314,7 +314,7 @@ Type: [Date32](/docs/en/sql-reference/data-types/date32.md). SELECT toDate32('1955-01-01') AS value, toTypeName(value); ``` -``` text +```response ┌──────value─┬─toTypeName(toDate32('1925-01-01'))─┐ │ 1955-01-01 │ Date32 │ └────────────┴────────────────────────────────────┘ @@ -326,7 +326,7 @@ SELECT toDate32('1955-01-01') AS value, toTypeName(value); SELECT toDate32('1899-01-01') AS value, toTypeName(value); ``` -``` text +```response ┌──────value─┬─toTypeName(toDate32('1899-01-01'))─┐ │ 1900-01-01 │ Date32 │ └────────────┴────────────────────────────────────┘ @@ -338,7 +338,7 @@ SELECT toDate32('1899-01-01') AS value, toTypeName(value); SELECT toDate32(toDate('1899-01-01')) AS value, toTypeName(value); ``` -``` text +```response ┌──────value─┬─toTypeName(toDate32(toDate('1899-01-01')))─┐ │ 1970-01-01 │ Date32 │ └────────────┴────────────────────────────────────────────┘ @@ -358,7 +358,7 @@ SELECT toDate32OrZero('1899-01-01'), toDate32OrZero(''); Result: -``` text +```response ┌─toDate32OrZero('1899-01-01')─┬─toDate32OrZero('')─┐ │ 1900-01-01 │ 1900-01-01 │ └──────────────────────────────┴────────────────────┘ @@ -378,7 +378,7 @@ SELECT toDate32OrNull('1955-01-01'), toDate32OrNull(''); Result: -``` text +```response ┌─toDate32OrNull('1955-01-01')─┬─toDate32OrNull('')─┐ │ 1955-01-01 │ ᴺᵁᴸᴸ │ └──────────────────────────────┴────────────────────┘ @@ -400,7 +400,7 @@ SELECT Result: -``` text +```response ┌─toDate32OrDefault('1930-01-01', toDate32('2020-01-01'))─┬─toDate32OrDefault('xx1930-01-01', toDate32('2020-01-01'))─┐ │ 1930-01-01 │ 2020-01-01 │ └─────────────────────────────────────────────────────────┴───────────────────────────────────────────────────────────┘ @@ -436,7 +436,7 @@ Type: [DateTime64](/docs/en/sql-reference/data-types/datetime64.md). SELECT toDateTime64('1955-01-01 00:00:00.000', 3) AS value, toTypeName(value); ``` -``` text +```response ┌───────────────────value─┬─toTypeName(toDateTime64('1955-01-01 00:00:00.000', 3))─┐ │ 1955-01-01 00:00:00.000 │ DateTime64(3) │ └─────────────────────────┴────────────────────────────────────────────────────────┘ @@ -448,7 +448,7 @@ SELECT toDateTime64('1955-01-01 00:00:00.000', 3) AS value, toTypeName(value); SELECT toDateTime64(1546300800.000, 3) AS value, toTypeName(value); ``` -``` text +```response ┌───────────────────value─┬─toTypeName(toDateTime64(1546300800., 3))─┐ │ 2019-01-01 00:00:00.000 │ DateTime64(3) │ └─────────────────────────┴──────────────────────────────────────────┘ @@ -460,7 +460,7 @@ Without the decimal point the value is still treated as Unix Timestamp in second SELECT toDateTime64(1546300800000, 3) AS value, toTypeName(value); ``` -``` text +```response ┌───────────────────value─┬─toTypeName(toDateTime64(1546300800000, 3))─┐ │ 2282-12-31 00:00:00.000 │ DateTime64(3) │ └─────────────────────────┴────────────────────────────────────────────┘ @@ -473,7 +473,7 @@ SELECT toDateTime64(1546300800000, 3) AS value, toTypeName(value); SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Asia/Istanbul') AS value, toTypeName(value); ``` -``` text +```response ┌───────────────────value─┬─toTypeName(toDateTime64('2019-01-01 00:00:00', 3, 'Asia/Istanbul'))─┐ │ 2019-01-01 00:00:00.000 │ DateTime64(3, 'Asia/Istanbul') │ └─────────────────────────┴─────────────────────────────────────────────────────────────────────┘ @@ -522,7 +522,7 @@ SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val); Result: -``` text +```response ┌────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┐ │ -1.111 │ Nullable(Decimal(9, 5)) │ └────────┴────────────────────────────────────────────────────┘ @@ -536,7 +536,7 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); Result: -``` text +```response ┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┐ │ ᴺᵁᴸᴸ │ Nullable(Decimal(9, 2)) │ └──────┴────────────────────────────────────────────────────┘ @@ -576,7 +576,7 @@ SELECT toDecimal32OrDefault(toString(-1.111), 5) AS val, toTypeName(val); Result: -``` text +```response ┌────val─┬─toTypeName(toDecimal32OrDefault(toString(-1.111), 5))─┐ │ -1.111 │ Decimal(9, 5) │ └────────┴───────────────────────────────────────────────────────┘ @@ -590,7 +590,7 @@ SELECT toDecimal32OrDefault(toString(-1.111), 2) AS val, toTypeName(val); Result: -``` text +```response ┌─val─┬─toTypeName(toDecimal32OrDefault(toString(-1.111), 2))─┐ │ 0 │ Decimal(9, 2) │ └─────┴───────────────────────────────────────────────────────┘ @@ -629,7 +629,7 @@ SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val); Result: -``` text +```response ┌────val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 5))─┐ │ -1.111 │ Decimal(9, 5) │ └────────┴────────────────────────────────────────────────────┘ @@ -643,7 +643,7 @@ SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val); Result: -``` text +```response ┌──val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 2))─┐ │ 0.00 │ Decimal(9, 2) │ └──────┴────────────────────────────────────────────────────┘ @@ -661,7 +661,7 @@ When converting dates with times to numbers or vice versa, the date with time co The date and date-with-time formats for the toDate/toDateTime functions are defined as follows: -``` text +```response YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` @@ -686,7 +686,7 @@ SELECT Result: -``` text +```response ┌───────────now_local─┬─now_yekat───────────┐ │ 2016-06-15 00:11:21 │ 2016-06-15 02:11:21 │ └─────────────────────┴─────────────────────┘ @@ -713,7 +713,7 @@ SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut; Result: -``` text +```response ┌─s─────────────┬─s_cut─┐ │ foo\0\0\0\0\0 │ foo │ └───────────────┴───────┘ @@ -727,7 +727,7 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; Result: -``` text +```response ┌─s──────────┬─s_cut─┐ │ foo\0bar\0 │ foo │ └────────────┴───────┘ @@ -755,6 +755,10 @@ This function accepts a number or date or date with time and returns a FixedStri ## reinterpretAsUUID +:::note +In addition to the UUID functions listed here, there is dedicated [UUID function documentation](/docs/en/sql-reference/functions/uuid-functions.md). +::: + Accepts 16 bytes string and returns UUID containing bytes representing the corresponding value in network byte order (big-endian). If the string isn't long enough, the function works as if the string is padded with the necessary number of null bytes to the end. If the string is longer than 16 bytes, the extra bytes at the end are ignored. **Syntax** @@ -783,7 +787,7 @@ SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))); Result: -``` text +```response ┌─reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f')))─┐ │ 08090a0b-0c0d-0e0f-0001-020304050607 │ └───────────────────────────────────────────────────────────────────────┘ @@ -803,7 +807,7 @@ SELECT uuid = uuid2; Result: -``` text +```response ┌─equals(uuid, uuid2)─┐ │ 1 │ └─────────────────────┘ @@ -904,7 +908,7 @@ SELECT Result: -``` text +```response ┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ └─────────────────────┴─────────────────────┴────────────┴─────────────────────┴───────────────────────────┘ @@ -924,7 +928,7 @@ SELECT toTypeName(x) FROM t_null; Result: -``` text +```response ┌─toTypeName(x)─┐ │ Int8 │ │ Int8 │ @@ -939,7 +943,7 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; Result: -``` text +```response ┌─toTypeName(CAST(x, 'Nullable(UInt16)'))─┐ │ Nullable(UInt16) │ │ Nullable(UInt16) │ @@ -966,7 +970,7 @@ SELECT cast(-1, 'UInt8') as uint8; Result: -``` text +```response ┌─uint8─┐ │ 255 │ └───────┘ @@ -980,7 +984,7 @@ SELECT accurateCast(-1, 'UInt8') as uint8; Result: -``` text +```response Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in column Int8 cannot be safely converted into type UInt8: While processing accurateCast(-1, 'UInt8') AS uint8. ``` @@ -1013,7 +1017,7 @@ SELECT toTypeName(accurateCastOrNull(5, 'UInt8')); Result: -``` text +```response ┌─toTypeName(accurateCastOrNull(5, 'UInt8'))─┐ │ Nullable(UInt8) │ └────────────────────────────────────────────┘ @@ -1030,7 +1034,7 @@ SELECT Result: -``` text +```response ┌─uint8─┬─int8─┬─fixed_string─┐ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ └───────┴──────┴──────────────┘ @@ -1067,7 +1071,7 @@ SELECT toTypeName(accurateCastOrDefault(5, 'UInt8')); Result: -``` text +```response ┌─toTypeName(accurateCastOrDefault(5, 'UInt8'))─┐ │ UInt8 │ └───────────────────────────────────────────────┘ @@ -1087,7 +1091,7 @@ SELECT Result: -``` text +```response ┌─uint8─┬─uint8_default─┬─int8─┬─int8_default─┬─fixed_string─┬─fixed_string_default─┐ │ 0 │ 5 │ 0 │ 5 │ │ Te │ └───────┴───────────────┴──────┴──────────────┴──────────────┴──────────────────────┘ @@ -1134,7 +1138,7 @@ SELECT Result: -``` text +```response ┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐ │ 2019-01-08 │ 2019-01-08 │ └───────────────────────────┴──────────────────────────────┘ @@ -1183,7 +1187,7 @@ AS parseDateTimeBestEffort; Result: -``` text +```response ┌─parseDateTimeBestEffort─┐ │ 2020-10-23 12:12:57 │ └─────────────────────────┘ @@ -1198,7 +1202,7 @@ AS parseDateTimeBestEffort; Result: -``` text +```response ┌─parseDateTimeBestEffort─┐ │ 2018-08-18 10:22:16 │ └─────────────────────────┘ @@ -1213,7 +1217,7 @@ AS parseDateTimeBestEffort; Result: -``` text +```response ┌─parseDateTimeBestEffort─┐ │ 2015-07-07 12:04:41 │ └─────────────────────────┘ @@ -1228,7 +1232,7 @@ AS parseDateTimeBestEffort; Result: -``` text +```response ┌─parseDateTimeBestEffort─┐ │ 2018-10-23 10:12:12 │ └─────────────────────────┘ @@ -1242,7 +1246,7 @@ SELECT parseDateTimeBestEffort('10 20:19'); Result: -``` text +```response ┌─parseDateTimeBestEffort('10 20:19')─┐ │ 2000-01-10 20:19:00 │ └─────────────────────────────────────┘ @@ -1376,7 +1380,7 @@ SELECT toLowCardinality('1'); Result: -``` text +```response ┌─toLowCardinality('1')─┐ │ 1 │ └───────────────────────┘ @@ -1419,7 +1423,7 @@ SELECT toUnixTimestamp64Milli(dt64); Result: -``` text +```response ┌─toUnixTimestamp64Milli(dt64)─┐ │ 1568650812345 │ └──────────────────────────────┘ @@ -1434,7 +1438,7 @@ SELECT toUnixTimestamp64Nano(dt64); Result: -``` text +```response ┌─toUnixTimestamp64Nano(dt64)─┐ │ 1568650812345678000 │ └─────────────────────────────┘ @@ -1474,7 +1478,7 @@ SELECT fromUnixTimestamp64Milli(i64, 'UTC'); Result: -``` text +```response ┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ │ 2009-02-13 23:31:31.011 │ └──────────────────────────────────────┘ @@ -1510,7 +1514,7 @@ FROM numbers(3); Result: -``` text +```response ┌─formatRow('CSV', number, 'good')─┐ │ 0,"good" │ @@ -1535,7 +1539,7 @@ SETTINGS format_custom_result_before_delimiter='\n', format_custom_resul Result: -``` text +```response ┌─formatRow('CustomSeparated', number, 'good')─┐ │ 0 good @@ -1581,7 +1585,7 @@ FROM numbers(3); Result: -``` text +```response ┌─formatRowNoNewline('CSV', number, 'good')─┐ │ 0,"good" │ │ 1,"good" │ @@ -1618,7 +1622,7 @@ SELECT snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC'); Result: -``` text +```response ┌─snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC')─┐ │ 2021-08-15 10:57:56 │ @@ -1654,7 +1658,7 @@ SELECT snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC'); Result: -``` text +```response ┌─snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC')─┐ │ 2021-08-15 10:58:19.841 │ @@ -1689,7 +1693,7 @@ WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt SELECT dateTimeToS Result: -``` text +```response ┌─dateTimeToSnowflake(dt)─┐ │ 1426860702823350272 │ └─────────────────────────┘ @@ -1723,7 +1727,7 @@ WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 SELECT Result: -``` text +```response ┌─dateTime64ToSnowflake(dt64)─┐ │ 1426860704886947840 │ └─────────────────────────────┘ diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index 43542367cd5..474e3248d1f 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -38,7 +38,7 @@ INSERT INTO t_uuid SELECT generateUUIDv4() SELECT * FROM t_uuid ``` -``` text +```response ┌────────────────────────────────────x─┐ │ f4bf890f-f9dc-4332-ad5c-0c18e73f28e9 │ └──────────────────────────────────────┘ @@ -89,7 +89,7 @@ SELECT empty(generateUUIDv4()); Result: -```text +```response ┌─empty(generateUUIDv4())─┐ │ 0 │ └─────────────────────────┘ @@ -131,7 +131,7 @@ SELECT notEmpty(generateUUIDv4()); Result: -```text +```response ┌─notEmpty(generateUUIDv4())─┐ │ 1 │ └────────────────────────────┘ @@ -155,12 +155,56 @@ The UUID type value. SELECT toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0') AS uuid ``` -``` text +```response ┌─────────────────────────────────uuid─┐ │ 61f0c404-5cb3-11e7-907b-a6006ad3dba0 │ └──────────────────────────────────────┘ ``` +## toUUIDOrDefault (x,y) + +**Arguments** + +- `string` — String of 36 characters or FixedString(36). [String](../../sql-reference/syntax.md#string). +- `default` — UUID to be used as the default if the first argument cannot be converted to a UUID type. [UUID](/docs/en/sql-reference/data-types/uuid.md). + +**Returned value** + +UUID + +``` sql +toUUIDOrDefault(String, UUID) +``` + +**Returned value** + +The UUID type value. + +**Usage examples** + +This first example returns the first argument converted to a UUID type as it can be converted: + +``` sql +SELECT toUUIDOrDefault('61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); +``` +```response +┌─toUUIDOrDefault('61f0c404-5cb3-11e7-907b-a6006ad3dba0', CAST('59f0c404-5cb3-11e7-907b-a6006ad3dba0', 'UUID'))─┐ +│ 61f0c404-5cb3-11e7-907b-a6006ad3dba0 │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +This second example returns the second argument (the provided default UUID) as the first argument cannot be converted to a UUID type: + +```sql +SELECT toUUIDOrDefault('-----61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); +``` + +```response +┌─toUUIDOrDefault('-----61f0c404-5cb3-11e7-907b-a6006ad3dba0', CAST('59f0c404-5cb3-11e7-907b-a6006ad3dba0', 'UUID'))─┐ +│ 59f0c404-5cb3-11e7-907b-a6006ad3dba0 │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + ## toUUIDOrNull (x) It takes an argument of type String and tries to parse it into UUID. If failed, returns NULL. @@ -179,7 +223,7 @@ The Nullable(UUID) type value. SELECT toUUIDOrNull('61f0c404-5cb3-11e7-907b-a6006ad3dba0T') AS uuid ``` -``` text +```response ┌─uuid─┐ │ ᴺᵁᴸᴸ │ └──────┘ @@ -203,7 +247,7 @@ The UUID type value. SELECT toUUIDOrZero('61f0c404-5cb3-11e7-907b-a6006ad3dba0T') AS uuid ``` -``` text +```response ┌─────────────────────────────────uuid─┐ │ 00000000-0000-0000-0000-000000000000 │ └──────────────────────────────────────┘ @@ -236,7 +280,7 @@ SELECT UUIDStringToNum(uuid) AS bytes ``` -``` text +```response ┌─uuid─────────────────────────────────┬─bytes────────────┐ │ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ └──────────────────────────────────────┴──────────────────┘ @@ -248,7 +292,7 @@ SELECT UUIDStringToNum(uuid, 2) AS bytes ``` -``` text +```response ┌─uuid─────────────────────────────────┬─bytes────────────┐ │ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ @ Date: Wed, 4 Jan 2023 11:36:38 -0500 Subject: [PATCH 76/78] add example with file globbing --- .../sql-reference/statements/insert-into.md | 28 +++++++++++++------ 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 764ea9b0292..c1437226814 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -91,11 +91,11 @@ INSERT INTO t FORMAT TabSeparated You can insert data separately from the query by using the command-line client or the HTTP interface. For more information, see the section “[Interfaces](../../interfaces)”. -### Constraints +## Constraints If table has [constraints](../../sql-reference/statements/create/table.md#constraints), their expressions will be checked for each row of inserted data. If any of those constraints is not satisfied — server will raise an exception containing constraint name and expression, the query will be stopped. -### Inserting the Results of `SELECT` +## Inserting the Results of `SELECT` **Syntax** @@ -114,7 +114,7 @@ However, you can delete old data using `ALTER TABLE ... DROP PARTITION`. To insert a default value instead of `NULL` into a column with not nullable data type, enable [insert_null_as_default](../../operations/settings/settings.md#insert_null_as_default) setting. -### Inserting Data from a File +## Inserting Data from a File **Syntax** @@ -122,14 +122,15 @@ To insert a default value instead of `NULL` into a column with not nullable data INSERT INTO [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name ``` -Use the syntax above to insert data from a file stored on a **client** side. `file_name` and `type` are string literals. Input file [format](../../interfaces/formats.md) must be set in the `FORMAT` clause. +Use the syntax above to insert data from a file, or files, stored on the **client** side. `file_name` and `type` are string literals. Input file [format](../../interfaces/formats.md) must be set in the `FORMAT` clause. -Compressed files are supported. Compression type is detected by the extension of the file name. Or it can be explicitly specified in a `COMPRESSION` clause. Supported types are: `'none'`, `'gzip'`, `'deflate'`, `'br'`, `'xz'`, `'zstd'`, `'lz4'`, `'bz2'`. +Compressed files are supported. The compression type is detected by the extension of the file name. Or it can be explicitly specified in a `COMPRESSION` clause. Supported types are: `'none'`, `'gzip'`, `'deflate'`, `'br'`, `'xz'`, `'zstd'`, `'lz4'`, `'bz2'`. This functionality is available in the [command-line client](../../interfaces/cli.md) and [clickhouse-local](../../operations/utilities/clickhouse-local.md). -**Example** +**Examples** +### Single file with FROM INFILE Execute the following queries using [command-line client](../../interfaces/cli.md): ```bash @@ -148,7 +149,18 @@ Result: └────┴──────┘ ``` -### Inserting into Table Function +### Multiple files with FROM INFILE using globs + +This example is very similar to the previous one but inserts from multiple files using `FROM INFILE 'input_*.csv`. + +```bash +echo 1,A > input_1.csv ; echo 2,B > input_2.csv +clickhouse-client --query="CREATE TABLE infile_globs (id UInt32, text String) ENGINE=MergeTree() ORDER BY id;" +clickhouse-client --query="INSERT INTO infile_globs FROM INFILE 'input_*.csv' FORMAT CSV;" +clickhouse-client --query="SELECT * FROM infile_globs FORMAT PrettyCompact;" +``` + +## Inserting into Table Function Data can be inserted into tables referenced by [table functions](../../sql-reference/table-functions/index.md). @@ -176,7 +188,7 @@ Result: └─────┴───────────────────────┘ ``` -### Performance Considerations +## Performance Considerations `INSERT` sorts the input data by primary key and splits them into partitions by a partition key. If you insert data into several partitions at once, it can significantly reduce the performance of the `INSERT` query. To avoid this: From 51f1d3ba1ecb3ade8fa46f0747e435d4941dd5cc Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 4 Jan 2023 14:53:02 -0500 Subject: [PATCH 77/78] revert doc removal --- docs/en/sql-reference/functions/geo/h3.md | 180 ++++++++++++++++++++++ 1 file changed, 180 insertions(+) diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index 78f7b4dd694..1149535434d 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -1027,6 +1027,186 @@ Result: └─────────────┘ ``` +## h3PointDistM + +Returns the "great circle" or "haversine" distance between pairs of GeoCoord points (latitude/longitude) pairs in meters. + +**Syntax** + +``` sql +h3PointDistM(lat1, lon1, lat2, lon2) +``` + +**Arguments** + +- `lat1`, `lon1` — Latitude and Longitude of point1 in degrees. Type: [Float64](../../../sql-reference/data-types/float.md). +- `lat2`, `lon2` — Latitude and Longitude of point2 in degrees. Type: [Float64](../../../sql-reference/data-types/float.md). + +**Returned values** + +- Haversine or great circle distance in meters. + +Type: [Float64](../../../sql-reference/data-types/float.md). + +**Example** + +Query: + +``` sql +select h3PointDistM(-10.0 ,0.0, 10.0, 0.0) as h3PointDistM; +``` + +Result: + +``` text +┌──────h3PointDistM─┐ +│ 2223901.039504589 │ +└───────────────────┘ +``` + +## h3PointDistKm + +Returns the "great circle" or "haversine" distance between pairs of GeoCoord points (latitude/longitude) pairs in kilometers. + +**Syntax** + +``` sql +h3PointDistKm(lat1, lon1, lat2, lon2) +``` + +**Arguments** + +- `lat1`, `lon1` — Latitude and Longitude of point1 in degrees. Type: [Float64](../../../sql-reference/data-types/float.md). +- `lat2`, `lon2` — Latitude and Longitude of point2 in degrees. Type: [Float64](../../../sql-reference/data-types/float.md). + +**Returned values** + +- Haversine or great circle distance in kilometers. + +Type: [Float64](../../../sql-reference/data-types/float.md). + +**Example** + +Query: + +``` sql +select h3PointDistKm(-10.0 ,0.0, 10.0, 0.0) as h3PointDistKm; +``` + +Result: + +``` text +┌─────h3PointDistKm─┐ +│ 2223.901039504589 │ +└───────────────────┘ +``` + +## h3PointDistRads + +Returns the "great circle" or "haversine" distance between pairs of GeoCoord points (latitude/longitude) pairs in radians. + +**Syntax** + +``` sql +h3PointDistRads(lat1, lon1, lat2, lon2) +``` + +**Arguments** + +- `lat1`, `lon1` — Latitude and Longitude of point1 in degrees. Type: [Float64](../../../sql-reference/data-types/float.md). +- `lat2`, `lon2` — Latitude and Longitude of point2 in degrees. Type: [Float64](../../../sql-reference/data-types/float.md). + +**Returned values** + +- Haversine or great circle distance in radians. + +Type: [Float64](../../../sql-reference/data-types/float.md). + +**Example** + +Query: + +``` sql +select h3PointDistRads(-10.0 ,0.0, 10.0, 0.0) as h3PointDistRads; +``` + +Result: + +``` text +┌────h3PointDistRads─┐ +│ 0.3490658503988659 │ +└────────────────────┘ +``` + +## h3GetRes0Indexes + +Returns an array of all the resolution 0 H3 indexes. + +**Syntax** + +``` sql +h3GetRes0Indexes() +``` + +**Returned values** + +- Array of all the resolution 0 H3 indexes. + +Type: [Array](../../../sql-reference/data-types/array.md)([UInt64](../../../sql-reference/data-types/int-uint.md)). + + +**Example** + +Query: + +``` sql +select h3GetRes0Indexes as indexes ; +``` + +Result: + +``` text +┌─indexes─────────────────────────────────────┐ +│ [576495936675512319,576531121047601151,....]│ +└─────────────────────────────────────────────┘ +``` + +## h3GetPentagonIndexes + +Returns all the pentagon H3 indexes at the specified resolution. + +**Syntax** + +``` sql +h3GetPentagonIndexes(resolution) +``` + +**Parameter** + +- `resolution` — Index resolution. Range: `[0, 15]`. Type: [UInt8](../../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Array of all pentagon H3 indexes. + +Type: [Array](../../../sql-reference/data-types/array.md)([UInt64](../../../sql-reference/data-types/int-uint.md)). + +**Example** + +Query: + +``` sql +SELECT h3GetPentagonIndexes(3) AS indexes; +``` + +Result: + +``` text +┌─indexes────────────────────────────────────────────────────────┐ +│ [590112357393367039,590464201114255359,590816044835143679,...] │ +└────────────────────────────────────────────────────────────────┘ +``` + ## h3Line Returns the line of indices between the two indices that are provided. From 2397318b1a823340db01c6f0a94d10db7d97e15d Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 4 Jan 2023 15:37:13 -0500 Subject: [PATCH 78/78] add more about matching --- docs/en/sql-reference/statements/insert-into.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index c1437226814..609740dec56 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -160,6 +160,15 @@ clickhouse-client --query="INSERT INTO infile_globs FROM INFILE 'input_*.csv' FO clickhouse-client --query="SELECT * FROM infile_globs FORMAT PrettyCompact;" ``` +:::tip +In addition to selecting multiple files with `*`, you can use ranges (`{1,2}` or `{1..9}`) and other [glob substitutions](/docs/en/sql-reference/table-functions/file.md/#globs-in-path). These three all would work with the above example: +```sql +INSERT INTO infile_globs FROM INFILE 'input_*.csv' FORMAT CSV; +INSERT INTO infile_globs FROM INFILE 'input_{1,2}.csv' FORMAT CSV; +INSERT INTO infile_globs FROM INFILE 'input_?.csv' FORMAT CSV; +``` +::: + ## Inserting into Table Function Data can be inserted into tables referenced by [table functions](../../sql-reference/table-functions/index.md).