temp-commit

This commit is contained in:
Konstantin Bogdanov 2024-09-11 16:25:05 +02:00
parent 512b871d87
commit a734d124b8
No known key found for this signature in database
4 changed files with 64 additions and 50 deletions

View File

@ -97,21 +97,21 @@
#include <Common/getHashOfLoadedBinary.h>
#include <Common/filesystemHelpers.h>
#include <Compression/CompressionCodecEncrypted.h>
#include <Server/ACMEClient.h>
#include <Server/CloudPlacementInfo.h>
#include <Server/HTTP/HTTPServer.h>
#include <Server/HTTP/HTTPServerConnectionFactory.h>
#include <Server/KeeperReadinessHandler.h>
#include <Server/MySQLHandlerFactory.h>
#include <Server/PostgreSQLHandlerFactory.h>
#include <Server/ProtocolServerAdapter.h>
#include <Server/ProxyV1HandlerFactory.h>
#include <Server/TLSHandlerFactory.h>
#include <Server/ProtocolServerAdapter.h>
#include <Server/KeeperReadinessHandler.h>
#include <Server/HTTP/HTTPServer.h>
#include <Server/CloudPlacementInfo.h>
#include <Interpreters/AsynchronousInsertQueue.h>
#include <Core/ServerSettings.h>
#include <filesystem>
#include <unordered_set>
#include "Server/ACMEClient.h"
#include "config.h"
#include <Common/config_version.h>
@ -1748,8 +1748,8 @@ try
CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "encryption_codecs");
#if USE_SSL
ACMEClient::ACMEClient::instance().initialize(*config);
CertificateReloader::instance().tryReloadAll(*config);
ACMEClient::ACMEClient::instance().reload(*config);
#endif
NamedCollectionFactory::instance().reloadFromConfig(*config);
@ -2150,8 +2150,8 @@ try
"to configuration file.)");
#if USE_SSL
ACMEClient::ACMEClient::instance().initialize(config());
CertificateReloader::instance().tryLoad(config());
ACMEClient::ACMEClient::instance().reload(config());
#endif
/// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread.

View File

