From e82a5d43b5304d30452b7f5fbcf03f9019fe7243 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 9 Nov 2022 09:02:04 +0100 Subject: [PATCH 01/62] Extract HTTPContext for HTTPServer --- programs/keeper/Keeper.cpp | 74 +++++++++++++++++++ programs/keeper/Keeper.h | 4 + programs/server/Server.cpp | 70 ++++++++++++++++-- programs/server/Server.h | 4 + src/Server/HTTP/HTTPContext.h | 24 ++++++ src/Server/HTTP/HTTPServer.cpp | 2 +- src/Server/HTTP/HTTPServer.h | 5 +- src/Server/HTTP/HTTPServerConnection.cpp | 6 +- src/Server/HTTP/HTTPServerConnection.h | 5 +- .../HTTP/HTTPServerConnectionFactory.cpp | 4 +- src/Server/HTTP/HTTPServerConnectionFactory.h | 5 +- src/Server/HTTP/HTTPServerRequest.cpp | 16 ++-- src/Server/HTTP/HTTPServerRequest.h | 3 +- src/Server/HTTPHandlerFactory.cpp | 10 +-- src/Server/HTTPHandlerFactory.h | 7 ++ src/Server/PrometheusRequestHandler.cpp | 15 ++++ src/Server/PrometheusRequestHandler.h | 1 + 17 files changed, 216 insertions(+), 39 deletions(-) create mode 100644 src/Server/HTTP/HTTPContext.h diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index e1d03b40b66..ddff0fa2ad7 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -24,6 +24,9 @@ #include #include +#include + +#include "Core/Defines.h" #include "config.h" #include "config_version.h" @@ -273,6 +276,54 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options) BaseDaemon::defineOptions(options); } +struct Keeper::KeeperHTTPContext : public IHTTPContext +{ + uint64_t getMaxHstsAge() const override + { + return 0; + } + + uint64_t getMaxUriSize() const override + { + return 1048576; + } + + uint64_t getMaxFields() const override + { + return 1000000; + } + + uint64_t getMaxFieldNameSize() const override + { + return 1048576; + } + + uint64_t getMaxFieldValueSize() const override + { + return 1048576; + } + + uint64_t getMaxChunkSize() const override + { + return 100_GiB; + } + + Poco::Timespan getReceiveTimeout() const override + { + return DEFAULT_HTTP_READ_BUFFER_TIMEOUT; + } + + Poco::Timespan getSendTimeout() const override + { + return DEFAULT_HTTP_READ_BUFFER_TIMEOUT; + } +}; + +HTTPContextPtr Keeper::httpContext() +{ + return std::make_shared(); +} + int Keeper::main(const std::vector & /*args*/) { Poco::Logger * log = &logger(); @@ -412,6 +463,29 @@ int Keeper::main(const std::vector & /*args*/) ErrorCodes::SUPPORT_IS_DISABLED}; #endif }); + + const auto & config = config_getter(); + Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0); + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC); + http_params->setKeepAliveTimeout(keep_alive_timeout); + + /// Prometheus (if defined and not setup yet with http_port) + port_name = "prometheus.port"; + createServer(listen_host, port_name, listen_try, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + // TODO(antonio2368): use config + socket.setReceiveTimeout(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC); + socket.setSendTimeout(DBMS_DEFAULT_SEND_TIMEOUT_SEC); + return ProtocolServerAdapter( + listen_host, + port_name, + "Prometheus: http://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); + }); } for (auto & server : *servers) diff --git a/programs/keeper/Keeper.h b/programs/keeper/Keeper.h index 75cd9b825d0..60481b02bdb 100644 --- a/programs/keeper/Keeper.h +++ b/programs/keeper/Keeper.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include "TinyContext.h" @@ -58,6 +59,9 @@ protected: private: TinyContext tiny_context; + struct KeeperHTTPContext; + static HTTPContextPtr httpContext(); + Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; using CreateServerFunc = std::function; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b412b579539..74cd64c442b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1878,15 +1878,15 @@ std::unique_ptr Server::buildProtocolStackFromConfig( return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory(*this)); if (type == "http") return TCPServerConnectionFactory::Ptr( - new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory")) + new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory")) ); if (type == "prometheus") return TCPServerConnectionFactory::Ptr( - new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory")) + new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory")) ); if (type == "interserver") return TCPServerConnectionFactory::Ptr( - new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory")) + new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory")) ); throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type); @@ -1927,6 +1927,60 @@ std::unique_ptr Server::buildProtocolStackFromConfig( return stack; } +struct Server::ServerHTTPContext : public IHTTPContext +{ + explicit ServerHTTPContext(ContextPtr context_) + : context(Context::createCopy(context_)) + {} + + uint64_t getMaxHstsAge() const override + { + return context->getSettingsRef().hsts_max_age; + } + + uint64_t getMaxUriSize() const override + { + return context->getSettingsRef().http_max_uri_size; + } + + uint64_t getMaxFields() const override + { + return context->getSettingsRef().http_max_fields; + } + + uint64_t getMaxFieldNameSize() const override + { + return context->getSettingsRef().http_max_field_name_size; + } + + uint64_t getMaxFieldValueSize() const override + { + return context->getSettingsRef().http_max_field_value_size; + } + + uint64_t getMaxChunkSize() const override + { + return context->getSettingsRef().http_max_chunk_size; + } + + Poco::Timespan getReceiveTimeout() const override + { + return context->getSettingsRef().http_receive_timeout; + } + + Poco::Timespan getSendTimeout() const override + { + return context->getSettingsRef().http_send_timeout; + } + + ContextPtr context; +}; + +HTTPContextPtr Server::httpContext() const +{ + return std::make_shared(context()); +} + void Server::createServers( Poco::Util::AbstractConfiguration & config, const Strings & listen_hosts, @@ -2009,7 +2063,7 @@ void Server::createServers( port_name, "http://" + address.toString(), std::make_unique( - context(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); + httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); }); /// HTTPS @@ -2026,7 +2080,7 @@ void Server::createServers( port_name, "https://" + address.toString(), std::make_unique( - context(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); + httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); #else UNUSED(port); throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.", @@ -2151,7 +2205,7 @@ void Server::createServers( port_name, "Prometheus: http://" + address.toString(), std::make_unique( - context(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); + httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); }); } @@ -2171,7 +2225,7 @@ void Server::createServers( port_name, "replica communication (interserver): http://" + address.toString(), std::make_unique( - context(), + httpContext(), createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), server_pool, socket, @@ -2191,7 +2245,7 @@ void Server::createServers( port_name, "secure replica communication (interserver): https://" + address.toString(), std::make_unique( - context(), + httpContext(), createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"), server_pool, socket, diff --git a/programs/server/Server.h b/programs/server/Server.h index 53841b1fcd4..10d413d9717 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -3,6 +3,7 @@ #include #include +#include "Server/HTTP/HTTPContext.h" #include #include @@ -72,6 +73,9 @@ private: /// Updated/recent config, to compare http_handlers ConfigurationPtr latest_config; + struct ServerHTTPContext; + HTTPContextPtr httpContext() const; + Poco::Net::SocketAddress socketBindListen( const Poco::Util::AbstractConfiguration & config, Poco::Net::ServerSocket & socket, diff --git a/src/Server/HTTP/HTTPContext.h b/src/Server/HTTP/HTTPContext.h new file mode 100644 index 00000000000..09c46ed188c --- /dev/null +++ b/src/Server/HTTP/HTTPContext.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +namespace DB +{ + +struct IHTTPContext +{ + virtual uint64_t getMaxHstsAge() const = 0; + virtual uint64_t getMaxUriSize() const = 0; + virtual uint64_t getMaxFields() const = 0; + virtual uint64_t getMaxFieldNameSize() const = 0; + virtual uint64_t getMaxFieldValueSize() const = 0; + virtual uint64_t getMaxChunkSize() const = 0; + virtual Poco::Timespan getReceiveTimeout() const = 0; + virtual Poco::Timespan getSendTimeout() const = 0; + + virtual ~IHTTPContext() = default; +}; + +using HTTPContextPtr = std::shared_ptr; + +} diff --git a/src/Server/HTTP/HTTPServer.cpp b/src/Server/HTTP/HTTPServer.cpp index 2e91fad1c0f..46734933263 100644 --- a/src/Server/HTTP/HTTPServer.cpp +++ b/src/Server/HTTP/HTTPServer.cpp @@ -6,7 +6,7 @@ namespace DB { HTTPServer::HTTPServer( - ContextPtr context, + HTTPContextPtr context, HTTPRequestHandlerFactoryPtr factory_, Poco::ThreadPool & thread_pool, Poco::Net::ServerSocket & socket_, diff --git a/src/Server/HTTP/HTTPServer.h b/src/Server/HTTP/HTTPServer.h index 07ad54d267f..adfb21e7c62 100644 --- a/src/Server/HTTP/HTTPServer.h +++ b/src/Server/HTTP/HTTPServer.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -11,13 +12,11 @@ namespace DB { -class Context; - class HTTPServer : public TCPServer { public: explicit HTTPServer( - ContextPtr context, + HTTPContextPtr context, HTTPRequestHandlerFactoryPtr factory, Poco::ThreadPool & thread_pool, Poco::Net::ServerSocket & socket, diff --git a/src/Server/HTTP/HTTPServerConnection.cpp b/src/Server/HTTP/HTTPServerConnection.cpp index 92a994b3a4e..926d37a11ee 100644 --- a/src/Server/HTTP/HTTPServerConnection.cpp +++ b/src/Server/HTTP/HTTPServerConnection.cpp @@ -7,12 +7,12 @@ namespace DB { HTTPServerConnection::HTTPServerConnection( - ContextPtr context_, + HTTPContextPtr context_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_) - : TCPServerConnection(socket), context(Context::createCopy(context_)), tcp_server(tcp_server_), params(params_), factory(factory_), stopped(false) + : TCPServerConnection(socket), context(std::move(context_)), tcp_server(tcp_server_), params(params_), factory(factory_), stopped(false) { poco_check_ptr(factory); } @@ -36,7 +36,7 @@ void HTTPServerConnection::run() if (request.isSecure()) { - size_t hsts_max_age = context->getSettingsRef().hsts_max_age.value; + size_t hsts_max_age = context->getMaxHstsAge(); if (hsts_max_age > 0) response.add("Strict-Transport-Security", "max-age=" + std::to_string(hsts_max_age)); diff --git a/src/Server/HTTP/HTTPServerConnection.h b/src/Server/HTTP/HTTPServerConnection.h index db3969f6ffb..c4ecb821ff1 100644 --- a/src/Server/HTTP/HTTPServerConnection.h +++ b/src/Server/HTTP/HTTPServerConnection.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -15,7 +16,7 @@ class HTTPServerConnection : public Poco::Net::TCPServerConnection { public: HTTPServerConnection( - ContextPtr context, + HTTPContextPtr context, TCPServer & tcp_server, const Poco::Net::StreamSocket & socket, Poco::Net::HTTPServerParams::Ptr params, @@ -27,7 +28,7 @@ protected: static void sendErrorResponse(Poco::Net::HTTPServerSession & session, Poco::Net::HTTPResponse::HTTPStatus status); private: - ContextPtr context; + HTTPContextPtr context; TCPServer & tcp_server; Poco::Net::HTTPServerParams::Ptr params; HTTPRequestHandlerFactoryPtr factory; diff --git a/src/Server/HTTP/HTTPServerConnectionFactory.cpp b/src/Server/HTTP/HTTPServerConnectionFactory.cpp index 008da222c79..7e4edbbf542 100644 --- a/src/Server/HTTP/HTTPServerConnectionFactory.cpp +++ b/src/Server/HTTP/HTTPServerConnectionFactory.cpp @@ -5,8 +5,8 @@ namespace DB { HTTPServerConnectionFactory::HTTPServerConnectionFactory( - ContextPtr context_, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_) - : context(Context::createCopy(context_)), params(params_), factory(factory_) + HTTPContextPtr context_, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_) + : context(std::move(context_)), params(params_), factory(factory_) { poco_check_ptr(factory); } diff --git a/src/Server/HTTP/HTTPServerConnectionFactory.h b/src/Server/HTTP/HTTPServerConnectionFactory.h index a19dc6d4d5c..b9b2ce0ad07 100644 --- a/src/Server/HTTP/HTTPServerConnectionFactory.h +++ b/src/Server/HTTP/HTTPServerConnectionFactory.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -12,12 +13,12 @@ namespace DB class HTTPServerConnectionFactory : public TCPServerConnectionFactory { public: - HTTPServerConnectionFactory(ContextPtr context, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory); + HTTPServerConnectionFactory(HTTPContextPtr context, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory); Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override; private: - ContextPtr context; + HTTPContextPtr context; Poco::Net::HTTPServerParams::Ptr params; HTTPRequestHandlerFactoryPtr factory; }; diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index 3b8df07b772..e70a1f93149 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -21,11 +21,11 @@ namespace DB { -HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session) - : max_uri_size(context->getSettingsRef().http_max_uri_size) - , max_fields_number(context->getSettingsRef().http_max_fields) - , max_field_name_size(context->getSettingsRef().http_max_field_name_size) - , max_field_value_size(context->getSettingsRef().http_max_field_value_size) +HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session) + : max_uri_size(context->getMaxUriSize()) + , max_fields_number(context->getMaxFields()) + , max_field_name_size(context->getMaxFieldNameSize()) + , max_field_value_size(context->getMaxFieldValueSize()) { response.attachRequest(this); @@ -34,8 +34,8 @@ HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & re server_address = session.serverAddress(); secure = session.socket().secure(); - auto receive_timeout = context->getSettingsRef().http_receive_timeout; - auto send_timeout = context->getSettingsRef().http_send_timeout; + auto receive_timeout = context->getReceiveTimeout(); + auto send_timeout = context->getSendTimeout(); session.socket().setReceiveTimeout(receive_timeout); session.socket().setSendTimeout(send_timeout); @@ -46,7 +46,7 @@ HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & re readRequest(*in); /// Try parse according to RFC7230 if (getChunkedTransferEncoding()) - stream = std::make_unique(std::move(in), context->getSettingsRef().http_max_chunk_size); + stream = std::make_unique(std::move(in), context->getMaxChunkSize()); else if (hasContentLength()) stream = std::make_unique(std::move(in), getContentLength(), false); else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE) diff --git a/src/Server/HTTP/HTTPServerRequest.h b/src/Server/HTTP/HTTPServerRequest.h index 7ddbd296280..1f38334c745 100644 --- a/src/Server/HTTP/HTTPServerRequest.h +++ b/src/Server/HTTP/HTTPServerRequest.h @@ -3,6 +3,7 @@ #include #include #include +#include #include "config.h" #include @@ -18,7 +19,7 @@ class ReadBufferFromPocoSocket; class HTTPServerRequest : public HTTPRequest { public: - HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session); + HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session); /// FIXME: it's a little bit inconvenient interface. The rationale is that all other ReadBuffer's wrap each other /// via unique_ptr - but we can't inherit HTTPServerRequest from ReadBuffer and pass it around, diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index ac8f8332a9e..d3b41b25b2b 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -144,15 +144,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco:: else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") return createInterserverHTTPHandlerFactory(server, name); else if (name == "PrometheusHandler-factory") - { - auto factory = std::make_shared(name); - auto handler = std::make_shared>( - server, PrometheusMetricsWriter(config, "prometheus", async_metrics)); - handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); - handler->allowGetAndHeadRequest(); - factory->addHandler(handler); - return factory; - } + return createPrometheusMainHandlerFactory(server, config, async_metrics, name); throw Exception("LOGICAL ERROR: Unknown HTTP handler factory name.", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index 9f306e787b0..fce54a2cece 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -148,6 +149,12 @@ createPrometheusHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & config_prefix); +HTTPRequestHandlerFactoryPtr +createPrometheusMainHandlerFactory(IServer & server, + const Poco::Util::AbstractConfiguration & config, + AsynchronousMetrics & async_metrics, + const std::string & name); + /// @param server - used in handlers to check IServer::isCancelled() /// @param config - not the same as server.config(), since it can be newer /// @param async_metrics - used for prometheus (in case of prometheus.asynchronous_metrics=true) diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 896efcca674..79025624206 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -53,4 +53,19 @@ createPrometheusHandlerFactory(IServer & server, return factory; } +HTTPRequestHandlerFactoryPtr +createPrometheusMainHandlerFactory(IServer & server, + const Poco::Util::AbstractConfiguration & config, + AsynchronousMetrics & async_metrics, + const std::string & name) +{ + auto factory = std::make_shared(name); + auto handler = std::make_shared>( + server, PrometheusMetricsWriter(config, "prometheus", async_metrics)); + handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); + handler->allowGetAndHeadRequest(); + factory->addHandler(handler); + return factory; +} + } diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index 1fb3d9f0f59..cce4ca17864 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include "PrometheusMetricsWriter.h" From c92ae5a385d3aee15eaff607ff7d3081a6ce8019 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 9 Nov 2022 13:37:42 +0100 Subject: [PATCH 02/62] Add suport for AsyncMetrics in standalone Keeper --- programs/keeper/CMakeLists.txt | 3 +- programs/keeper/Keeper.cpp | 36 +- programs/keeper/Keeper.h | 16 +- programs/server/MetricsTransmitter.cpp | 2 +- programs/server/Server.cpp | 4 +- .../AsynchronousMetrics.cpp | 428 +----------------- .../AsynchronousMetrics.h | 40 +- .../KeeperAsynchronousMetrics.cpp | 110 +++++ src/Coordination/KeeperAsynchronousMetrics.h | 25 + .../Coordination}/TinyContext.cpp | 2 +- .../keeper => src/Coordination}/TinyContext.h | 2 +- src/Interpreters/AsynchronousMetricLog.cpp | 2 +- .../ServerAsynchronousMetrics.cpp | 351 ++++++++++++++ src/Interpreters/ServerAsynchronousMetrics.h | 37 ++ src/Server/HTTPHandlerFactory.h | 2 +- src/Server/PrometheusMetricsWriter.h | 2 +- .../StorageSystemAsynchronousMetrics.cpp | 2 +- 17 files changed, 597 insertions(+), 467 deletions(-) rename src/{Interpreters => Common}/AsynchronousMetrics.cpp (72%) rename src/{Interpreters => Common}/AsynchronousMetrics.h (90%) create mode 100644 src/Coordination/KeeperAsynchronousMetrics.cpp create mode 100644 src/Coordination/KeeperAsynchronousMetrics.h rename {programs/keeper => src/Coordination}/TinyContext.cpp (98%) rename {programs/keeper => src/Coordination}/TinyContext.h (93%) create mode 100644 src/Interpreters/ServerAsynchronousMetrics.cpp create mode 100644 src/Interpreters/ServerAsynchronousMetrics.h diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 9266a4ca419..67ea245cc5b 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -13,7 +13,6 @@ clickhouse_embed_binaries( set(CLICKHOUSE_KEEPER_SOURCES Keeper.cpp - TinyContext.cpp ) set (CLICKHOUSE_KEEPER_LINK @@ -49,6 +48,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/TinyContext.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/pathUtils.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SummingStateMachine.cpp @@ -96,7 +96,6 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp Keeper.cpp - TinyContext.cpp clickhouse-keeper.cpp ) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index ddff0fa2ad7..4e7e123845a 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -22,9 +21,13 @@ #include #include #include + #include +#include #include +#include +#include #include "Core/Defines.h" #include "config.h" @@ -397,6 +400,25 @@ int Keeper::main(const std::vector & /*args*/) DNSResolver::instance().setDisableCacheFlag(); Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); + std::mutex servers_lock; + auto servers = std::make_shared>(); + + /// This object will periodically calculate some metrics. + KeeperAsynchronousMetrics async_metrics( + tiny_context, + config().getUInt("asynchronous_metrics_update_period_s", 1), + [&]() -> std::vector + { + std::vector metrics; + + std::lock_guard lock(servers_lock); + metrics.reserve(servers->size()); + for (const auto & server : *servers) + metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + return metrics; + } + ); + std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); @@ -408,8 +430,6 @@ int Keeper::main(const std::vector & /*args*/) listen_try = true; } - auto servers = std::make_shared>(); - /// Initialize keeper RAFT. Do nothing if no keeper_server in config. tiny_context.initializeKeeperDispatcher(/* start_async = */ true); FourLetterCommandFactory::registerCommands(*tiny_context.getKeeperDispatcher()); @@ -472,19 +492,19 @@ int Keeper::main(const std::vector & /*args*/) /// Prometheus (if defined and not setup yet with http_port) port_name = "prometheus.port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) -> ProtocolServerAdapter + createServer(listen_host, port_name, listen_try, [&](UInt16 port) { Poco::Net::ServerSocket socket; auto address = socketBindListen(socket, listen_host, port); // TODO(antonio2368): use config socket.setReceiveTimeout(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC); socket.setSendTimeout(DBMS_DEFAULT_SEND_TIMEOUT_SEC); - return ProtocolServerAdapter( + servers->emplace_back( listen_host, port_name, "Prometheus: http://" + address.toString(), std::make_unique( - httpContext(), createHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); + httpContext(), createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); }); } @@ -494,6 +514,8 @@ int Keeper::main(const std::vector & /*args*/) LOG_INFO(log, "Listening for {}", server.getDescription()); } + async_metrics.start(); + zkutil::EventPtr unused_event = std::make_shared(); zkutil::ZooKeeperNodeCache unused_cache([] { return nullptr; }); /// ConfigReloader have to strict parameters which are redundant in our case @@ -514,6 +536,8 @@ int Keeper::main(const std::vector & /*args*/) LOG_INFO(log, "Shutting down."); main_config_reloader.reset(); + async_metrics.stop(); + LOG_DEBUG(log, "Waiting for current connections to Keeper to finish."); size_t current_connections = 0; for (auto & server : *servers) diff --git a/programs/keeper/Keeper.h b/programs/keeper/Keeper.h index 60481b02bdb..cd0944421bb 100644 --- a/programs/keeper/Keeper.h +++ b/programs/keeper/Keeper.h @@ -3,7 +3,7 @@ #include #include #include -#include "TinyContext.h" +#include namespace Poco { @@ -19,26 +19,32 @@ namespace DB /// standalone clickhouse-keeper server (replacement for ZooKeeper). Uses the same /// config as clickhouse-server. Serves requests on TCP ports with or without /// SSL using ZooKeeper protocol. -class Keeper : public BaseDaemon +class Keeper : public BaseDaemon, public IServer { public: using ServerApplication::run; - Poco::Util::LayeredConfiguration & config() const + Poco::Util::LayeredConfiguration & config() const override { return BaseDaemon::config(); } - Poco::Logger & logger() const + Poco::Logger & logger() const override { return BaseDaemon::logger(); } - bool isCancelled() const + bool isCancelled() const override { return BaseDaemon::isCancelled(); } + /// Returns global application's context. + ContextMutablePtr context() const override + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot fetch context for Keeper"); + } + void defineOptions(Poco::Util::OptionSet & _options) override; protected: diff --git a/programs/server/MetricsTransmitter.cpp b/programs/server/MetricsTransmitter.cpp index 8ad519ba5aa..7e81d05a3e8 100644 --- a/programs/server/MetricsTransmitter.cpp +++ b/programs/server/MetricsTransmitter.cpp @@ -1,6 +1,6 @@ #include "MetricsTransmitter.h" -#include +#include #include #include diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 74cd64c442b..9473a3cca16 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -46,7 +46,7 @@ #include #include #include -#include +#include #include #include #include @@ -751,7 +751,7 @@ int Server::main(const std::vector & /*args*/) std::vector servers; std::vector servers_to_start_before_tables; /// This object will periodically calculate some metrics. - AsynchronousMetrics async_metrics( + ServerAsynchronousMetrics async_metrics( global_context, config().getUInt("asynchronous_metrics_update_period_s", 1), config().getUInt("asynchronous_heavy_metrics_update_period_s", 120), diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp similarity index 72% rename from src/Interpreters/AsynchronousMetrics.cpp rename to src/Common/AsynchronousMetrics.cpp index 488ac77e956..e775b3aa263 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -1,29 +1,16 @@ -#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 @@ -68,15 +55,11 @@ static std::unique_ptr openFileIfExists(const std::stri AsynchronousMetrics::AsynchronousMetrics( - ContextPtr global_context_, int update_period_seconds, - int heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : WithContext(global_context_) - , update_period(update_period_seconds) - , heavy_metric_update_period(heavy_metrics_update_period_seconds) - , protocol_server_metrics_func(protocol_server_metrics_func_) + : update_period(update_period_seconds) , log(&Poco::Logger::get("AsynchronousMetrics")) + , protocol_server_metrics_func(protocol_server_metrics_func_) { #if defined(OS_LINUX) openFileIfExists("/proc/meminfo", meminfo); @@ -360,22 +343,6 @@ void AsynchronousMetrics::run() } } - -template -static void calculateMax(Max & max, T x) -{ - if (Max(x) > max) - max = x; -} - -template -static void calculateMaxAndSum(Max & max, Sum & sum, T x) -{ - sum += x; - if (Max(x) > max) - max = x; -} - #if USE_JEMALLOC uint64_t updateJemallocEpoch() { @@ -572,85 +539,6 @@ void AsynchronousMetrics::update(TimePoint update_time) /// This is also a good indicator of system responsiveness. new_values["Jitter"] = std::chrono::duration_cast(current_time - update_time).count() / 1e9; - { - if (auto mark_cache = getContext()->getMarkCache()) - { - new_values["MarkCacheBytes"] = mark_cache->weight(); - new_values["MarkCacheFiles"] = mark_cache->count(); - } - } - - { - if (auto uncompressed_cache = getContext()->getUncompressedCache()) - { - new_values["UncompressedCacheBytes"] = uncompressed_cache->weight(); - new_values["UncompressedCacheCells"] = uncompressed_cache->count(); - } - } - - { - if (auto index_mark_cache = getContext()->getIndexMarkCache()) - { - new_values["IndexMarkCacheBytes"] = index_mark_cache->weight(); - new_values["IndexMarkCacheFiles"] = index_mark_cache->count(); - } - } - - { - if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache()) - { - new_values["IndexUncompressedCacheBytes"] = index_uncompressed_cache->weight(); - new_values["IndexUncompressedCacheCells"] = index_uncompressed_cache->count(); - } - } - - { - if (auto mmap_cache = getContext()->getMMappedFileCache()) - { - new_values["MMapCacheCells"] = mmap_cache->count(); - } - } - - { - auto caches = FileCacheFactory::instance().getAll(); - for (const auto & [_, cache_data] : caches) - { - new_values["FilesystemCacheBytes"] = cache_data->cache->getUsedCacheSize(); - new_values["FilesystemCacheFiles"] = cache_data->cache->getFileSegmentsNum(); - } - } - -#if USE_ROCKSDB - { - if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache()) - { - new_values["MergeTreeMetadataCacheSize"] = metadata_cache->getEstimateNumKeys(); - } - } -#endif - -#if USE_EMBEDDED_COMPILER - { - if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) - { - new_values["CompiledExpressionCacheBytes"] = compiled_expression_cache->weight(); - new_values["CompiledExpressionCacheCount"] = compiled_expression_cache->count(); - } - } -#endif - - - new_values["Uptime"] = getContext()->getUptimeSeconds(); - - { - if (const auto stats = getHashTablesCacheStatistics()) - { - new_values["HashTableStatsCacheEntries"] = stats->entries; - new_values["HashTableStatsCacheHits"] = stats->hits; - new_values["HashTableStatsCacheMisses"] = stats->misses; - } - } - #if defined(OS_LINUX) || defined(OS_FREEBSD) MemoryStatisticsOS::Data memory_statistics_data = memory_stat.get(); #endif @@ -1342,147 +1230,7 @@ void AsynchronousMetrics::update(TimePoint update_time) } #endif - /// Free space in filesystems at data path and logs path. { - auto stat = getStatVFS(getContext()->getPath()); - - new_values["FilesystemMainPathTotalBytes"] = stat.f_blocks * stat.f_frsize; - new_values["FilesystemMainPathAvailableBytes"] = stat.f_bavail * stat.f_frsize; - new_values["FilesystemMainPathUsedBytes"] = (stat.f_blocks - stat.f_bavail) * stat.f_frsize; - new_values["FilesystemMainPathTotalINodes"] = stat.f_files; - new_values["FilesystemMainPathAvailableINodes"] = stat.f_favail; - new_values["FilesystemMainPathUsedINodes"] = stat.f_files - stat.f_favail; - } - - { - /// Current working directory of the server is the directory with logs. - auto stat = getStatVFS("."); - - new_values["FilesystemLogsPathTotalBytes"] = stat.f_blocks * stat.f_frsize; - new_values["FilesystemLogsPathAvailableBytes"] = stat.f_bavail * stat.f_frsize; - new_values["FilesystemLogsPathUsedBytes"] = (stat.f_blocks - stat.f_bavail) * stat.f_frsize; - new_values["FilesystemLogsPathTotalINodes"] = stat.f_files; - new_values["FilesystemLogsPathAvailableINodes"] = stat.f_favail; - new_values["FilesystemLogsPathUsedINodes"] = stat.f_files - stat.f_favail; - } - - /// Free and total space on every configured disk. - { - DisksMap disks_map = getContext()->getDisksMap(); - for (const auto & [name, disk] : disks_map) - { - auto total = disk->getTotalSpace(); - - /// Some disks don't support information about the space. - if (!total) - continue; - - auto available = disk->getAvailableSpace(); - auto unreserved = disk->getUnreservedSpace(); - - new_values[fmt::format("DiskTotal_{}", name)] = total; - new_values[fmt::format("DiskUsed_{}", name)] = total - available; - new_values[fmt::format("DiskAvailable_{}", name)] = available; - new_values[fmt::format("DiskUnreserved_{}", name)] = unreserved; - } - } - - { - auto databases = DatabaseCatalog::instance().getDatabases(); - - size_t max_queue_size = 0; - size_t max_inserts_in_queue = 0; - size_t max_merges_in_queue = 0; - - size_t sum_queue_size = 0; - size_t sum_inserts_in_queue = 0; - size_t sum_merges_in_queue = 0; - - size_t max_absolute_delay = 0; - size_t max_relative_delay = 0; - - size_t max_part_count_for_partition = 0; - - size_t number_of_databases = databases.size(); - size_t total_number_of_tables = 0; - - size_t total_number_of_bytes = 0; - size_t total_number_of_rows = 0; - size_t total_number_of_parts = 0; - - for (const auto & db : databases) - { - /// Check if database can contain MergeTree tables - if (!db.second->canContainMergeTreeTables()) - continue; - - for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) - { - ++total_number_of_tables; - const auto & table = iterator->table(); - if (!table) - continue; - - if (MergeTreeData * table_merge_tree = dynamic_cast(table.get())) - { - const auto & settings = getContext()->getSettingsRef(); - - calculateMax(max_part_count_for_partition, table_merge_tree->getMaxPartsCountAndSizeForPartition().first); - total_number_of_bytes += table_merge_tree->totalBytes(settings).value(); - total_number_of_rows += table_merge_tree->totalRows(settings).value(); - total_number_of_parts += table_merge_tree->getPartsCount(); - } - - if (StorageReplicatedMergeTree * table_replicated_merge_tree = typeid_cast(table.get())) - { - StorageReplicatedMergeTree::Status status; - table_replicated_merge_tree->getStatus(status, false); - - calculateMaxAndSum(max_queue_size, sum_queue_size, status.queue.queue_size); - calculateMaxAndSum(max_inserts_in_queue, sum_inserts_in_queue, status.queue.inserts_in_queue); - calculateMaxAndSum(max_merges_in_queue, sum_merges_in_queue, status.queue.merges_in_queue); - - if (!status.is_readonly) - { - try - { - time_t absolute_delay = 0; - time_t relative_delay = 0; - table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay); - - calculateMax(max_absolute_delay, absolute_delay); - calculateMax(max_relative_delay, relative_delay); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__, - "Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name())); - } - } - } - } - } - - new_values["ReplicasMaxQueueSize"] = max_queue_size; - new_values["ReplicasMaxInsertsInQueue"] = max_inserts_in_queue; - new_values["ReplicasMaxMergesInQueue"] = max_merges_in_queue; - - new_values["ReplicasSumQueueSize"] = sum_queue_size; - new_values["ReplicasSumInsertsInQueue"] = sum_inserts_in_queue; - new_values["ReplicasSumMergesInQueue"] = sum_merges_in_queue; - - new_values["ReplicasMaxAbsoluteDelay"] = max_absolute_delay; - new_values["ReplicasMaxRelativeDelay"] = max_relative_delay; - - new_values["MaxPartCountForPartition"] = max_part_count_for_partition; - - new_values["NumberOfDatabases"] = number_of_databases; - new_values["NumberOfTables"] = total_number_of_tables; - - new_values["TotalBytesOfMergeTreeTables"] = total_number_of_bytes; - new_values["TotalRowsOfMergeTreeTables"] = total_number_of_rows; - new_values["TotalPartsOfMergeTreeTables"] = total_number_of_parts; - auto get_metric_name = [](const String & name) -> const char * { static std::map metric_map = @@ -1496,7 +1244,9 @@ void AsynchronousMetrics::update(TimePoint update_time) {"mysql_port", "MySQLThreads"}, {"postgresql_port", "PostgreSQLThreads"}, {"grpc_port", "GRPCThreads"}, - {"prometheus.port", "PrometheusThreads"} + {"prometheus.port", "PrometheusThreads"}, + {"keeper_server.tcp_port", "KeeperTCPThreads"}, + {"keeper_server.tcp_port_secure", "KeeperTCPSecureThreads"} }; auto it = metric_map.find(name); if (it == metric_map.end()) @@ -1512,102 +1262,14 @@ void AsynchronousMetrics::update(TimePoint update_time) new_values[name] = server_metric.current_threads; } } -#if USE_NURAFT - { - auto keeper_dispatcher = getContext()->tryGetKeeperDispatcher(); - if (keeper_dispatcher) - { - size_t is_leader = 0; - size_t is_follower = 0; - size_t is_observer = 0; - size_t is_standalone = 0; - size_t znode_count = 0; - size_t watch_count =0; - size_t ephemerals_count = 0; - size_t approximate_data_size =0; - size_t key_arena_size = 0; - size_t latest_snapshot_size =0; - size_t open_file_descriptor_count =0; - size_t max_file_descriptor_count =0; - size_t followers =0; - size_t synced_followers = 0; - size_t zxid = 0; - size_t session_with_watches = 0; - size_t paths_watched = 0; - size_t snapshot_dir_size = 0; - size_t log_dir_size = 0; - - if (keeper_dispatcher->isServerActive()) - { - auto keeper_info = keeper_dispatcher -> getKeeper4LWInfo(); - is_standalone = static_cast(keeper_info.is_standalone); - is_leader = static_cast(keeper_info.is_leader); - is_observer = static_cast(keeper_info.is_observer); - is_follower = static_cast(keeper_info.is_follower); - - zxid = keeper_info.last_zxid; - const auto & state_machine = keeper_dispatcher->getStateMachine(); - znode_count = state_machine.getNodesCount(); - watch_count = state_machine.getTotalWatchesCount(); - ephemerals_count = state_machine.getTotalEphemeralNodesCount(); - approximate_data_size = state_machine.getApproximateDataSize(); - key_arena_size = state_machine.getKeyArenaSize(); - latest_snapshot_size = state_machine.getLatestSnapshotBufSize(); - session_with_watches = state_machine.getSessionsWithWatchesCount(); - paths_watched = state_machine.getWatchedPathsCount(); - snapshot_dir_size = keeper_dispatcher->getSnapDirSize(); - log_dir_size = keeper_dispatcher->getLogDirSize(); - - #if defined(__linux__) || defined(__APPLE__) - open_file_descriptor_count = getCurrentProcessFDCount(); - max_file_descriptor_count = getMaxFileDescriptorCount(); - #endif - - if (keeper_info.is_leader) - { - followers = keeper_info.follower_count; - synced_followers = keeper_info.synced_follower_count; - } - } - - new_values["KeeperIsLeader"] = is_leader; - new_values["KeeperIsFollower"] = is_follower; - new_values["KeeperIsObserver"] = is_observer; - new_values["KeeperIsStandalone"] = is_standalone; - - new_values["KeeperZnodeCount"] = znode_count; - new_values["KeeperWatchCount"] = watch_count; - new_values["KeeperEphemeralsCount"] = ephemerals_count; - - new_values["KeeperApproximateDataSize"] = approximate_data_size; - new_values["KeeperKeyArenaSize"] = key_arena_size; - new_values["KeeperLatestSnapshotSize"] = latest_snapshot_size; - - new_values["KeeperOpenFileDescriptorCount"] = open_file_descriptor_count; - new_values["KeeperMaxFileDescriptorCount"] = max_file_descriptor_count; - - new_values["KeeperFollowers"] = followers; - new_values["KeeperSyncedFollowers"] = synced_followers; - new_values["KeeperZxid"] = zxid; - new_values["KeeperSessionWithWatches"] = session_with_watches; - new_values["KeeperPathsWatched"] = paths_watched; - new_values["KeeperSnapshotDirSize"] = snapshot_dir_size; - new_values["KeeperLogDirSize"] = log_dir_size; - } - } -#endif - - updateHeavyMetricsIfNeeded(current_time, update_time, new_values); /// Add more metrics as you wish. + updateImpl(new_values); + new_values["AsynchronousMetricsCalculationTimeSpent"] = watch.elapsedSeconds(); - /// Log the new metrics. - if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) - { - asynchronous_metric_log->addValues(new_values); - } + logImpl(new_values); first_run = false; @@ -1616,76 +1278,4 @@ void AsynchronousMetrics::update(TimePoint update_time) values = new_values; } -void AsynchronousMetrics::updateDetachedPartsStats() -{ - DetachedPartsStats current_values{}; - - for (const auto & db : DatabaseCatalog::instance().getDatabases()) - { - if (!db.second->canContainMergeTreeTables()) - continue; - - for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) - { - const auto & table = iterator->table(); - if (!table) - continue; - - if (MergeTreeData * table_merge_tree = dynamic_cast(table.get())) - { - for (const auto & detached_part: table_merge_tree->getDetachedParts()) - { - if (!detached_part.valid_name) - continue; - - if (detached_part.prefix.empty()) - ++current_values.detached_by_user; - - ++current_values.count; - } - } - } - } - - detached_parts_stats = current_values; -} - -void AsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values) -{ - const auto time_after_previous_update = current_time - heavy_metric_previous_update_time; - const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run; - - if (update_heavy_metric) - { - heavy_metric_previous_update_time = update_time; - - Stopwatch watch; - - /// Test shows that listing 100000 entries consuming around 0.15 sec. - updateDetachedPartsStats(); - - watch.stop(); - - /// Normally heavy metrics don't delay the rest of the metrics calculation - /// otherwise log the warning message - auto log_level = std::make_pair(DB::LogsLevel::trace, Poco::Message::PRIO_TRACE); - if (watch.elapsedSeconds() > (update_period.count() / 2.)) - log_level = std::make_pair(DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG); - else if (watch.elapsedSeconds() > (update_period.count() / 4. * 3)) - log_level = std::make_pair(DB::LogsLevel::warning, Poco::Message::PRIO_WARNING); - LOG_IMPL(log, log_level.first, log_level.second, - "Update heavy metrics. " - "Update period {} sec. " - "Update heavy metrics period {} sec. " - "Heavy metrics calculation elapsed: {} sec.", - update_period.count(), - heavy_metric_update_period.count(), - watch.elapsedSeconds()); - - } - - new_values["NumberOfDetachedParts"] = detached_parts_stats.count; - new_values["NumberOfDetachedByUserParts"] = detached_parts_stats.detached_by_user; -} - } diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h similarity index 90% rename from src/Interpreters/AsynchronousMetrics.h rename to src/Common/AsynchronousMetrics.h index 6e32bdb43b8..37ee3389da6 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -43,17 +42,15 @@ struct ProtocolServerMetrics * This includes both ClickHouse-related metrics (like memory usage of ClickHouse process) * and common OS-related metrics (like total memory usage on the server). */ -class AsynchronousMetrics : WithContext +class AsynchronousMetrics { public: using ProtocolServerMetricsFunc = std::function()>; AsynchronousMetrics( - ContextPtr global_context_, int update_period_seconds, - int heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); - ~AsynchronousMetrics(); + virtual ~AsynchronousMetrics(); /// Separate method allows to initialize the `servers` variable beforehand. void start(); @@ -63,12 +60,22 @@ public: /// Returns copy of all values. AsynchronousMetricValues getValues() const; -private: +protected: using Duration = std::chrono::seconds; using TimePoint = std::chrono::system_clock::time_point; const Duration update_period; - const Duration heavy_metric_update_period; + + /// Some values are incremental and we have to calculate the difference. + /// On first run we will only collect the values to subtract later. + bool first_run = true; + TimePoint previous_update_time; + + Poco::Logger * log; +private: + virtual void updateImpl(AsynchronousMetricValues & new_values) = 0; + virtual void logImpl(AsynchronousMetricValues &) {} + ProtocolServerMetricsFunc protocol_server_metrics_func; mutable std::mutex mutex; @@ -76,20 +83,6 @@ private: bool quit {false}; AsynchronousMetricValues values; - /// Some values are incremental and we have to calculate the difference. - /// On first run we will only collect the values to subtract later. - bool first_run = true; - TimePoint previous_update_time; - TimePoint heavy_metric_previous_update_time; - - struct DetachedPartsStats - { - size_t count; - size_t detached_by_user; - }; - - DetachedPartsStats detached_parts_stats{}; - #if defined(OS_LINUX) || defined(OS_FREEBSD) MemoryStatisticsOS memory_stat; #endif @@ -200,11 +193,6 @@ private: void run(); void update(TimePoint update_time); - - void updateDetachedPartsStats(); - void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values); - - Poco::Logger * log; }; } diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp new file mode 100644 index 00000000000..0812c31a154 --- /dev/null +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -0,0 +1,110 @@ +#include + +#include + +#include +#include + +namespace DB +{ + +void updateKeeperInformation(const KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values) +{ +#if USE_NURAFT + size_t is_leader = 0; + size_t is_follower = 0; + size_t is_observer = 0; + size_t is_standalone = 0; + size_t znode_count = 0; + size_t watch_count = 0; + size_t ephemerals_count = 0; + size_t approximate_data_size = 0; + size_t key_arena_size = 0; + size_t latest_snapshot_size = 0; + size_t open_file_descriptor_count = 0; + size_t max_file_descriptor_count = 0; + size_t followers = 0; + size_t synced_followers = 0; + size_t zxid = 0; + size_t session_with_watches = 0; + size_t paths_watched = 0; + size_t snapshot_dir_size = 0; + size_t log_dir_size = 0; + + if (keeper_dispatcher.isServerActive()) + { + auto keeper_info = keeper_dispatcher.getKeeper4LWInfo(); + is_standalone = static_cast(keeper_info.is_standalone); + is_leader = static_cast(keeper_info.is_leader); + is_observer = static_cast(keeper_info.is_observer); + is_follower = static_cast(keeper_info.is_follower); + + zxid = keeper_info.last_zxid; + const auto & state_machine = keeper_dispatcher.getStateMachine(); + znode_count = state_machine.getNodesCount(); + watch_count = state_machine.getTotalWatchesCount(); + ephemerals_count = state_machine.getTotalEphemeralNodesCount(); + approximate_data_size = state_machine.getApproximateDataSize(); + key_arena_size = state_machine.getKeyArenaSize(); + latest_snapshot_size = state_machine.getLatestSnapshotBufSize(); + session_with_watches = state_machine.getSessionsWithWatchesCount(); + paths_watched = state_machine.getWatchedPathsCount(); + snapshot_dir_size = keeper_dispatcher.getSnapDirSize(); + log_dir_size = keeper_dispatcher.getLogDirSize(); + +# if defined(__linux__) || defined(__APPLE__) + open_file_descriptor_count = getCurrentProcessFDCount(); + max_file_descriptor_count = getMaxFileDescriptorCount(); +# endif + + if (keeper_info.is_leader) + { + followers = keeper_info.follower_count; + synced_followers = keeper_info.synced_follower_count; + } + } + + new_values["KeeperIsLeader"] = is_leader; + new_values["KeeperIsFollower"] = is_follower; + new_values["KeeperIsObserver"] = is_observer; + new_values["KeeperIsStandalone"] = is_standalone; + + new_values["KeeperZnodeCount"] = znode_count; + new_values["KeeperWatchCount"] = watch_count; + new_values["KeeperEphemeralsCount"] = ephemerals_count; + + new_values["KeeperApproximateDataSize"] = approximate_data_size; + new_values["KeeperKeyArenaSize"] = key_arena_size; + new_values["KeeperLatestSnapshotSize"] = latest_snapshot_size; + + new_values["KeeperOpenFileDescriptorCount"] = open_file_descriptor_count; + new_values["KeeperMaxFileDescriptorCount"] = max_file_descriptor_count; + + new_values["KeeperFollowers"] = followers; + new_values["KeeperSyncedFollowers"] = synced_followers; + new_values["KeeperZxid"] = zxid; + new_values["KeeperSessionWithWatches"] = session_with_watches; + new_values["KeeperPathsWatched"] = paths_watched; + new_values["KeeperSnapshotDirSize"] = snapshot_dir_size; + new_values["KeeperLogDirSize"] = log_dir_size; +#endif +} + +KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( + const TinyContext & tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), tiny_context(tiny_context_) +{ +} + +void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values) +{ +#if USE_NURAFT + { + auto keeper_dispatcher = tiny_context.tryGetKeeperDispatcher(); + if (keeper_dispatcher) + updateKeeperInformation(*keeper_dispatcher, new_values); + } +#endif +} + +} diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h new file mode 100644 index 00000000000..30d04dc3e17 --- /dev/null +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class KeeperDispatcher; +void updateKeeperInformation(const KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values); + +class KeeperAsynchronousMetrics : public AsynchronousMetrics +{ +public: + KeeperAsynchronousMetrics( + const TinyContext & tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + +private: + const TinyContext & tiny_context; + + void updateImpl(AsynchronousMetricValues & new_values) override; +}; + + +} diff --git a/programs/keeper/TinyContext.cpp b/src/Coordination/TinyContext.cpp similarity index 98% rename from programs/keeper/TinyContext.cpp rename to src/Coordination/TinyContext.cpp index 09174838c04..967e6b23d70 100644 --- a/programs/keeper/TinyContext.cpp +++ b/src/Coordination/TinyContext.cpp @@ -1,4 +1,4 @@ -#include "TinyContext.h" +#include #include #include diff --git a/programs/keeper/TinyContext.h b/src/Coordination/TinyContext.h similarity index 93% rename from programs/keeper/TinyContext.h rename to src/Coordination/TinyContext.h index 1cbbc725090..7ccbe0ec8a6 100644 --- a/programs/keeper/TinyContext.h +++ b/src/Coordination/TinyContext.h @@ -10,7 +10,7 @@ namespace DB class KeeperDispatcher; -class TinyContext: public std::enable_shared_from_this +class TinyContext : public std::enable_shared_from_this { public: std::shared_ptr getKeeperDispatcher() const; diff --git a/src/Interpreters/AsynchronousMetricLog.cpp b/src/Interpreters/AsynchronousMetricLog.cpp index 228934d5f4d..a018759991f 100644 --- a/src/Interpreters/AsynchronousMetricLog.cpp +++ b/src/Interpreters/AsynchronousMetricLog.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp new file mode 100644 index 00000000000..2fc20463aa8 --- /dev/null +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -0,0 +1,351 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include + +#include +#include + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace +{ + +template +static void calculateMax(Max & max, T x) +{ + if (Max(x) > max) + max = x; +} + +template +static void calculateMaxAndSum(Max & max, Sum & sum, T x) +{ + sum += x; + if (Max(x) > max) + max = x; +} + +} + +ServerAsynchronousMetrics::ServerAsynchronousMetrics( + ContextPtr global_context_, + int update_period_seconds, + int heavy_metrics_update_period_seconds, + const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) + , WithContext(global_context_) + , heavy_metric_update_period(heavy_metrics_update_period_seconds) +{} + +void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values) +{ + if (auto mark_cache = getContext()->getMarkCache()) + { + new_values["MarkCacheBytes"] = mark_cache->weight(); + new_values["MarkCacheFiles"] = mark_cache->count(); + } + + if (auto uncompressed_cache = getContext()->getUncompressedCache()) + { + new_values["UncompressedCacheBytes"] = uncompressed_cache->weight(); + new_values["UncompressedCacheCells"] = uncompressed_cache->count(); + } + + if (auto index_mark_cache = getContext()->getIndexMarkCache()) + { + new_values["IndexMarkCacheBytes"] = index_mark_cache->weight(); + new_values["IndexMarkCacheFiles"] = index_mark_cache->count(); + } + + if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache()) + { + new_values["IndexUncompressedCacheBytes"] = index_uncompressed_cache->weight(); + new_values["IndexUncompressedCacheCells"] = index_uncompressed_cache->count(); + } + + if (auto mmap_cache = getContext()->getMMappedFileCache()) + { + new_values["MMapCacheCells"] = mmap_cache->count(); + } + + { + auto caches = FileCacheFactory::instance().getAll(); + for (const auto & [_, cache_data] : caches) + { + new_values["FilesystemCacheBytes"] = cache_data->cache->getUsedCacheSize(); + new_values["FilesystemCacheFiles"] = cache_data->cache->getFileSegmentsNum(); + } + } + +#if USE_ROCKSDB + if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache()) + { + new_values["MergeTreeMetadataCacheSize"] = metadata_cache->getEstimateNumKeys(); + } +#endif + +#if USE_EMBEDDED_COMPILER + if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) + { + new_values["CompiledExpressionCacheBytes"] = compiled_expression_cache->weight(); + new_values["CompiledExpressionCacheCount"] = compiled_expression_cache->count(); + } +#endif + + + new_values["Uptime"] = getContext()->getUptimeSeconds(); + + if (const auto stats = getHashTablesCacheStatistics()) + { + new_values["HashTableStatsCacheEntries"] = stats->entries; + new_values["HashTableStatsCacheHits"] = stats->hits; + new_values["HashTableStatsCacheMisses"] = stats->misses; + } + + /// Free space in filesystems at data path and logs path. + { + auto stat = getStatVFS(getContext()->getPath()); + + new_values["FilesystemMainPathTotalBytes"] = stat.f_blocks * stat.f_frsize; + new_values["FilesystemMainPathAvailableBytes"] = stat.f_bavail * stat.f_frsize; + new_values["FilesystemMainPathUsedBytes"] = (stat.f_blocks - stat.f_bavail) * stat.f_frsize; + new_values["FilesystemMainPathTotalINodes"] = stat.f_files; + new_values["FilesystemMainPathAvailableINodes"] = stat.f_favail; + new_values["FilesystemMainPathUsedINodes"] = stat.f_files - stat.f_favail; + } + + { + /// Current working directory of the server is the directory with logs. + auto stat = getStatVFS("."); + + new_values["FilesystemLogsPathTotalBytes"] = stat.f_blocks * stat.f_frsize; + new_values["FilesystemLogsPathAvailableBytes"] = stat.f_bavail * stat.f_frsize; + new_values["FilesystemLogsPathUsedBytes"] = (stat.f_blocks - stat.f_bavail) * stat.f_frsize; + new_values["FilesystemLogsPathTotalINodes"] = stat.f_files; + new_values["FilesystemLogsPathAvailableINodes"] = stat.f_favail; + new_values["FilesystemLogsPathUsedINodes"] = stat.f_files - stat.f_favail; + } + + /// Free and total space on every configured disk. + { + DisksMap disks_map = getContext()->getDisksMap(); + for (const auto & [name, disk] : disks_map) + { + auto total = disk->getTotalSpace(); + + /// Some disks don't support information about the space. + if (!total) + continue; + + auto available = disk->getAvailableSpace(); + auto unreserved = disk->getUnreservedSpace(); + + new_values[fmt::format("DiskTotal_{}", name)] = total; + new_values[fmt::format("DiskUsed_{}", name)] = total - available; + new_values[fmt::format("DiskAvailable_{}", name)] = available; + new_values[fmt::format("DiskUnreserved_{}", name)] = unreserved; + } + } + + { + auto databases = DatabaseCatalog::instance().getDatabases(); + + size_t max_queue_size = 0; + size_t max_inserts_in_queue = 0; + size_t max_merges_in_queue = 0; + + size_t sum_queue_size = 0; + size_t sum_inserts_in_queue = 0; + size_t sum_merges_in_queue = 0; + + size_t max_absolute_delay = 0; + size_t max_relative_delay = 0; + + size_t max_part_count_for_partition = 0; + + size_t number_of_databases = databases.size(); + size_t total_number_of_tables = 0; + + size_t total_number_of_bytes = 0; + size_t total_number_of_rows = 0; + size_t total_number_of_parts = 0; + + for (const auto & db : databases) + { + /// Check if database can contain MergeTree tables + if (!db.second->canContainMergeTreeTables()) + continue; + + for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) + { + ++total_number_of_tables; + const auto & table = iterator->table(); + if (!table) + continue; + + if (MergeTreeData * table_merge_tree = dynamic_cast(table.get())) + { + const auto & settings = getContext()->getSettingsRef(); + + calculateMax(max_part_count_for_partition, table_merge_tree->getMaxPartsCountAndSizeForPartition().first); + total_number_of_bytes += table_merge_tree->totalBytes(settings).value(); + total_number_of_rows += table_merge_tree->totalRows(settings).value(); + total_number_of_parts += table_merge_tree->getPartsCount(); + } + + if (StorageReplicatedMergeTree * table_replicated_merge_tree = typeid_cast(table.get())) + { + StorageReplicatedMergeTree::Status status; + table_replicated_merge_tree->getStatus(status, false); + + calculateMaxAndSum(max_queue_size, sum_queue_size, status.queue.queue_size); + calculateMaxAndSum(max_inserts_in_queue, sum_inserts_in_queue, status.queue.inserts_in_queue); + calculateMaxAndSum(max_merges_in_queue, sum_merges_in_queue, status.queue.merges_in_queue); + + if (!status.is_readonly) + { + try + { + time_t absolute_delay = 0; + time_t relative_delay = 0; + table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay); + + calculateMax(max_absolute_delay, absolute_delay); + calculateMax(max_relative_delay, relative_delay); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__, + "Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name())); + } + } + } + } + } + + new_values["ReplicasMaxQueueSize"] = max_queue_size; + new_values["ReplicasMaxInsertsInQueue"] = max_inserts_in_queue; + new_values["ReplicasMaxMergesInQueue"] = max_merges_in_queue; + + new_values["ReplicasSumQueueSize"] = sum_queue_size; + new_values["ReplicasSumInsertsInQueue"] = sum_inserts_in_queue; + new_values["ReplicasSumMergesInQueue"] = sum_merges_in_queue; + + new_values["ReplicasMaxAbsoluteDelay"] = max_absolute_delay; + new_values["ReplicasMaxRelativeDelay"] = max_relative_delay; + + new_values["MaxPartCountForPartition"] = max_part_count_for_partition; + + new_values["NumberOfDatabases"] = number_of_databases; + new_values["NumberOfTables"] = total_number_of_tables; + + new_values["TotalBytesOfMergeTreeTables"] = total_number_of_bytes; + new_values["TotalRowsOfMergeTreeTables"] = total_number_of_rows; + new_values["TotalPartsOfMergeTreeTables"] = total_number_of_parts; + } + +#if USE_NURAFT + { + auto keeper_dispatcher = getContext()->tryGetKeeperDispatcher(); + if (keeper_dispatcher) + updateKeeperInformation(*keeper_dispatcher, new_values); + } +#endif +} + +void ServerAsynchronousMetrics::logImpl(AsynchronousMetricValues & new_values) +{ + /// Log the new metrics. + if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) + asynchronous_metric_log->addValues(new_values); +} + +void ServerAsynchronousMetrics::updateDetachedPartsStats() +{ + DetachedPartsStats current_values{}; + + for (const auto & db : DatabaseCatalog::instance().getDatabases()) + { + if (!db.second->canContainMergeTreeTables()) + continue; + + for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) + { + const auto & table = iterator->table(); + if (!table) + continue; + + if (MergeTreeData * table_merge_tree = dynamic_cast(table.get())) + { + for (const auto & detached_part: table_merge_tree->getDetachedParts()) + { + if (!detached_part.valid_name) + continue; + + if (detached_part.prefix.empty()) + ++current_values.detached_by_user; + + ++current_values.count; + } + } + } + } + + detached_parts_stats = current_values; +} + +void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values) +{ + const auto time_after_previous_update = current_time - heavy_metric_previous_update_time; + const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run; + + if (update_heavy_metric) + { + heavy_metric_previous_update_time = update_time; + + Stopwatch watch; + + /// Test shows that listing 100000 entries consuming around 0.15 sec. + updateDetachedPartsStats(); + + watch.stop(); + + /// Normally heavy metrics don't delay the rest of the metrics calculation + /// otherwise log the warning message + auto log_level = std::make_pair(DB::LogsLevel::trace, Poco::Message::PRIO_TRACE); + if (watch.elapsedSeconds() > (update_period.count() / 2.)) + log_level = std::make_pair(DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG); + else if (watch.elapsedSeconds() > (update_period.count() / 4. * 3)) + log_level = std::make_pair(DB::LogsLevel::warning, Poco::Message::PRIO_WARNING); + LOG_IMPL(log, log_level.first, log_level.second, + "Update heavy metrics. " + "Update period {} sec. " + "Update heavy metrics period {} sec. " + "Heavy metrics calculation elapsed: {} sec.", + update_period.count(), + heavy_metric_update_period.count(), + watch.elapsedSeconds()); + + } + + new_values["NumberOfDetachedParts"] = detached_parts_stats.count; + new_values["NumberOfDetachedByUserParts"] = detached_parts_stats.detached_by_user; +} + +} diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h new file mode 100644 index 00000000000..cd19483032a --- /dev/null +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ServerAsynchronousMetrics : public AsynchronousMetrics, WithContext +{ +public: + ServerAsynchronousMetrics( + ContextPtr global_context_, + int update_period_seconds, + int heavy_metrics_update_period_seconds, + const ProtocolServerMetricsFunc & protocol_server_metrics_func_); +private: + void updateImpl(AsynchronousMetricValues & new_values) override; + void logImpl(AsynchronousMetricValues & new_values) override; + + const Duration heavy_metric_update_period; + TimePoint heavy_metric_previous_update_time; + + struct DetachedPartsStats + { + size_t count; + size_t detached_by_user; + }; + + DetachedPartsStats detached_parts_stats{}; + + void updateDetachedPartsStats(); + void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values); +}; + +} diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index fce54a2cece..ce4e349e34b 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Server/PrometheusMetricsWriter.h b/src/Server/PrometheusMetricsWriter.h index 4422ced625e..f96d5435f82 100644 --- a/src/Server/PrometheusMetricsWriter.h +++ b/src/Server/PrometheusMetricsWriter.h @@ -2,7 +2,7 @@ #include -#include +#include #include diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp index 70e12440678..14a082a8308 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include From 1ed3930809f4898736eb50e896e5c2e56d19bf69 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 9 Nov 2022 15:51:41 +0100 Subject: [PATCH 03/62] Add more metrics for Keeper --- programs/keeper/Keeper.cpp | 25 +++++---- programs/keeper/Keeper.h | 2 +- programs/server/Server.cpp | 51 +------------------ programs/server/Server.h | 1 - src/Bridge/IBridge.cpp | 2 +- src/Common/ProfileEvents.cpp | 9 ++++ .../KeeperAsynchronousMetrics.cpp | 19 ++++++- src/Coordination/KeeperAsynchronousMetrics.h | 2 +- src/Coordination/KeeperStorage.cpp | 27 ++++++++++ src/Interpreters/Context.h | 51 +++++++++++++++++++ .../ServerAsynchronousMetrics.cpp | 2 +- src/Server/HTTPHandlerFactory.h | 1 - src/Server/PrometheusRequestHandler.h | 1 - 13 files changed, 125 insertions(+), 68 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 4e7e123845a..a28a96ddb3b 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -281,50 +281,56 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options) struct Keeper::KeeperHTTPContext : public IHTTPContext { + explicit KeeperHTTPContext(const TinyContext & context_) + : context(context_) + {} + uint64_t getMaxHstsAge() const override { - return 0; + return context.getConfigRef().getUInt64("keeper_server.hsts_max_age", 0); } uint64_t getMaxUriSize() const override { - return 1048576; + return context.getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576); } uint64_t getMaxFields() const override { - return 1000000; + return context.getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000); } uint64_t getMaxFieldNameSize() const override { - return 1048576; + return context.getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 1048576); } uint64_t getMaxFieldValueSize() const override { - return 1048576; + return context.getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 1048576); } uint64_t getMaxChunkSize() const override { - return 100_GiB; + return context.getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB); } Poco::Timespan getReceiveTimeout() const override { - return DEFAULT_HTTP_READ_BUFFER_TIMEOUT; + return context.getConfigRef().getUInt64("keeper_server.http_receive_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); } Poco::Timespan getSendTimeout() const override { - return DEFAULT_HTTP_READ_BUFFER_TIMEOUT; + return context.getConfigRef().getUInt64("keeper_server.http_send_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); } + + const TinyContext & context; }; HTTPContextPtr Keeper::httpContext() { - return std::make_shared(); + return std::make_shared(tiny_context); } int Keeper::main(const std::vector & /*args*/) @@ -419,7 +425,6 @@ int Keeper::main(const std::vector & /*args*/) } ); - std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); bool listen_try = config().getBool("listen_try", false); diff --git a/programs/keeper/Keeper.h b/programs/keeper/Keeper.h index cd0944421bb..585e5a04641 100644 --- a/programs/keeper/Keeper.h +++ b/programs/keeper/Keeper.h @@ -66,7 +66,7 @@ private: TinyContext tiny_context; struct KeeperHTTPContext; - static HTTPContextPtr httpContext(); + HTTPContextPtr httpContext(); Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9473a3cca16..3b06f478f75 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1927,58 +1927,9 @@ std::unique_ptr Server::buildProtocolStackFromConfig( return stack; } -struct Server::ServerHTTPContext : public IHTTPContext -{ - explicit ServerHTTPContext(ContextPtr context_) - : context(Context::createCopy(context_)) - {} - - uint64_t getMaxHstsAge() const override - { - return context->getSettingsRef().hsts_max_age; - } - - uint64_t getMaxUriSize() const override - { - return context->getSettingsRef().http_max_uri_size; - } - - uint64_t getMaxFields() const override - { - return context->getSettingsRef().http_max_fields; - } - - uint64_t getMaxFieldNameSize() const override - { - return context->getSettingsRef().http_max_field_name_size; - } - - uint64_t getMaxFieldValueSize() const override - { - return context->getSettingsRef().http_max_field_value_size; - } - - uint64_t getMaxChunkSize() const override - { - return context->getSettingsRef().http_max_chunk_size; - } - - Poco::Timespan getReceiveTimeout() const override - { - return context->getSettingsRef().http_receive_timeout; - } - - Poco::Timespan getSendTimeout() const override - { - return context->getSettingsRef().http_send_timeout; - } - - ContextPtr context; -}; - HTTPContextPtr Server::httpContext() const { - return std::make_shared(context()); + return std::make_shared(context()); } void Server::createServers( diff --git a/programs/server/Server.h b/programs/server/Server.h index 10d413d9717..e9ae6d8d937 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -73,7 +73,6 @@ private: /// Updated/recent config, to compare http_handlers ConfigurationPtr latest_config; - struct ServerHTTPContext; HTTPContextPtr httpContext() const; Poco::Net::SocketAddress socketBindListen( diff --git a/src/Bridge/IBridge.cpp b/src/Bridge/IBridge.cpp index 04d904d0a00..afaaf11b26a 100644 --- a/src/Bridge/IBridge.cpp +++ b/src/Bridge/IBridge.cpp @@ -236,7 +236,7 @@ int IBridge::main(const std::vector & /*args*/) SensitiveDataMasker::setInstance(std::make_unique(config(), "query_masking_rules")); auto server = HTTPServer( - context, + std::make_shared(context), getHandlerFactoryPtr(context), server_pool, socket, diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index e30a6bb6aaf..2c433fece82 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -433,6 +433,15 @@ The server successfully detected this situation and will download merged part fr M(KeeperSnapshotApplysFailed, "Number of failed snapshot applying")\ M(KeeperReadSnapshot, "Number of snapshot read(serialization)")\ M(KeeperSaveSnapshot, "Number of snapshot save")\ + M(KeeperCreateRequest, "Number of create requests")\ + M(KeeperRemoveRequest, "Number of remove requests")\ + M(KeeperSetRequest, "Number of set requests")\ + M(KeeperCheckRequest, "Number of check requests")\ + M(KeeperMultiRequest, "Number of multi requests")\ + M(KeeperMultiReadRequest, "Number of multi read requests")\ + M(KeeperGetRequest, "Number of get requests")\ + M(KeeperListRequest, "Number of list requests")\ + M(KeeperExistsRequest, "Number of exists requests")\ \ M(OverflowBreak, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'break' and the result is incomplete.") \ M(OverflowThrow, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'throw' and exception was thrown.") \ diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 0812c31a154..e26f9e1c9e8 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -8,7 +8,7 @@ namespace DB { -void updateKeeperInformation(const KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values) +void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values) { #if USE_NURAFT size_t is_leader = 0; @@ -87,6 +87,23 @@ void updateKeeperInformation(const KeeperDispatcher & keeper_dispatcher, Asynchr new_values["KeeperPathsWatched"] = paths_watched; new_values["KeeperSnapshotDirSize"] = snapshot_dir_size; new_values["KeeperLogDirSize"] = log_dir_size; + + auto keeper_log_info = keeper_dispatcher.getKeeperLogInfo(); + + new_values["KeeperLastLogIdx"] = keeper_log_info.last_log_idx; + new_values["KeeperLastLogTerm"] = keeper_log_info.last_log_term; + + new_values["KeeperLastCommittedLogIdx"] = keeper_log_info.last_committed_log_idx; + new_values["KeeperTargetCommitLogIdx"] = keeper_log_info.target_committed_log_idx; + new_values["KeeperLastSnapshotIdx"] = keeper_log_info.last_snapshot_idx; + + auto & keeper_connection_stats = keeper_dispatcher.getKeeperConnectionStats(); + + new_values["KeeperMinLatency"] = keeper_connection_stats.getMinLatency(); + new_values["KeeperMaxLatency"] = keeper_connection_stats.getMaxLatency(); + new_values["KeeperAvgLatency"] = keeper_connection_stats.getAvgLatency(); + new_values["KeeperPacketsReceived"] = keeper_connection_stats.getAvgLatency(); + new_values["KeeperPacketsSent"] = keeper_connection_stats.getAvgLatency(); #endif } diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index 30d04dc3e17..579131c25b4 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -7,7 +7,7 @@ namespace DB { class KeeperDispatcher; -void updateKeeperInformation(const KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values); +void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values); class KeeperAsynchronousMetrics : public AsynchronousMetrics { diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 3153d17899d..fb472201aec 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -27,6 +28,19 @@ #include #include +namespace ProfileEvents +{ + extern const Event KeeperCreateRequest; + extern const Event KeeperRemoveRequest; + extern const Event KeeperSetRequest; + extern const Event KeeperCheckRequest; + extern const Event KeeperMultiRequest; + extern const Event KeeperMultiReadRequest; + extern const Event KeeperGetRequest; + extern const Event KeeperListRequest; + extern const Event KeeperExistsRequest; +} + namespace DB { @@ -865,6 +879,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr std::vector preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { + ProfileEvents::increment(ProfileEvents::KeeperCreateRequest); Coordination::ZooKeeperCreateRequest & request = dynamic_cast(*zk_request); std::vector new_deltas; @@ -986,6 +1001,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce std::vector preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { + ProfileEvents::increment(ProfileEvents::KeeperGetRequest); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); if (request.path == Coordination::keeper_api_version_path) @@ -1040,6 +1056,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override { + ProfileEvents::increment(ProfileEvents::KeeperGetRequest); return processImpl(storage, zxid); } }; @@ -1055,6 +1072,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr std::vector preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override { + ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); Coordination::ZooKeeperRemoveRequest & request = dynamic_cast(*zk_request); std::vector new_deltas; @@ -1145,6 +1163,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr std::vector preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { + ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*zk_request); if (!storage.uncommitted_state.getNode(request.path)) @@ -1194,6 +1213,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override { + ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); return processImpl(storage, zxid); } }; @@ -1209,6 +1229,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce std::vector preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { + ProfileEvents::increment(ProfileEvents::KeeperSetRequest); Coordination::ZooKeeperSetRequest & request = dynamic_cast(*zk_request); std::vector new_deltas; @@ -1301,6 +1322,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc std::vector preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { + ProfileEvents::increment(ProfileEvents::KeeperListRequest); Coordination::ZooKeeperListRequest & request = dynamic_cast(*zk_request); if (!storage.uncommitted_state.getNode(request.path)) @@ -1387,6 +1409,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override { + ProfileEvents::increment(ProfileEvents::KeeperListRequest); return processImpl(storage, zxid); } }; @@ -1402,6 +1425,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro std::vector preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { + ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*zk_request); if (!storage.uncommitted_state.getNode(request.path)) @@ -1463,6 +1487,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override { + ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); return processImpl(storage, zxid); } }; @@ -1689,6 +1714,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro std::vector preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { + ProfileEvents::increment(ProfileEvents::KeeperMultiRequest); std::vector response_errors; response_errors.reserve(concrete_requests.size()); uint64_t current_digest = digest; @@ -1756,6 +1782,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override { + ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index eeb9e8da148..1e69bf92cc7 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -18,6 +18,8 @@ #include #include +#include + #include "config.h" @@ -1062,4 +1064,53 @@ private: DiskSelectorPtr getDiskSelector(std::lock_guard & /* lock */) const; }; +struct HTTPContext : public IHTTPContext +{ + explicit HTTPContext(ContextPtr context_) + : context(Context::createCopy(context_)) + {} + + uint64_t getMaxHstsAge() const override + { + return context->getSettingsRef().hsts_max_age; + } + + uint64_t getMaxUriSize() const override + { + return context->getSettingsRef().http_max_uri_size; + } + + uint64_t getMaxFields() const override + { + return context->getSettingsRef().http_max_fields; + } + + uint64_t getMaxFieldNameSize() const override + { + return context->getSettingsRef().http_max_field_name_size; + } + + uint64_t getMaxFieldValueSize() const override + { + return context->getSettingsRef().http_max_field_value_size; + } + + uint64_t getMaxChunkSize() const override + { + return context->getSettingsRef().http_max_chunk_size; + } + + Poco::Timespan getReceiveTimeout() const override + { + return context->getSettingsRef().http_receive_timeout; + } + + Poco::Timespan getSendTimeout() const override + { + return context->getSettingsRef().http_send_timeout; + } + + ContextPtr context; +}; + } diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 2fc20463aa8..23a5d95e0f7 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -52,7 +52,7 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( , WithContext(global_context_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) {} - + void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values) { if (auto mark_cache = getContext()->getMarkCache()) diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index ce4e349e34b..bcc27b6454c 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index cce4ca17864..1fb3d9f0f59 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include "PrometheusMetricsWriter.h" From a0c12c2428f213c7de7af0578d05c13eb52e7294 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 9 Nov 2022 16:06:25 +0100 Subject: [PATCH 04/62] Define LOGICAL_ERROR --- programs/keeper/Keeper.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/programs/keeper/Keeper.h b/programs/keeper/Keeper.h index 585e5a04641..224990cc28e 100644 --- a/programs/keeper/Keeper.h +++ b/programs/keeper/Keeper.h @@ -16,6 +16,11 @@ namespace Poco namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + /// standalone clickhouse-keeper server (replacement for ZooKeeper). Uses the same /// config as clickhouse-server. Serves requests on TCP ports with or without /// SSL using ZooKeeper protocol. From 36d763011b60c2fea2e86f231275abf1f0b7d19c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 10 Nov 2022 09:23:52 +0100 Subject: [PATCH 05/62] Compile standalone keeper --- programs/keeper/CMakeLists.txt | 13 ++++++- src/Coordination/KeeperDispatcher.cpp | 1 + src/Coordination/KeeperStorage.cpp | 1 + src/Server/HTTP/HTTPServerConnection.h | 1 - src/Server/HTTP/HTTPServerConnectionFactory.h | 1 - src/Server/HTTP/HTTPServerRequest.cpp | 1 - src/Server/HTTPHandlerFactory.cpp | 30 --------------- src/Server/HTTPHandlerFactory.h | 18 +-------- src/Server/HTTPRequestHandlerFactoryMain.cpp | 38 +++++++++++++++++++ src/Server/HTTPRequestHandlerFactoryMain.h | 27 +++++++++++++ 10 files changed, 80 insertions(+), 51 deletions(-) create mode 100644 src/Server/HTTPRequestHandlerFactoryMain.cpp create mode 100644 src/Server/HTTPRequestHandlerFactoryMain.h diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 67ea245cc5b..9b01e6920a4 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -48,6 +48,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperAsynchronousMetrics.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/TinyContext.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/pathUtils.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp @@ -64,7 +65,18 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/NotFoundHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ProtocolServerAdapter.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusMetricsWriter.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPRequestHandlerFactoryMain.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/ReadHeaders.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerConnection.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerRequest.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerResponse.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerConnectionFactory.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CachedCompressedReadBuffer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CheckingCompressedReadBuffer.cpp @@ -97,7 +109,6 @@ if (BUILD_STANDALONE_KEEPER) Keeper.cpp clickhouse-keeper.cpp - ) clickhouse_add_executable(clickhouse-keeper ${CLICKHOUSE_KEEPER_STANDALONE_SOURCES}) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 6e9116d4b75..b8f83e6ebcd 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -214,6 +214,7 @@ void KeeperDispatcher::snapshotThread() void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) { std::lock_guard lock(session_to_response_callback_mutex); + LOG_INFO(&Poco::Logger::get("LOGGER"), "Sending {} for {}", session_id, response->getOpNum()); /// Special new session response. if (response->xid != Coordination::WATCH_XID && response->getOpNum() == Coordination::OpNum::SessionID) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index fb472201aec..f47b8f76b53 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -774,6 +774,7 @@ struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageReques Coordination::ZooKeeperResponsePtr process(KeeperStorage & /* storage */, int64_t /* zxid */) const override { + LOG_INFO(&Poco::Logger::get("LOGGER"), "Processing heartbeat"); return zk_request->makeResponse(); } }; diff --git a/src/Server/HTTP/HTTPServerConnection.h b/src/Server/HTTP/HTTPServerConnection.h index c4ecb821ff1..cce4f44f203 100644 --- a/src/Server/HTTP/HTTPServerConnection.h +++ b/src/Server/HTTP/HTTPServerConnection.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Server/HTTP/HTTPServerConnectionFactory.h b/src/Server/HTTP/HTTPServerConnectionFactory.h index b9b2ce0ad07..03648ce7be7 100644 --- a/src/Server/HTTP/HTTPServerConnectionFactory.h +++ b/src/Server/HTTP/HTTPServerConnectionFactory.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index e70a1f93149..a82eb95aee1 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index d3b41b25b2b..e4da7941b50 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include @@ -33,35 +32,6 @@ static void addDefaultHandlersFactory( const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics); -HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_) - : log(&Poco::Logger::get(name_)), name(name_) -{ -} - -std::unique_ptr HTTPRequestHandlerFactoryMain::createRequestHandler(const HTTPServerRequest & request) -{ - LOG_TRACE(log, "HTTP Request for {}. Method: {}, Address: {}, User-Agent: {}{}, Content Type: {}, Transfer Encoding: {}, X-Forwarded-For: {}", - name, request.getMethod(), request.clientAddress().toString(), request.get("User-Agent", "(none)"), - (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")), - request.getContentType(), request.getTransferEncoding(), request.get("X-Forwarded-For", "(none)")); - - for (auto & handler_factory : child_factories) - { - auto handler = handler_factory->createRequestHandler(request); - if (handler) - return handler; - } - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - { - return std::unique_ptr(new NotFoundHandler); - } - - return nullptr; -} - static inline auto createHandlersFactoryFromConfig( IServer & server, const Poco::Util::AbstractConfiguration & config, diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index bcc27b6454c..f56c712c615 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -19,23 +20,6 @@ namespace ErrorCodes class IServer; -/// Handle request using child handlers -class HTTPRequestHandlerFactoryMain : public HTTPRequestHandlerFactory -{ -public: - explicit HTTPRequestHandlerFactoryMain(const std::string & name_); - - void addHandler(HTTPRequestHandlerFactoryPtr child_factory) { child_factories.emplace_back(child_factory); } - - std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; - -private: - Poco::Logger * log; - std::string name; - - std::vector child_factories; -}; - template class HandlingRuleHTTPHandlerFactory : public HTTPRequestHandlerFactory { diff --git a/src/Server/HTTPRequestHandlerFactoryMain.cpp b/src/Server/HTTPRequestHandlerFactoryMain.cpp new file mode 100644 index 00000000000..61a2909d30f --- /dev/null +++ b/src/Server/HTTPRequestHandlerFactoryMain.cpp @@ -0,0 +1,38 @@ +#include +#include + +#include + +namespace DB +{ + +HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_) + : log(&Poco::Logger::get(name_)), name(name_) +{ +} + +std::unique_ptr HTTPRequestHandlerFactoryMain::createRequestHandler(const HTTPServerRequest & request) +{ + LOG_TRACE(log, "HTTP Request for {}. Method: {}, Address: {}, User-Agent: {}{}, Content Type: {}, Transfer Encoding: {}, X-Forwarded-For: {}", + name, request.getMethod(), request.clientAddress().toString(), request.get("User-Agent", "(none)"), + (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")), + request.getContentType(), request.getTransferEncoding(), request.get("X-Forwarded-For", "(none)")); + + for (auto & handler_factory : child_factories) + { + auto handler = handler_factory->createRequestHandler(request); + if (handler) + return handler; + } + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + { + return std::unique_ptr(new NotFoundHandler); + } + + return nullptr; +} + +} diff --git a/src/Server/HTTPRequestHandlerFactoryMain.h b/src/Server/HTTPRequestHandlerFactoryMain.h new file mode 100644 index 00000000000..b0e57bd6b3b --- /dev/null +++ b/src/Server/HTTPRequestHandlerFactoryMain.h @@ -0,0 +1,27 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +/// Handle request using child handlers +class HTTPRequestHandlerFactoryMain : public HTTPRequestHandlerFactory +{ +public: + explicit HTTPRequestHandlerFactoryMain(const std::string & name_); + + void addHandler(HTTPRequestHandlerFactoryPtr child_factory) { child_factories.emplace_back(child_factory); } + + std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; + +private: + Poco::Logger * log; + std::string name; + + std::vector child_factories; +}; + +} From 8aeb406d826f8921515472036aaffb65ecf7abf8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 10 Nov 2022 10:49:00 +0100 Subject: [PATCH 06/62] Fix standalone build --- programs/keeper/Keeper.cpp | 8 ++++++++ src/Coordination/KeeperDispatcher.cpp | 1 - src/Coordination/KeeperStorage.cpp | 1 - 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index a28a96ddb3b..22185fc2c96 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -58,6 +58,14 @@ int mainEntryClickHouseKeeper(int argc, char ** argv) } } +#ifdef KEEPER_STANDALONE_BUILD + +void collectCrashLog( + Int32, UInt64, const String &, const StackTrace &) +{} + +#endif + namespace DB { diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index b8f83e6ebcd..6e9116d4b75 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -214,7 +214,6 @@ void KeeperDispatcher::snapshotThread() void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) { std::lock_guard lock(session_to_response_callback_mutex); - LOG_INFO(&Poco::Logger::get("LOGGER"), "Sending {} for {}", session_id, response->getOpNum()); /// Special new session response. if (response->xid != Coordination::WATCH_XID && response->getOpNum() == Coordination::OpNum::SessionID) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index f47b8f76b53..fb472201aec 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -774,7 +774,6 @@ struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageReques Coordination::ZooKeeperResponsePtr process(KeeperStorage & /* storage */, int64_t /* zxid */) const override { - LOG_INFO(&Poco::Logger::get("LOGGER"), "Processing heartbeat"); return zk_request->makeResponse(); } }; From 89fc3f52f33f8af347ef34c8c43b29001efff0f0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 10 Nov 2022 10:41:49 +0000 Subject: [PATCH 07/62] Add missing include --- src/Interpreters/ServerAsynchronousMetrics.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 23a5d95e0f7..ecb2c44d844 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -27,14 +28,14 @@ namespace { template -static void calculateMax(Max & max, T x) +void calculateMax(Max & max, T x) { if (Max(x) > max) max = x; } template -static void calculateMaxAndSum(Max & max, Sum & sum, T x) +void calculateMaxAndSum(Max & max, Sum & sum, T x) { sum += x; if (Max(x) > max) From 94aab4276c72dd62fa1b99d80bacdbfa6fe85f37 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 10 Nov 2022 12:56:27 +0100 Subject: [PATCH 08/62] Bring back heavy metrics --- src/Common/AsynchronousMetrics.cpp | 2 +- src/Common/AsynchronousMetrics.h | 2 +- src/Coordination/KeeperAsynchronousMetrics.cpp | 2 +- src/Coordination/KeeperAsynchronousMetrics.h | 2 +- src/Interpreters/ServerAsynchronousMetrics.cpp | 4 +++- src/Interpreters/ServerAsynchronousMetrics.h | 2 +- 6 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index e775b3aa263..05143cbc666 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -1265,7 +1265,7 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Add more metrics as you wish. - updateImpl(new_values); + updateImpl(new_values, update_time, current_time); new_values["AsynchronousMetricsCalculationTimeSpent"] = watch.elapsedSeconds(); diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 37ee3389da6..3f0eab61304 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -73,7 +73,7 @@ protected: Poco::Logger * log; private: - virtual void updateImpl(AsynchronousMetricValues & new_values) = 0; + virtual void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) = 0; virtual void logImpl(AsynchronousMetricValues &) {} ProtocolServerMetricsFunc protocol_server_metrics_func; diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index e26f9e1c9e8..4efb90a7995 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -113,7 +113,7 @@ KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( { } -void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values) +void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint /*update_time*/, TimePoint /*current_time*/) { #if USE_NURAFT { diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index 579131c25b4..5b703951065 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -18,7 +18,7 @@ public: private: const TinyContext & tiny_context; - void updateImpl(AsynchronousMetricValues & new_values) override; + void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; }; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index ecb2c44d844..cc8fd7e14b2 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -54,7 +54,7 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( , heavy_metric_update_period(heavy_metrics_update_period_seconds) {} -void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values) +void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) { if (auto mark_cache = getContext()->getMarkCache()) { @@ -268,6 +268,8 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values updateKeeperInformation(*keeper_dispatcher, new_values); } #endif + + updateHeavyMetricsIfNeeded(current_time, update_time, new_values); } void ServerAsynchronousMetrics::logImpl(AsynchronousMetricValues & new_values) diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index cd19483032a..81047e2fdf9 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -16,7 +16,7 @@ public: int heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); private: - void updateImpl(AsynchronousMetricValues & new_values) override; + void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; void logImpl(AsynchronousMetricValues & new_values) override; const Duration heavy_metric_update_period; From 84aea78d0f269610865242791bbc7ea5fb59b3af Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 16 Nov 2022 13:11:05 +0800 Subject: [PATCH 09/62] fix array first / last or null with nullable column --- src/Functions/array/arrayFirstLast.cpp | 18 +++++++++++++++--- .../02241_array_first_last_or_null.reference | 6 ++++++ .../02241_array_first_last_or_null.sql | 8 ++++++++ 3 files changed, 29 insertions(+), 3 deletions(-) diff --git a/src/Functions/array/arrayFirstLast.cpp b/src/Functions/array/arrayFirstLast.cpp index 8160234a6b0..9b193756f4a 100644 --- a/src/Functions/array/arrayFirstLast.cpp +++ b/src/Functions/array/arrayFirstLast.cpp @@ -3,6 +3,8 @@ #include #include +#include "Columns/IColumn.h" +#include "Core/Types.h" #include "FunctionArrayMapped.h" @@ -43,6 +45,16 @@ struct ArrayFirstLastImpl return array_element; } + static ColumnPtr createNullableColumn(MutableColumnPtr && column, ColumnUInt8::MutablePtr && null_map) + { + if (auto * nullable_column = typeid_cast(column.get())) + { + nullable_column->applyNullMap(*null_map); + return std::move(column); + } + return ColumnNullable::create(std::move(column), std::move(null_map)); + } + static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped) { const auto * column_filter = typeid_cast(&*mapped); @@ -94,7 +106,7 @@ struct ArrayFirstLastImpl } if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null) - return ColumnNullable::create(std::move(out), std::move(col_null_map_to)); + return createNullableColumn(std::move(out), std::move(col_null_map_to)); return out; } @@ -106,7 +118,7 @@ struct ArrayFirstLastImpl if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null) { auto col_null_map_to = ColumnUInt8::create(out->size(), true); - return ColumnNullable::create(std::move(out), std::move(col_null_map_to)); + return createNullableColumn(std::move(out), std::move(col_null_map_to)); } return out; @@ -172,7 +184,7 @@ struct ArrayFirstLastImpl } if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null) - return ColumnNullable::create(std::move(out), std::move(col_null_map_to)); + return createNullableColumn(std::move(out), std::move(col_null_map_to)); return out; } diff --git a/tests/queries/0_stateless/02241_array_first_last_or_null.reference b/tests/queries/0_stateless/02241_array_first_last_or_null.reference index 2906b04ecd0..fc4a5ff8af5 100644 --- a/tests/queries/0_stateless/02241_array_first_last_or_null.reference +++ b/tests/queries/0_stateless/02241_array_first_last_or_null.reference @@ -7,6 +7,9 @@ ArrayFirst non constant predicate \N 2 2 +ArrayFirst with Null +2 +\N ArrayLast constant predicate \N \N @@ -16,3 +19,6 @@ ArrayLast non constant predicate \N 3 3 +ArrayLast with Null +2 +\N diff --git a/tests/queries/0_stateless/02241_array_first_last_or_null.sql b/tests/queries/0_stateless/02241_array_first_last_or_null.sql index 3230e4d483a..aa8f0cdbf92 100644 --- a/tests/queries/0_stateless/02241_array_first_last_or_null.sql +++ b/tests/queries/0_stateless/02241_array_first_last_or_null.sql @@ -9,6 +9,10 @@ SELECT arrayFirstOrNull(x -> x >= 2, emptyArrayUInt8()); SELECT arrayFirstOrNull(x -> x >= 2, [1, 2, 3]); SELECT arrayFirstOrNull(x -> x >= 2, materialize([1, 2, 3])); +SELECT 'ArrayFirst with Null'; +SELECT arrayFirstOrNull((x,f) -> f, [1,2,3,NULL], [0,1,0,0]); +SELECT arrayFirstOrNull((x,f) -> f, [1,2,3,NULL], [0,0,0,1]); + SELECT 'ArrayLast constant predicate'; SELECT arrayLastOrNull(x -> 1, emptyArrayUInt8()); SELECT arrayLastOrNull(x -> 0, emptyArrayUInt8()); @@ -19,3 +23,7 @@ SELECT 'ArrayLast non constant predicate'; SELECT arrayLastOrNull(x -> x >= 2, emptyArrayUInt8()); SELECT arrayLastOrNull(x -> x >= 2, [1, 2, 3]); SELECT arrayLastOrNull(x -> x >= 2, materialize([1, 2, 3])); + +SELECT 'ArrayLast with Null'; +SELECT arrayLastOrNull((x,f) -> f, [1,2,3,NULL], [0,1,0,0]); +SELECT arrayLastOrNull((x,f) -> f, [1,2,3,NULL], [0,1,0,1]); \ No newline at end of file From fa70e335dbd4dcffd60bb9d433b4771c53d6ceb6 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 21 Nov 2022 10:53:54 +0800 Subject: [PATCH 10/62] minor fix --- src/Functions/array/arrayFirstLast.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Functions/array/arrayFirstLast.cpp b/src/Functions/array/arrayFirstLast.cpp index 9b193756f4a..fa72ecba161 100644 --- a/src/Functions/array/arrayFirstLast.cpp +++ b/src/Functions/array/arrayFirstLast.cpp @@ -3,8 +3,6 @@ #include #include -#include "Columns/IColumn.h" -#include "Core/Types.h" #include "FunctionArrayMapped.h" @@ -47,7 +45,7 @@ struct ArrayFirstLastImpl static ColumnPtr createNullableColumn(MutableColumnPtr && column, ColumnUInt8::MutablePtr && null_map) { - if (auto * nullable_column = typeid_cast(column.get())) + if (auto * nullable_column = typeid_cast(column.get())) { nullable_column->applyNullMap(*null_map); return std::move(column); From 4ea833afa1f2abae7dc91e52b9637d39ad5c6e31 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Nov 2022 17:29:03 +0100 Subject: [PATCH 11/62] Fix bug with projections and zero-copy replication --- .../MergeTree/DataPartStorageOnDisk.cpp | 11 ++- src/Storages/MergeTree/IDataPartStorage.h | 3 + src/Storages/MergeTree/MergeTreeData.cpp | 18 +++++ ...ction_and_mutation_work_together.reference | 4 + ..._projection_and_mutation_work_together.sql | 79 +++++++++++++++++++ 5 files changed, 114 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.reference create mode 100644 tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index e68a04fd9bc..347ea16950e 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -101,6 +101,7 @@ public: bool isValid() const override { return it->isValid(); } bool isFile() const override { return isValid() && disk->isFile(it->path()); } std::string name() const override { return it->name(); } + std::string path() const override { return it->path(); } private: DiskPtr disk; @@ -259,9 +260,17 @@ void DataPartStorageOnDisk::remove( std::string proj_dir_name = projection.name + proj_suffix; projection_directories.emplace(proj_dir_name); + NameSet files_not_to_remove_for_projection; + for (const auto & file_name : can_remove_description->files_not_to_remove) + { + if (file_name.starts_with(proj_dir_name)) + files_not_to_remove_for_projection.emplace(fs::path(file_name).filename()); + } + LOG_DEBUG(log, "Will not remove files [{}] for projection {}", fmt::join(files_not_to_remove_for_projection, ", "), projection.name); + clearDirectory( fs::path(to) / proj_dir_name, - can_remove_description->can_remove_anything, can_remove_description->files_not_to_remove, projection.checksums, {}, is_temp, state, log, true); + can_remove_description->can_remove_anything, files_not_to_remove_for_projection, projection.checksums, {}, is_temp, state, log, true); } /// It is possible that we are removing the part which have a written but not loaded projection. diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index c6669908db4..53ee2738fc6 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -39,6 +39,9 @@ public: /// Name of the file that the iterator currently points to. virtual std::string name() const = 0; + /// Path of the file that the iterator currently points to. + virtual std::string path() const = 0; + virtual ~IDataPartStorageIterator() = default; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 088821c6188..534ad9bd972 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6412,7 +6412,25 @@ std::pair MergeTreeData::cloneAn if (!files_to_copy_instead_of_hardlinks.contains(it->name()) && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { hardlinked_files->hardlinks_from_source_part.insert(it->name()); + } + } + + auto projections = src_part->getProjectionParts(); + for (const auto & [name, projection_part] : projections) + { + const auto & projection_storage = projection_part->getDataPartStorage(); + for (auto it = projection_storage.iterate(); it->isValid(); it->next()) + { + auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); + if (!files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); + } + } } } diff --git a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.reference b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.reference new file mode 100644 index 00000000000..726e74146fc --- /dev/null +++ b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.reference @@ -0,0 +1,4 @@ +199 +199 +1990 199 +1990 199 diff --git a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql new file mode 100644 index 00000000000..7a51d86dd30 --- /dev/null +++ b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql @@ -0,0 +1,79 @@ +DROP TABLE IF EXISTS wikistat1; +DROP TABLE IF EXISTS wikistat2; + +CREATE TABLE wikistat1 +( + time DateTime, + project LowCardinality(String), + subproject LowCardinality(String), + path String, + hits UInt64, + PROJECTION total + ( + SELECT + project, + subproject, + path, + sum(hits), + count() + GROUP BY + project, + subproject, + path + ) +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/02494_zero_copy_and_projection', '1') +ORDER BY (path, time) +SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, allow_remote_fs_zero_copy_replication=1, min_bytes_for_wide_part=0; + +CREATE TABLE wikistat2 +( + time DateTime, + project LowCardinality(String), + subproject LowCardinality(String), + path String, + hits UInt64, + PROJECTION total + ( + SELECT + project, + subproject, + path, + sum(hits), + count() + GROUP BY + project, + subproject, + path + ) +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/02494_zero_copy_and_projection', '2') +ORDER BY (path, time) +SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, allow_remote_fs_zero_copy_replication=1, min_bytes_for_wide_part=0; + +INSERT INTO wikistat1 SELECT toDateTime('2020-10-01 00:00:00'), 'hello', 'world', '/data/path', 10 from numbers(100); + +INSERT INTO wikistat1 SELECT toDateTime('2020-10-01 00:00:00'), 'hello', 'world', '/data/path', 10 from numbers(99, 99); + +SYSTEM SYNC REPLICA wikistat2; + +SELECT COUNT() from wikistat1 WHERE NOT ignore(*); +SELECT COUNT() from wikistat2 WHERE NOT ignore(*); + +SYSTEM STOP REPLICATION QUEUES wikistat2; + +ALTER TABLE wikistat1 DELETE where time = toDateTime('2022-12-20 00:00:00') SETTINGS mutations_sync = 1; + +SYSTEM START REPLICATION QUEUES wikistat2; + +SYSTEM SYNC REPLICA wikistat2; + +-- it doesn't make test flaky, rarely we will not delete the parts because of cleanup thread was slow. +-- Such condition will lead to successful queries. +SELECT 0 FROM numbers(5) WHERE sleepEachRow(1) = 1; + +select sum(hits), count() from wikistat1 GROUP BY project, subproject, path settings allow_experimental_projection_optimization = 1, force_optimize_projection = 1; +select sum(hits), count() from wikistat2 GROUP BY project, subproject, path settings allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + +DROP TABLE wikistat1; +DROP TABLE wikistat2; From 7914e2674470bb3aa85efa3b91d9ee93f023d8cd Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Nov 2022 12:50:20 +0100 Subject: [PATCH 12/62] Fix another part of code --- src/Storages/MergeTree/MutateTask.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e5ba771a198..916befdf78f 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1322,9 +1322,11 @@ private: for (auto p_it = projection_data_part_storage_src->iterate(); p_it->isValid(); p_it->next()) { + auto file_name_with_projection_prefix = fs::path(projection_data_part_storage_src->getPartDirectory()) / it->name(); projection_data_part_storage_dst->createHardLinkFrom( *projection_data_part_storage_src, p_it->name(), p_it->name()); - hardlinked_files.insert(p_it->name()); + + hardlinked_files.insert(file_name_with_projection_prefix); } } } From c058f07fabb37d379ffc02f88e75029e0b8ba715 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Nov 2022 16:40:52 +0100 Subject: [PATCH 13/62] Add mypy config, fix git_helper.py --- tests/ci/.mypy.ini | 16 ++++++++++++++++ tests/ci/git_helper.py | 29 ++++++++++++++++------------- 2 files changed, 32 insertions(+), 13 deletions(-) create mode 100644 tests/ci/.mypy.ini diff --git a/tests/ci/.mypy.ini b/tests/ci/.mypy.ini new file mode 100644 index 00000000000..7326675067c --- /dev/null +++ b/tests/ci/.mypy.ini @@ -0,0 +1,16 @@ +[mypy] +warn_no_return = False +warn_unused_configs = True +disallow_subclassing_any = True +disallow_untyped_calls = False +disallow_untyped_defs = False +disallow_incomplete_defs = True +check_untyped_defs = True +disallow_untyped_decorators = True +no_implicit_optional = True +warn_redundant_casts = True +warn_unused_ignores = True +warn_return_any = True +no_implicit_reexport = True +strict_equality = True +strict_concatenate = True diff --git a/tests/ci/git_helper.py b/tests/ci/git_helper.py index 77c2fc9cf05..eb5e835eab3 100644 --- a/tests/ci/git_helper.py +++ b/tests/ci/git_helper.py @@ -4,7 +4,7 @@ import logging import os.path as p import re import subprocess -from typing import List, Optional +from typing import Any, List, Optional logger = logging.getLogger(__name__) @@ -21,19 +21,19 @@ TWEAK = 1 # Py 3.8 removeprefix and removesuffix -def removeprefix(string: str, prefix: str): +def removeprefix(string: str, prefix: str) -> str: if string.startswith(prefix): return string[len(prefix) :] # noqa: ignore E203, false positive return string -def removesuffix(string: str, suffix: str): +def removesuffix(string: str, suffix: str) -> str: if string.endswith(suffix): return string[: -len(suffix)] return string -def commit(name: str): +def commit(name: str) -> str: r = re.compile(SHA_REGEXP) if not r.match(name): raise argparse.ArgumentTypeError( @@ -42,7 +42,7 @@ def commit(name: str): return name -def release_branch(name: str): +def release_branch(name: str) -> str: r = re.compile(RELEASE_BRANCH_REGEXP) if not r.match(name): raise argparse.ArgumentTypeError("release branch should be as 12.1") @@ -55,20 +55,23 @@ class Runner: def __init__(self, cwd: str = CWD): self._cwd = cwd - def run(self, cmd: str, cwd: Optional[str] = None, **kwargs) -> str: + def run(self, cmd: str, cwd: Optional[str] = None, **kwargs: Any) -> str: if cwd is None: cwd = self.cwd logger.debug("Running command: %s", cmd) - return subprocess.check_output( - cmd, shell=True, cwd=cwd, encoding="utf-8", **kwargs - ).strip() + output = str( + subprocess.check_output( + cmd, shell=True, cwd=cwd, encoding="utf-8", **kwargs + ).strip() + ) + return output @property def cwd(self) -> str: return self._cwd @cwd.setter - def cwd(self, value: str): + def cwd(self, value: str) -> None: # Set _cwd only once, then set it to readonly if self._cwd != CWD: return @@ -139,7 +142,7 @@ class Git: ) @staticmethod - def check_tag(value: str): + def check_tag(value: str) -> None: if value == "": return if not Git._tag_pattern.match(value): @@ -150,7 +153,7 @@ class Git: return self._latest_tag @latest_tag.setter - def latest_tag(self, value: str): + def latest_tag(self, value: str) -> None: self.check_tag(value) self._latest_tag = value @@ -159,7 +162,7 @@ class Git: return self._new_tag @new_tag.setter - def new_tag(self, value: str): + def new_tag(self, value: str) -> None: self.check_tag(value) self._new_tag = value From 9d173ca3696be5e60ec77732d890c326e99bf154 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Nov 2022 16:47:44 +0100 Subject: [PATCH 14/62] Fix version_helper.py --- tests/ci/version_helper.py | 27 ++++++++++++++++----------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 162bab6a50a..dc4674b3699 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -2,9 +2,9 @@ import logging import os.path as p from argparse import ArgumentParser, ArgumentDefaultsHelpFormatter, ArgumentTypeError -from typing import Dict, List, Optional, Tuple, Union +from typing import Any, Dict, List, Literal, Optional, Tuple, Union -from git_helper import TWEAK, Git, get_tags, git_runner, removeprefix +from git_helper import TWEAK, Git as Git, get_tags, git_runner, removeprefix FILE_WITH_VERSION_PATH = "cmake/autogenerated_versions.txt" CHANGELOG_IN_PATH = "debian/changelog.in" @@ -45,7 +45,7 @@ class ClickHouseVersion: patch: Union[int, str], revision: Union[int, str], git: Optional[Git], - tweak: str = None, + tweak: Optional[str] = None, ): self._major = int(major) self._minor = int(minor) @@ -59,10 +59,15 @@ class ClickHouseVersion: self._tweak = self._git.tweak self._describe = "" - def update(self, part: str) -> "ClickHouseVersion": + def update(self, part: Literal["major", "minor", "patch"]) -> "ClickHouseVersion": """If part is valid, returns a new version""" - method = getattr(self, f"{part}_update") - return method() + if part == "major": + return self.major_update() + if part == "minor": + return self.minor_update() + if part == "patch": + return self.patch_update() + raise KeyError(f"wrong part {part} is used") def major_update(self) -> "ClickHouseVersion": if self._git is not None: @@ -139,10 +144,10 @@ class ClickHouseVersion: raise ValueError(f"version type {version_type} not in {VersionType.VALID}") self._describe = f"v{self.string}-{version_type}" - def __eq__(self, other) -> bool: + def __eq__(self, other: Any) -> bool: if not isinstance(self, type(other)): return NotImplemented - return ( + return ( # type: ignore self.major == other.major and self.minor == other.minor and self.patch == other.patch @@ -170,7 +175,7 @@ class VersionType: VALID = (TESTING, PRESTABLE, STABLE, LTS) -def validate_version(version: str): +def validate_version(version: str) -> None: parts = version.split(".") if len(parts) != 4: raise ValueError(f"{version} does not contain 4 parts") @@ -259,7 +264,7 @@ def get_tagged_versions() -> List[ClickHouseVersion]: def update_cmake_version( version: ClickHouseVersion, versions_path: str = FILE_WITH_VERSION_PATH, -): +) -> None: path_to_file = get_abs_path(versions_path) with open(path_to_file, "w", encoding="utf-8") as f: f.write(VERSIONS_TEMPLATE.format_map(version.as_dict())) @@ -269,7 +274,7 @@ def update_contributors( relative_contributors_path: str = GENERATED_CONTRIBUTORS, force: bool = False, raise_error: bool = False, -): +) -> None: # Check if we have shallow checkout by comparing number of lines # '--is-shallow-repository' is in git since 2.15, 2017-10-30 if git_runner.run("git rev-parse --is-shallow-repository") == "true" and not force: From 78e560f78e83c49c82ecae25a2e6338903db340f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Nov 2022 16:57:01 +0100 Subject: [PATCH 15/62] Fix docker_image_check and docker_test --- tests/ci/docker_images_check.py | 6 +++--- tests/ci/docker_test.py | 20 ++++++++++---------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 873aee9aabf..0618969f94c 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -8,7 +8,7 @@ import shutil import subprocess import time import sys -from typing import Dict, List, Optional, Set, Tuple, Union +from typing import Any, Dict, List, Optional, Set, Tuple, Union from github import Github @@ -52,7 +52,7 @@ class DockerImage: and self.only_amd64 == other.only_amd64 ) - def __lt__(self, other) -> bool: + def __lt__(self, other: Any) -> bool: if not isinstance(other, DockerImage): return False if self.parent and not other.parent: @@ -270,7 +270,7 @@ def build_and_push_one_image( def process_single_image( image: DockerImage, versions: List[str], - additional_cache, + additional_cache: str, push: bool, child: bool, ) -> List[Tuple[str, str, str]]: diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 1848300e2f6..8b18a580ed7 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -43,55 +43,55 @@ class TestDockerImageCheck(unittest.TestCase): "docker/test/stateless", "clickhouse/stateless-test", False, - "clickhouse/test-base", + "clickhouse/test-base", # type: ignore ), di.DockerImage( "docker/test/integration/base", "clickhouse/integration-test", False, - "clickhouse/test-base", + "clickhouse/test-base", # type: ignore ), di.DockerImage( "docker/test/fuzzer", "clickhouse/fuzzer", False, - "clickhouse/test-base", + "clickhouse/test-base", # type: ignore ), di.DockerImage( "docker/test/keeper-jepsen", "clickhouse/keeper-jepsen-test", False, - "clickhouse/test-base", + "clickhouse/test-base", # type: ignore ), di.DockerImage( "docker/docs/check", "clickhouse/docs-check", False, - "clickhouse/docs-builder", + "clickhouse/docs-builder", # type: ignore ), di.DockerImage( "docker/docs/release", "clickhouse/docs-release", False, - "clickhouse/docs-builder", + "clickhouse/docs-builder", # type: ignore ), di.DockerImage( "docker/test/stateful", "clickhouse/stateful-test", False, - "clickhouse/stateless-test", + "clickhouse/stateless-test", # type: ignore ), di.DockerImage( "docker/test/unit", "clickhouse/unit-test", False, - "clickhouse/stateless-test", + "clickhouse/stateless-test", # type: ignore ), di.DockerImage( "docker/test/stress", "clickhouse/stress-test", False, - "clickhouse/stateful-test", + "clickhouse/stateful-test", # type: ignore ), ] ) @@ -277,7 +277,7 @@ class TestDockerServer(unittest.TestCase): ds.gen_tags(version, "auto") @patch("docker_server.get_tagged_versions") - def test_auto_release_type(self, mock_tagged_versions: MagicMock): + def test_auto_release_type(self, mock_tagged_versions: MagicMock) -> None: mock_tagged_versions.return_value = [ get_version_from_string("1.1.1.1"), get_version_from_string("1.2.1.1"), From 9dff6a80ab5303a7b33a93b1dbb0100c71591a12 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Nov 2022 17:11:23 +0100 Subject: [PATCH 16/62] Fix run_check.py and dependencies --- tests/ci/build_download_helper.py | 12 ++++----- tests/ci/commit_status_helper.py | 16 ++++++------ tests/ci/get_robot_token.py | 26 ++++++++++++++----- tests/ci/pr_info.py | 6 ++--- tests/ci/run_check.py | 2 +- tests/ci/workflow_approve_rerun_lambda/app.py | 4 +-- 6 files changed, 39 insertions(+), 27 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 58997bed253..1a2fdedefed 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -5,7 +5,7 @@ import logging import os import sys import time -from typing import List, Optional +from typing import Any, List, Optional import requests # type: ignore @@ -18,7 +18,7 @@ def get_with_retries( url: str, retries: int = DOWNLOAD_RETRIES_COUNT, sleep: int = 3, - **kwargs, + **kwargs: Any, ) -> requests.Response: logging.info( "Getting URL with %i tries and sleep %i in between: %s", retries, sleep, url @@ -41,18 +41,18 @@ def get_with_retries( return response -def get_build_name_for_check(check_name) -> str: - return CI_CONFIG["tests_config"][check_name]["required_build"] +def get_build_name_for_check(check_name: str) -> str: + return CI_CONFIG["tests_config"][check_name]["required_build"] # type: ignore -def read_build_urls(build_name, reports_path) -> List[str]: +def read_build_urls(build_name: str, reports_path: str) -> List[str]: for root, _, files in os.walk(reports_path): for f in files: if build_name in f: logging.info("Found build report json %s", f) with open(os.path.join(root, f), "r", encoding="utf-8") as file_handler: build_report = json.load(file_handler) - return build_report["build_urls"] + return build_report["build_urls"] # type: ignore return [] diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 185dc64daa9..785250c3904 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -17,7 +17,7 @@ RETRY = 5 CommitStatuses = List[CommitStatus] -def override_status(status: str, check_name: str, invert=False) -> str: +def override_status(status: str, check_name: str, invert: bool = False) -> str: if CI_CONFIG["tests_config"].get(check_name, {}).get("force_tests", False): return "success" @@ -45,7 +45,7 @@ def get_commit(gh: Github, commit_sha: str, retry_count: int = RETRY) -> Commit: def post_commit_status( gh: Github, sha: str, check_name: str, description: str, state: str, report_url: str -): +) -> None: for i in range(RETRY): try: commit = get_commit(gh, sha, 1) @@ -64,7 +64,7 @@ def post_commit_status( def post_commit_status_to_file( file_path: str, description: str, state: str, report_url: str -): +) -> None: if os.path.exists(file_path): raise Exception(f'File "{file_path}" already exists!') with open(file_path, "w", encoding="utf-8") as f: @@ -88,21 +88,21 @@ def get_commit_filtered_statuses(commit: Commit) -> CommitStatuses: return list(filtered.values()) -def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): +def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]) -> None: repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) for label in labels_names: pull_request.remove_from_labels(label) -def post_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): +def post_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]) -> None: repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) for label in labels_names: pull_request.add_to_labels(label) -def fail_mergeable_check(commit: Commit, description: str): +def fail_mergeable_check(commit: Commit, description: str) -> None: commit.create_status( context="Mergeable Check", description=description, @@ -111,7 +111,7 @@ def fail_mergeable_check(commit: Commit, description: str): ) -def reset_mergeable_check(commit: Commit, description: str = ""): +def reset_mergeable_check(commit: Commit, description: str = "") -> None: commit.create_status( context="Mergeable Check", description=description, @@ -120,7 +120,7 @@ def reset_mergeable_check(commit: Commit, description: str = ""): ) -def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str): +def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str) -> None: if SKIP_MERGEABLE_CHECK_LABEL in pr_info.labels: return diff --git a/tests/ci/get_robot_token.py b/tests/ci/get_robot_token.py index 163e1ce071e..6ecaf468ed1 100644 --- a/tests/ci/get_robot_token.py +++ b/tests/ci/get_robot_token.py @@ -1,8 +1,17 @@ #!/usr/bin/env python3 import logging +from dataclasses import dataclass import boto3 # type: ignore -from github import Github # type: ignore +from github import Github +from github.AuthenticatedUser import AuthenticatedUser + + +@dataclass +class Token: + user: AuthenticatedUser + value: str + rest: int def get_parameter_from_ssm(name, decrypt=True, client=None): @@ -19,7 +28,7 @@ def get_best_robot_token(token_prefix_env_name="github_robot_token_"): ] )["Parameters"] assert parameters - token = {"login": "", "value": "", "rest": 0} + token = None for token_name in [p["Name"] for p in parameters]: value = get_parameter_from_ssm(token_name, True, client) @@ -29,12 +38,15 @@ def get_best_robot_token(token_prefix_env_name="github_robot_token_"): user = gh.get_user() rest, _ = gh.rate_limiting logging.info("Get token with %s remaining requests", rest) - if token["rest"] < rest: - token = {"user": user, "value": value, "rest": rest} + if token is None: + token = Token(user, value, rest) + continue + if token.rest < rest: + token.user, token.value, token.rest = user, value, rest - assert token["value"] + assert token logging.info( - "User %s with %s remaining requests is used", token["user"].login, token["rest"] + "User %s with %s remaining requests is used", token.user.login, token.rest ) - return token["value"] + return token.value diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 6a2fac0a291..f0a371c488f 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -146,7 +146,7 @@ class PRInfo: self.body = github_event["pull_request"]["body"] self.labels = { label["name"] for label in github_event["pull_request"]["labels"] - } + } # type: Set[str] self.user_login = github_event["pull_request"]["user"]["login"] self.user_orgs = set([]) @@ -178,7 +178,7 @@ class PRInfo: if pull_request is None or pull_request["state"] == "closed": # it's merged PR to master self.number = 0 - self.labels = {} + self.labels = set() self.pr_html_url = f"{repo_prefix}/commits/{ref}" self.base_ref = ref self.base_name = self.repo_full_name @@ -228,7 +228,7 @@ class PRInfo: print(json.dumps(github_event, sort_keys=True, indent=4)) self.sha = os.getenv("GITHUB_SHA") self.number = 0 - self.labels = {} + self.labels = set() repo_prefix = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}" self.task_url = GITHUB_RUN_URL self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 39dbc938c8f..7119f443719 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -112,7 +112,7 @@ def should_run_checks_for_pr(pr_info: PRInfo) -> Tuple[bool, str, str]: return True, "No special conditions apply", "pending" -def check_pr_description(pr_info) -> Tuple[str, str]: +def check_pr_description(pr_info: PRInfo) -> Tuple[str, str]: lines = list( map(lambda x: x.strip(), pr_info.body.split("\n") if pr_info.body else []) ) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 23e808b0861..d285e29943d 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -313,7 +313,7 @@ def check_suspicious_changed_files(changed_files): return False -def approve_run(workflow_description: WorkflowDescription, token): +def approve_run(workflow_description: WorkflowDescription, token: str) -> None: url = f"{workflow_description.api_url}/approve" _exec_post_with_retry(url, token) @@ -391,7 +391,7 @@ def rerun_workflow(workflow_description, token): def check_workflow_completed( - event_data, workflow_description: WorkflowDescription, token: str + event_data: dict, workflow_description: WorkflowDescription, token: str ) -> bool: if workflow_description.action == "completed": attempt = 0 From 080ff97627da25d88db072aa0ddab7f472ffc4ca Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 15:47:09 +0100 Subject: [PATCH 17/62] Fix mypy errors in report.py --- tests/ci/report.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index a6700f50dfc..6a29aed9913 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -229,8 +229,8 @@ def create_test_html_report( if has_test_logs and not with_raw_logs: headers.append("Logs") - headers = "".join(["" + h + "" for h in headers]) - test_part = HTML_TEST_PART.format(headers=headers, rows=rows_part) + headers_html = "".join(["" + h + "" for h in headers]) + test_part = HTML_TEST_PART.format(headers=headers_html, rows=rows_part) else: test_part = "" @@ -341,7 +341,7 @@ def create_build_html_report( if build_result.elapsed_seconds: delta = datetime.timedelta(seconds=build_result.elapsed_seconds) else: - delta = "unknown" + delta = "unknown" # type: ignore row += "{}".format(str(delta)) From 324c8e4b127a63d03e835d2387e5811b7fb00154 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 15:48:51 +0100 Subject: [PATCH 18/62] Use f-strings in report.py --- tests/ci/report.py | 36 ++++++++++++++---------------------- 1 file changed, 14 insertions(+), 22 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 6a29aed9913..2904a5519a9 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -101,7 +101,7 @@ def _format_header(header, branch_name, branch_url=None): result = "ClickHouse " + result result += " for " if branch_url: - result += '{name}'.format(url=branch_url, name=branch_name) + result += f'{branch_name}' else: result += branch_name return result @@ -140,9 +140,7 @@ def _get_html_url(url): if isinstance(url, tuple): href, name = url[0], _get_html_url_name(url) if href and name: - return '{name}'.format( - href=href, name=_get_html_url_name(url) - ) + return f'{_get_html_url_name(url)}' return "" @@ -199,13 +197,7 @@ def create_test_html_report( num_fails = num_fails + 1 is_fail_id = 'id="fail' + str(num_fails) + '" ' - row += ( - "'.format(style) - + test_status - + "" - ) + row += f'{test_status}' if test_time is not None: row += "" + test_time + "" @@ -317,33 +309,33 @@ def create_build_html_report( build_results, build_logs_urls, artifact_urls_list ): row = "" - row += "{}".format(build_result.compiler) + row += f"{build_result.compiler}" if build_result.build_type: - row += "{}".format(build_result.build_type) + row += f"{build_result.build_type}" else: - row += "{}".format("relwithdebuginfo") + row += "relwithdebuginfo" if build_result.sanitizer: - row += "{}".format(build_result.sanitizer) + row += f"{build_result.sanitizer}" else: - row += "{}".format("none") + row += "none" - row += "{}".format(build_result.libraries) + row += f"{build_result.libraries}" if build_result.status: style = _get_status_style(build_result.status) - row += '{}'.format(style, build_result.status) + row += f'{build_result.status}' else: style = _get_status_style("error") - row += '{}'.format(style, "error") + row += f'error' - row += 'link'.format(build_log_url) + row += f'link' if build_result.elapsed_seconds: delta = datetime.timedelta(seconds=build_result.elapsed_seconds) else: delta = "unknown" # type: ignore - row += "{}".format(str(delta)) + row += f"{delta}" links = "" link_separator = "
" @@ -355,7 +347,7 @@ def create_build_html_report( links += link_separator if links: links = links[: -len(link_separator)] - row += "{}".format(links) + row += f"{links}" row += "" rows += row From 1fb090bfe20bd76345ed20ca2b50a9852fa46dbc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 18:50:31 +0100 Subject: [PATCH 19/62] Add typing to upload_result_helper.py, remove stale code --- tests/ci/s3_helper.py | 6 ++-- tests/ci/upload_result_helper.py | 52 +++++++++----------------------- 2 files changed, 18 insertions(+), 40 deletions(-) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 24ff013d69a..03e855a0057 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -46,7 +46,7 @@ class S3Helper: self.host = host self.download_host = download_host - def _upload_file_to_s3(self, bucket_name, file_path, s3_path): + def _upload_file_to_s3(self, bucket_name: str, file_path: str, s3_path: str) -> str: logging.debug( "Start uploading %s to bucket=%s path=%s", file_path, bucket_name, s3_path ) @@ -110,7 +110,7 @@ class S3Helper: url = f"{self.download_host}/{bucket_name}/{s3_path}" return url.replace("+", "%2B").replace(" ", "%20") - def upload_test_report_to_s3(self, file_path, s3_path): + def upload_test_report_to_s3(self, file_path: str, s3_path: str) -> str: if CI: return self._upload_file_to_s3(S3_TEST_REPORTS_BUCKET, file_path, s3_path) else: @@ -296,7 +296,7 @@ class S3Helper: return False @staticmethod - def copy_file_to_local(bucket_name, file_path, s3_path): + def copy_file_to_local(bucket_name: str, file_path: str, s3_path: str) -> str: local_path = os.path.abspath( os.path.join(RUNNER_TEMP, "s3", bucket_name, s3_path) ) diff --git a/tests/ci/upload_result_helper.py b/tests/ci/upload_result_helper.py index e145df02f80..745633a9e4d 100644 --- a/tests/ci/upload_result_helper.py +++ b/tests/ci/upload_result_helper.py @@ -1,6 +1,7 @@ import os import logging -import ast + +from typing import List, Tuple from env_helper import ( GITHUB_JOB_URL, @@ -9,35 +10,15 @@ from env_helper import ( GITHUB_SERVER_URL, ) from report import ReportColorTheme, create_test_html_report +from s3_helper import S3Helper def process_logs( - s3_client, additional_logs, s3_path_prefix, test_results, with_raw_logs -): + s3_client: S3Helper, additional_logs: List[str], s3_path_prefix: str +) -> List[str]: logging.info("Upload files to s3 %s", additional_logs) - processed_logs = {} - # Firstly convert paths of logs from test_results to urls to s3. - for test_result in test_results: - if len(test_result) <= 3 or with_raw_logs: - continue - - # Convert from string repr of list to list. - test_log_paths = ast.literal_eval(test_result[3]) - test_log_urls = [] - for log_path in test_log_paths: - if log_path in processed_logs: - test_log_urls.append(processed_logs[log_path]) - elif log_path: - url = s3_client.upload_test_report_to_s3( - log_path, s3_path_prefix + "/" + os.path.basename(log_path) - ) - test_log_urls.append(url) - processed_logs[log_path] = url - - test_result[3] = test_log_urls - - additional_urls = [] + additional_urls = [] # type: List[str] for log_path in additional_logs: if log_path: additional_urls.append( @@ -50,21 +31,18 @@ def process_logs( def upload_results( - s3_client, - pr_number, - commit_sha, - test_results, - additional_files, - check_name, - with_raw_logs=True, - statuscolors=None, -): + s3_client: S3Helper, + pr_number: int, + commit_sha: str, + test_results: List[Tuple[str, str]], + additional_files: List[str], + check_name: str, + with_raw_logs: bool = True, +) -> str: s3_path_prefix = f"{pr_number}/{commit_sha}/" + check_name.lower().replace( " ", "_" ).replace("(", "_").replace(")", "_").replace(",", "_") - additional_urls = process_logs( - s3_client, additional_files, s3_path_prefix, test_results, with_raw_logs - ) + additional_urls = process_logs(s3_client, additional_files, s3_path_prefix) branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commits/master" branch_name = "master" From 88381026576eecc3d01e70a71d201d1a956487f9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 18:52:38 +0100 Subject: [PATCH 20/62] Fix issue in docker_manifests_merge.py --- tests/ci/docker_manifests_merge.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index 09b7a99da78..2ba5a99de0a 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -70,7 +70,7 @@ def parse_args() -> argparse.Namespace: def load_images(path: str, suffix: str) -> Images: with open(os.path.join(path, CHANGED_IMAGES.format(suffix)), "rb") as images: - return json.load(images) + return json.load(images) # type: ignore def strip_suffix(suffix: str, images: Images) -> Images: From 02b8da2a0ffb3e5e89135ea33c2c1c3df4888fac Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 18:59:01 +0100 Subject: [PATCH 21/62] Add typing for TeePopen --- tests/ci/tee_popen.py | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py index 7270cd6fb03..61404847bff 100644 --- a/tests/ci/tee_popen.py +++ b/tests/ci/tee_popen.py @@ -3,6 +3,7 @@ from subprocess import Popen, PIPE, STDOUT from threading import Thread from time import sleep +from typing import Optional import logging import os import sys @@ -18,7 +19,7 @@ class TeePopen: self.command = command self.log_file = log_file self.env = env - self.process = None + self._process = None # type: Optional[Popen] self.timeout = timeout def _check_timeout(self): @@ -51,7 +52,7 @@ class TeePopen: return self def __exit__(self, t, value, traceback): - for line in self.process.stdout: + for line in self.process.stdout: # type: ignore sys.stdout.write(line) self.log_file.write(line) @@ -59,8 +60,18 @@ class TeePopen: self.log_file.close() def wait(self): - for line in self.process.stdout: + for line in self.process.stdout: # type: ignore sys.stdout.write(line) self.log_file.write(line) return self.process.wait() + + @property + def process(self) -> Popen: + if self._process is not None: + return self._process + raise AttributeError("process is not created yet") + + @process.setter + def process(self, process: Popen) -> None: + self._process = process From 6e00bb2e744f6d38f930442f08b3809b69d0ef4f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 20:02:33 +0100 Subject: [PATCH 22/62] Add and fix typing for docker_pull_helper, github_helper and style_check --- tests/ci/docker_pull_helper.py | 18 ++++++++++++------ tests/ci/github_helper.py | 11 ++++++----- tests/ci/style_check.py | 16 ++++++++++------ 3 files changed, 28 insertions(+), 17 deletions(-) diff --git a/tests/ci/docker_pull_helper.py b/tests/ci/docker_pull_helper.py index 04817ed7de3..5336966b3eb 100644 --- a/tests/ci/docker_pull_helper.py +++ b/tests/ci/docker_pull_helper.py @@ -6,11 +6,11 @@ import time import subprocess import logging -from typing import Optional +from typing import List, Optional class DockerImage: - def __init__(self, name, version: Optional[str] = None): + def __init__(self, name: str, version: Optional[str] = None): self.name = name if version is None: self.version = "latest" @@ -22,8 +22,11 @@ class DockerImage: def get_images_with_versions( - reports_path, required_image, pull=True, version: Optional[str] = None -): + reports_path: str, + required_images: List[str], + pull: bool = True, + version: Optional[str] = None, +) -> List[DockerImage]: images_path = None for root, _, files in os.walk(reports_path): for f in files: @@ -45,12 +48,13 @@ def get_images_with_versions( images = {} docker_images = [] - for image_name in required_image: + for image_name in required_images: docker_image = DockerImage(image_name, version) if image_name in images: docker_image.version = images[image_name] docker_images.append(docker_image) + latest_error = Exception("predefined to avoid access before created") if pull: for docker_image in docker_images: for i in range(10): @@ -75,6 +79,8 @@ def get_images_with_versions( return docker_images -def get_image_with_version(reports_path, image, pull=True, version=None): +def get_image_with_version( + reports_path: str, image: str, pull: bool = True, version: Optional[str] = None +) -> DockerImage: logging.info("Looking for images file in %s", reports_path) return get_images_with_versions(reports_path, [image], pull, version=version)[0] diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 685d9f2c841..5a862a9fdca 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -90,7 +90,7 @@ class GitHub(github.Github): raise exception # pylint: enable=signature-differs - def get_pulls_from_search(self, *args, **kwargs) -> PullRequests: + def get_pulls_from_search(self, *args, **kwargs) -> PullRequests: # type: ignore """The search api returns actually issues, so we need to fetch PullRequests""" issues = self.search_issues(*args, **kwargs) repos = {} @@ -168,7 +168,7 @@ class GitHub(github.Github): self.dump(user, prfd) # type: ignore return user - def _get_cached(self, path: Path): + def _get_cached(self, path: Path): # type: ignore with open(path, "rb") as ob_fd: return self.load(ob_fd) # type: ignore @@ -190,11 +190,11 @@ class GitHub(github.Github): return False, cached_obj @property - def cache_path(self): + def cache_path(self) -> Path: return self._cache_path @cache_path.setter - def cache_path(self, value: str): + def cache_path(self, value: str) -> None: self._cache_path = Path(value) if self._cache_path.exists(): assert self._cache_path.is_dir() @@ -208,5 +208,6 @@ class GitHub(github.Github): return self._retries @retries.setter - def retries(self, value: int): + def retries(self, value: int) -> None: + assert isinstance(value, int) self._retries = value diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 23a1dd467d7..58d06f890af 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -1,11 +1,13 @@ #!/usr/bin/env python3 import argparse +import atexit import csv import logging import os import subprocess import sys -import atexit + +from typing import List, Tuple from clickhouse_helper import ( @@ -29,8 +31,10 @@ from upload_result_helper import upload_results NAME = "Style Check" -def process_result(result_folder): - test_results = [] +def process_result( + result_folder: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible @@ -57,7 +61,7 @@ def process_result(result_folder): try: results_path = os.path.join(result_folder, "test_results.tsv") with open(results_path, "r", encoding="utf-8") as fd: - test_results = list(csv.reader(fd, delimiter="\t")) + test_results = list(csv.reader(fd, delimiter="\t")) # type: ignore if len(test_results) == 0: raise Exception("Empty results") @@ -81,7 +85,7 @@ def parse_args(): return parser.parse_args() -def checkout_head(pr_info: PRInfo): +def checkout_head(pr_info: PRInfo) -> None: # It works ONLY for PRs, and only over ssh, so either # ROBOT_CLICKHOUSE_SSH_KEY should be set or ssh-agent should work assert pr_info.number @@ -107,7 +111,7 @@ def checkout_head(pr_info: PRInfo): git_runner(f"git checkout -f head-{pr_info.head_ref}") -def commit_push_staged(pr_info: PRInfo): +def commit_push_staged(pr_info: PRInfo) -> None: # It works ONLY for PRs, and only over ssh, so either # ROBOT_CLICKHOUSE_SSH_KEY should be set or ssh-agent should work assert pr_info.number From da71dbe040f90fa6e67c4101f24d1cb5be5001b5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 13:01:27 +0100 Subject: [PATCH 23/62] Fix issues in build_report_check.py --- tests/ci/build_report_check.py | 26 +++++++++++++------------- tests/ci/pr_info.py | 3 ++- 2 files changed, 15 insertions(+), 14 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 673b0204864..03e18d7766e 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -19,7 +19,7 @@ from env_helper import ( from report import create_build_html_report from s3_helper import S3Helper from get_robot_token import get_best_robot_token -from pr_info import PRInfo +from pr_info import NeedsDataType, PRInfo from commit_status_helper import ( get_commit, update_mergeable_check, @@ -28,7 +28,7 @@ from ci_config import CI_CONFIG from rerun_helper import RerunHelper -NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH") +NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "") class BuildResult: @@ -98,7 +98,7 @@ def get_failed_report( def process_report( - build_report, + build_report: dict, ) -> Tuple[List[BuildResult], List[List[str]], List[str]]: build_config = build_report["build_config"] build_result = BuildResult( @@ -144,16 +144,14 @@ def main(): os.makedirs(temp_path) build_check_name = sys.argv[1] - needs_data = None + needs_data = {} # type: NeedsDataType required_builds = 0 if os.path.exists(NEEDS_DATA_PATH): with open(NEEDS_DATA_PATH, "rb") as file_handler: needs_data = json.load(file_handler) required_builds = len(needs_data) - if needs_data is not None and all( - i["result"] == "skipped" for i in needs_data.values() - ): + if needs_data and all(i["result"] == "skipped" for i in needs_data.values()): logging.info("All builds are skipped, exiting") sys.exit(0) @@ -218,19 +216,21 @@ def main(): build_logs = [] for build_report in build_reports: - build_result, build_artifacts_url, build_logs_url = process_report(build_report) - logging.info( - "Got %s artifact groups for build report report", len(build_result) + _build_results, build_artifacts_url, build_logs_url = process_report( + build_report ) - build_results.extend(build_result) + logging.info( + "Got %s artifact groups for build report report", len(_build_results) + ) + build_results.extend(_build_results) build_artifacts.extend(build_artifacts_url) build_logs.extend(build_logs_url) for failed_job in missing_build_names: - build_result, build_artifacts_url, build_logs_url = get_failed_report( + _build_results, build_artifacts_url, build_logs_url = get_failed_report( failed_job ) - build_results.extend(build_result) + build_results.extend(_build_results) build_artifacts.extend(build_artifacts_url) build_logs.extend(build_logs_url) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index f0a371c488f..942edeaa81c 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -2,7 +2,7 @@ import json import logging import os -from typing import Set +from typing import Dict, Set, Union from unidiff import PatchSet # type: ignore @@ -16,6 +16,7 @@ from env_helper import ( FORCE_TESTS_LABEL = "force tests" SKIP_MERGEABLE_CHECK_LABEL = "skip mergeable check" +NeedsDataType = Dict[str, Dict[str, Union[str, Dict[str, str]]]] DIFF_IN_DOCUMENTATION_EXT = [ ".html", From 4fc5bb7a639eeefc99fb23a94d3b0407288caf09 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 13:10:13 +0100 Subject: [PATCH 24/62] Fix release.py typing issue --- tests/ci/release.py | 38 ++++++++++++++++++++++++-------------- 1 file changed, 24 insertions(+), 14 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 8024091e300..dec97f2a54f 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -11,7 +11,7 @@ On another hand, PyGithub is used for convenient getting commit's status from AP from contextlib import contextmanager -from typing import List, Optional +from typing import Any, Iterator, List, Literal, Optional import argparse import logging import subprocess @@ -48,7 +48,7 @@ class Repo: return self._url @url.setter - def url(self, protocol: str): + def url(self, protocol: str) -> None: if protocol == "ssh": self._url = f"git@github.com:{self}.git" elif protocol == "https": @@ -68,17 +68,23 @@ class Release: CMAKE_PATH = get_abs_path(FILE_WITH_VERSION_PATH) CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS) - def __init__(self, repo: Repo, release_commit: str, release_type: str): + def __init__( + self, + repo: Repo, + release_commit: str, + release_type: Literal["major", "minor", "patch"], + ): self.repo = repo self._release_commit = "" self.release_commit = release_commit + assert release_type in self.BIG + self.SMALL self.release_type = release_type self._git = git self._version = get_version_from_repo(git=self._git) self._release_branch = "" self._rollback_stack = [] # type: List[str] - def run(self, cmd: str, cwd: Optional[str] = None, **kwargs) -> str: + def run(self, cmd: str, cwd: Optional[str] = None, **kwargs: Any) -> str: cwd_text = "" if cwd: cwd_text = f" (CWD='{cwd}')" @@ -153,7 +159,9 @@ class Release: self.check_commit_release_ready() - def do(self, check_dirty: bool, check_branch: bool, with_release_branch: bool): + def do( + self, check_dirty: bool, check_branch: bool, with_release_branch: bool + ) -> None: self.check_prerequisites() if check_dirty: @@ -310,7 +318,7 @@ class Release: return self._version @version.setter - def version(self, version: ClickHouseVersion): + def version(self, version: ClickHouseVersion) -> None: if not isinstance(version, ClickHouseVersion): raise ValueError(f"version must be ClickHouseVersion, not {type(version)}") self._version = version @@ -320,7 +328,7 @@ class Release: return self._release_branch @release_branch.setter - def release_branch(self, branch: str): + def release_branch(self, branch: str) -> None: self._release_branch = release_branch(branch) @property @@ -328,7 +336,7 @@ class Release: return self._release_commit @release_commit.setter - def release_commit(self, release_commit: str): + def release_commit(self, release_commit: str) -> None: self._release_commit = commit(release_commit) @contextmanager @@ -367,7 +375,7 @@ class Release: yield @contextmanager - def _bump_testing_version(self, helper_branch: str): + def _bump_testing_version(self, helper_branch: str) -> Iterator[None]: self.read_version() self.version = self.version.update(self.release_type) self.version.with_description(VersionType.TESTING) @@ -387,7 +395,7 @@ class Release: yield @contextmanager - def _checkout(self, ref: str, with_checkout_back: bool = False): + def _checkout(self, ref: str, with_checkout_back: bool = False) -> Iterator[None]: orig_ref = self._git.branch or self._git.sha need_rollback = False if ref not in (self._git.branch, self._git.sha): @@ -406,7 +414,7 @@ class Release: self.run(rollback_cmd) @contextmanager - def _create_branch(self, name: str, start_point: str = ""): + def _create_branch(self, name: str, start_point: str = "") -> Iterator[None]: self.run(f"git branch {name} {start_point}") rollback_cmd = f"git branch -D {name}" self._rollback_stack.append(rollback_cmd) @@ -418,7 +426,7 @@ class Release: raise @contextmanager - def _create_gh_label(self, label: str, color_hex: str): + def _create_gh_label(self, label: str, color_hex: str) -> Iterator[None]: # API call, https://docs.github.com/en/rest/reference/issues#create-a-label self.run( f"gh api repos/{self.repo}/labels -f name={label} -f color={color_hex}" @@ -433,7 +441,7 @@ class Release: raise @contextmanager - def _create_gh_release(self, as_prerelease: bool): + def _create_gh_release(self, as_prerelease: bool) -> Iterator[None]: with self._create_tag(): # Preserve tag if version is changed tag = self.version.describe @@ -468,7 +476,9 @@ class Release: raise @contextmanager - def _push(self, ref: str, with_rollback_on_fail: bool = True, remote_ref: str = ""): + def _push( + self, ref: str, with_rollback_on_fail: bool = True, remote_ref: str = "" + ) -> Iterator[None]: if remote_ref == "": remote_ref = ref From ec0684dfce09ed94b87f27c1fabb0f7e5fa80a4a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 14:03:16 +0100 Subject: [PATCH 25/62] Fix typing issues in build_check.py --- tests/ci/build_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index d668dbe0498..c9e8dac2c00 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -121,7 +121,7 @@ def check_for_success_run( s3_prefix: str, build_name: str, build_config: BuildConfig, -): +) -> None: logged_prefix = os.path.join(S3_BUILDS_BUCKET, s3_prefix) logging.info("Checking for artifacts in %s", logged_prefix) try: @@ -174,7 +174,7 @@ def create_json_artifact( build_config: BuildConfig, elapsed: int, success: bool, -): +) -> None: subprocess.check_call( f"echo 'BUILD_URLS=build_urls_{build_name}' >> $GITHUB_ENV", shell=True ) @@ -218,7 +218,7 @@ def upload_master_static_binaries( build_config: BuildConfig, s3_helper: S3Helper, build_output_path: str, -): +) -> None: """Upload binary artifacts to a static S3 links""" static_binary_name = build_config.get("static_binary_name", False) if pr_info.number != 0: From ba9e51d6865844f79fc380e083e06078535d5821 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 14:46:48 +0100 Subject: [PATCH 26/62] Fix cherry-pick.py typing issues --- tests/ci/cherry_pick.py | 17 ++++++++++------- tests/ci/github_helper.py | 17 ++++++++++++----- 2 files changed, 22 insertions(+), 12 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index d1c9d3d394c..20d2d780484 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -92,7 +92,8 @@ Merge it only if you intend to backport changes to the target branch, otherwise if branch_updated: self._backported = True - def pop_prs(self, prs: PullRequests): + def pop_prs(self, prs: PullRequests) -> None: + """the method processes all prs and pops the ReleaseBranch related prs""" to_pop = [] # type: List[int] for i, pr in enumerate(prs): if self.name not in pr.head.ref: @@ -105,14 +106,14 @@ Merge it only if you intend to backport changes to the target branch, otherwise to_pop.append(i) else: logging.error( - "PR #%s doesn't head ref starting with known suffix", + "head ref of PR #%s isn't starting with known suffix", pr.number, ) for i in reversed(to_pop): # Going from the tail to keep the order and pop greater index first prs.pop(i) - def process(self, dry_run: bool): + def process(self, dry_run: bool) -> None: if self.backported: return if not self.cherrypick_pr: @@ -209,6 +210,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise self._assign_new_pr(self.cherrypick_pr) def create_backport(self): + assert self.cherrypick_pr is not None # Checkout the backport branch from the remote and make all changes to # apply like they are only one cherry-pick commit on top of release git_runner(f"{self.git_prefix} checkout -f {self.backport_branch}") @@ -239,7 +241,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise self.backport_pr.add_to_labels(Labels.BACKPORT) self._assign_new_pr(self.backport_pr) - def _assign_new_pr(self, new_pr: PullRequest): + def _assign_new_pr(self, new_pr: PullRequest) -> None: """Assign `new_pr` to author, merger and assignees of an original PR""" # It looks there some race when multiple .add_to_assignees are executed, # so we'll add all at once @@ -340,7 +342,7 @@ class Backport: ) self.error = e - def process_pr(self, pr: PullRequest): + def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] if Labels.MUST_BACKPORT in pr_labels: branches = [ @@ -403,7 +405,7 @@ class Backport: # And check it after the running self.mark_pr_backported(pr) - def mark_pr_backported(self, pr: PullRequest): + def mark_pr_backported(self, pr: PullRequest) -> None: if self.dry_run: logging.info("DRY RUN: would mark PR #%s as done", pr.number) return @@ -488,7 +490,8 @@ def main(): gh = GitHub(token, per_page=100) bp = Backport(gh, args.repo, args.dry_run) - bp.gh.cache_path = str(f"{TEMP_PATH}/gh_cache") + # https://github.com/python/mypy/issues/3004 + bp.gh.cache_path = f"{TEMP_PATH}/gh_cache" # type: ignore bp.receive_release_prs() bp.receive_prs_for_backport() bp.process_backports() diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 5a862a9fdca..bd740827b34 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -8,11 +8,18 @@ from time import sleep from typing import List, Optional, Tuple import github -from github.GithubException import RateLimitExceededException -from github.Issue import Issue -from github.NamedUser import NamedUser -from github.PullRequest import PullRequest -from github.Repository import Repository + +# explicit reimport +# pylint: disable=useless-import-alias +from github.GithubException import ( + RateLimitExceededException as RateLimitExceededException, +) +from github.Issue import Issue as Issue +from github.NamedUser import NamedUser as NamedUser +from github.PullRequest import PullRequest as PullRequest +from github.Repository import Repository as Repository + +# pylint: enable=useless-import-alias CACHE_PATH = p.join(p.dirname(p.realpath(__file__)), "gh_cache") From 459dde989bb6166f6b78532872ea73b8d9b5a344 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 14:52:40 +0100 Subject: [PATCH 27/62] A tiny fix for missing types --- tests/ci/get_previous_release_tag.py | 2 +- tests/ci/jepsen_check.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index bfce69a17d9..b9ad51379d2 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -3,7 +3,7 @@ import re import logging -import requests +import requests # type: ignore CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags" VERSION_PATTERN = r"(v(?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)" diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 4116d15bba6..69964c0a0bc 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -7,9 +7,9 @@ import sys import argparse -import boto3 +import boto3 # type: ignore +import requests # type: ignore from github import Github -import requests from env_helper import REPO_COPY, TEMP_PATH, S3_BUILDS_BUCKET, S3_DOWNLOAD from stopwatch import Stopwatch From d4859275f4156b9881944ca7ad25c9041034ab5e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 17:35:15 +0100 Subject: [PATCH 28/62] Fix typing in version_test.py --- tests/ci/version_test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/ci/version_test.py b/tests/ci/version_test.py index 86a2d58c3c8..abd0f9349f4 100644 --- a/tests/ci/version_test.py +++ b/tests/ci/version_test.py @@ -17,9 +17,9 @@ class TestFunctions(unittest.TestCase): ("v1.1.1.2-testing", vh.get_version_from_string("1.1.1.2")), ("refs/tags/v1.1.1.2-testing", vh.get_version_from_string("1.1.1.2")), ) - for case in cases: - version = vh.version_arg(case[0]) - self.assertEqual(case[1], version) + for test_case in cases: + version = vh.version_arg(test_case[0]) + self.assertEqual(test_case[1], version) error_cases = ( "0.0.0", "1.1.1.a", @@ -28,6 +28,6 @@ class TestFunctions(unittest.TestCase): "v1.1.1.2-testin", "refs/tags/v1.1.1.2-testin", ) - for case in error_cases: + for error_case in error_cases: with self.assertRaises(ArgumentTypeError): - version = vh.version_arg(case[0]) + version = vh.version_arg(error_case[0]) From 40b8c8559f44f5c63da8e7cc260cf2850ab219b5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 17:49:35 +0100 Subject: [PATCH 29/62] Simplify finish_check.py --- tests/ci/finish_check.py | 27 ++++++++------------------- 1 file changed, 8 insertions(+), 19 deletions(-) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index a0b7f14ecfb..ea2f5eb3136 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -5,27 +5,11 @@ from github import Github from env_helper import GITHUB_RUN_URL from pr_info import PRInfo from get_robot_token import get_best_robot_token -from commit_status_helper import get_commit +from commit_status_helper import get_commit, get_commit_filtered_statuses NAME = "Run Check" -def filter_statuses(statuses): - """ - Squash statuses to latest state - 1. context="first", state="success", update_time=1 - 2. context="second", state="success", update_time=2 - 3. context="first", stat="failure", update_time=3 - =========> - 1. context="second", state="success" - 2. context="first", stat="failure" - """ - filt = {} - for status in sorted(statuses, key=lambda x: x.updated_at): - filt[status.context] = status - return filt - - if __name__ == "__main__": logging.basicConfig(level=logging.INFO) @@ -34,8 +18,13 @@ if __name__ == "__main__": commit = get_commit(gh, pr_info.sha) url = GITHUB_RUN_URL - statuses = filter_statuses(list(commit.get_statuses())) - if NAME in statuses and statuses[NAME].state == "pending": + statuses = get_commit_filtered_statuses(commit) + pending_status = any( # find NAME status in pending state + True + for status in statuses + if status.context == NAME and status.state == "pending" + ) + if pending_status: commit.create_status( context=NAME, description="All checks finished", From eb738fa13e68a38369c02fa782aacb11cc899f1d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 17:57:56 +0100 Subject: [PATCH 30/62] Use PRInfo and TEMP_PATH in codebrowser --- tests/ci/codebrowser_check.py | 38 ++++++++++++++++++----------------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/tests/ci/codebrowser_check.py b/tests/ci/codebrowser_check.py index 97036c6fc7b..412bcdf8818 100644 --- a/tests/ci/codebrowser_check.py +++ b/tests/ci/codebrowser_check.py @@ -7,14 +7,21 @@ import logging from github import Github -from env_helper import IMAGES_PATH, REPO_COPY, S3_TEST_REPORTS_BUCKET, S3_DOWNLOAD -from stopwatch import Stopwatch -from upload_result_helper import upload_results -from s3_helper import S3Helper -from get_robot_token import get_best_robot_token +from env_helper import ( + IMAGES_PATH, + REPO_COPY, + S3_DOWNLOAD, + S3_TEST_REPORTS_BUCKET, + TEMP_PATH, +) from commit_status_helper import post_commit_status from docker_pull_helper import get_image_with_version +from get_robot_token import get_best_robot_token +from pr_info import PRInfo +from s3_helper import S3Helper +from stopwatch import Stopwatch from tee_popen import TeePopen +from upload_result_helper import upload_results NAME = "Woboq Build" @@ -33,17 +40,16 @@ if __name__ == "__main__": stopwatch = Stopwatch() - temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - gh = Github(get_best_robot_token(), per_page=100) + pr_info = PRInfo() - if not os.path.exists(temp_path): - os.makedirs(temp_path) + if not os.path.exists(TEMP_PATH): + os.makedirs(TEMP_PATH) docker_image = get_image_with_version(IMAGES_PATH, "clickhouse/codebrowser") s3_helper = S3Helper() - result_path = os.path.join(temp_path, "result_path") + result_path = os.path.join(TEMP_PATH, "result_path") if not os.path.exists(result_path): os.makedirs(result_path) @@ -51,7 +57,7 @@ if __name__ == "__main__": logging.info("Going to run codebrowser: %s", run_command) - run_log_path = os.path.join(temp_path, "runlog.log") + run_log_path = os.path.join(TEMP_PATH, "runlog.log") with TeePopen(run_command, run_log_path) as process: retcode = process.wait() @@ -60,7 +66,7 @@ if __name__ == "__main__": else: logging.info("Run failed") - subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {TEMP_PATH}", shell=True) report_path = os.path.join(result_path, "html_report") logging.info("Report path %s", report_path) @@ -76,12 +82,8 @@ if __name__ == "__main__": test_results = [(index_html, "Look at the report")] - report_url = upload_results( - s3_helper, 0, os.getenv("GITHUB_SHA"), test_results, [], NAME - ) + report_url = upload_results(s3_helper, 0, pr_info.sha, test_results, [], NAME) print(f"::notice ::Report url: {report_url}") - post_commit_status( - gh, os.getenv("GITHUB_SHA"), NAME, "Report built", "success", report_url - ) + post_commit_status(gh, pr_info.sha, NAME, "Report built", "success", report_url) From 33424bc411c593bb6caf9ebb667e9aee762a6fed Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 18:02:42 +0100 Subject: [PATCH 31/62] Add typing to fast tests --- tests/ci/fast_test_check.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 03e42726808..2a6a0d5fa57 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -6,6 +6,7 @@ import os import csv import sys import atexit +from typing import List, Tuple from github import Github @@ -50,8 +51,10 @@ def get_fasttest_cmd( ) -def process_results(result_folder): - test_results = [] +def process_results( + result_folder: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible for content of @@ -78,7 +81,7 @@ def process_results(result_folder): results_path = os.path.join(result_folder, "test_results.tsv") if os.path.exists(results_path): with open(results_path, "r", encoding="utf-8") as results_file: - test_results = list(csv.reader(results_file, delimiter="\t")) + test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore if len(test_results) == 0: return "error", "Empty test_results.tsv", test_results, additional_files @@ -172,7 +175,7 @@ if __name__ == "__main__": "test_log.txt" in test_output_files or "test_result.txt" in test_output_files ) test_result_exists = "test_results.tsv" in test_output_files - test_results = [] + test_results = [] # type: List[Tuple[str, str]] if "submodule_log.txt" not in test_output_files: description = "Cannot clone repository" state = "failure" From 1e5fec1157ee1400749e731086454d9f591a24c7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:01:17 +0100 Subject: [PATCH 32/62] Fix functional_test_check.py typing --- tests/ci/functional_test_check.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index f7d3288c316..87833d688af 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -7,6 +7,7 @@ import os import subprocess import sys import atexit +from typing import List, Tuple from github import Github @@ -122,8 +123,11 @@ def get_tests_to_run(pr_info): return list(result) -def process_results(result_folder, server_log_path): - test_results = [] +def process_results( + result_folder: str, + server_log_path: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible for content of result_folder. @@ -166,7 +170,7 @@ def process_results(result_folder, server_log_path): return "error", "Not found test_results.tsv", test_results, additional_files with open(results_path, "r", encoding="utf-8") as results_file: - test_results = list(csv.reader(results_file, delimiter="\t")) + test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore if len(test_results) == 0: return "error", "Empty test_results.tsv", test_results, additional_files @@ -232,8 +236,8 @@ if __name__ == "__main__": sys.exit(0) if "RUN_BY_HASH_NUM" in os.environ: - run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM")) - run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL")) + run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0")) + run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "0")) check_name_with_group = ( check_name + f" [{run_by_hash_num + 1}/{run_by_hash_total}]" ) From d0d23af63e3d2b3de7d0a84fad56b6e9950229f2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:08:53 +0100 Subject: [PATCH 33/62] Fix integration_test_check.py typing --- tests/ci/integration_test_check.py | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index cba428cbcf5..e61117a4b45 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -7,6 +7,7 @@ import logging import os import subprocess import sys +from typing import List, Tuple from github import Github @@ -87,8 +88,10 @@ def get_env_for_runner(build_path, repo_path, result_path, work_path): return my_env -def process_results(result_folder): - test_results = [] +def process_results( + result_folder: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible for content of result_folder. @@ -115,7 +118,7 @@ def process_results(result_folder): results_path = os.path.join(result_folder, "test_results.tsv") if os.path.exists(results_path): with open(results_path, "r", encoding="utf-8") as results_file: - test_results = list(csv.reader(results_file, delimiter="\t")) + test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore if len(test_results) == 0: return "error", "Empty test_results.tsv", test_results, additional_files @@ -153,8 +156,8 @@ if __name__ == "__main__": validate_bugix_check = args.validate_bugfix if "RUN_BY_HASH_NUM" in os.environ: - run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM")) - run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL")) + run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0")) + run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "0")) check_name_with_group = ( check_name + f" [{run_by_hash_num + 1}/{run_by_hash_total}]" ) From 5af2bc6660fc876d0a098fbbf3b9f03fd207b24d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:12:07 +0100 Subject: [PATCH 34/62] Fix unit_tests_check.py typing --- tests/ci/unit_tests_check.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index c2dfab9dddc..4777296da18 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -5,6 +5,7 @@ import os import sys import subprocess import atexit +from typing import List, Tuple from github import Github @@ -37,14 +38,16 @@ def get_test_name(line): raise Exception(f"No test name in line '{line}'") -def process_result(result_folder): +def process_results( + result_folder: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: OK_SIGN = "OK ]" FAILED_SIGN = "FAILED ]" SEGFAULT = "Segmentation fault" SIGNAL = "received signal SIG" PASSED = "PASSED" - summary = [] + summary = [] # type: List[Tuple[str, str]] total_counter = 0 failed_counter = 0 result_log_path = f"{result_folder}/test_result.txt" @@ -151,7 +154,7 @@ if __name__ == "__main__": subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) s3_helper = S3Helper() - state, description, test_results, additional_logs = process_result(test_output) + state, description, test_results, additional_logs = process_results(test_output) ch_helper = ClickHouseHelper() mark_flaky_tests(ch_helper, check_name, test_results) From e1a9f2976a7d76d7012d34174dae102877acf1d0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:24:10 +0100 Subject: [PATCH 35/62] Fix stress_check.py typing --- tests/ci/stress_check.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 8f310eaa99d..c02128d114f 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -5,6 +5,7 @@ import logging import subprocess import os import sys +from typing import List, Tuple from github import Github @@ -44,8 +45,10 @@ def get_run_command( return cmd -def process_results(result_folder, server_log_path, run_log_path): - test_results = [] +def process_results( + result_folder: str, server_log_path: str, run_log_path: str +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible for content @@ -89,7 +92,7 @@ def process_results(result_folder, server_log_path, run_log_path): results_path = os.path.join(result_folder, "test_results.tsv") with open(results_path, "r", encoding="utf-8") as results_file: - test_results = list(csv.reader(results_file, delimiter="\t")) + test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore if len(test_results) == 0: raise Exception("Empty results") From db44e40989469f23ba472acabc84121466bf63c8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:24:32 +0100 Subject: [PATCH 36/62] Fix sqlancer_check.py typing --- tests/ci/sqlancer_check.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index 63c7d18fe46..5e94969d4b1 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -4,6 +4,7 @@ import logging import subprocess import os import sys +from typing import List, Tuple from github import Github @@ -137,7 +138,7 @@ if __name__ == "__main__": report_url = GITHUB_RUN_URL status = "success" - test_results = [] + test_results = [] # type: List[Tuple[str, str]] # Try to get status message saved by the SQLancer try: # with open( @@ -145,7 +146,7 @@ if __name__ == "__main__": # ) as status_f: # status = status_f.readline().rstrip("\n") if os.path.exists(os.path.join(workspace_path, "server_crashed.log")): - test_results.append("Server crashed", "FAIL") + test_results.append(("Server crashed", "FAIL")) with open( os.path.join(workspace_path, "summary.tsv"), "r", encoding="utf-8" ) as summary_f: From f4a7ff409e7057f6d980b56d377723f2a8fc9d01 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 13:52:31 +0100 Subject: [PATCH 37/62] Fix push_to_artifactory.py --- tests/ci/push_to_artifactory.py | 29 +++++++++++++++++------------ 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/tests/ci/push_to_artifactory.py b/tests/ci/push_to_artifactory.py index dd8081227bf..97971f207ce 100755 --- a/tests/ci/push_to_artifactory.py +++ b/tests/ci/push_to_artifactory.py @@ -5,7 +5,7 @@ import logging import os import re from collections import namedtuple -from typing import Dict, List, Tuple +from typing import Dict, List, Optional, Tuple from artifactory import ArtifactorySaaSPath # type: ignore from build_download_helper import download_build_with_progress @@ -14,7 +14,7 @@ from git_helper import TAG_REGEXP, commit, removeprefix, removesuffix # Necessary ENV variables -def getenv(name: str, default: str = None): +def getenv(name: str, default: Optional[str] = None) -> str: env = os.getenv(name, default) if env is not None: return env @@ -62,7 +62,7 @@ class Packages: raise ValueError(f"{deb_pkg} not in {self.deb}") return removesuffix(deb_pkg, ".deb").split("_")[-1] - def replace_with_fallback(self, name: str): + def replace_with_fallback(self, name: str) -> None: if name.endswith(".deb"): suffix = self.deb.pop(name) self.deb[self.fallback_to_all(name)] = self.fallback_to_all(suffix) @@ -80,7 +80,7 @@ class Packages: return os.path.join(TEMP_PATH, package_file) @staticmethod - def fallback_to_all(url_or_name: str): + def fallback_to_all(url_or_name: str) -> str: """Until July 2022 we had clickhouse-server and clickhouse-client with arch 'all'""" # deb @@ -111,7 +111,7 @@ class S3: self.force_download = force_download self.packages = Packages(version) - def download_package(self, package_file: str, s3_path_suffix: str): + def download_package(self, package_file: str, s3_path_suffix: str) -> None: path = Packages.path(package_file) fallback_path = Packages.fallback_to_all(path) if not self.force_download and ( @@ -186,7 +186,12 @@ class Release: class Artifactory: def __init__( - self, url: str, release: str, deb_repo="deb", rpm_repo="rpm", tgz_repo="tgz" + self, + url: str, + release: str, + deb_repo: str = "deb", + rpm_repo: str = "rpm", + tgz_repo: str = "tgz", ): self._url = url self._release = release @@ -196,7 +201,7 @@ class Artifactory: # check the credentials ENVs for early exit self.__path_helper("_deb", "") - def deploy_deb(self, packages: Packages): + def deploy_deb(self, packages: Packages) -> None: for package_file in packages.deb: path = packages.path(package_file) dist = self._release @@ -212,13 +217,13 @@ class Artifactory: ) self.deb_path(package_file).deploy_deb(path, dist, comp, arch) - def deploy_rpm(self, packages: Packages): + def deploy_rpm(self, packages: Packages) -> None: for package_file in packages.rpm: path = packages.path(package_file) logging.info("Deploy %s to artifactory", path) self.rpm_path(package_file).deploy_file(path) - def deploy_tgz(self, packages: Packages): + def deploy_tgz(self, packages: Packages) -> None: for package_file in packages.tgz: path = packages.path(package_file) logging.info("Deploy %s to artifactory", path) @@ -316,19 +321,19 @@ def parse_args() -> argparse.Namespace: return args -def process_deb(s3: S3, art_clients: List[Artifactory]): +def process_deb(s3: S3, art_clients: List[Artifactory]) -> None: s3.download_deb() for art_client in art_clients: art_client.deploy_deb(s3.packages) -def process_rpm(s3: S3, art_clients: List[Artifactory]): +def process_rpm(s3: S3, art_clients: List[Artifactory]) -> None: s3.download_rpm() for art_client in art_clients: art_client.deploy_rpm(s3.packages) -def process_tgz(s3: S3, art_clients: List[Artifactory]): +def process_tgz(s3: S3, art_clients: List[Artifactory]) -> None: s3.download_tgz() for art_client in art_clients: art_client.deploy_tgz(s3.packages) From 3bf3dacb4d6f2493824931641beba56c2d50f26f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 14:11:32 +0100 Subject: [PATCH 38/62] Add __init__.py to lambdas for mypy --- tests/ci/cancel_and_rerun_workflow_lambda/__init__.py | 0 tests/ci/runner_token_rotation_lambda/__init__.py | 0 tests/ci/team_keys_lambda/__init__.py | 0 tests/ci/terminate_runner_lambda/__init__.py | 0 4 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/ci/cancel_and_rerun_workflow_lambda/__init__.py create mode 100644 tests/ci/runner_token_rotation_lambda/__init__.py create mode 100644 tests/ci/team_keys_lambda/__init__.py create mode 100644 tests/ci/terminate_runner_lambda/__init__.py diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/__init__.py b/tests/ci/cancel_and_rerun_workflow_lambda/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/ci/runner_token_rotation_lambda/__init__.py b/tests/ci/runner_token_rotation_lambda/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/ci/team_keys_lambda/__init__.py b/tests/ci/team_keys_lambda/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/ci/terminate_runner_lambda/__init__.py b/tests/ci/terminate_runner_lambda/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From eacf7e01a1c79a16ff2bc57fca8047aa27a08ab6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 28 Nov 2022 17:17:45 +0100 Subject: [PATCH 39/62] Fix typing in team_keys_lambda --- tests/ci/team_keys_lambda/app.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/team_keys_lambda/app.py b/tests/ci/team_keys_lambda/app.py index 9e73a3f0993..870d41c441e 100644 --- a/tests/ci/team_keys_lambda/app.py +++ b/tests/ci/team_keys_lambda/app.py @@ -14,7 +14,7 @@ import boto3 # type: ignore class Keys(set): def __init__(self, *args, **kwargs): super().__init__(*args, **kwargs) - self.updated_at = 0 + self.updated_at = 0.0 def update_now(self): self.updated_at = datetime.now().timestamp() @@ -88,7 +88,7 @@ def get_token_from_aws() -> str: ) get_secret_value_response = client.get_secret_value(SecretId=secret_name) data = json.loads(get_secret_value_response["SecretString"]) - return data["clickhouse_robot_token"] + return data["clickhouse_robot_token"] # type: ignore def main(token: str, org: str, team_slug: str) -> str: From 3feb8e3657e11a1131b7e9a638c1d0a9c1f87bd9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 28 Nov 2022 17:58:50 +0100 Subject: [PATCH 40/62] Add type hinting to ci_runners_metrics_lambda --- tests/ci/ci_runners_metrics_lambda/app.py | 58 ++++++++++++++--------- 1 file changed, 35 insertions(+), 23 deletions(-) diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index c1b20beb599..2bc568bb462 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -12,11 +12,12 @@ import json import time from collections import namedtuple from datetime import datetime +from typing import Dict, List, Tuple import jwt -import requests -import boto3 -from botocore.exceptions import ClientError +import requests # type: ignore +import boto3 # type: ignore +from botocore.exceptions import ClientError # type: ignore UNIVERSAL_LABEL = "universal" RUNNER_TYPE_LABELS = [ @@ -29,8 +30,13 @@ RUNNER_TYPE_LABELS = [ "style-checker-aarch64", ] +RunnerDescription = namedtuple( + "RunnerDescription", ["id", "name", "tags", "offline", "busy"] +) +RunnerDescriptions = List[RunnerDescription] -def get_dead_runners_in_ec2(runners): + +def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions: ids = { runner.name: runner for runner in runners @@ -92,7 +98,7 @@ def get_dead_runners_in_ec2(runners): return result_to_delete -def get_lost_ec2_instances(runners): +def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]: client = boto3.client("ec2") reservations = client.describe_instances( Filters=[{"Name": "tag-key", "Values": ["github:runner-type"]}] @@ -130,7 +136,7 @@ def get_lost_ec2_instances(runners): return lost_instances -def get_key_and_app_from_aws(): +def get_key_and_app_from_aws() -> Tuple[str, int]: secret_name = "clickhouse_github_secret_key" session = boto3.session.Session() client = session.client( @@ -146,7 +152,7 @@ def handler(event, context): main(private_key, app_id, True, True) -def get_installation_id(jwt_token): +def get_installation_id(jwt_token: str) -> int: headers = { "Authorization": f"Bearer {jwt_token}", "Accept": "application/vnd.github.v3+json", @@ -157,10 +163,12 @@ def get_installation_id(jwt_token): for installation in data: if installation["account"]["login"] == "ClickHouse": installation_id = installation["id"] - return installation_id + break + + return installation_id # type: ignore -def get_access_token(jwt_token, installation_id): +def get_access_token(jwt_token: str, installation_id: int) -> str: headers = { "Authorization": f"Bearer {jwt_token}", "Accept": "application/vnd.github.v3+json", @@ -171,15 +179,10 @@ def get_access_token(jwt_token, installation_id): ) response.raise_for_status() data = response.json() - return data["token"] + return data["token"] # type: ignore -RunnerDescription = namedtuple( - "RunnerDescription", ["id", "name", "tags", "offline", "busy"] -) - - -def list_runners(access_token): +def list_runners(access_token: str) -> RunnerDescriptions: headers = { "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", @@ -225,8 +228,10 @@ def list_runners(access_token): return result -def group_runners_by_tag(listed_runners): - result = {} +def group_runners_by_tag( + listed_runners: RunnerDescriptions, +) -> Dict[str, RunnerDescriptions]: + result = {} # type: Dict[str, RunnerDescriptions] def add_to_result(tag, runner): if tag not in result: @@ -248,7 +253,9 @@ def group_runners_by_tag(listed_runners): return result -def push_metrics_to_cloudwatch(listed_runners, namespace): +def push_metrics_to_cloudwatch( + listed_runners: RunnerDescriptions, namespace: str +) -> None: client = boto3.client("cloudwatch") metrics_data = [] busy_runners = sum( @@ -278,7 +285,7 @@ def push_metrics_to_cloudwatch(listed_runners, namespace): } ) if total_active_runners == 0: - busy_ratio = 100 + busy_ratio = 100.0 else: busy_ratio = busy_runners / total_active_runners * 100 @@ -293,7 +300,7 @@ def push_metrics_to_cloudwatch(listed_runners, namespace): client.put_metric_data(Namespace=namespace, MetricData=metrics_data) -def delete_runner(access_token, runner): +def delete_runner(access_token: str, runner: RunnerDescription) -> bool: headers = { "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", @@ -305,10 +312,15 @@ def delete_runner(access_token, runner): ) response.raise_for_status() print(f"Response code deleting {runner.name} is {response.status_code}") - return response.status_code == 204 + return bool(response.status_code == 204) -def main(github_secret_key, github_app_id, push_to_cloudwatch, delete_offline_runners): +def main( + github_secret_key: str, + github_app_id: int, + push_to_cloudwatch: bool, + delete_offline_runners: bool, +) -> None: payload = { "iat": int(time.time()) - 60, "exp": int(time.time()) + (10 * 60), From 23002bc808b6cfb8600d4fec77516bd6fc1385ab Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 13:24:45 +0100 Subject: [PATCH 41/62] Add typing to terminate_runner_lambda --- tests/ci/terminate_runner_lambda/app.py | 52 ++++++++++++++----------- 1 file changed, 29 insertions(+), 23 deletions(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index 4140690e891..223555ced74 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -1,17 +1,18 @@ #!/usr/bin/env python3 -import requests import argparse -import jwt import sys import json import time from collections import namedtuple +from typing import Any, Dict, List, Tuple + +import boto3 # type: ignore +import requests # type: ignore +import jwt -def get_key_and_app_from_aws(): - import boto3 - +def get_key_and_app_from_aws() -> Tuple[str, int]: secret_name = "clickhouse_github_secret_key" session = boto3.session.Session() client = session.client( @@ -22,7 +23,7 @@ def get_key_and_app_from_aws(): return data["clickhouse-app-key"], int(data["clickhouse-app-id"]) -def get_installation_id(jwt_token): +def get_installation_id(jwt_token: str) -> int: headers = { "Authorization": f"Bearer {jwt_token}", "Accept": "application/vnd.github.v3+json", @@ -33,10 +34,12 @@ def get_installation_id(jwt_token): for installation in data: if installation["account"]["login"] == "ClickHouse": installation_id = installation["id"] - return installation_id + break + + return installation_id # type: ignore -def get_access_token(jwt_token, installation_id): +def get_access_token(jwt_token: str, installation_id: int) -> str: headers = { "Authorization": f"Bearer {jwt_token}", "Accept": "application/vnd.github.v3+json", @@ -47,15 +50,16 @@ def get_access_token(jwt_token, installation_id): ) response.raise_for_status() data = response.json() - return data["token"] + return data["token"] # type: ignore RunnerDescription = namedtuple( "RunnerDescription", ["id", "name", "tags", "offline", "busy"] ) +RunnerDescriptions = List[RunnerDescription] -def list_runners(access_token): +def list_runners(access_token: str) -> RunnerDescriptions: headers = { "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", @@ -94,9 +98,9 @@ def list_runners(access_token): return result -def how_many_instances_to_kill(event_data): +def how_many_instances_to_kill(event_data: dict) -> Dict[str, int]: data_array = event_data["CapacityToTerminate"] - to_kill_by_zone = {} + to_kill_by_zone = {} # type: Dict[str, int] for av_zone in data_array: zone_name = av_zone["AvailabilityZone"] to_kill = av_zone["Capacity"] @@ -104,15 +108,16 @@ def how_many_instances_to_kill(event_data): to_kill_by_zone[zone_name] = 0 to_kill_by_zone[zone_name] += to_kill + return to_kill_by_zone -def get_candidates_to_be_killed(event_data): +def get_candidates_to_be_killed(event_data: dict) -> Dict[str, List[str]]: data_array = event_data["Instances"] - instances_by_zone = {} + instances_by_zone = {} # type: Dict[str, List[str]] for instance in data_array: zone_name = instance["AvailabilityZone"] - instance_id = instance["InstanceId"] + instance_id = instance["InstanceId"] # type: str if zone_name not in instances_by_zone: instances_by_zone[zone_name] = [] instances_by_zone[zone_name].append(instance_id) @@ -120,7 +125,7 @@ def get_candidates_to_be_killed(event_data): return instances_by_zone -def delete_runner(access_token, runner): +def delete_runner(access_token: str, runner: RunnerDescription) -> bool: headers = { "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", @@ -134,10 +139,12 @@ def delete_runner(access_token, runner): print( f"Response code deleting {runner.name} with id {runner.id} is {response.status_code}" ) - return response.status_code == 204 + return bool(response.status_code == 204) -def main(github_secret_key, github_app_id, event): +def main( + github_secret_key: str, github_app_id: int, event: dict +) -> Dict[str, List[str]]: print("Got event", json.dumps(event, sort_keys=True, indent=4)) to_kill_by_zone = how_many_instances_to_kill(event) instances_by_zone = get_candidates_to_be_killed(event) @@ -156,17 +163,16 @@ def main(github_secret_key, github_app_id, event): to_delete_runners = [] instances_to_kill = [] - for zone in to_kill_by_zone: - num_to_kill = to_kill_by_zone[zone] + for zone, num_to_kill in to_kill_by_zone.items(): candidates = instances_by_zone[zone] if num_to_kill > len(candidates): raise Exception( f"Required to kill {num_to_kill}, but have only {len(candidates)} candidates in AV {zone}" ) - delete_for_av = [] + delete_for_av = [] # type: RunnerDescriptions for candidate in candidates: - if candidate not in set([runner.name for runner in runners]): + if candidate not in set(runner.name for runner in runners): print( f"Candidate {candidate} was not in runners list, simply delete it" ) @@ -214,7 +220,7 @@ def main(github_secret_key, github_app_id, event): return response -def handler(event, context): +def handler(event: dict, context: Any) -> Dict[str, List[str]]: private_key, app_id = get_key_and_app_from_aws() return main(private_key, app_id, event) From 0f2704703ff245d4b70d787d59d0ff374cc804cf Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 13:33:45 +0100 Subject: [PATCH 42/62] Fix typing and bug in cancel_and_rerun_workflow_lambda --- tests/ci/cancel_and_rerun_workflow_lambda/app.py | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index ebdfe2fdb5b..d93a9062a3b 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -106,7 +106,7 @@ def _exec_get_with_retry(url: str, token: str) -> dict: try: response = requests.get(url, headers=headers) response.raise_for_status() - return response.json() + return response.json() # type: ignore except Exception as ex: print("Got exception executing request", ex) time.sleep(i + 1) @@ -130,8 +130,7 @@ WorkflowDescription = namedtuple( def get_workflows_description_for_pull_request( - pull_request_event, - token, + pull_request_event: dict, token: str ) -> List[WorkflowDescription]: head_repo = pull_request_event["head"]["repo"]["full_name"] head_branch = pull_request_event["head"]["ref"] @@ -193,7 +192,7 @@ def get_workflows_description_for_pull_request( def get_workflow_description_fallback( - pull_request_event, token + pull_request_event: dict, token: str ) -> List[WorkflowDescription]: head_repo = pull_request_event["head"]["repo"]["full_name"] head_branch = pull_request_event["head"]["ref"] @@ -241,7 +240,7 @@ def get_workflow_description_fallback( WorkflowDescription( url=wf["url"], run_id=wf["id"], - name=workflow["name"], + name=wf["name"], head_sha=wf["head_sha"], status=wf["status"], rerun_url=wf["rerun_url"], @@ -254,7 +253,7 @@ def get_workflow_description_fallback( return workflow_descriptions -def get_workflow_description(workflow_url, token) -> WorkflowDescription: +def get_workflow_description(workflow_url: str, token: str) -> WorkflowDescription: workflow = _exec_get_with_retry(workflow_url, token) return WorkflowDescription( url=workflow["url"], @@ -331,7 +330,7 @@ def main(event): workflow_descriptions or get_workflow_description_fallback(pull_request, token) ) - workflow_descriptions.sort(key=lambda x: x.run_id) + workflow_descriptions.sort(key=lambda x: x.run_id) # type: ignore most_recent_workflow = workflow_descriptions[-1] if ( most_recent_workflow.status == "completed" @@ -376,7 +375,7 @@ def main(event): print("Not found any workflows") return - workflow_descriptions.sort(key=lambda x: x.run_id) + workflow_descriptions.sort(key=lambda x: x.run_id) # type: ignore most_recent_workflow = workflow_descriptions[-1] print("Latest workflow", most_recent_workflow) if ( From e46f615176ab629d696cfd0c1d822374a92307a1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 14:08:04 +0100 Subject: [PATCH 43/62] Simplify checks list in style-check image --- .../test/style/process_style_check_result.py | 19 ++++++++++--------- docker/test/style/run.sh | 4 ++-- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/docker/test/style/process_style_check_result.py b/docker/test/style/process_style_check_result.py index 8c2110d64e5..e07e6153cee 100755 --- a/docker/test/style/process_style_check_result.py +++ b/docker/test/style/process_style_check_result.py @@ -11,17 +11,18 @@ def process_result(result_folder): description = "" test_results = [] checks = ( - ("header duplicates", "duplicate_output.txt"), - ("shellcheck", "shellcheck_output.txt"), - ("style", "style_output.txt"), - ("black", "black_output.txt"), - ("typos", "typos_output.txt"), - ("whitespaces", "whitespaces_output.txt"), - ("workflows", "workflows_output.txt"), - ("doc typos", "doc_spell_output.txt"), + "header duplicates", + "shellcheck", + "style", + "black", + "typos", + "whitespaces", + "workflows", + "doc typos", ) - for name, out_file in checks: + for name in checks: + out_file = name.replace(" ", "_") + "_output.txt" full_path = os.path.join(result_folder, out_file) if not os.path.exists(full_path): logging.info("No %s check log on path %s", name, full_path) diff --git a/docker/test/style/run.sh b/docker/test/style/run.sh index 06ecadbfebf..9f0490af120 100755 --- a/docker/test/style/run.sh +++ b/docker/test/style/run.sh @@ -4,7 +4,7 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv echo "Check duplicates" | ts -./check-duplicate-includes.sh |& tee /test_output/duplicate_output.txt +./check-duplicate-includes.sh |& tee /test_output/header_duplicates_output.txt echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt echo "Check python formatting with black" | ts @@ -12,7 +12,7 @@ echo "Check python formatting with black" | ts echo "Check typos" | ts ./check-typos |& tee /test_output/typos_output.txt echo "Check docs spelling" | ts -./check-doc-aspell |& tee /test_output/doc_spell_output.txt +./check-doc-aspell |& tee /test_output/doc_typos_output.txt echo "Check whitespaces" | ts ./check-whitespaces -n |& tee /test_output/whitespaces_output.txt echo "Check workflows" | ts From 9cb2aa1c464420309c1a610bcfd213dee8ee93ad Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 15:48:52 +0100 Subject: [PATCH 44/62] Add python mypy check to CI --- docker/test/style/Dockerfile | 2 +- .../test/style/process_style_check_result.py | 1 + docker/test/style/run.sh | 2 ++ utils/check-style/check-mypy | 23 +++++++++++++++++++ 4 files changed, 27 insertions(+), 1 deletion(-) create mode 100755 utils/check-style/check-mypy diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index cb8c914e53d..e8c5e17024c 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -17,7 +17,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ shellcheck \ yamllint \ - && pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory PyGithub unidiff pylint==2.6.2 \ + && pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory mypy PyGithub unidiff pylint==2.6.2 \ && apt-get clean \ && rm -rf /root/.cache/pip diff --git a/docker/test/style/process_style_check_result.py b/docker/test/style/process_style_check_result.py index e07e6153cee..d914ceffb86 100755 --- a/docker/test/style/process_style_check_result.py +++ b/docker/test/style/process_style_check_result.py @@ -15,6 +15,7 @@ def process_result(result_folder): "shellcheck", "style", "black", + "mypy", "typos", "whitespaces", "workflows", diff --git a/docker/test/style/run.sh b/docker/test/style/run.sh index 9f0490af120..911536ed03b 100755 --- a/docker/test/style/run.sh +++ b/docker/test/style/run.sh @@ -9,6 +9,8 @@ echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt echo "Check python formatting with black" | ts ./check-black -n |& tee /test_output/black_output.txt +echo "Check python type hinting with mypy" | ts +./check-mypy -n |& tee /test_output/mypy_output.txt echo "Check typos" | ts ./check-typos |& tee /test_output/typos_output.txt echo "Check docs spelling" | ts diff --git a/utils/check-style/check-mypy b/utils/check-style/check-mypy new file mode 100755 index 00000000000..42cb7fbbd15 --- /dev/null +++ b/utils/check-style/check-mypy @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +# The mypy supports pyproject.toml, but unfortunately it doesn't support it recursively +# https://github.com/python/mypy/issues/10613 +# +# Unless it's done, mypy only runs against tests/ci +# Let's leave here a room for improvement and redo it when mypy will test anything else + +GIT_ROOT=$(git rev-parse --show-cdup) +GIT_ROOT=${GIT_ROOT:-.} +CONFIG="$GIT_ROOT/tests/ci/.mypy.ini" +DIRS=("$GIT_ROOT/tests/ci/" "$GIT_ROOT/tests/ci/"*/) +tmp=$(mktemp) +for dir in "${DIRS[@]}"; do + if ! compgen -G "$dir"/*.py > /dev/null; then + continue + fi + if ! mypy --config-file="$CONFIG" --sqlite-cache "$dir"/*.py > "$tmp" 2>&1; then + echo "Errors while processing $dir": + cat "$tmp" + fi +done +rm -rf "$tmp" From fe8107e95f68f2571264e29eb755a2f979f1fa61 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 17:18:29 +0100 Subject: [PATCH 45/62] Fix type ignorance in version_helper.py --- tests/ci/version_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index dc4674b3699..69cfba64be3 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -147,7 +147,7 @@ class ClickHouseVersion: def __eq__(self, other: Any) -> bool: if not isinstance(self, type(other)): return NotImplemented - return ( # type: ignore + return bool( self.major == other.major and self.minor == other.minor and self.patch == other.patch From d6ca97c8d0dcf3c18917eb0914db5c19820aefc8 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 30 Nov 2022 11:58:04 +0200 Subject: [PATCH 46/62] Fix multipart upload for large S3 object --- src/Backups/BackupIO_S3.cpp | 3 ++- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 8342749e230..9b5711d5595 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -166,7 +166,8 @@ void BackupWriterS3::copyObjectImpl( auto outcome = client->CopyObject(request); - if (!outcome.IsSuccess() && outcome.GetError().GetExceptionName() == "EntityTooLarge") + if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge" + || outcome.GetError().GetExceptionName() == "InvalidRequest")) { // Can't come here with MinIO, MinIO allows single part upload for large objects. copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata); return; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 996268079e8..ed7b8182622 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -480,7 +480,8 @@ void S3ObjectStorage::copyObjectImpl( auto outcome = client_ptr->CopyObject(request); - if (!outcome.IsSuccess() && outcome.GetError().GetExceptionName() == "EntityTooLarge") + if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge" + || outcome.GetError().GetExceptionName() == "InvalidRequest")) { // Can't come here with MinIO, MinIO allows single part upload for large objects. copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata); return; From 7b4025ba7ceabe48ea90d1c56d9caf15486e4439 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Nov 2022 12:43:46 +0100 Subject: [PATCH 47/62] Fix bug --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 916befdf78f..f6befe67fd4 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1322,7 +1322,7 @@ private: for (auto p_it = projection_data_part_storage_src->iterate(); p_it->isValid(); p_it->next()) { - auto file_name_with_projection_prefix = fs::path(projection_data_part_storage_src->getPartDirectory()) / it->name(); + auto file_name_with_projection_prefix = fs::path(projection_data_part_storage_src->getPartDirectory()) / p_it->name(); projection_data_part_storage_dst->createHardLinkFrom( *projection_data_part_storage_src, p_it->name(), p_it->name()); From fbda86ddc80eefee87522f0e6e3f2fe937352b74 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 30 Nov 2022 13:24:08 +0100 Subject: [PATCH 48/62] PR fixes --- programs/keeper/Keeper.cpp | 47 ++++++++++--------- programs/keeper/Keeper.h | 2 +- .../KeeperAsynchronousMetrics.cpp | 6 +-- src/Coordination/KeeperAsynchronousMetrics.h | 4 +- src/Coordination/TinyContext.h | 2 + src/Interpreters/InterpreterCreateQuery.cpp | 1 - 6 files changed, 33 insertions(+), 29 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 37717721cea..25452b808e2 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -60,6 +60,8 @@ int mainEntryClickHouseKeeper(int argc, char ** argv) #ifdef KEEPER_STANDALONE_BUILD +// Weak symbols don't work correctly on Darwin +// so we have a stub implementation to avoid linker errors void collectCrashLog( Int32, UInt64, const String &, const StackTrace &) {} @@ -277,51 +279,51 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options) struct Keeper::KeeperHTTPContext : public IHTTPContext { - explicit KeeperHTTPContext(const TinyContext & context_) - : context(context_) + explicit KeeperHTTPContext(TinyContextPtr context_) + : context(std::move(context_)) {} uint64_t getMaxHstsAge() const override { - return context.getConfigRef().getUInt64("keeper_server.hsts_max_age", 0); + return context->getConfigRef().getUInt64("keeper_server.hsts_max_age", 0); } uint64_t getMaxUriSize() const override { - return context.getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576); + return context->getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576); } uint64_t getMaxFields() const override { - return context.getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000); + return context->getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000); } uint64_t getMaxFieldNameSize() const override { - return context.getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 1048576); + return context->getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 1048576); } uint64_t getMaxFieldValueSize() const override { - return context.getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 1048576); + return context->getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 1048576); } uint64_t getMaxChunkSize() const override { - return context.getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB); + return context->getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB); } Poco::Timespan getReceiveTimeout() const override { - return context.getConfigRef().getUInt64("keeper_server.http_receive_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); + return context->getConfigRef().getUInt64("keeper_server.http_receive_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); } Poco::Timespan getSendTimeout() const override { - return context.getConfigRef().getUInt64("keeper_server.http_send_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); + return context->getConfigRef().getUInt64("keeper_server.http_send_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); } - const TinyContext & context; + TinyContextPtr context; }; HTTPContextPtr Keeper::httpContext() @@ -406,6 +408,7 @@ try std::mutex servers_lock; auto servers = std::make_shared>(); + tiny_context = std::make_shared(); /// This object will periodically calculate some metrics. KeeperAsynchronousMetrics async_metrics( tiny_context, @@ -433,12 +436,12 @@ try } /// Initialize keeper RAFT. Do nothing if no keeper_server in config. - tiny_context.initializeKeeperDispatcher(/* start_async = */ true); - FourLetterCommandFactory::registerCommands(*tiny_context.getKeeperDispatcher()); + tiny_context->initializeKeeperDispatcher(/* start_async = */ true); + FourLetterCommandFactory::registerCommands(*tiny_context->getKeeperDispatcher()); auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration & { - return tiny_context.getConfigRef(); + return tiny_context->getConfigRef(); }; for (const auto & listen_host : listen_hosts) @@ -457,7 +460,7 @@ try "Keeper (tcp): " + address.toString(), std::make_unique( new KeeperTCPHandlerFactory( - config_getter, tiny_context.getKeeperDispatcher(), + config_getter, tiny_context->getKeeperDispatcher(), config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), false), server_pool, socket)); }); @@ -476,7 +479,7 @@ try "Keeper with secure protocol (tcp_secure): " + address.toString(), std::make_unique( new KeeperTCPHandlerFactory( - config_getter, tiny_context.getKeeperDispatcher(), + config_getter, tiny_context->getKeeperDispatcher(), config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), true), server_pool, socket)); #else @@ -498,15 +501,15 @@ try { Poco::Net::ServerSocket socket; auto address = socketBindListen(socket, listen_host, port); - // TODO(antonio2368): use config - socket.setReceiveTimeout(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC); - socket.setSendTimeout(DBMS_DEFAULT_SEND_TIMEOUT_SEC); + auto http_context = httpContext(); + socket.setReceiveTimeout(http_context->getReceiveTimeout()); + socket.setSendTimeout(http_context->getSendTimeout()); servers->emplace_back( listen_host, port_name, "Prometheus: http://" + address.toString(), std::make_unique( - httpContext(), createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); + std::move(http_context), createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); }); } @@ -530,7 +533,7 @@ try [&](ConfigurationPtr config, bool /* initial_loading */) { if (config->has("keeper_server")) - tiny_context.updateKeeperConfiguration(*config); + tiny_context->updateKeeperConfiguration(*config); }, /* already_loaded = */ false); /// Reload it right now (initial loading) @@ -561,7 +564,7 @@ try else LOG_INFO(log, "Closed connections to Keeper."); - tiny_context.shutdownKeeperDispatcher(); + tiny_context->shutdownKeeperDispatcher(); /// Wait server pool to avoid use-after-free of destroyed context in the handlers server_pool.joinAll(); diff --git a/programs/keeper/Keeper.h b/programs/keeper/Keeper.h index 224990cc28e..8a7724acb85 100644 --- a/programs/keeper/Keeper.h +++ b/programs/keeper/Keeper.h @@ -68,7 +68,7 @@ protected: std::string getDefaultConfigFileName() const override; private: - TinyContext tiny_context; + TinyContextPtr tiny_context; struct KeeperHTTPContext; HTTPContextPtr httpContext(); diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 4531fbfb3f6..2d523a26dcc 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -108,8 +108,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM } KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( - const TinyContext & tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), tiny_context(tiny_context_) + TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), tiny_context(std::move(tiny_context_)) { } @@ -117,7 +117,7 @@ void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values { #if USE_NURAFT { - auto keeper_dispatcher = tiny_context.tryGetKeeperDispatcher(); + auto keeper_dispatcher = tiny_context->tryGetKeeperDispatcher(); if (keeper_dispatcher) updateKeeperInformation(*keeper_dispatcher, new_values); } diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index 5b703951065..8fa27336bc5 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -13,10 +13,10 @@ class KeeperAsynchronousMetrics : public AsynchronousMetrics { public: KeeperAsynchronousMetrics( - const TinyContext & tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); private: - const TinyContext & tiny_context; + TinyContextPtr tiny_context; void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; }; diff --git a/src/Coordination/TinyContext.h b/src/Coordination/TinyContext.h index 7ccbe0ec8a6..b966d445004 100644 --- a/src/Coordination/TinyContext.h +++ b/src/Coordination/TinyContext.h @@ -31,4 +31,6 @@ private: ConfigurationPtr config TSA_GUARDED_BY(keeper_dispatcher_mutex); }; +using TinyContextPtr = std::shared_ptr; + } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ac6f6c0b228..e9cf06c5c69 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1167,7 +1167,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) getContext() )) { - LOG_INFO(&Poco::Logger::get("LOGGER"), "{}", create.select->dumpTree()); Block input_block = InterpreterSelectWithUnionQuery( create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); From 5a67ba85982ef633012bbd9cd24a5ee6277a7b76 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 30 Nov 2022 15:21:14 +0000 Subject: [PATCH 49/62] Increase some logging level for keeper client. --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index ebab18b5ed7..427088675fb 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -466,7 +466,7 @@ void ZooKeeper::connect( } else { - LOG_TEST(log, "Connected to ZooKeeper at {} with session_id {}{}", socket.peerAddress().toString(), session_id, fail_reasons.str()); + LOG_INFO(log, "Connected to ZooKeeper at {} with session_id {}{}", socket.peerAddress().toString(), session_id, fail_reasons.str()); } } @@ -867,7 +867,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea /// If some thread (send/receive) already finalizing session don't try to do it bool already_started = finalization_started.test_and_set(); - LOG_TEST(log, "Finalizing session {}: finalization_started={}, queue_finished={}, reason={}", + LOG_INFO(log, "Finalizing session {}: finalization_started={}, queue_finished={}, reason={}", session_id, already_started, requests_queue.isFinished(), reason); if (already_started) From ecc45e5b1f692111b1e99d334c3481b3dfd74748 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 30 Nov 2022 16:15:40 +0000 Subject: [PATCH 50/62] Review fixes. --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 427088675fb..7cbe7d7b0f2 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -867,12 +867,12 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea /// If some thread (send/receive) already finalizing session don't try to do it bool already_started = finalization_started.test_and_set(); - LOG_INFO(log, "Finalizing session {}: finalization_started={}, queue_finished={}, reason={}", - session_id, already_started, requests_queue.isFinished(), reason); - if (already_started) return; + LOG_INFO(log, "Finalizing session {}: finalization_started={}, queue_finished={}, reason={}", + session_id, already_started, requests_queue.isFinished(), reason); + auto expire_session_if_not_expired = [&] { /// No new requests will appear in queue after finish() From 98fe3c6c02b10c67f3d51c52bf2a9362cb792006 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Nov 2022 17:45:59 +0000 Subject: [PATCH 51/62] Temporary files evict fs cache --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 58 +++------- src/Common/filesystemHelpers.cpp | 6 +- src/Common/filesystemHelpers.h | 2 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 42 +++++-- .../IO/CachedOnDiskWriteBufferFromFile.h | 8 +- src/Disks/IO/FileCachePlaceholder.cpp | 7 ++ src/Disks/IO/FileCachePlaceholder.h | 104 ++++++++++++++++++ .../Cached/CachedObjectStorage.h | 2 + .../ObjectStorages/DiskObjectStorage.cpp | 8 ++ src/Disks/ObjectStorages/DiskObjectStorage.h | 1 + src/Disks/TemporaryFileInPath.cpp | 10 ++ src/Disks/TemporaryFileInPath.h | 27 +++++ src/Disks/TemporaryFileOnDisk.cpp | 2 +- src/Disks/TemporaryFileOnDisk.h | 15 ++- src/Formats/NativeWriter.cpp | 8 +- src/Formats/NativeWriter.h | 2 +- src/Interpreters/Cache/FileCache.cpp | 8 ++ src/Interpreters/Cache/FileCache.h | 2 + src/Interpreters/Cache/FileCacheFactory.cpp | 11 +- src/Interpreters/Cache/FileCacheFactory.h | 1 + src/Interpreters/Cache/FileSegment.cpp | 49 ++++++--- src/Interpreters/Cache/FileSegment.h | 15 +-- src/Interpreters/Context.cpp | 104 ++++++++++++++---- src/Interpreters/Context.h | 4 +- src/Interpreters/TemporaryDataOnDisk.cpp | 76 +++++++++---- src/Interpreters/TemporaryDataOnDisk.h | 22 +++- .../tests/gtest_lru_file_cache.cpp | 44 ++++++++ 28 files changed, 499 insertions(+), 141 deletions(-) create mode 100644 src/Disks/IO/FileCachePlaceholder.cpp create mode 100644 src/Disks/IO/FileCachePlaceholder.h create mode 100644 src/Disks/TemporaryFileInPath.cpp create mode 100644 src/Disks/TemporaryFileInPath.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index ce7e27026f1..22c94e01a51 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -207,7 +207,7 @@ void LocalServer::tryInitPath() global_context->setPath(path); - global_context->setTemporaryStorage(path + "tmp", "", 0); + global_context->setTemporaryStoragePath(path + "tmp/", 0); global_context->setFlagsPath(path + "flags"); global_context->setUserFilesPath(""); // user's files are everywhere diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b6ce358a5ef..e772208c30e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -203,46 +203,6 @@ int mainEntryClickHouseServer(int argc, char ** argv) namespace { -void setupTmpPath(Poco::Logger * log, const std::string & path) -try -{ - LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); - - fs::create_directories(path); - - /// Clearing old temporary files. - fs::directory_iterator dir_end; - size_t unknown_files = 0; - for (fs::directory_iterator it(path); it != dir_end; ++it) - { - if (it->is_regular_file() && startsWith(it->path().filename(), "tmp")) - { - LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); - fs::remove(it->path()); - } - else - { - unknown_files++; - if (unknown_files < 100) - LOG_DEBUG(log, "Found unknown {} {} in temporary path", - it->is_regular_file() ? "file" : (it->is_directory() ? "directory" : "element"), - it->path().string()); - } - } - - if (unknown_files) - LOG_DEBUG(log, "Found {} unknown files in temporary path", unknown_files); -} -catch (...) -{ - DB::tryLogCurrentException( - log, - fmt::format( - "Caught exception while setup temporary path: {}. It is ok to skip this exception as cleaning old temporary files is not " - "necessary", - path)); -} - size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait) { const size_t sleep_max_ms = 1000 * seconds_to_wait; @@ -1037,13 +997,21 @@ try LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); /// Storage with temporary data for processing of heavy queries. + if (auto temporary_policy = config().getString("tmp_policy", ""); !temporary_policy.empty()) + { + size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); + global_context->setTemporaryStoragePolicy(temporary_policy, max_size); + } + else if (auto temporary_cache = config().getString("tmp_cache", ""); !temporary_cache.empty()) + { + size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); + global_context->setTemporaryStorageInCache(temporary_cache, max_size); + } + else { std::string temporary_path = config().getString("tmp_path", path / "tmp/"); - std::string temporary_policy = config().getString("tmp_policy", ""); size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); - const VolumePtr & volume = global_context->setTemporaryStorage(temporary_path, temporary_policy, max_size); - for (const DiskPtr & disk : volume->getDisks()) - setupTmpPath(log, disk->getPath()); + global_context->setTemporaryStoragePath(temporary_path, max_size); } /** Directory with 'flags': files indicating temporary settings for the server set by system administrator. @@ -1442,7 +1410,7 @@ try } catch (...) { - tryLogCurrentException(log); + tryLogCurrentException(log, "Caught exception while setting up access control."); throw; } diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 07a08dc7fbc..b259d1929ea 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -64,11 +64,11 @@ bool enoughSpaceInDirectory(const std::string & path, size_t data_size) return data_size <= free_space; } -std::unique_ptr createTemporaryFile(const std::string & path) +std::unique_ptr createTemporaryFile(const std::string & folder_path) { ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal); - fs::create_directories(path); - return std::make_unique(path); + fs::create_directories(folder_path); + return std::make_unique(folder_path); } #if !defined(OS_LINUX) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 0e6e16941bb..fd080771b52 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -17,7 +17,7 @@ namespace DB using TemporaryFile = Poco::TemporaryFile; bool enoughSpaceInDirectory(const std::string & path, size_t data_size); -std::unique_ptr createTemporaryFile(const std::string & path); +std::unique_ptr createTemporaryFile(const std::string & folder_path); // Determine what block device is responsible for specified path diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 994bb743c5f..609ef2cdaa7 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -52,9 +52,16 @@ FileSegmentRangeWriter::FileSegmentRangeWriter( } bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, bool is_persistent) +{ + size_t written_size = tryWrite(data, size, offset, is_persistent, true); + chassert(written_size == 0 || written_size == size); + return written_size == size; +} + +size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict) { if (finalized) - return false; + return 0; auto & file_segments = file_segments_holder.file_segments; @@ -93,8 +100,8 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset file_segment->completePartAndResetDownloader(); }); - bool reserved = file_segment->reserve(size); - if (!reserved) + size_t reserved_size = file_segment->tryReserve(size, strict); + if (strict && reserved_size != size) { file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); appendFilesystemCacheLog(*file_segment); @@ -104,9 +111,15 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset "Unsuccessful space reservation attempt (size: {}, file segment info: {}", size, file_segment->getInfoForLog()); - return false; + return 0; } + if (reserved_size == 0) + return 0; + + /// shrink + size = reserved_size; + try { file_segment->write(data, size, offset); @@ -120,15 +133,26 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset file_segment->completePartAndResetDownloader(); current_file_segment_write_offset += size; - return true; + return size; } -void FileSegmentRangeWriter::finalize() +void FileSegmentRangeWriter::finalize(bool clear) { if (finalized) return; auto & file_segments = file_segments_holder.file_segments; + + /// Set all segments state to SKIP_CACHE to remove it from cache immediately on complete + /// Note: if segments are hold by someone else, it won't be removed + if (clear) + { + for (auto & file_segment : file_segments) + completeFileSegment(*file_segment, FileSegment::State::SKIP_CACHE); + finalized = true; + return; + } + if (file_segments.empty() || current_file_segment_it == file_segments.end()) return; @@ -196,12 +220,16 @@ void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_s } } -void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment) +void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment, std::optional state) { /// File segment can be detached if space reservation failed. if (file_segment.isDetached()) return; + // if (state.has_value()) + // file_segment.setDownloadState(*state); + UNUSED(state); + file_segment.completeWithoutState(); appendFilesystemCacheLog(file_segment); } diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index cec7305ab1b..4f5bebb92e8 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace Poco { @@ -33,8 +34,11 @@ public: * it until it is full and then allocate next file segment. */ bool write(const char * data, size_t size, size_t offset, bool is_persistent); + size_t tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict = false); - void finalize(); + void finalize(bool clear = false); + + size_t currentOffset() const { return current_file_segment_write_offset; } ~FileSegmentRangeWriter(); @@ -43,7 +47,7 @@ private: void appendFilesystemCacheLog(const FileSegment & file_segment); - void completeFileSegment(FileSegment & file_segment); + void completeFileSegment(FileSegment & file_segment, std::optional state = {}); FileCache * cache; FileSegment::Key key; diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp new file mode 100644 index 00000000000..a2640aaffdc --- /dev/null +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -0,0 +1,7 @@ +#include + +namespace DB +{ + + +} diff --git a/src/Disks/IO/FileCachePlaceholder.h b/src/Disks/IO/FileCachePlaceholder.h new file mode 100644 index 00000000000..3c5c2bb1723 --- /dev/null +++ b/src/Disks/IO/FileCachePlaceholder.h @@ -0,0 +1,104 @@ +#pragma once + +#include +#include + +#include + +namespace fs = std::filesystem; + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int NOT_ENOUGH_SPACE; +} + + +class ISpacePlaceholder +{ + +public: + void reserveCapacity(size_t requested_capacity) + { + chassert(used_space <= capacity); + + size_t remaining_space = capacity - used_space; + if (requested_capacity <= remaining_space) + return; + + size_t capacity_to_reserve = requested_capacity - remaining_space; + reserveImpl(capacity_to_reserve); + capacity += capacity_to_reserve; + } + + void setUsed(size_t size) + { + if (used_space + size > capacity) + { + LOG_WARNING(&Poco::Logger::get("ISpacePlaceholder"), "Used space is greater than capacity. It may lead to not enough space error"); + reserveCapacity(used_space + size - capacity); + } + + used_space = used_space + size; + } + + virtual ~ISpacePlaceholder() = default; + +private: + virtual void reserveImpl(size_t size) = 0; + + size_t capacity = 0; + size_t used_space = 0; +}; + +class FileCachePlaceholder : public ISpacePlaceholder +{ +public: + FileCachePlaceholder(FileCache * cache, const String & name) + : key(cache->hash(name)) + , directory(cache->getPathInLocalCache(key)) + , cache_writer(cache, key, nullptr, "", name) + { + + } + + fs::path getDirectoryPath() const + { + return directory; + } + + void reserveImpl(size_t size) override + { + while (size > 0) + { + size_t current_offset = cache_writer.currentOffset(); + size_t written = cache_writer.tryWrite(nullptr, size, current_offset, /* is_persistent */ false); + if (written == 0) + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Cannot reserve space in file cache ({} bytes required)", size); + size -= written; + } + } + + ~FileCachePlaceholder() override + { + try + { + cache_writer.finalize(/* clear */ true); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + +private: + FileSegment::Key key; + fs::path directory; + + FileSegmentRangeWriter cache_writer; +}; + +} diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 2d67203be0f..119dc25c66b 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -113,6 +113,8 @@ public: WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override; + FileCachePtr getCache() const { return cache; } + private: FileCache::Key getCacheKey(const std::string & path) const; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 263a9a9d0e1..4230fb6254a 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -519,6 +519,14 @@ void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSetting object_storage = std::make_shared(object_storage, cache, cache_settings, layer_name); } +FileCachePtr DiskObjectStorage::getCache() const +{ + const auto * cached_object_storage = typeid_cast(object_storage.get()); + if (!cached_object_storage) + return nullptr; + return cached_object_storage->getCache(); +} + NameSet DiskObjectStorage::getCacheLayersNames() const { NameSet cache_layers; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 00e3cf98142..a24acc270c0 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -186,6 +186,7 @@ public: /// There can be any number of cache layers: /// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...)) void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name); + FileCachePtr getCache() const; /// Get structure of object storage this disk works with. Examples: /// DiskObjectStorage(S3ObjectStorage) diff --git a/src/Disks/TemporaryFileInPath.cpp b/src/Disks/TemporaryFileInPath.cpp new file mode 100644 index 00000000000..b7d5d7ed85e --- /dev/null +++ b/src/Disks/TemporaryFileInPath.cpp @@ -0,0 +1,10 @@ +#include + + + +namespace DB +{ + + + +} diff --git a/src/Disks/TemporaryFileInPath.h b/src/Disks/TemporaryFileInPath.h new file mode 100644 index 00000000000..540711e59c7 --- /dev/null +++ b/src/Disks/TemporaryFileInPath.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Wrapper around Poco::TemporaryFile to implement ITemporaryFile. +class TemporaryFileInPath : public ITemporaryFile +{ +public: + explicit TemporaryFileInPath(const String & folder_path) + : tmp_file(createTemporaryFile(folder_path)) + { + chassert(tmp_file); + } + + String getPath() const override { return tmp_file->path(); } + +private: + std::unique_ptr tmp_file; +}; + + + +} diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index 4f348519037..af1f3f87c71 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -15,7 +16,6 @@ namespace CurrentMetrics extern const Metric TotalTemporaryFiles; } - namespace DB { diff --git a/src/Disks/TemporaryFileOnDisk.h b/src/Disks/TemporaryFileOnDisk.h index 9ba59c3eaf0..fa3a0383297 100644 --- a/src/Disks/TemporaryFileOnDisk.h +++ b/src/Disks/TemporaryFileOnDisk.h @@ -9,21 +9,30 @@ namespace DB { using DiskPtr = std::shared_ptr; +class ITemporaryFile +{ +public: + virtual String getPath() const = 0; + virtual ~ITemporaryFile() = default; +}; + +using TemporaryFileHolder = std::unique_ptr; + /// This class helps with the handling of temporary files or directories. /// A unique name for the temporary file or directory is automatically chosen based on a specified prefix. /// Create a directory in the constructor. /// The destructor always removes the temporary file or directory with all contained files. -class TemporaryFileOnDisk +class TemporaryFileOnDisk : public ITemporaryFile { public: explicit TemporaryFileOnDisk(const DiskPtr & disk_); explicit TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Value metric_scope); explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix); - ~TemporaryFileOnDisk(); + ~TemporaryFileOnDisk() override; DiskPtr getDisk() const { return disk; } - String getPath() const; + String getPath() const override; private: DiskPtr disk; diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index c4dea371afd..e932bb88c2d 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -64,8 +64,10 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co } -void NativeWriter::write(const Block & block) +size_t NativeWriter::write(const Block & block) { + size_t written_before = ostr.count(); + /// Additional information about the block. if (client_revision > 0) block.info.write(ostr); @@ -161,6 +163,10 @@ void NativeWriter::write(const Block & block) if (index) index->blocks.emplace_back(std::move(index_block)); + + size_t written_after = ostr.count(); + size_t written_size = written_after - written_before; + return written_size; } } diff --git a/src/Formats/NativeWriter.h b/src/Formats/NativeWriter.h index 010a03ec722..eba7ac8443d 100644 --- a/src/Formats/NativeWriter.h +++ b/src/Formats/NativeWriter.h @@ -27,7 +27,7 @@ public: IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0); Block getHeader() const { return header; } - void write(const Block & block); + size_t write(const Block & block); void flush(); static String getContentType() { return "application/octet-stream"; } diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 72fa1b3c324..56c3eef5015 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -12,6 +13,7 @@ #include #include + namespace fs = std::filesystem; namespace DB @@ -1009,6 +1011,12 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock fs::directory_iterator offset_it{key_it->path()}; for (; offset_it != fs::directory_iterator(); ++offset_it) { + if (offset_it->is_directory()) + { + LOG_DEBUG(log, "Unexpected directory: {}. Expected a file", offset_it->path().string()); + continue; + } + auto offset_with_suffix = offset_it->path().filename().string(); auto delim_pos = offset_with_suffix.find('_'); bool parsed; diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 706762b6915..1df634b48ed 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -221,6 +221,8 @@ private: FileSegmentCell * getCell(const Key & key, size_t offset, std::lock_guard & cache_lock); + /// Returns non-owened pointer to the cell stored in the `files` map. + /// Doesn't reserve any space. FileSegmentCell * addCell( const Key & key, size_t offset, diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index b276760c0dd..e120fe3fc27 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -31,14 +31,21 @@ const FileCacheSettings & FileCacheFactory::getSettings(const std::string & cach } -FileCachePtr FileCacheFactory::get(const std::string & cache_base_path) +FileCachePtr FileCacheFactory::tryGet(const std::string & cache_base_path) { std::lock_guard lock(mutex); auto it = caches_by_path.find(cache_base_path); if (it == caches_by_path.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path); + return nullptr; return it->second->cache; +} +FileCachePtr FileCacheFactory::get(const std::string & cache_base_path) +{ + auto file_cache_ptr = tryGet(cache_base_path); + if (!file_cache_ptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path); + return file_cache_ptr; } FileCachePtr FileCacheFactory::getOrCreate( diff --git a/src/Interpreters/Cache/FileCacheFactory.h b/src/Interpreters/Cache/FileCacheFactory.h index 82e0ec8f928..32ecd05f019 100644 --- a/src/Interpreters/Cache/FileCacheFactory.h +++ b/src/Interpreters/Cache/FileCacheFactory.h @@ -33,6 +33,7 @@ public: FileCachePtr getOrCreate(const std::string & cache_base_path, const FileCacheSettings & file_cache_settings, const std::string & name); + FileCachePtr tryGet(const std::string & cache_base_path); FileCachePtr get(const std::string & cache_base_path); CacheByBasePath getAll(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 418bcee05d9..73cbaaaca57 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -99,6 +99,7 @@ size_t FileSegment::getFirstNonDownloadedOffsetUnlocked(std::unique_lock 0) throw Exception( @@ -324,11 +325,13 @@ void FileSegment::write(const char * from, size_t size, size_t offset) try { - cache_writer->write(from, size); + if (cache_writer && from != nullptr) + cache_writer->write(from, size); std::unique_lock download_lock(download_mutex); - cache_writer->next(); + if (cache_writer && from != nullptr) + cache_writer->next(); downloaded_size += size; } @@ -379,6 +382,13 @@ FileSegment::State FileSegment::wait() } bool FileSegment::reserve(size_t size_to_reserve) +{ + size_t reserved = tryReserve(size_to_reserve, true); + assert(reserved == 0 || reserved == size_to_reserve); + return reserved == size_to_reserve; +} + +size_t FileSegment::tryReserve(size_t size_to_reserve, bool strict) { if (!size_to_reserve) throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed"); @@ -394,10 +404,16 @@ bool FileSegment::reserve(size_t size_to_reserve) expected_downloaded_size = getDownloadedSizeUnlocked(segment_lock); if (expected_downloaded_size + size_to_reserve > range().size()) - throw Exception( - ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})", - size_to_reserve, range().toString(), downloaded_size); + { + if (strict) + { + throw Exception( + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + "Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})", + size_to_reserve, range().toString(), downloaded_size); + } + size_to_reserve = range().size() - expected_downloaded_size; + } chassert(reserved_size >= expected_downloaded_size); } @@ -415,17 +431,16 @@ bool FileSegment::reserve(size_t size_to_reserve) { std::lock_guard cache_lock(cache->mutex); - size_to_reserve = size_to_reserve - already_reserved_size; - reserved = cache->tryReserve(key(), offset(), size_to_reserve, cache_lock); + size_t need_to_reserve = size_to_reserve - already_reserved_size; + reserved = cache->tryReserve(key(), offset(), need_to_reserve, cache_lock); - if (reserved) - { - std::lock_guard segment_lock(mutex); - reserved_size += size_to_reserve; - } + if (!reserved) + return 0; + + std::lock_guard segment_lock(mutex); + reserved_size += need_to_reserve; } - - return reserved; + return size_to_reserve; } void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock & segment_lock) @@ -783,6 +798,8 @@ FileSegmentsHolder::~FileSegmentsHolder() if (!cache) cache = file_segment->cache; + assert(cache == file_segment->cache); /// all segments should belong to the same cache + try { bool is_detached = false; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 8f9c0097d77..02ae7cd5c8d 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -183,18 +183,13 @@ public: void assertCorrectness() const; - /** - * ========== Methods for _only_ file segment's `writer` ====================== - */ - - void synchronousWrite(const char * from, size_t size, size_t offset); - /** * ========== Methods for _only_ file segment's `downloader` ================== */ /// Try to reserve exactly `size` bytes. bool reserve(size_t size_to_reserve); + size_t tryReserve(size_t size_to_reserve, bool strict); /// Write data into reserved space. void write(const char * from, size_t size, size_t offset); @@ -247,9 +242,9 @@ private: void assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock & segment_lock) const; void assertCorrectnessUnlocked(std::unique_lock & segment_lock) const; - /// complete() without any completion state is called from destructor of - /// FileSegmentsHolder. complete() might check if the caller of the method - /// is the last alive holder of the segment. Therefore, complete() and destruction + /// completeWithoutStateUnlocked() is called from destructor of FileSegmentsHolder. + /// Function might check if the caller of the method + /// is the last alive holder of the segment. Therefore, completion and destruction /// of the file segment pointer must be done under the same cache mutex. void completeWithoutStateUnlocked(std::lock_guard & cache_lock); void completeBasedOnCurrentState(std::lock_guard & cache_lock, std::unique_lock & segment_lock); @@ -295,7 +290,7 @@ private: /// In general case, all file segments are owned by cache. bool is_detached = false; - bool is_downloaded{false}; + bool is_downloaded = false; std::atomic hits_count = 0; /// cache hits. std::atomic ref_count = 0; /// Used for getting snapshot state diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 913b0535358..796b93998a9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -102,6 +103,7 @@ #include #include #include +#include #include #include @@ -746,28 +748,65 @@ void Context::setPath(const String & path) shared->user_scripts_path = shared->path + "user_scripts/"; } -VolumePtr Context::setTemporaryStorage(const String & path, const String & policy_name, size_t max_size) +static void setupTmpPath(Poco::Logger * log, const std::string & path) +try +{ + LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); + + fs::create_directories(path); + + /// Clearing old temporary files. + fs::directory_iterator dir_end; + for (fs::directory_iterator it(path); it != dir_end; ++it) + { + if (it->is_regular_file() && startsWith(it->path().filename(), "tmp")) + { + LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); + fs::remove(it->path()); + } + else + LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string()); + } +} +catch (...) +{ + DB::tryLogCurrentException(log, fmt::format( + "Caught exception while setup temporary path: {}. " + "It is ok to skip this exception as cleaning old temporary files is not necessary", path)); +} + +static VolumePtr createLocalSingleDiskVolume(const std::string & path) +{ + auto disk = std::make_shared("_tmp_default", path, 0); + VolumePtr volume = std::make_shared("_tmp_default", disk, 0); + return volume; +} + +void Context::setTemporaryStoragePath(const String & path, size_t max_size) +{ + shared->tmp_path = path; + if (!shared->tmp_path.ends_with('/')) + shared->tmp_path += '/'; + + VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path); + + for (const auto & disk : volume->getDisks()) + { + setupTmpPath(shared->log, disk->getPath()); + } + + shared->temp_data_on_disk = std::make_shared(volume, nullptr, max_size); +} + +void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_size) { std::lock_guard lock(shared->storage_policies_mutex); - VolumePtr volume; - if (policy_name.empty()) - { - shared->tmp_path = path; - if (!shared->tmp_path.ends_with('/')) - shared->tmp_path += '/'; - - auto disk = std::make_shared("_tmp_default", shared->tmp_path, 0); - volume = std::make_shared("_tmp_default", disk, 0); - } - else - { - StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name); - if (tmp_policy->getVolumes().size() != 1) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name); - volume = tmp_policy->getVolume(0); - } + StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name); + if (tmp_policy->getVolumes().size() != 1) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name); + VolumePtr volume = tmp_policy->getVolume(0); if (volume->getDisks().empty()) throw Exception("No disks volume for temporary files", ErrorCodes::NO_ELEMENTS_IN_CONFIG); @@ -789,10 +828,33 @@ VolumePtr Context::setTemporaryStorage(const String & path, const String & polic "Disk '{}' ({}) is not local and can't be used for temporary files", disk_ptr->getName(), typeid(*disk_raw_ptr).name()); } + + setupTmpPath(shared->log, disk->getPath()); } - shared->temp_data_on_disk = std::make_shared(volume, max_size); - return volume; + shared->temp_data_on_disk = std::make_shared(volume, nullptr, max_size); +} + + +void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size) +{ + auto disk_ptr = getDisk(cache_disk_name); + if (!disk_ptr) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' is not found", cache_disk_name); + + const auto * disk_object_storage_ptr = dynamic_cast(disk_ptr.get()); + if (!disk_object_storage_ptr) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' does not use cache", cache_disk_name); + + auto file_cache = disk_object_storage_ptr->getCache(); + if (!file_cache) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Cache '{}' is not found", file_cache->getBasePath()); + + LOG_DEBUG(shared->log, "Using file cache ({}) for temporary files", file_cache->getBasePath()); + + shared->tmp_path = file_cache->getBasePath(); + VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path); + shared->temp_data_on_disk = std::make_shared(volume, file_cache.get(), max_size); } void Context::setFlagsPath(const String & path) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index bc89ce36edc..2c3122e4109 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -461,7 +461,9 @@ public: void addWarningMessage(const String & msg) const; - VolumePtr setTemporaryStorage(const String & path, const String & policy_name, size_t max_size); + void setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size); + void setTemporaryStoragePolicy(const String & policy_name, size_t max_size); + void setTemporaryStoragePath(const String & path, size_t max_size); using ConfigurationPtr = Poco::AutoPtr; diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index c5ae6f6c885..9d07d51bad8 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -43,24 +44,14 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size) { - DiskPtr disk; - if (max_file_size > 0) - { - auto reservation = volume->reserve(max_file_size); - if (!reservation) - throw Exception("Not enough space on temporary disk", ErrorCodes::NOT_ENOUGH_SPACE); - disk = reservation->getDisk(); - } + TemporaryFileStreamPtr tmp_stream; + if (cache) + tmp_stream = TemporaryFileStream::create(cache, header, max_file_size, this); else - { - disk = volume->getDisk(); - } - - auto tmp_file = std::make_unique(disk, current_metric_scope); + tmp_stream = TemporaryFileStream::create(volume, header, max_file_size, this); std::lock_guard lock(mutex); - TemporaryFileStreamPtr & tmp_stream = streams.emplace_back(std::make_unique(std::move(tmp_file), header, this)); - return *tmp_stream; + return *streams.emplace_back(std::move(tmp_stream)); } @@ -89,12 +80,13 @@ struct TemporaryFileStream::OutputWriter { } - void write(const Block & block) + size_t write(const Block & block) { if (finalized) throw Exception("Cannot write to finalized stream", ErrorCodes::LOGICAL_ERROR); - out_writer.write(block); + size_t written_bytes = out_writer.write(block); num_rows += block.rows(); + return written_bytes; } void finalize() @@ -155,21 +147,65 @@ struct TemporaryFileStream::InputReader NativeReader in_reader; }; -TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_) +TemporaryFileStreamPtr TemporaryFileStream::create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_) +{ + DiskPtr disk; + if (max_file_size > 0) + { + auto reservation = volume->reserve(max_file_size); + if (!reservation) + throw Exception("Not enough space on temporary disk", ErrorCodes::NOT_ENOUGH_SPACE); + disk = reservation->getDisk(); + } + else + { + disk = volume->getDisk(); + } + + auto tmp_file = std::make_unique(disk, parent_->getMetricScope()); + return std::make_unique(std::move(tmp_file), header, /* cache_placeholder */ nullptr, /* parent */ parent_); +} + +TemporaryFileStreamPtr TemporaryFileStream::create(FileCache * cache, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_) +{ + auto tmp_file = std::make_unique(fs::path(cache->getBasePath()) / "tmp"); + + auto cache_placeholder = std::make_unique(cache, tmp_file->getPath()); + cache_placeholder->reserveCapacity(max_file_size); + + return std::make_unique(std::move(tmp_file), header, std::move(cache_placeholder), parent_); +} + +TemporaryFileStream::TemporaryFileStream( + TemporaryFileHolder file_, + const Block & header_, + std::unique_ptr space_holder_, + TemporaryDataOnDisk * parent_) : parent(parent_) , header(header_) , file(std::move(file_)) + , space_holder(std::move(space_holder_)) , out_writer(std::make_unique(file->getPath(), header)) { } -void TemporaryFileStream::write(const Block & block) +size_t TemporaryFileStream::write(const Block & block) { if (!out_writer) throw Exception("Writing has been finished", ErrorCodes::LOGICAL_ERROR); + size_t block_size_in_memory = block.allocatedBytes(); + + if (space_holder) + space_holder->reserveCapacity(block_size_in_memory); + updateAllocAndCheck(); - out_writer->write(block); + + size_t bytes_written = out_writer->write(block); + if (space_holder) + space_holder->setUsed(bytes_written); + + return bytes_written; } TemporaryFileStream::Stat TemporaryFileStream::finishWriting() diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 11edc8700d2..33d1ba183b9 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -40,12 +41,12 @@ public: std::atomic uncompressed_size; }; - explicit TemporaryDataOnDiskScope(VolumePtr volume_, size_t limit_) - : volume(std::move(volume_)), limit(limit_) + explicit TemporaryDataOnDiskScope(VolumePtr volume_, FileCache * cache_, size_t limit_) + : volume(std::move(volume_)), cache(cache_), limit(limit_) {} explicit TemporaryDataOnDiskScope(TemporaryDataOnDiskScopePtr parent_, size_t limit_) - : parent(std::move(parent_)), volume(parent->volume), limit(limit_) + : parent(std::move(parent_)), volume(parent->volume), cache(parent->cache), limit(limit_) {} /// TODO: remove @@ -56,7 +57,9 @@ protected: void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta); TemporaryDataOnDiskScopePtr parent = nullptr; + VolumePtr volume; + FileCache * cache = nullptr; StatAtomic stat; size_t limit = 0; @@ -91,6 +94,7 @@ public: bool empty() const; const StatAtomic & getStat() const { return stat; } + CurrentMetrics::Value getMetricScope() const { return current_metric_scope; } private: mutable std::mutex mutex; @@ -116,9 +120,14 @@ public: size_t num_rows = 0; }; - TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_); + static TemporaryFileStreamPtr create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_); + static TemporaryFileStreamPtr create(FileCache * cache, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_); + + TemporaryFileStream(TemporaryFileHolder file_, const Block & header_, std::unique_ptr space_holder, TemporaryDataOnDisk * parent_); + + /// Returns number of written bytes + size_t write(const Block & block); - void write(const Block & block); Stat finishWriting(); bool isWriteFinished() const; @@ -142,7 +151,8 @@ private: Block header; - TemporaryFileOnDiskHolder file; + TemporaryFileHolder file; + std::unique_ptr space_holder; Stat stat; diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 22150b9f656..71557a99897 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -15,6 +15,8 @@ #include #include +#include + namespace fs = std::filesystem; fs::path caches_dir = fs::current_path() / "lru_cache_test"; @@ -517,3 +519,45 @@ TEST(FileCache, get) } } + + +TEST(FileCache, rangeWriter) +{ + if (fs::exists(cache_base_path)) + fs::remove_all(cache_base_path); + fs::create_directories(cache_base_path); + + DB::FileCacheSettings settings; + settings.max_size = 25; + settings.max_elements = 5; + settings.max_file_segment_size = 10; + + auto cache = DB::FileCache(cache_base_path, settings); + cache.initialize(); + auto key = cache.hash("key1"); + + DB::FileSegmentRangeWriter writer(&cache, key, nullptr, "", "key1"); + + std::string data(100, '\xf0'); + + /// Write first segment + ASSERT_EQ(writer.tryWrite(data.data(), 5, 0, false, false), 5); + ASSERT_EQ(writer.tryWrite(data.data(), 3, 5, false, false), 3); + ASSERT_EQ(writer.tryWrite(data.data(), 1, 8, false, false), 1); + ASSERT_EQ(writer.tryWrite(data.data(), 1, 9, false, false), 1); + + /// Second segment starts + ASSERT_EQ(writer.tryWrite(data.data(), 1, 10, false, false), 1); + ASSERT_EQ(writer.tryWrite(data.data(), 1, 11, false, false), 1); + /// Can't write 10 bytes into the rest of current segment + ASSERT_EQ(writer.tryWrite(data.data(), 10, 12, false, false), 8); + + /// Rest can be written into the next segment + ASSERT_EQ(writer.tryWrite(data.data(), 2, 20, false, false), 2); + /// Only 3 bytes left, can't write 4 and nothing should be written + ASSERT_EQ(writer.tryWrite(data.data(), 4, 22, false, false), 0); + ASSERT_EQ(writer.tryWrite(data.data(), 4, 22, false, false), 0); + ASSERT_EQ(writer.tryWrite(data.data(), 3, 22, false, false), 3); + + writer.finalize(); +} From 7a814d285432c2c96b5c4932f5ff3669fffccc89 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 23 Nov 2022 14:48:51 +0000 Subject: [PATCH 52/62] wip: temporary files evict fs cache --- src/Common/filesystemHelpers.cpp | 4 +- src/Common/filesystemHelpers.h | 4 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 17 +- src/Disks/IO/FileCachePlaceholder.cpp | 79 ++++++++ src/Disks/IO/FileCachePlaceholder.h | 75 ++----- src/Disks/TemporaryFileInPath.cpp | 14 +- src/Disks/TemporaryFileInPath.h | 16 +- src/IO/WriteBufferFromTemporaryFile.cpp | 6 +- src/IO/WriteBufferFromTemporaryFile.h | 4 +- src/Interpreters/Cache/FileCache.cpp | 25 ++- src/Interpreters/Cache/FileCache.h | 2 + src/Interpreters/Cache/FileSegment.h | 2 + src/Interpreters/TemporaryDataOnDisk.cpp | 10 +- src/Interpreters/TemporaryDataOnDisk.h | 7 +- .../tests/gtest_lru_file_cache.cpp | 189 +++++++++++++++--- src/Storages/MergeTree/MergeTask.h | 4 +- tests/integration/test_tmp_policy/test.py | 2 +- 18 files changed, 323 insertions(+), 139 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index b259d1929ea..43f88dd7faa 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -64,11 +64,11 @@ bool enoughSpaceInDirectory(const std::string & path, size_t data_size) return data_size <= free_space; } -std::unique_ptr createTemporaryFile(const std::string & folder_path) +std::unique_ptr createTemporaryFile(const std::string & folder_path) { ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal); fs::create_directories(folder_path); - return std::make_unique(folder_path); + return std::make_unique(folder_path); } #if !defined(OS_LINUX) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index fd080771b52..14ee5f54322 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -14,10 +14,10 @@ namespace fs = std::filesystem; namespace DB { -using TemporaryFile = Poco::TemporaryFile; +using PocoTemporaryFile = Poco::TemporaryFile; bool enoughSpaceInDirectory(const std::string & path, size_t data_size); -std::unique_ptr createTemporaryFile(const std::string & folder_path); +std::unique_ptr createTemporaryFile(const std::string & folder_path); // Determine what block device is responsible for specified path diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 5b5d746ab55..6b51481e833 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -951,7 +951,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } else { - LOG_TRACE(log, "No space left in cache, will continue without cache download"); + LOG_TRACE(log, "No space left in cache to reserve {} bytes, will continue without cache download", size); file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); } diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 609ef2cdaa7..1eb7ef0090e 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -101,7 +101,7 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o }); size_t reserved_size = file_segment->tryReserve(size, strict); - if (strict && reserved_size != size) + if (reserved_size == 0 || (strict && reserved_size != size)) { file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); appendFilesystemCacheLog(*file_segment); @@ -114,8 +114,6 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return 0; } - if (reserved_size == 0) - return 0; /// shrink size = reserved_size; @@ -147,10 +145,12 @@ void FileSegmentRangeWriter::finalize(bool clear) /// Note: if segments are hold by someone else, it won't be removed if (clear) { - for (auto & file_segment : file_segments) - completeFileSegment(*file_segment, FileSegment::State::SKIP_CACHE); + for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end(); ++file_segment_it) + { + completeFileSegment(**file_segment_it, FileSegment::State::SKIP_CACHE); + file_segments.erase(file_segment_it); + } finalized = true; - return; } if (file_segments.empty() || current_file_segment_it == file_segments.end()) @@ -226,9 +226,8 @@ void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment, std if (file_segment.isDetached()) return; - // if (state.has_value()) - // file_segment.setDownloadState(*state); - UNUSED(state); + if (state.has_value()) + file_segment.setDownloadState(*state); file_segment.completeWithoutState(); appendFilesystemCacheLog(file_segment); diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp index a2640aaffdc..33e8d13b648 100644 --- a/src/Disks/IO/FileCachePlaceholder.cpp +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -3,5 +3,84 @@ namespace DB { +void ISpacePlaceholder::reserveCapacity(size_t requested_capacity) +{ + chassert(used_space <= capacity); + + size_t remaining_space = capacity - used_space; + /// TODO LOG_TEST + LOG_DEBUG(&Poco::Logger::get("ISpacePlaceholder"), "reserving {} bytes (used_space: {}, capacity: {})", requested_capacity, used_space, capacity); + + if (requested_capacity <= remaining_space) + return; + + size_t capacity_to_reserve = requested_capacity - remaining_space; + reserveImpl(capacity_to_reserve); + capacity += capacity_to_reserve; +} + +void ISpacePlaceholder::setUsed(size_t size) +{ + /// TODO LOG_TEST + LOG_DEBUG(&Poco::Logger::get("ISpacePlaceholder"), "using {} bytes ({} already used, {} capacity)", size, used_space, capacity); + + if (used_space + size > capacity) + { + LOG_WARNING(&Poco::Logger::get("ISpacePlaceholder"), "Used space is greater than capacity. It may lead to not enough space error"); + reserveCapacity(size); + } + + used_space = used_space + size; + +} + +FileCachePlaceholder::FileCachePlaceholder(FileCache * cache, const String & name) + : key_name(name) + , file_cache(cache) +{ +} + + +void FileCachePlaceholder::reserveImpl(size_t requested_size) +{ + String key = fmt::format("{}_{}", key_name, cache_writers.size()); + auto cache_writer = std::make_unique(file_cache, + file_cache->hash(key), + /* cache_log_ */ nullptr, + /* query_id_ */ "", + /* source_path_ */ key); + + while (requested_size > 0) + { + size_t current_offset = cache_writer->currentOffset(); + size_t written = cache_writer->tryWrite(nullptr, requested_size, current_offset, /* is_persistent */ false, /* strict */ false); + if (written == 0) + { + cache_writer->finalize(/* clear */ true); + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, + "Cannot reserve space in file cache ({} bytes required, {} / {} bytes used, {} / {} elements used)", + requested_size, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize(), + file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); + } + requested_size -= written; + } + + cache_writers.push_back(std::move(cache_writer)); +} + +FileCachePlaceholder::~FileCachePlaceholder() +{ + try + { + for (auto & cache_writer : cache_writers) + { + cache_writer->finalize(/* clear */ true); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} } diff --git a/src/Disks/IO/FileCachePlaceholder.h b/src/Disks/IO/FileCachePlaceholder.h index 3c5c2bb1723..26c0ddc2375 100644 --- a/src/Disks/IO/FileCachePlaceholder.h +++ b/src/Disks/IO/FileCachePlaceholder.h @@ -3,6 +3,9 @@ #include #include +#include +#include + #include namespace fs = std::filesystem; @@ -10,40 +13,16 @@ namespace fs = std::filesystem; namespace DB { - namespace ErrorCodes { extern const int NOT_ENOUGH_SPACE; } - class ISpacePlaceholder { - public: - void reserveCapacity(size_t requested_capacity) - { - chassert(used_space <= capacity); - - size_t remaining_space = capacity - used_space; - if (requested_capacity <= remaining_space) - return; - - size_t capacity_to_reserve = requested_capacity - remaining_space; - reserveImpl(capacity_to_reserve); - capacity += capacity_to_reserve; - } - - void setUsed(size_t size) - { - if (used_space + size > capacity) - { - LOG_WARNING(&Poco::Logger::get("ISpacePlaceholder"), "Used space is greater than capacity. It may lead to not enough space error"); - reserveCapacity(used_space + size - capacity); - } - - used_space = used_space + size; - } + void reserveCapacity(size_t requested_capacity); + void setUsed(size_t size); virtual ~ISpacePlaceholder() = default; @@ -54,51 +33,21 @@ private: size_t used_space = 0; }; + class FileCachePlaceholder : public ISpacePlaceholder { public: - FileCachePlaceholder(FileCache * cache, const String & name) - : key(cache->hash(name)) - , directory(cache->getPathInLocalCache(key)) - , cache_writer(cache, key, nullptr, "", name) - { + FileCachePlaceholder(FileCache * cache, const String & name); - } + void reserveImpl(size_t requested_size) override; - fs::path getDirectoryPath() const - { - return directory; - } - - void reserveImpl(size_t size) override - { - while (size > 0) - { - size_t current_offset = cache_writer.currentOffset(); - size_t written = cache_writer.tryWrite(nullptr, size, current_offset, /* is_persistent */ false); - if (written == 0) - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Cannot reserve space in file cache ({} bytes required)", size); - size -= written; - } - } - - ~FileCachePlaceholder() override - { - try - { - cache_writer.finalize(/* clear */ true); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + ~FileCachePlaceholder() override; private: - FileSegment::Key key; - fs::path directory; + std::string key_name; + FileCache * file_cache; - FileSegmentRangeWriter cache_writer; + std::vector> cache_writers; }; } diff --git a/src/Disks/TemporaryFileInPath.cpp b/src/Disks/TemporaryFileInPath.cpp index b7d5d7ed85e..eae7fa66855 100644 --- a/src/Disks/TemporaryFileInPath.cpp +++ b/src/Disks/TemporaryFileInPath.cpp @@ -1,10 +1,20 @@ #include - - +#include namespace DB { +TemporaryFileInPath::TemporaryFileInPath(const String & folder_path) + : tmp_file(createTemporaryFile(folder_path)) +{ + chassert(tmp_file); +} +String TemporaryFileInPath::getPath() const +{ + return tmp_file->path(); +} + +TemporaryFileInPath::~TemporaryFileInPath() = default; } diff --git a/src/Disks/TemporaryFileInPath.h b/src/Disks/TemporaryFileInPath.h index 540711e59c7..503247e3f89 100644 --- a/src/Disks/TemporaryFileInPath.h +++ b/src/Disks/TemporaryFileInPath.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { @@ -10,18 +10,12 @@ namespace DB class TemporaryFileInPath : public ITemporaryFile { public: - explicit TemporaryFileInPath(const String & folder_path) - : tmp_file(createTemporaryFile(folder_path)) - { - chassert(tmp_file); - } - - String getPath() const override { return tmp_file->path(); } + explicit TemporaryFileInPath(const String & folder_path); + String getPath() const override; + ~TemporaryFileInPath() override; private: - std::unique_ptr tmp_file; + std::unique_ptr tmp_file; }; - - } diff --git a/src/IO/WriteBufferFromTemporaryFile.cpp b/src/IO/WriteBufferFromTemporaryFile.cpp index f93c79ca587..4562ad512b3 100644 --- a/src/IO/WriteBufferFromTemporaryFile.cpp +++ b/src/IO/WriteBufferFromTemporaryFile.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes } -WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file_) +WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file_) : WriteBufferFromFile(tmp_file_->path(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, 0600), tmp_file(std::move(tmp_file_)) {} @@ -40,11 +40,11 @@ public: return std::make_shared(fd, file_name, std::move(origin->tmp_file)); } - ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr && tmp_file_) + ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr && tmp_file_) : ReadBufferFromFile(fd_, file_name_), tmp_file(std::move(tmp_file_)) {} - std::unique_ptr tmp_file; + std::unique_ptr tmp_file; }; diff --git a/src/IO/WriteBufferFromTemporaryFile.h b/src/IO/WriteBufferFromTemporaryFile.h index 06e2911db26..a4e83b95ac6 100644 --- a/src/IO/WriteBufferFromTemporaryFile.h +++ b/src/IO/WriteBufferFromTemporaryFile.h @@ -20,11 +20,11 @@ public: ~WriteBufferFromTemporaryFile() override; private: - explicit WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file); + explicit WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file); std::shared_ptr getReadBufferImpl() override; - std::unique_ptr tmp_file; + std::unique_ptr tmp_file; friend class ReadBufferFromTemporaryWriteBuffer; }; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 56c3eef5015..59a07df8439 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -542,9 +542,6 @@ FileSegmentPtr FileCache::createFileSegmentForDownload( assertCacheCorrectness(key, cache_lock); #endif - if (size > max_file_segment_size) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Requested size exceeds max file segment size"); - auto * cell = getCell(key, offset, cache_lock); if (cell) throw Exception( @@ -1001,9 +998,17 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock fs::directory_iterator key_it{key_prefix_it->path()}; for (; key_it != fs::directory_iterator(); ++key_it) { - if (!key_it->is_directory()) + if (key_it->is_regular_file()) { - LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string()); + if (key_prefix_it->path().filename() == "tmp" && startsWith(key_it->path().filename(), "tmp")) + { + LOG_DEBUG(log, "Found temporary file '{}', will remove it", key_it->path().string()); + fs::remove(key_it->path()); + } + else + { + LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string()); + } continue; } @@ -1181,6 +1186,16 @@ size_t FileCache::getAvailableCacheSizeUnlocked(std::lock_guard & ca return max_size - getUsedCacheSizeUnlocked(cache_lock); } +size_t FileCache::getTotalMaxSize() const +{ + return max_size; +} + +size_t FileCache::getTotalMaxElements() const +{ + return max_element_size; +} + size_t FileCache::getFileSegmentsNum() const { std::lock_guard cache_lock(mutex); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 1df634b48ed..4c9c68569bd 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -89,8 +89,10 @@ public: size_t capacity() const { return max_size; } size_t getUsedCacheSize() const; + size_t getTotalMaxSize() const; size_t getFileSegmentsNum() const; + size_t getTotalMaxElements() const; static bool isReadOnly(); diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 02ae7cd5c8d..4c780eecdc3 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -308,6 +308,8 @@ struct FileSegmentsHolder : private boost::noncopyable FileSegmentsHolder(FileSegmentsHolder && other) noexcept : file_segments(std::move(other.file_segments)) {} + void reset() { file_segments.clear(); } + ~FileSegmentsHolder(); String toString(); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 9d07d51bad8..07fc3d22546 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -36,7 +36,8 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz size_t new_consumprion = stat.compressed_size + compressed_delta; if (compressed_delta > 0 && limit && new_consumprion > limit) - throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES, "Limit for temporary files size exceeded"); + throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES, + "Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumprion, limit); stat.compressed_size += compressed_delta; stat.uncompressed_size += uncompressed_delta; @@ -149,6 +150,9 @@ struct TemporaryFileStream::InputReader TemporaryFileStreamPtr TemporaryFileStream::create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_) { + if (!volume) + throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); + DiskPtr disk; if (max_file_size > 0) { @@ -173,7 +177,7 @@ TemporaryFileStreamPtr TemporaryFileStream::create(FileCache * cache, const Bloc auto cache_placeholder = std::make_unique(cache, tmp_file->getPath()); cache_placeholder->reserveCapacity(max_file_size); - return std::make_unique(std::move(tmp_file), header, std::move(cache_placeholder), parent_); + return std::make_unique(std::move(tmp_file), header, std::move(cache_placeholder), parent_); } TemporaryFileStream::TemporaryFileStream( @@ -194,7 +198,7 @@ size_t TemporaryFileStream::write(const Block & block) if (!out_writer) throw Exception("Writing has been finished", ErrorCodes::LOGICAL_ERROR); - size_t block_size_in_memory = block.allocatedBytes(); + size_t block_size_in_memory = block.bytes(); if (space_holder) space_holder->reserveCapacity(block_size_in_memory); diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 33d1ba183b9..24e7e0506bc 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -51,7 +51,12 @@ public: /// TODO: remove /// Refactor all code that uses volume directly to use TemporaryDataOnDisk. - VolumePtr getVolume() const { return volume; } + VolumePtr getVolume() const + { + if (!volume) + throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); + return volume; + } protected: void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta); diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 71557a99897..a4b446bdd90 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -14,13 +15,14 @@ #include #include #include +#include #include namespace fs = std::filesystem; +using namespace DB; -fs::path caches_dir = fs::current_path() / "lru_cache_test"; -String cache_base_path = caches_dir / "cache1" / ""; +static constexpr auto TEST_LOG_LEVEL = "debug"; void assertRange( [[maybe_unused]] size_t assert_n, DB::FileSegmentPtr file_segment, @@ -55,7 +57,7 @@ String getFileSegmentPath(const String & base_path, const DB::FileCache::Key & k return fs::path(base_path) / key_str.substr(0, 3) / key_str / DB::toString(offset); } -void download(DB::FileSegmentPtr file_segment) +void download(const std::string & cache_base_path, DB::FileSegmentPtr file_segment) { const auto & key = file_segment->key(); size_t size = file_segment->range().size(); @@ -69,30 +71,54 @@ void download(DB::FileSegmentPtr file_segment) file_segment->write(data.data(), size, file_segment->getCurrentWriteOffset()); } -void prepareAndDownload(DB::FileSegmentPtr file_segment) +void prepareAndDownload(const std::string & cache_base_path, DB::FileSegmentPtr file_segment) { - // std::cerr << "Reserving: " << file_segment->range().size() << " for: " << file_segment->range().toString() << "\n"; ASSERT_TRUE(file_segment->reserve(file_segment->range().size())); - download(file_segment); + download(cache_base_path, file_segment); } -void complete(const DB::FileSegmentsHolder & holder) +void complete(const std::string & cache_base_path, const DB::FileSegmentsHolder & holder) { for (const auto & file_segment : holder.file_segments) { ASSERT_TRUE(file_segment->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(file_segment); + prepareAndDownload(cache_base_path, file_segment); file_segment->completeWithState(DB::FileSegment::State::DOWNLOADED); } } - -TEST(FileCache, get) +class FileCacheTest : public ::testing::Test { - if (fs::exists(cache_base_path)) - fs::remove_all(cache_base_path); - fs::create_directories(cache_base_path); +public: + static void setupLogs(const std::string & level) + { + Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel(level); + } + + void SetUp() override + { + setupLogs(TEST_LOG_LEVEL); + + if (fs::exists(cache_base_path)) + fs::remove_all(cache_base_path); + fs::create_directories(cache_base_path); + } + + void TearDown() override + { + if (fs::exists(cache_base_path)) + fs::remove_all(cache_base_path); + } + + fs::path caches_dir = fs::current_path() / "lru_cache_test"; + std::string cache_base_path = caches_dir / "cache1" / ""; +}; + +TEST_F(FileCacheTest, get) +{ DB::ThreadStatus thread_status; /// To work with cache need query_id and query context. @@ -128,7 +154,7 @@ TEST(FileCache, get) ASSERT_TRUE(segments[0]->reserve(segments[0]->range().size())); assertRange(2, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADING); - download(segments[0]); + download(cache_base_path, segments[0]); segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); assertRange(3, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); } @@ -149,7 +175,7 @@ TEST(FileCache, get) assertRange(5, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::EMPTY); ASSERT_TRUE(segments[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[1]); + prepareAndDownload(cache_base_path, segments[1]); segments[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); assertRange(6, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } @@ -182,8 +208,8 @@ TEST(FileCache, get) assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } - complete(cache.getOrSet(key, 17, 4, {})); /// Get [17, 20] - complete(cache.getOrSet(key, 24, 3, {})); /// Get [24, 26] + complete(cache_base_path, cache.getOrSet(key, 17, 4, {})); /// Get [17, 20] + complete(cache_base_path, cache.getOrSet(key, 24, 3, {})); /// Get [24, 26] /// completeWithState(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] /// Current cache: [__________][_____] [____] [___][] @@ -205,7 +231,7 @@ TEST(FileCache, get) assertRange(13, segments[2], DB::FileSegment::Range(15, 16), DB::FileSegment::State::EMPTY); ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[2]); + prepareAndDownload(cache_base_path, segments[2]); segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); @@ -246,7 +272,7 @@ TEST(FileCache, get) assertRange(21, segments[3], DB::FileSegment::Range(21, 21), DB::FileSegment::State::EMPTY); ASSERT_TRUE(segments[3]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[3]); + prepareAndDownload(cache_base_path, segments[3]); segments[3]->completeWithState(DB::FileSegment::State::DOWNLOADED); ASSERT_TRUE(segments[3]->state() == DB::FileSegment::State::DOWNLOADED); @@ -269,8 +295,8 @@ TEST(FileCache, get) ASSERT_TRUE(segments[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[0]); - prepareAndDownload(segments[2]); + prepareAndDownload(cache_base_path, segments[0]); + prepareAndDownload(cache_base_path, segments[2]); segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); } @@ -292,8 +318,8 @@ TEST(FileCache, get) ASSERT_TRUE(s5[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); ASSERT_TRUE(s1[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(s5[0]); - prepareAndDownload(s1[0]); + prepareAndDownload(cache_base_path, s5[0]); + prepareAndDownload(cache_base_path, s1[0]); s5[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); s1[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); @@ -396,7 +422,7 @@ TEST(FileCache, get) cv.wait(lock, [&]{ return lets_start_download; }); } - prepareAndDownload(segments[2]); + prepareAndDownload(cache_base_path, segments[2]); segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADED); @@ -461,7 +487,7 @@ TEST(FileCache, get) ASSERT_TRUE(segments_2[1]->state() == DB::FileSegment::State::PARTIALLY_DOWNLOADED); ASSERT_TRUE(segments_2[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments_2[1]); + prepareAndDownload(cache_base_path, segments_2[1]); segments_2[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); }); @@ -520,19 +546,14 @@ TEST(FileCache, get) } - -TEST(FileCache, rangeWriter) +TEST_F(FileCacheTest, rangeWriter) { - if (fs::exists(cache_base_path)) - fs::remove_all(cache_base_path); - fs::create_directories(cache_base_path); - DB::FileCacheSettings settings; settings.max_size = 25; settings.max_elements = 5; settings.max_file_segment_size = 10; - auto cache = DB::FileCache(cache_base_path, settings); + DB::FileCache cache(cache_base_path, settings); cache.initialize(); auto key = cache.hash("key1"); @@ -561,3 +582,107 @@ TEST(FileCache, rangeWriter) writer.finalize(); } + +static Block generateBlock(size_t size = 0) +{ + Block block; + ColumnWithTypeAndName column; + column.name = "x"; + column.type = std::make_shared(); + + { + MutableColumnPtr mut_col = column.type->createColumn(); + for (size_t i = 0; i < size; ++i) + mut_col->insert(i); + column.column = std::move(mut_col); + } + + block.insert(column); + + LOG_DEBUG(&Poco::Logger::get("FileCacheTest"), "generated block {} bytes", block.bytes()); + return block; +} + +static size_t readAllTemporaryData(TemporaryFileStream & stream) +{ + Block block; + size_t read_rows = 0; + do + { + block = stream.read(); + read_rows += block.rows(); + } while (block); + return read_rows; +} + +TEST_F(FileCacheTest, temporaryData) +{ + DB::FileCacheSettings settings; + settings.max_size = 10240; + settings.max_file_segment_size = 1024; + + DB::FileCache file_cache(cache_base_path, settings); + file_cache.initialize(); + + auto tmp_data_scope = std::make_shared(nullptr, &file_cache, 0); + + auto some_data_holder = file_cache.getOrSet(file_cache.hash("some_data"), 0, 1024 * 5, CreateFileSegmentSettings{}); + + { + auto segments = fromHolder(some_data_holder); + ASSERT_EQ(segments.size(), 5); + for (auto & segment : segments) + { + ASSERT_TRUE(segment->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_TRUE(segment->reserve(segment->range().size())); + download(cache_base_path, segment); + segment->completeWithState(DB::FileSegment::State::DOWNLOADED); + } + } + + size_t size_used_before_temporary_data = file_cache.getUsedCacheSize(); + size_t segments_used_before_temporary_data = file_cache.getFileSegmentsNum(); + ASSERT_GT(size_used_before_temporary_data, 0); + ASSERT_GT(segments_used_before_temporary_data, 0); + + size_t size_used_with_temporary_data; + size_t segments_used_with_temporary_data; + { + auto tmp_data = std::make_unique(tmp_data_scope); + + auto & stream = tmp_data->createStream(generateBlock()); + + ASSERT_GT(stream.write(generateBlock(100)), 0); + + EXPECT_GT(file_cache.getUsedCacheSize(), 0); + EXPECT_GT(file_cache.getFileSegmentsNum(), 0); + + size_t used_size_before_attempt = file_cache.getUsedCacheSize(); + /// data can't be evicted because it is still held by `some_data_holder` + ASSERT_THROW(stream.write(generateBlock(1000)), DB::Exception); + + ASSERT_EQ(file_cache.getUsedCacheSize(), used_size_before_attempt); + + some_data_holder.reset(); + + stream.write(generateBlock(1011)); + + auto stat = stream.finishWriting(); + + EXPECT_EQ(stat.num_rows, 1111); + EXPECT_EQ(readAllTemporaryData(stream), 1111); + + size_used_with_temporary_data = file_cache.getUsedCacheSize(); + segments_used_with_temporary_data = file_cache.getFileSegmentsNum(); + EXPECT_GT(size_used_with_temporary_data, 0); + EXPECT_GT(segments_used_with_temporary_data, 0); + } + + /// All temp data should be evicted after removing temporary files + EXPECT_LE(file_cache.getUsedCacheSize(), size_used_with_temporary_data); + EXPECT_LE(file_cache.getFileSegmentsNum(), segments_used_with_temporary_data); + + /// Some segments reserved by `some_data_holder` was eviced by temporary data + EXPECT_LE(file_cache.getUsedCacheSize(), size_used_before_temporary_data); + EXPECT_LE(file_cache.getFileSegmentsNum(), segments_used_before_temporary_data); +} diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 6a29cdbb5ca..2b8d9055cff 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -192,7 +192,7 @@ private: bool force_ttl{false}; CompressionCodecPtr compression_codec{nullptr}; size_t sum_input_rows_upper_bound{0}; - std::unique_ptr rows_sources_file{nullptr}; + std::unique_ptr rows_sources_file{nullptr}; std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; std::unique_ptr rows_sources_write_buf{nullptr}; std::optional column_sizes{}; @@ -257,7 +257,7 @@ private: /// Begin dependencies from previous stage std::unique_ptr rows_sources_write_buf{nullptr}; std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; - std::unique_ptr rows_sources_file; + std::unique_ptr rows_sources_file; std::optional column_sizes; CompressionCodecPtr compression_codec; DiskPtr tmp_disk{nullptr}; diff --git a/tests/integration/test_tmp_policy/test.py b/tests/integration/test_tmp_policy/test.py index c919d9a0c3d..870a70b127a 100644 --- a/tests/integration/test_tmp_policy/test.py +++ b/tests/integration/test_tmp_policy/test.py @@ -23,7 +23,7 @@ def start_cluster(): cluster.shutdown() -def test_different_versions(start_cluster): +def test_disk_selection(start_cluster): query = "SELECT count(ignore(*)) FROM (SELECT * FROM system.numbers LIMIT 1e7) GROUP BY number" settings = { "max_bytes_before_external_group_by": 1 << 20, From 08b8d335f06dd35eddee434a6e583bea86351d6e Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 23 Nov 2022 17:52:00 +0000 Subject: [PATCH 53/62] add integration/test_temporary_data_in_cache --- .../test_temporary_data_in_cache/__init__.py | 0 .../config.d/storage_configuration.xml | 39 +++++++++ .../test_temporary_data_in_cache/test.py | 81 +++++++++++++++++++ 3 files changed, 120 insertions(+) create mode 100644 tests/integration/test_temporary_data_in_cache/__init__.py create mode 100644 tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml create mode 100644 tests/integration/test_temporary_data_in_cache/test.py diff --git a/tests/integration/test_temporary_data_in_cache/__init__.py b/tests/integration/test_temporary_data_in_cache/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml new file mode 100644 index 00000000000..8ccd705c6f1 --- /dev/null +++ b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml @@ -0,0 +1,39 @@ + + + + + local + /local_disk/ + + + + cache + local_disk + /tiny_local_cache/ + 10M + 1M + 1 + 0 + + + + + + local + /tiny_local_cache/ + + + + + + +
+ tiny_local_cache +
+
+
+
+
+ + tiny_local_cache +
diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py new file mode 100644 index 00000000000..ba57348ee37 --- /dev/null +++ b/tests/integration/test_temporary_data_in_cache/test.py @@ -0,0 +1,81 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name + +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/config.d/storage_configuration.xml"], + tmpfs=["/local_disk:size=50M", "/tiny_local_cache:size=12M"], +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_cache_evicted_by_temporary_data(start_cluster): + q = node.query + qi = lambda query: int(node.query(query).strip()) + + cache_size_initial = qi("SELECT sum(size) FROM system.filesystem_cache") + assert cache_size_initial == 0 + + free_space_initial = qi( + "SELECT free_space FROM system.disks WHERE name = 'tiny_local_cache_local_disk'" + ) + assert free_space_initial > 8 * 1024 * 1024 + + q( + "CREATE TABLE t1 (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" + ) + q("INSERT INTO t1 SELECT number FROM numbers(1024 * 1024)") + + # To be sure that nothing is reading the cache and entries for t1 can be evited + q("OPTIMIZE TABLE t1 FINAL") + q("SYSTEM STOP MERGES t1") + + # Read some data to fill the cache + q("SELECT sum(x) FROM t1") + + cache_size_with_t1 = qi("SELECT sum(size) FROM system.filesystem_cache") + assert cache_size_with_t1 > 8 * 1024 * 1024 + + # Almost all disk space is occupied by t1 cache + free_space_with_t1 = qi( + "SELECT free_space FROM system.disks WHERE name = 'tiny_local_cache_local_disk'" + ) + assert free_space_with_t1 < 4 * 1024 * 1024 + + # Try to sort the table, but fail because of lack of disk space + with pytest.raises(QueryRuntimeException) as exc: + q( + "SELECT ignore(*) FROM numbers(10 * 1024 * 1024) ORDER BY sipHash64(number)", + settings={ + "max_bytes_before_external_group_by": "4M", + "max_bytes_before_external_sort": "4M", + }, + ) + assert "Cannot reserve space in file cache" in str(exc.value) + + # Some data evicted from cache by temporary data + cache_size_after_eviction = qi("SELECT sum(size) FROM system.filesystem_cache") + assert cache_size_after_eviction < cache_size_with_t1 + + # Disk space freed, at least 3 MB, because temporary data tried to write 4 MB + free_space_after_eviction = qi( + "SELECT free_space FROM system.disks WHERE name = 'tiny_local_cache_local_disk'" + ) + assert free_space_after_eviction > free_space_with_t1 + 3 * 1024 * 1024 + + q("DROP TABLE IF EXISTS t1") From 816af3dc1657dd9f12d727b6b69495375293d9ce Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 23 Nov 2022 19:14:36 +0000 Subject: [PATCH 54/62] wip: temporary files evict fs cache --- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 6 +++++- .../IO/CachedOnDiskWriteBufferFromFile.h | 12 +++++++++++ src/Disks/IO/FileCachePlaceholder.cpp | 21 ++++++++++++------- src/Disks/IO/FileCachePlaceholder.h | 19 ++++++++++++----- src/Formats/NativeWriter.h | 2 ++ src/Interpreters/Cache/FileSegment.cpp | 1 + src/Interpreters/TemporaryDataOnDisk.cpp | 8 ++++++- src/Interpreters/TemporaryDataOnDisk.h | 7 +------ 8 files changed, 56 insertions(+), 20 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 1eb7ef0090e..3283c9424d2 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -114,7 +114,6 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return 0; } - /// shrink size = reserved_size; @@ -134,6 +133,11 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return size; } +size_t FileSegmentRangeWriter::tryReserve(size_t size, size_t offset, bool is_persistent, bool strict) +{ + return tryWrite(nullptr, size, offset, is_persistent, strict); +} + void FileSegmentRangeWriter::finalize(bool clear) { if (finalized) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 4f5bebb92e8..b8b7d519b78 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -34,8 +34,20 @@ public: * it until it is full and then allocate next file segment. */ bool write(const char * data, size_t size, size_t offset, bool is_persistent); + + /* Tries to write data to current file segment. + * Size of written data may be less than requested_size, because current file segment may not have enough space. + * In strict mode, if current file segment doesn't have enough space, then exception is thrown. + * + * Returns size of written data. + * If returned non zero value, then we can try to write again. + * If no space is available, returns zero. + */ size_t tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict = false); + /// Same as tryWrite, but doesn't write anything, just reserves some space in cache + size_t tryReserve(size_t size, size_t offset, bool is_persistent, bool strict = false); + void finalize(bool clear = false); size_t currentOffset() const { return current_file_segment_write_offset; } diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp index 33e8d13b648..4877d9532c2 100644 --- a/src/Disks/IO/FileCachePlaceholder.cpp +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -3,13 +3,17 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_ENOUGH_SPACE; +} + void ISpacePlaceholder::reserveCapacity(size_t requested_capacity) { chassert(used_space <= capacity); size_t remaining_space = capacity - used_space; - /// TODO LOG_TEST - LOG_DEBUG(&Poco::Logger::get("ISpacePlaceholder"), "reserving {} bytes (used_space: {}, capacity: {})", requested_capacity, used_space, capacity); + LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Reserving {} bytes (used_space: {}, capacity: {})", requested_capacity, used_space, capacity); if (requested_capacity <= remaining_space) return; @@ -21,8 +25,7 @@ void ISpacePlaceholder::reserveCapacity(size_t requested_capacity) void ISpacePlaceholder::setUsed(size_t size) { - /// TODO LOG_TEST - LOG_DEBUG(&Poco::Logger::get("ISpacePlaceholder"), "using {} bytes ({} already used, {} capacity)", size, used_space, capacity); + LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Using {} bytes ({} already used, {} capacity)", size, used_space, capacity); if (used_space + size > capacity) { @@ -31,7 +34,6 @@ void ISpacePlaceholder::setUsed(size_t size) } used_space = used_space + size; - } FileCachePlaceholder::FileCachePlaceholder(FileCache * cache, const String & name) @@ -40,9 +42,9 @@ FileCachePlaceholder::FileCachePlaceholder(FileCache * cache, const String & nam { } - void FileCachePlaceholder::reserveImpl(size_t requested_size) { + /// We create new cache_writer and will try to reserve requested_size in it String key = fmt::format("{}_{}", key_name, cache_writers.size()); auto cache_writer = std::make_unique(file_cache, file_cache->hash(key), @@ -50,10 +52,15 @@ void FileCachePlaceholder::reserveImpl(size_t requested_size) /* query_id_ */ "", /* source_path_ */ key); + /* Sometimes several calls of tryReserve are required, + * because it allocates space in current segment and its size may be less than requested_size. + * If we can't write any data, we need to free all space occupied by current reservation. + * To do this we will just destroy current cache_writer. + */ while (requested_size > 0) { size_t current_offset = cache_writer->currentOffset(); - size_t written = cache_writer->tryWrite(nullptr, requested_size, current_offset, /* is_persistent */ false, /* strict */ false); + size_t written = cache_writer->tryReserve(requested_size, current_offset, /* is_persistent */ false, /* strict */ false); if (written == 0) { cache_writer->finalize(/* clear */ true); diff --git a/src/Disks/IO/FileCachePlaceholder.h b/src/Disks/IO/FileCachePlaceholder.h index 26c0ddc2375..589aec289fe 100644 --- a/src/Disks/IO/FileCachePlaceholder.h +++ b/src/Disks/IO/FileCachePlaceholder.h @@ -13,15 +13,21 @@ namespace fs = std::filesystem; namespace DB { -namespace ErrorCodes -{ - extern const int NOT_ENOUGH_SPACE; -} +/* ISpacePlaceholder is a base class for all classes that need to reserve space in some storage. + * You should resrve space with call reserveCapacity() before writing to it. + * After writing you should call setUsed() to let ISpacePlaceholder know how much space was used. + * It can be different because in some cases you don't know exact size of data you will write (because of compression, for example). + * It's better to reserve more space in advance not to overuse space. + */ class ISpacePlaceholder { public: + /// Reserve space in storage void reserveCapacity(size_t requested_capacity); + + /// Indicate that some space is used + /// It uses reserved space if it is possible, otherwise it reserves more space void setUsed(size_t size); virtual ~ISpacePlaceholder() = default; @@ -33,7 +39,9 @@ private: size_t used_space = 0; }; - +/* FileCachePlaceholder is a class that reserves space in FileCache. + * Data is written externally, and FileCachePlaceholder is only used to hold space in FileCache. + */ class FileCachePlaceholder : public ISpacePlaceholder { public: @@ -47,6 +55,7 @@ private: std::string key_name; FileCache * file_cache; + /// On each reserveImpl() call we create new FileSegmentRangeWriter that would be hold space std::vector> cache_writers; }; diff --git a/src/Formats/NativeWriter.h b/src/Formats/NativeWriter.h index eba7ac8443d..7bb377d2e4a 100644 --- a/src/Formats/NativeWriter.h +++ b/src/Formats/NativeWriter.h @@ -27,6 +27,8 @@ public: IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0); Block getHeader() const { return header; } + + /// Returns the number of bytes written. size_t write(const Block & block); void flush(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 73cbaaaca57..36600f1fca4 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -325,6 +325,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) try { + /// if `from` is nullptr, then we just allocate and hold space by current segment and it was (or would) be written outside if (cache_writer && from != nullptr) cache_writer->write(from, size); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 07fc3d22546..a039053b011 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -43,6 +43,13 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz stat.uncompressed_size += uncompressed_delta; } +VolumePtr TemporaryDataOnDiskScope::getVolume() const +{ + if (!volume) + throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); + return volume; +} + TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size) { TemporaryFileStreamPtr tmp_stream; @@ -55,7 +62,6 @@ TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, si return *streams.emplace_back(std::move(tmp_stream)); } - std::vector TemporaryDataOnDisk::getStreams() const { std::vector res; diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 24e7e0506bc..2790529754f 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -51,12 +51,7 @@ public: /// TODO: remove /// Refactor all code that uses volume directly to use TemporaryDataOnDisk. - VolumePtr getVolume() const - { - if (!volume) - throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); - return volume; - } + VolumePtr getVolume() const; protected: void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta); From 35c4e8ac842cb229a3124a87965cf057dee21413 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 25 Nov 2022 20:46:56 +0000 Subject: [PATCH 55/62] wip: temporary files evict fs cache --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 5 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 52 +++++++++++++------ .../IO/CachedOnDiskWriteBufferFromFile.h | 32 +++++++----- src/Disks/IO/FileCachePlaceholder.cpp | 25 +++------ src/Interpreters/Cache/FileCache.cpp | 29 ++++++++--- src/Interpreters/Cache/FileCache.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 29 +++++++++-- src/Interpreters/Cache/FileSegment.h | 43 +++++++++++++-- .../tests/gtest_lru_file_cache.cpp | 34 ++++++------ .../System/StorageSystemFilesystemCache.cpp | 7 +-- 10 files changed, 171 insertions(+), 87 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 6b51481e833..0dd40e7f153 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -118,10 +118,7 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) } else { - CreateFileSegmentSettings create_settings{ - .is_persistent = is_persistent - }; - + CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular); file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings)); } diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 3283c9424d2..c9960e75c16 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -51,14 +51,31 @@ FileSegmentRangeWriter::FileSegmentRangeWriter( { } -bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, bool is_persistent) +bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind) { - size_t written_size = tryWrite(data, size, offset, is_persistent, true); - chassert(written_size == 0 || written_size == size); + size_t written_size = tryWrite(data, size, offset, segment_kind, true); return written_size == size; } -size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict) +size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict) +{ + size_t total_written_size = 0; + while (size > 0) + { + size_t written_size = tryWriteImpl(data, size, offset, segment_kind, strict); + chassert(written_size <= size); + if (written_size == 0) + break; + + data += written_size; + size -= written_size; + offset += written_size; + total_written_size += written_size; + } + return total_written_size; +} + +size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict) { if (finalized) return 0; @@ -67,7 +84,7 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o if (current_file_segment_it == file_segments.end()) { - current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent); + current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind); } else { @@ -85,7 +102,7 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o if (file_segment->range().size() == file_segment->getDownloadedSize()) { completeFileSegment(*file_segment); - current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent); + current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind); } } @@ -103,8 +120,11 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o size_t reserved_size = file_segment->tryReserve(size, strict); if (reserved_size == 0 || (strict && reserved_size != size)) { - file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); - appendFilesystemCacheLog(*file_segment); + if (strict) + { + file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); + appendFilesystemCacheLog(*file_segment); + } LOG_DEBUG( &Poco::Logger::get("FileSegmentRangeWriter"), @@ -114,7 +134,7 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return 0; } - /// shrink + /// Shrink to reserved size, because we can't write more than reserved size = reserved_size; try @@ -133,9 +153,9 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return size; } -size_t FileSegmentRangeWriter::tryReserve(size_t size, size_t offset, bool is_persistent, bool strict) +bool FileSegmentRangeWriter::reserve(size_t size, size_t offset) { - return tryWrite(nullptr, size, offset, is_persistent, strict); + return write(nullptr, size, offset, FileSegmentKind::Temporary); } void FileSegmentRangeWriter::finalize(bool clear) @@ -177,7 +197,7 @@ FileSegmentRangeWriter::~FileSegmentRangeWriter() } } -FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, bool is_persistent) +FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSegmentKind segment_kind) { /** * Allocate a new file segment starting `offset`. @@ -186,10 +206,7 @@ FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset std::lock_guard cache_lock(cache->mutex); - CreateFileSegmentSettings create_settings - { - .is_persistent = is_persistent, - }; + CreateFileSegmentSettings create_settings(segment_kind); /// We set max_file_segment_size to be downloaded, /// if we have less size to write, file segment will be resized in complete() method. @@ -301,7 +318,8 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size) try { - if (!cache_writer->write(data, size, current_download_offset, is_persistent_cache_file)) + auto segment_kind = is_persistent_cache_file ? FileSegmentKind::Persistent : FileSegmentKind::Regular; + if (!cache_writer->write(data, size, current_download_offset, segment_kind)) { LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted"); return; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index b8b7d519b78..bb63e727ec8 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -29,24 +29,23 @@ public: FileCache * cache_, const FileSegment::Key & key_, std::shared_ptr cache_log_, const String & query_id_, const String & source_path_); - /** - * Write a range of file segments. Allocate file segment of `max_file_segment_size` and write to - * it until it is full and then allocate next file segment. - */ - bool write(const char * data, size_t size, size_t offset, bool is_persistent); + /* Write a range of file segments. + * Allocate file segment of `max_file_segment_size` and write to it until it is full and then allocate next file segment. + * If it's impossible to allocate new file segment and reserve space to write all data, then returns false. + * + * Note: the data that was written to file segments before the error occurred is not rolled back. + */ + bool write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind); /* Tries to write data to current file segment. - * Size of written data may be less than requested_size, because current file segment may not have enough space. - * In strict mode, if current file segment doesn't have enough space, then exception is thrown. + * Size of written data may be less than requested_size, because it may not be enough space. * * Returns size of written data. - * If returned non zero value, then we can try to write again. - * If no space is available, returns zero. */ - size_t tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict = false); + size_t tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind = FileSegmentKind::Regular, bool strict = false); - /// Same as tryWrite, but doesn't write anything, just reserves some space in cache - size_t tryReserve(size_t size, size_t offset, bool is_persistent, bool strict = false); + /// Same as `write`, but doesn't write anything, just reserves some space in cache + bool reserve(size_t size, size_t offset); void finalize(bool clear = false); @@ -55,12 +54,19 @@ public: ~FileSegmentRangeWriter(); private: - FileSegments::iterator allocateFileSegment(size_t offset, bool is_persistent); + FileSegments::iterator allocateFileSegment(size_t offset, FileSegmentKind segment_kind); void appendFilesystemCacheLog(const FileSegment & file_segment); void completeFileSegment(FileSegment & file_segment, std::optional state = {}); + /* Writes data to current file segment as much as possible and returns size of written data, do not allocate new file segments + * In `strict` mode it will write all data or nothing, otherwise it will write as much as possible + * If returned non zero value, then we can try to write again to next file segment. + * If no space is available, returns zero. + */ + size_t tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict); + FileCache * cache; FileSegment::Key key; diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp index 4877d9532c2..63dc4ffb1c2 100644 --- a/src/Disks/IO/FileCachePlaceholder.cpp +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -52,24 +52,15 @@ void FileCachePlaceholder::reserveImpl(size_t requested_size) /* query_id_ */ "", /* source_path_ */ key); - /* Sometimes several calls of tryReserve are required, - * because it allocates space in current segment and its size may be less than requested_size. - * If we can't write any data, we need to free all space occupied by current reservation. - * To do this we will just destroy current cache_writer. - */ - while (requested_size > 0) + size_t current_offset = cache_writer->currentOffset(); + bool is_reserved = cache_writer->reserve(requested_size, current_offset); + if (!is_reserved) { - size_t current_offset = cache_writer->currentOffset(); - size_t written = cache_writer->tryReserve(requested_size, current_offset, /* is_persistent */ false, /* strict */ false); - if (written == 0) - { - cache_writer->finalize(/* clear */ true); - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, - "Cannot reserve space in file cache ({} bytes required, {} / {} bytes used, {} / {} elements used)", - requested_size, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize(), - file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); - } - requested_size -= written; + cache_writer->finalize(/* clear */ true); + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, + "Cannot reserve space in file cache ({} bytes required, {} / {} bytes used, {} / {} elements used)", + requested_size, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize(), + file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); } cache_writers.push_back(std::move(cache_writer)); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 59a07df8439..e2a0e839f19 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -48,13 +48,27 @@ FileCache::Key FileCache::hash(const String & path) return Key(sipHash128(path.data(), path.size())); } -String FileCache::getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const +String FileCache::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const { + String file_suffix; + switch (segment_kind) + { + case FileSegmentKind::Persistent: + file_suffix = "_persistent"; + break; + case FileSegmentKind::Temporary: + file_suffix = "_temporary"; + break; + case FileSegmentKind::Regular: + file_suffix = ""; + break; + } + auto key_str = key.toString(); return fs::path(cache_base_path) / key_str.substr(0, 3) / key_str - / (std::to_string(offset) + (is_persistent ? "_persistent" : "")); + / (std::to_string(offset) + file_suffix); } String FileCache::getPathInLocalCache(const Key & key) const @@ -1025,14 +1039,17 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock auto offset_with_suffix = offset_it->path().filename().string(); auto delim_pos = offset_with_suffix.find('_'); bool parsed; - bool is_persistent = false; + FileSegmentKind segment_kind = FileSegmentKind::Regular; if (delim_pos == std::string::npos) parsed = tryParse(offset, offset_with_suffix); else { parsed = tryParse(offset, offset_with_suffix.substr(0, delim_pos)); - is_persistent = offset_with_suffix.substr(delim_pos+1) == "persistent"; + if (offset_with_suffix.substr(delim_pos+1) == "persistent") + segment_kind = FileSegmentKind::Persistent; + if (offset_with_suffix.substr(delim_pos+1) == "temporary") + segment_kind = FileSegmentKind::Temporary; } if (!parsed) @@ -1052,7 +1069,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock { auto * cell = addCell( key, offset, size, FileSegment::State::DOWNLOADED, - CreateFileSegmentSettings{ .is_persistent = is_persistent }, cache_lock); + CreateFileSegmentSettings(segment_kind), cache_lock); if (cell) queue_entries.emplace_back(cell->queue_iterator, cell->file_segment); @@ -1164,7 +1181,7 @@ std::vector FileCache::tryGetCachePaths(const Key & key) for (const auto & [offset, cell] : cells_by_offset) { if (cell.file_segment->state() == FileSegment::State::DOWNLOADED) - cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->isPersistent())); + cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->getKind())); } return cache_paths; diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 4c9c68569bd..4a2610fd76b 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -80,7 +80,7 @@ public: static Key hash(const String & path); - String getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const; + String getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const; String getPathInLocalCache(const Key & key) const; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 36600f1fca4..19ea6784c02 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -23,6 +23,19 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +String toString(FileSegmentKind type) +{ + switch (type) + { + case FileSegmentKind::Regular: + return "Regular"; + case FileSegmentKind::Persistent: + return "Persistent"; + case FileSegmentKind::Temporary: + return "Temporary"; + } +} + FileSegment::FileSegment( size_t offset_, size_t size_, @@ -39,7 +52,7 @@ FileSegment::FileSegment( #else , log(&Poco::Logger::get("FileSegment")) #endif - , is_persistent(settings.is_persistent) + , segment_kind(settings.type) { /// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING. switch (download_state) @@ -73,7 +86,7 @@ FileSegment::FileSegment( String FileSegment::getPathInLocalCache() const { - return cache->getPathInLocalCache(key(), offset(), isPersistent()); + return cache->getPathInLocalCache(key(), offset(), segment_kind); } FileSegment::State FileSegment::state() const @@ -99,7 +112,6 @@ size_t FileSegment::getFirstNonDownloadedOffsetUnlocked(std::unique_lock & cach resetDownloaderUnlocked(segment_lock); } + if (segment_kind == FileSegmentKind::Temporary && is_last_holder) + { + cache->remove(key(), offset(), cache_lock, segment_lock); + LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock)); + return; + } + switch (download_state) { case State::SKIP_CACHE: @@ -642,7 +661,7 @@ String FileSegment::getInfoForLogUnlocked(std::unique_lock & segment info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(segment_lock) << ", "; info << "caller id: " << getCallerId() << ", "; info << "detached: " << is_detached << ", "; - info << "persistent: " << is_persistent; + info << "kind: " << toString(segment_kind); return info.str(); } @@ -737,7 +756,7 @@ FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std snapshot->ref_count = file_segment.use_count(); snapshot->downloaded_size = file_segment->getDownloadedSizeUnlocked(segment_lock); snapshot->download_state = file_segment->download_state; - snapshot->is_persistent = file_segment->isPersistent(); + snapshot->segment_kind = file_segment->getKind(); return snapshot; } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 4c780eecdc3..8915b2f0a36 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -30,9 +30,38 @@ using FileSegmentPtr = std::shared_ptr; using FileSegments = std::list; +/* + * FileSegmentKind is used to specify the eviction policy for file segments. + */ +enum class FileSegmentKind +{ + /* `Regular` file segment is still in cache after usage, and can be evicted + * (unless there're some holders). + */ + Regular, + + /* `Persistent` file segment can't be evicted from cache, + * it should be removed manually. + */ + Persistent, + + /* `Temporary` file segment is removed right after relesing. + * Also corresponding files are removed during cache loading (if any). + */ + Temporary, +}; + +String toString(FileSegmentKind type); + struct CreateFileSegmentSettings { - bool is_persistent = false; + FileSegmentKind type = FileSegmentKind::Regular; + + CreateFileSegmentSettings() = default; + + explicit CreateFileSegmentSettings(FileSegmentKind type_) + : type(type_) + {} }; class FileSegment : private boost::noncopyable, public std::enable_shared_from_this @@ -127,7 +156,8 @@ public: size_t offset() const { return range().left; } - bool isPersistent() const { return is_persistent; } + FileSegmentKind getKind() const { return segment_kind; } + bool isPersistent() const { return segment_kind == FileSegmentKind::Persistent; } using UniqueId = std::pair; UniqueId getUniqueId() const { return std::pair(key(), offset()); } @@ -188,8 +218,13 @@ public: */ /// Try to reserve exactly `size` bytes. + /// Returns true if reservation was successful, false otherwise. bool reserve(size_t size_to_reserve); - size_t tryReserve(size_t size_to_reserve, bool strict); + + /// Try to reserve at max `size` bytes. + /// Returns actual size reserved. + /// In strict mode throws an error on attempt to reserve space too much space + size_t tryReserve(size_t size_to_reserve, bool strict = false); /// Write data into reserved space. void write(const char * from, size_t size, size_t offset); @@ -295,7 +330,7 @@ private: std::atomic hits_count = 0; /// cache hits. std::atomic ref_count = 0; /// Used for getting snapshot state - bool is_persistent; + FileSegmentKind segment_kind; CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments}; }; diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index a4b446bdd90..cb66aeafa0a 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -100,7 +100,10 @@ public: void SetUp() override { - setupLogs(TEST_LOG_LEVEL); + if(const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) + setupLogs(test_log_level); + else + setupLogs(TEST_LOG_LEVEL); if (fs::exists(cache_base_path)) fs::remove_all(cache_base_path); @@ -561,24 +564,21 @@ TEST_F(FileCacheTest, rangeWriter) std::string data(100, '\xf0'); - /// Write first segment - ASSERT_EQ(writer.tryWrite(data.data(), 5, 0, false, false), 5); - ASSERT_EQ(writer.tryWrite(data.data(), 3, 5, false, false), 3); - ASSERT_EQ(writer.tryWrite(data.data(), 1, 8, false, false), 1); - ASSERT_EQ(writer.tryWrite(data.data(), 1, 9, false, false), 1); + size_t total_written = 0; + for (const size_t size : {3, 5, 8, 1, 1, 3}) + { + total_written += size; + ASSERT_EQ(writer.tryWrite(data.data(), size, writer.currentOffset()), size); + } + ASSERT_LT(total_written, settings.max_size); - /// Second segment starts - ASSERT_EQ(writer.tryWrite(data.data(), 1, 10, false, false), 1); - ASSERT_EQ(writer.tryWrite(data.data(), 1, 11, false, false), 1); - /// Can't write 10 bytes into the rest of current segment - ASSERT_EQ(writer.tryWrite(data.data(), 10, 12, false, false), 8); + size_t offset_before_unsuccessful_write = writer.currentOffset(); + size_t space_left = settings.max_size - total_written; + ASSERT_EQ(writer.tryWrite(data.data(), space_left + 1, writer.currentOffset()), 0); - /// Rest can be written into the next segment - ASSERT_EQ(writer.tryWrite(data.data(), 2, 20, false, false), 2); - /// Only 3 bytes left, can't write 4 and nothing should be written - ASSERT_EQ(writer.tryWrite(data.data(), 4, 22, false, false), 0); - ASSERT_EQ(writer.tryWrite(data.data(), 4, 22, false, false), 0); - ASSERT_EQ(writer.tryWrite(data.data(), 3, 22, false, false), 3); + ASSERT_EQ(writer.currentOffset(), offset_before_unsuccessful_write); + + ASSERT_EQ(writer.tryWrite(data.data(), space_left, writer.currentOffset()), space_left); writer.finalize(); } diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index cd9324b3253..5220a2a59e6 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -24,7 +24,8 @@ NamesAndTypesList StorageSystemFilesystemCache::getNamesAndTypes() {"cache_hits", std::make_shared()}, {"references", std::make_shared()}, {"downloaded_size", std::make_shared()}, - {"persistent", std::make_shared>()} + {"persistent", std::make_shared>()}, + {"kind", std::make_shared()}, }; } @@ -45,8 +46,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex for (const auto & file_segment : file_segments) { res_columns[0]->insert(cache_base_path); - res_columns[1]->insert( - cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->isPersistent())); + res_columns[1]->insert(file_segment->getPathInLocalCache()); const auto & range = file_segment->range(); res_columns[2]->insert(range.left); @@ -57,6 +57,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex res_columns[7]->insert(file_segment->getRefCount()); res_columns[8]->insert(file_segment->getDownloadedSize()); res_columns[9]->insert(file_segment->isPersistent()); + res_columns[10]->insert(toString(file_segment->getKind())); } } } From 1da1adc88475eb9e288e144143bb81dba8363477 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 29 Nov 2022 11:13:14 +0000 Subject: [PATCH 56/62] Better finalization in FileSegmentRangeWriter --- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 23 ++++++-------- .../IO/CachedOnDiskWriteBufferFromFile.h | 5 +-- src/Disks/IO/FileCachePlaceholder.cpp | 31 ++++++------------- src/Disks/IO/FileCachePlaceholder.h | 3 +- src/Interpreters/Cache/FileSegment.cpp | 4 ++- 5 files changed, 26 insertions(+), 40 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index c9960e75c16..b11edd7e701 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -67,7 +67,9 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o if (written_size == 0) break; - data += written_size; + if (data) + data += written_size; + size -= written_size; offset += written_size; total_written_size += written_size; @@ -158,25 +160,18 @@ bool FileSegmentRangeWriter::reserve(size_t size, size_t offset) return write(nullptr, size, offset, FileSegmentKind::Temporary); } -void FileSegmentRangeWriter::finalize(bool clear) +size_t FileSegmentRangeWriter::tryReserve(size_t size, size_t offset) +{ + return tryWrite(nullptr, size, offset, FileSegmentKind::Temporary); +} + +void FileSegmentRangeWriter::finalize() { if (finalized) return; auto & file_segments = file_segments_holder.file_segments; - /// Set all segments state to SKIP_CACHE to remove it from cache immediately on complete - /// Note: if segments are hold by someone else, it won't be removed - if (clear) - { - for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end(); ++file_segment_it) - { - completeFileSegment(**file_segment_it, FileSegment::State::SKIP_CACHE); - file_segments.erase(file_segment_it); - } - finalized = true; - } - if (file_segments.empty() || current_file_segment_it == file_segments.end()) return; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index bb63e727ec8..38ec2b46a5d 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -44,10 +44,11 @@ public: */ size_t tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind = FileSegmentKind::Regular, bool strict = false); - /// Same as `write`, but doesn't write anything, just reserves some space in cache + /// Same as `write/tryWrite`, but doesn't write anything, just reserves some space in cache bool reserve(size_t size, size_t offset); + size_t tryReserve(size_t size, size_t offset); - void finalize(bool clear = false); + void finalize(); size_t currentOffset() const { return current_file_segment_write_offset; } diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp index 63dc4ffb1c2..d45d7b2d83d 100644 --- a/src/Disks/IO/FileCachePlaceholder.cpp +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -53,32 +53,21 @@ void FileCachePlaceholder::reserveImpl(size_t requested_size) /* source_path_ */ key); size_t current_offset = cache_writer->currentOffset(); - bool is_reserved = cache_writer->reserve(requested_size, current_offset); - if (!is_reserved) + size_t reserved_size = cache_writer->tryReserve(requested_size, current_offset); + if (reserved_size != requested_size) { - cache_writer->finalize(/* clear */ true); throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, - "Cannot reserve space in file cache ({} bytes required, {} / {} bytes used, {} / {} elements used)", - requested_size, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize(), - file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); + "Cannot reserve space in file cache " + "({} bytes required, got {} reserved " + "{} / {} bytes used, " + "{} / {} elements used)" + , requested_size, reserved_size + , file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize() + , file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); } - + /// Add to cache_writers only if we successfully reserved space, otherwise free reserved_size back cache_writers.push_back(std::move(cache_writer)); } -FileCachePlaceholder::~FileCachePlaceholder() -{ - try - { - for (auto & cache_writer : cache_writers) - { - cache_writer->finalize(/* clear */ true); - } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} } diff --git a/src/Disks/IO/FileCachePlaceholder.h b/src/Disks/IO/FileCachePlaceholder.h index 589aec289fe..6ddeb85286b 100644 --- a/src/Disks/IO/FileCachePlaceholder.h +++ b/src/Disks/IO/FileCachePlaceholder.h @@ -49,13 +49,12 @@ public: void reserveImpl(size_t requested_size) override; - ~FileCachePlaceholder() override; - private: std::string key_name; FileCache * file_cache; /// On each reserveImpl() call we create new FileSegmentRangeWriter that would be hold space + /// It's required to easily release already reserved space on unsuccessful attempt std::vector> cache_writers; }; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 19ea6784c02..e098a994ba2 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -575,8 +575,10 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach if (segment_kind == FileSegmentKind::Temporary && is_last_holder) { - cache->remove(key(), offset(), cache_lock, segment_lock); LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock)); + detach(cache_lock, segment_lock); + setDownloadState(State::SKIP_CACHE); + cache->remove(key(), offset(), cache_lock, segment_lock); return; } From 9bdf01a55cd85bdc8db42f3eb0c44fc6902c67ae Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 30 Nov 2022 12:58:52 +0000 Subject: [PATCH 57/62] Do not use in FileSegment::getPathInLocalCache in StorageSystemFilesystemCache --- src/Interpreters/Cache/FileSegment.cpp | 1 + src/Interpreters/tests/gtest_lru_file_cache.cpp | 2 +- src/Storages/System/StorageSystemFilesystemCache.cpp | 6 +++++- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index e098a994ba2..e070317e454 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -86,6 +86,7 @@ FileSegment::FileSegment( String FileSegment::getPathInLocalCache() const { + chassert(cache); return cache->getPathInLocalCache(key(), offset(), segment_kind); } diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index cb66aeafa0a..2d408bd9b34 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -100,7 +100,7 @@ public: void SetUp() override { - if(const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) + if(const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) // NOLINT(concurrency-mt-unsafe) setupLogs(test_log_level); else setupLogs(TEST_LOG_LEVEL); diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index 5220a2a59e6..bec92a60436 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -46,7 +46,11 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex for (const auto & file_segment : file_segments) { res_columns[0]->insert(cache_base_path); - res_columns[1]->insert(file_segment->getPathInLocalCache()); + + /// Do not use `file_segment->getPathInLocalCache` here because it will lead to nullptr dereference + /// (because file_segments in getSnapshot doesn't have `cache` field set) + res_columns[1]->insert( + cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->getKind())); const auto & range = file_segment->range(); res_columns[2]->insert(range.left); From 5d7cb6b85a75eb2e4db9305e113c8ed88a5ba753 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Dec 2022 13:01:35 +0100 Subject: [PATCH 58/62] Change email for robot-clickhouse to immutable one --- docs/tools/release.sh | 2 +- tests/ci/cherry_pick.py | 2 +- tests/ci/style_check.py | 25 ++++++++++--------------- 3 files changed, 12 insertions(+), 17 deletions(-) diff --git a/docs/tools/release.sh b/docs/tools/release.sh index 1d344457bf1..67499631baa 100755 --- a/docs/tools/release.sh +++ b/docs/tools/release.sh @@ -19,7 +19,7 @@ then # Will make a repository with website content as the only commit. git init git remote add origin "${GIT_PROD_URI}" - git config user.email "robot-clickhouse@clickhouse.com" + git config user.email "robot-clickhouse@users.noreply.github.com" git config user.name "robot-clickhouse" # Add files. diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index d1c9d3d394c..9e7b250db75 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -79,7 +79,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise self.backport_pr = None # type: Optional[PullRequest] self._backported = None # type: Optional[bool] self.git_prefix = ( # All commits to cherrypick are done as robot-clickhouse - "git -c user.email=robot-clickhouse@clickhouse.com " + "git -c user.email=robot-clickhouse@users.noreply.github.com " "-c user.name=robot-clickhouse -c commit.gpgsign=false" ) self.pre_check() diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 23a1dd467d7..a4bf5d54105 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -28,6 +28,13 @@ from upload_result_helper import upload_results NAME = "Style Check" +GIT_PREFIX = ( # All commits to remote are done as robot-clickhouse + "git -c user.email=robot-clickhouse@users.noreply.github.com " + "-c user.name=robot-clickhouse -c commit.gpgsign=false " + "-c core.sshCommand=" + "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" +) + def process_result(result_folder): test_results = [] @@ -89,14 +96,8 @@ def checkout_head(pr_info: PRInfo): # We can't push to forks, sorry folks return remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] - git_prefix = ( # All commits to remote are done as robot-clickhouse - "git -c user.email=robot-clickhouse@clickhouse.com " - "-c user.name=robot-clickhouse -c commit.gpgsign=false " - "-c core.sshCommand=" - "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" - ) fetch_cmd = ( - f"{git_prefix} fetch --depth=1 " + f"{GIT_PREFIX} fetch --depth=1 " f"{remote_url} {pr_info.head_ref}:head-{pr_info.head_ref}" ) if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): @@ -118,15 +119,9 @@ def commit_push_staged(pr_info: PRInfo): if not git_staged: return remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] - git_prefix = ( # All commits to remote are done as robot-clickhouse - "git -c user.email=robot-clickhouse@clickhouse.com " - "-c user.name=robot-clickhouse -c commit.gpgsign=false " - "-c core.sshCommand=" - "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" - ) - git_runner(f"{git_prefix} commit -m 'Automatic style fix'") + git_runner(f"{GIT_PREFIX} commit -m 'Automatic style fix'") push_cmd = ( - f"{git_prefix} push {remote_url} head-{pr_info.head_ref}:{pr_info.head_ref}" + f"{GIT_PREFIX} push {remote_url} head-{pr_info.head_ref}:{pr_info.head_ref}" ) if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): From 6388bc5000043082f445119f4f43bd1e830542b4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 1 Dec 2022 13:36:40 +0100 Subject: [PATCH 59/62] Followup fixes for XML helpers (#43845) --- src/Common/XMLUtils.cpp | 62 ++++++++++++++++++++++++++++++----------- src/Common/XMLUtils.h | 29 ++++++++++++------- 2 files changed, 65 insertions(+), 26 deletions(-) diff --git a/src/Common/XMLUtils.cpp b/src/Common/XMLUtils.cpp index 3d15400461e..db84e00adce 100644 --- a/src/Common/XMLUtils.cpp +++ b/src/Common/XMLUtils.cpp @@ -42,15 +42,15 @@ public: return s; } - template + template static ValueType getValue(const Node * node, const std::string & path, - const std::optional & default_value, const ParseFunction & parse_function) + const ValueType & default_value, const ParseFunction & parse_function) { const auto * value_node = node->getNodeByPath(path); if (!value_node) { - if (default_value) - return *default_value; + if constexpr (ReturnDefault) + return default_value; else throw Poco::NotFoundException(path); } @@ -59,34 +59,64 @@ public: }; -std::string getString(const Node * node, const std::string & path, const std::optional & default_value) +std::string getString(const Node * node, const std::string & path) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseString); + return ParseHelper::getValue(node, path, {}, ParseHelper::parseString); } -Int64 getInt64(const Node * node, const std::string & path, const std::optional & default_value) +std::string getString(const Node * node, const std::string & path, const std::string & default_value) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseInt64); + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseString); } -UInt64 getUInt64(const Node * node, const std::string & path, const std::optional & default_value) +Int64 getInt64(const Node * node, const std::string & path) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseUInt64); + return ParseHelper::getValue(node, path, {}, ParseHelper::parseInt64); } -int getInt(const Node * node, const std::string & path, const std::optional & default_value) +Int64 getInt64(const Node * node, const std::string & path, Int64 default_value) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseInt); + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseInt64); } -unsigned getUInt(const Node * node, const std::string & path, const std::optional & default_value) +UInt64 getUInt64(const Node * node, const std::string & path) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseUInt); + return ParseHelper::getValue(node, path, {}, ParseHelper::parseUInt64); } -bool getBool(const Node * node, const std::string & path, const std::optional & default_value) +UInt64 getUInt64(const Node * node, const std::string & path, UInt64 default_value) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseBool); + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseUInt64); +} + +int getInt(const Node * node, const std::string & path) +{ + return ParseHelper::getValue(node, path, {}, ParseHelper::parseInt); +} + +int getInt(const Node * node, const std::string & path, int default_value) +{ + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseInt); +} + +unsigned getUInt(const Node * node, const std::string & path) +{ + return ParseHelper::getValue(node, path, {}, ParseHelper::parseUInt); +} + +unsigned getUInt(const Node * node, const std::string & path, unsigned default_value) +{ + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseUInt); +} + +bool getBool(const Node * node, const std::string & path) +{ + return ParseHelper::getValue(node, path, {}, ParseHelper::parseBool); +} + +bool getBool(const Node * node, const std::string & path, bool default_value) +{ + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseBool); } } diff --git a/src/Common/XMLUtils.h b/src/Common/XMLUtils.h index 24efc691704..af9613b67ad 100644 --- a/src/Common/XMLUtils.h +++ b/src/Common/XMLUtils.h @@ -7,17 +7,26 @@ namespace DB:: XMLUtils { +/// Returns root element of the document. Poco::XML::Node * getRootNode(Poco::XML::Document * document); -std::string getString(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); +/// Finds the element in the node's subtree by the specified path and returns its inner text +/// trying to parse it as the requested type. +/// Throws an exception if path is not found. +std::string getString(const Poco::XML::Node * node, const std::string & path); +Int64 getInt64(const Poco::XML::Node * node, const std::string & path); +UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path); +int getInt(const Poco::XML::Node * node, const std::string & path); +unsigned getUInt(const Poco::XML::Node * node, const std::string & path); +bool getBool(const Poco::XML::Node * node, const std::string & path); -Int64 getInt64(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); - -UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); - -int getInt(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); - -unsigned getUInt(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); - -bool getBool(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); +/// Finds the element in the node's subtree by the specified path and returns its inner text +/// trying to parse it as the requested type. +/// Returns the specified default value if path is not found. +std::string getString(const Poco::XML::Node * node, const std::string & path, const std::string & default_value); +Int64 getInt64(const Poco::XML::Node * node, const std::string & path, Int64 default_value); +UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path, UInt64 default_value); +int getInt(const Poco::XML::Node * node, const std::string & path, int default_value); +unsigned getUInt(const Poco::XML::Node * node, const std::string & path, unsigned default_value); +bool getBool(const Poco::XML::Node * node, const std::string & path, bool default_value); } From 4c1fb543e1993d755e343b22af345be51e17075d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 1 Dec 2022 13:42:59 +0100 Subject: [PATCH 60/62] Sync everything to persistent storage to avoid writeback affects perf tests (#43530) Signed-off-by: Azat Khuzhin Signed-off-by: Azat Khuzhin --- docker/test/performance-comparison/perf.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 7a034c741eb..cb23372d31f 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -295,6 +295,9 @@ if not args.use_existing_tables: reportStageEnd("create") +# Let's sync the data to avoid writeback affects performance +os.system("sync") + # By default, test all queries. queries_to_run = range(0, len(test_queries)) From 9ef54c6ecc39eee4ae96ae6cb5a2ed12675e2730 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 1 Dec 2022 13:50:56 +0100 Subject: [PATCH 61/62] Try to use exchanges during database replicated replica recovery process (#43628) --- src/Databases/DatabaseReplicated.cpp | 48 ++++++-- .../test_replicated_database/test.py | 104 +++++++++++------- 2 files changed, 104 insertions(+), 48 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index c0bc9d3f3a2..2a9f06e77fc 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -702,7 +702,18 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep /// We will drop or move tables which exist only in local metadata Strings tables_to_detach; - std::vector> replicated_tables_to_rename; + + struct RenameEdge + { + String from; + String intermediate; + String to; + }; + + /// This is needed to generate intermediate name + String salt = toString(thread_local_rng()); + + std::vector replicated_tables_to_rename; size_t total_tables = 0; std::vector replicated_ids; for (auto existing_tables_it = getTablesIterator(getContext(), {}); existing_tables_it->isValid(); @@ -719,8 +730,15 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep { if (name != it->second) { + String intermediate_name; + /// Possibly we failed to rename it on previous iteration + /// And this table was already renamed to an intermediate name + if (startsWith(name, ".rename-") && !startsWith(it->second, ".rename-")) + intermediate_name = name; + else + intermediate_name = fmt::format(".rename-{}-{}", name, sipHash64(fmt::format("{}-{}", name, salt))); /// Need just update table name - replicated_tables_to_rename.emplace_back(name, it->second); + replicated_tables_to_rename.push_back({name, intermediate_name, it->second}); } continue; } @@ -840,13 +858,13 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep tables_to_detach.size(), dropped_dictionaries, dropped_tables.size() - dropped_dictionaries, moved_tables); /// Now database is cleared from outdated tables, let's rename ReplicatedMergeTree tables to actual names - for (const auto & old_to_new : replicated_tables_to_rename) + /// We have to take into account that tables names could be changed with two general queries + /// 1) RENAME TABLE. There could be multiple pairs of tables (e.g. RENAME b TO c, a TO b, c TO d) + /// But it is equal to multiple subsequent RENAMEs each of which operates only with two tables + /// 2) EXCHANGE TABLE. This query swaps two names atomically and could not be represented with two separate RENAMEs + auto rename_table = [&](String from, String to) { - const String & from = old_to_new.first; - const String & to = old_to_new.second; - LOG_DEBUG(log, "Will RENAME TABLE {} TO {}", backQuoteIfNeed(from), backQuoteIfNeed(to)); - /// TODO Maybe we should do it in two steps: rename all tables to temporary names and then rename them to actual names? DDLGuardPtr table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::min(from, to)); DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::max(from, to)); @@ -858,7 +876,23 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep DatabaseAtomic::renameTable(make_query_context(), from, *this, to, false, false); tables_metadata_digest = new_digest; assert(checkDigestValid(getContext())); + }; + + LOG_DEBUG(log, "Starting first stage of renaming process. Will rename tables to intermediate names"); + for (auto & [from, intermediate, _] : replicated_tables_to_rename) + { + /// Due to some unknown failures there could be tables + /// which are already in an intermediate state + /// For them we skip the first stage + if (from == intermediate) + continue; + rename_table(from, intermediate); } + LOG_DEBUG(log, "Starting second stage of renaming process. Will rename tables from intermediate to desired names"); + for (auto & [_, intermediate, to] : replicated_tables_to_rename) + rename_table(intermediate, to); + + LOG_DEBUG(log, "Renames completed succesessfully"); for (const auto & id : dropped_tables) DatabaseCatalog::instance().waitTableFinallyDropped(id); diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index de5433d5beb..1e6a39ee1bd 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -592,60 +592,64 @@ def test_alters_from_different_replicas(started_cluster): def create_some_tables(db): settings = {"distributed_ddl_task_timeout": 0} - main_node.query( - "CREATE TABLE {}.t1 (n int) ENGINE=Memory".format(db), settings=settings - ) + main_node.query(f"CREATE TABLE {db}.t1 (n int) ENGINE=Memory", settings=settings) dummy_node.query( - "CREATE TABLE {}.t2 (s String) ENGINE=Memory".format(db), settings=settings + f"CREATE TABLE {db}.t2 (s String) ENGINE=Memory", settings=settings ) main_node.query( - "CREATE TABLE {}.mt1 (n int) ENGINE=MergeTree order by n".format(db), + f"CREATE TABLE {db}.mt1 (n int) ENGINE=MergeTree order by n", settings=settings, ) dummy_node.query( - "CREATE TABLE {}.mt2 (n int) ENGINE=MergeTree order by n".format(db), + f"CREATE TABLE {db}.mt2 (n int) ENGINE=MergeTree order by n", settings=settings, ) main_node.query( - "CREATE TABLE {}.rmt1 (n int) ENGINE=ReplicatedMergeTree order by n".format(db), + f"CREATE TABLE {db}.rmt1 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings, ) dummy_node.query( - "CREATE TABLE {}.rmt2 (n int) ENGINE=ReplicatedMergeTree order by n".format(db), + f"CREATE TABLE {db}.rmt2 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings, ) main_node.query( - "CREATE TABLE {}.rmt3 (n int) ENGINE=ReplicatedMergeTree order by n".format(db), + f"CREATE TABLE {db}.rmt3 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings, ) dummy_node.query( - "CREATE TABLE {}.rmt5 (n int) ENGINE=ReplicatedMergeTree order by n".format(db), + f"CREATE TABLE {db}.rmt5 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings, ) main_node.query( - "CREATE MATERIALIZED VIEW {}.mv1 (n int) ENGINE=ReplicatedMergeTree order by n AS SELECT n FROM recover.rmt1".format( - db - ), + f"CREATE MATERIALIZED VIEW {db}.mv1 (n int) ENGINE=ReplicatedMergeTree order by n AS SELECT n FROM recover.rmt1", settings=settings, ) dummy_node.query( - "CREATE MATERIALIZED VIEW {}.mv2 (n int) ENGINE=ReplicatedMergeTree order by n AS SELECT n FROM recover.rmt2".format( - db - ), + f"CREATE MATERIALIZED VIEW {db}.mv2 (n int) ENGINE=ReplicatedMergeTree order by n AS SELECT n FROM recover.rmt2", settings=settings, ) main_node.query( - "CREATE DICTIONARY {}.d1 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " + f"CREATE DICTIONARY {db}.d1 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())".format(db) + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())" ) dummy_node.query( - "CREATE DICTIONARY {}.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " + f"CREATE DICTIONARY {db}.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt2' PASSWORD '' DB 'recover')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())".format(db) + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())" ) +# These tables are used to check that DatabaseReplicated correctly renames all the tables in case when it restores from the lost state +def create_table_for_exchanges(db): + settings = {"distributed_ddl_task_timeout": 0} + for table in ["a1", "a2", "a3", "a4", "a5", "a6"]: + main_node.query( + f"CREATE TABLE {db}.{table} (s String) ENGINE=ReplicatedMergeTree order by s", + settings=settings, + ) + + def test_recover_staled_replica(started_cluster): main_node.query( "CREATE DATABASE recover ENGINE = Replicated('/clickhouse/databases/recover', 'shard1', 'replica1');" @@ -659,13 +663,20 @@ def test_recover_staled_replica(started_cluster): settings = {"distributed_ddl_task_timeout": 0} create_some_tables("recover") + create_table_for_exchanges("recover") for table in ["t1", "t2", "mt1", "mt2", "rmt1", "rmt2", "rmt3", "rmt5"]: - main_node.query("INSERT INTO recover.{} VALUES (42)".format(table)) + main_node.query(f"INSERT INTO recover.{table} VALUES (42)") for table in ["t1", "t2", "mt1", "mt2"]: - dummy_node.query("INSERT INTO recover.{} VALUES (42)".format(table)) + dummy_node.query(f"INSERT INTO recover.{table} VALUES (42)") + + for i, table in enumerate(["a1", "a2", "a3", "a4", "a5", "a6"]): + main_node.query(f"INSERT INTO recover.{table} VALUES ('{str(i + 1) * 10}')") + for table in ["rmt1", "rmt2", "rmt3", "rmt5"]: - main_node.query("SYSTEM SYNC REPLICA recover.{}".format(table)) + main_node.query(f"SYSTEM SYNC REPLICA recover.{table}") + for table in ["a1", "a2", "a3", "a4", "a5", "a6"]: + main_node.query(f"SYSTEM SYNC REPLICA recover.{table}") with PartitionManager() as pm: pm.drop_instance_zk_connections(dummy_node) @@ -699,19 +710,15 @@ def test_recover_staled_replica(started_cluster): ).strip() ) main_node.query_with_retry( - "ALTER TABLE recover.`{}` MODIFY COLUMN n int DEFAULT 42".format( - inner_table - ), + f"ALTER TABLE recover.`{inner_table}` MODIFY COLUMN n int DEFAULT 42", settings=settings, ) main_node.query_with_retry( - "ALTER TABLE recover.mv1 MODIFY QUERY SELECT m FROM recover.rmt1".format( - inner_table - ), + "ALTER TABLE recover.mv1 MODIFY QUERY SELECT m FROM recover.rmt1", settings=settings, ) main_node.query_with_retry( - "RENAME TABLE recover.mv2 TO recover.mv3".format(inner_table), + "RENAME TABLE recover.mv2 TO recover.mv3", settings=settings, ) @@ -727,11 +734,18 @@ def test_recover_staled_replica(started_cluster): "CREATE TABLE recover.tmp AS recover.m1", settings=settings ) + main_node.query("EXCHANGE TABLES recover.a1 AND recover.a2", settings=settings) + main_node.query("EXCHANGE TABLES recover.a3 AND recover.a4", settings=settings) + main_node.query("EXCHANGE TABLES recover.a5 AND recover.a4", settings=settings) + main_node.query("EXCHANGE TABLES recover.a6 AND recover.a3", settings=settings) + main_node.query("RENAME TABLE recover.a6 TO recover.a7", settings=settings) + main_node.query("RENAME TABLE recover.a1 TO recover.a8", settings=settings) + assert ( main_node.query( "SELECT name FROM system.tables WHERE database='recover' AND name NOT LIKE '.inner_id.%' ORDER BY name" ) - == "d1\nd2\nm1\nmt1\nmt2\nmv1\nmv3\nrmt1\nrmt2\nrmt4\nt2\ntmp\n" + == "a2\na3\na4\na5\na7\na8\nd1\nd2\nm1\nmt1\nmt2\nmv1\nmv3\nrmt1\nrmt2\nrmt4\nt2\ntmp\n" ) query = ( "SELECT name, uuid, create_table_query FROM system.tables WHERE database='recover' AND name NOT LIKE '.inner_id.%' " @@ -752,6 +766,12 @@ def test_recover_staled_replica(started_cluster): == "2\n" ) + # Check that Database Replicated renamed all the tables correctly + for i, table in enumerate(["a2", "a8", "a5", "a7", "a4", "a3"]): + assert ( + dummy_node.query(f"SELECT * FROM recover.{table}") == f"{str(i + 1) * 10}\n" + ) + for table in [ "m1", "t2", @@ -765,11 +785,11 @@ def test_recover_staled_replica(started_cluster): "mv1", "mv3", ]: - assert main_node.query("SELECT (*,).1 FROM recover.{}".format(table)) == "42\n" + assert main_node.query(f"SELECT (*,).1 FROM recover.{table}") == "42\n" for table in ["t2", "rmt1", "rmt2", "rmt4", "d1", "d2", "mt2", "mv1", "mv3"]: - assert dummy_node.query("SELECT (*,).1 FROM recover.{}".format(table)) == "42\n" + assert dummy_node.query(f"SELECT (*,).1 FROM recover.{table}") == "42\n" for table in ["m1", "mt1"]: - assert dummy_node.query("SELECT count() FROM recover.{}".format(table)) == "0\n" + assert dummy_node.query(f"SELECT count() FROM recover.{table}") == "0\n" global test_recover_staled_replica_run assert ( dummy_node.query( @@ -784,20 +804,22 @@ def test_recover_staled_replica(started_cluster): == f"{test_recover_staled_replica_run}\n" ) test_recover_staled_replica_run += 1 + + print(dummy_node.query("SHOW DATABASES")) + print(dummy_node.query("SHOW TABLES FROM recover_broken_tables")) + print(dummy_node.query("SHOW TABLES FROM recover_broken_replicated_tables")) + table = dummy_node.query( - "SHOW TABLES FROM recover_broken_tables LIKE 'mt1_29_%' LIMIT 1" + "SHOW TABLES FROM recover_broken_tables LIKE 'mt1_41_%' LIMIT 1" ).strip() assert ( - dummy_node.query("SELECT (*,).1 FROM recover_broken_tables.{}".format(table)) - == "42\n" + dummy_node.query(f"SELECT (*,).1 FROM recover_broken_tables.{table}") == "42\n" ) table = dummy_node.query( - "SHOW TABLES FROM recover_broken_replicated_tables LIKE 'rmt5_29_%' LIMIT 1" + "SHOW TABLES FROM recover_broken_replicated_tables LIKE 'rmt5_41_%' LIMIT 1" ).strip() assert ( - dummy_node.query( - "SELECT (*,).1 FROM recover_broken_replicated_tables.{}".format(table) - ) + dummy_node.query(f"SELECT (*,).1 FROM recover_broken_replicated_tables.{table}") == "42\n" ) From 1c5610bf65a28be927061f7a1c802682f28b3b64 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Dec 2022 14:06:17 +0100 Subject: [PATCH 62/62] Changes during the review --- docker/test/style/process_style_check_result.py | 4 ++-- docker/test/style/run.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/style/process_style_check_result.py b/docker/test/style/process_style_check_result.py index d914ceffb86..6dc3d05d051 100755 --- a/docker/test/style/process_style_check_result.py +++ b/docker/test/style/process_style_check_result.py @@ -11,7 +11,7 @@ def process_result(result_folder): description = "" test_results = [] checks = ( - "header duplicates", + "duplicate includes", "shellcheck", "style", "black", @@ -19,7 +19,7 @@ def process_result(result_folder): "typos", "whitespaces", "workflows", - "doc typos", + "docs spelling", ) for name in checks: diff --git a/docker/test/style/run.sh b/docker/test/style/run.sh index 911536ed03b..80911bf8627 100755 --- a/docker/test/style/run.sh +++ b/docker/test/style/run.sh @@ -4,7 +4,7 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv echo "Check duplicates" | ts -./check-duplicate-includes.sh |& tee /test_output/header_duplicates_output.txt +./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt echo "Check python formatting with black" | ts @@ -14,7 +14,7 @@ echo "Check python type hinting with mypy" | ts echo "Check typos" | ts ./check-typos |& tee /test_output/typos_output.txt echo "Check docs spelling" | ts -./check-doc-aspell |& tee /test_output/doc_typos_output.txt +./check-doc-aspell |& tee /test_output/docs_spelling_output.txt echo "Check whitespaces" | ts ./check-whitespaces -n |& tee /test_output/whitespaces_output.txt echo "Check workflows" | ts