From fbda86ddc80eefee87522f0e6e3f2fe937352b74 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 30 Nov 2022 13:24:08 +0100 Subject: [PATCH] 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();