@ -1,8 +1,6 @@
#include <memory>
#include <sstream>
#include <Server/ACMEClient.h>
#include "Common/ZooKeeper/ZooKeeperLock.h"
#include "Core/ServerUUID.h"
#if USE_SSL
#include <Core/BackgroundSchedulePool.h>
@ -19,9 +17,9 @@
#include <openssl/hmac.h>
#include <openssl/obj_mac.h>
#include <openssl/pem.h>
#include <openssl/provider.h>
#include <openssl/x509.h>
#include <openssl/x509v3.h>
#include <openssl/provider.h>
#include <Poco/Base64Encoder.h>
#include <Poco/Crypto/CryptoStream.h>
#include <Poco/Crypto/ECKey.h>
@ -40,10 +38,10 @@
#include <Poco/StreamCopier.h>
#include <Poco/String.h>
#include <Poco/URI.h>
#include <Common/OpenSSLHelpers.h>
#include <Common/Base64.h>
#include <Common/Exception.h>
#include <Common/HTTPConnectionPool.h>
#include <Common/OpenSSLHelpers.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/logger_useful.h>
@ -53,9 +51,9 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int OPENSSL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int OPENSSL_ERROR;
}
namespace ACMEClient
@ -209,15 +207,19 @@ void ACMEClient::requestCertificate(const Poco::Util::AbstractConfiguration &)
/// TODO
}
/// uninitialized -> load/generate key -> authorize -> ready
void ACMEClient::initialize(const Poco::Util::AbstractConfiguration & config)
try
{
if (initialized)
return;
auto http_port = config.getInt("http_port");
if (http_port != 80 && !initialized)
LOG_WARNING(log, "For ACME HTTP challenge HTTP port must be 80, but is {}", http_port);
if (private_acme_key && !key_id.empty())
return;
// auto http_port = config.getInt("http_port");
// if (http_port != 80 && !initialized)
// LOG_WARNING(log, "For ACME HTTP challenge HTTP port must be 80, but is {}", http_port);
directory_url = config.getString("acme.directory_url", LetsEncrypt::ACME_STAGING_DIRECTORY_URL);
contact_email = config.getString("acme.email", "");
@ -236,12 +238,12 @@ try
connection_timeout_settings = ConnectionTimeouts();
proxy_configuration = ProxyConfiguration();
keeper_dispatcher = Context::getGlobalContextInstance()->getZooKeeper();
keeper_dispatcher->get()->createIfNotExists(fs::path(ZOOKEEPER_ACME_BASE_PATH), "");
zookeeper = Context::getGlobalContextInstance()->getZooKeeper();
zookeeper->createIfNotExists(fs::path(ZOOKEEPER_ACME_BASE_PATH), "");
BackgroundSchedulePool & bgpool = context->getSchedulePool();
BackgroundSchedulePool & bgpool = Context::getGlobalContextInstance()->getSchedulePool();
lock = createSimpleZooKeeperLock(zk, ZOOKEEPER_ACME_BASE_PATH, "leader", "ACME lock", /*throw_if_lost*/true);
// lock = createSimpleZooKeeperLock(zk_client, ZOOKEEPER_ACME_BASE_PATH, "leader", "ACME lock", /*throw_if_lost*/true);
// auto do_election = [this, zk] {
// LOG_DEBUG(log, "Running election task");
@ -260,8 +262,34 @@ try
// do_election();
authentication_task = bgpool.createTask(
"ACMEAuth",
[this]
{
if (!key_id.empty())
return;
try
{
authenticate();
}
catch (...)
{
authentication_task->scheduleAfter(1000);
tryLogCurrentException("Failed to authenticate");
}
});
/// First, we need to get private key from Keeper,
/// or generate one if we have not already.
/// This blocks authorization, as key is required for this step.
/// Next, we start an authorization loop.
/// On finished authorization we can start accepting orders for certificates.
refresh_key_task = bgpool.createTask(
"ACMEClient",
"ACMEKeygen",
[this]
{
LOG_DEBUG(log, "Running ACMEClient key refresh task");
@ -303,17 +331,13 @@ try
chassert(!private_key.empty());
std::istringstream private_key_stream(private_key); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
std::istringstream private_key_stream(private_key); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
private_acme_key = std::make_shared<Poco::Crypto::RSAKey>(nullptr, &private_key_stream, "");
try{
authenticate();
}
catch(...)
{
refresh_key_task->scheduleAfter(1000);
tryLogCurrentException("Failed to authenticate");
}
LOG_DEBUG(log, "ACME private key successfully initialized");
refresh_key_task->deactivate();
authentication_task->activateAndSchedule();
});
refresh_key_task->activateAndSchedule();
@ -410,12 +434,7 @@ ACMEClient::doJWSRequest(const std::string & url, const std::string & payload, s
r.set("Content-Type", "application/jose+json");
r.set("Content-Length", std::to_string(request_data.size()));
auto session = makeHTTPSession(
HTTPConnectionGroupType::HTTP,
uri,
connection_timeout_settings,
proxy_configuration
);
auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, uri, connection_timeout_settings, proxy_configuration);
auto & ostream = session->sendRequest(r);
ostream << request_data;
@ -448,7 +467,7 @@ void ACMEClient::authenticate()
payload_object.set("contact", contact);
payload_object.set("termsOfServiceAgreed", terms_of_service_agreed);
std::ostringstream payload; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
std::ostringstream payload; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
payload.exceptions(std::ios::failbit);
Poco::JSON::Stringifier::stringify(payload_object, payload);
@ -484,7 +503,7 @@ std::string ACMEClient::order()
payload_object.set("identifiers", payload_identifiers);
std::ostringstream payload; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
std::ostringstream payload; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
payload.exceptions(std::ios::failbit);
Poco::JSON::Stringifier::stringify(payload_object, payload);
@ -592,12 +611,7 @@ std::string ACMEClient::requestNonce()
LOG_TRACE(log, "Requesting nonce from {}", uri.getPathAndQuery());
auto session = makeHTTPSession(
HTTPConnectionGroupType::HTTP,
uri,
connection_timeout_settings,
proxy_configuration
);
auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, uri, connection_timeout_settings, proxy_configuration);
session->sendRequest(r);
auto response = Poco::Net::HTTPResponse();

View File

@ -1,9 +1,9 @@
#pragma once
#include "Common/ZooKeeper/ZooKeeperLock.h"
#include "Coordination/KeeperDispatcher.h"
#include "config.h"
#include <Common/ZooKeeper/ZooKeeperLock.h>
#if USE_SSL
#include <boost/core/noncopyable.hpp>
@ -109,10 +109,10 @@ private:
DirectoryPtr directory;
BackgroundSchedulePoolTaskHolder election_task;
BackgroundSchedulePoolTaskHolder authentication_task;
BackgroundSchedulePoolTaskHolder refresh_key_task;
std::shared_ptr<KeeperDispatcher> keeper_dispatcher;
zkutil::ZooKeeperPtr zookeeper;
std::shared_ptr<zkutil::ZooKeeperLock> lock;
std::vector<std::string> domains;

View File

@ -34,7 +34,7 @@ namespace DB
class CertificateReloader
{
public:
using stat_t = struct stat;
// using stat_t = struct stat;
struct Data
{