This commit is contained in:
Antonio Andelic 2022-11-30 13:24:08 +01:00
parent 678958603b
commit fbda86ddc8
6 changed files with 33 additions and 29 deletions

View File

@ -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<std::vector<ProtocolServerAdapter>>();
tiny_context = std::make_shared<TinyContext>();
/// 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<TCPServer>(
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<TCPServer>(
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<HTTPServer>(
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();

View File

@ -68,7 +68,7 @@ protected:
std::string getDefaultConfigFileName() const override;
private:
TinyContext tiny_context;
TinyContextPtr tiny_context;
struct KeeperHTTPContext;
HTTPContextPtr httpContext();

View File

@ -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);
}

View File

@ -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;
};

View File

@ -31,4 +31,6 @@ private:
ConfigurationPtr config TSA_GUARDED_BY(keeper_dispatcher_mutex);
};
using TinyContextPtr = std::shared_ptr<TinyContext>;
}

View File

@ -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();