Support interserver credential rotation

Restarting a server instance to change the interserver password results
in many replicas being out of sync until all clusters are using the new
credential.

This commit adds dynamic credential loading for both the client
(Replicated* tables) and server (InterserverIOHTTPHandler).

This commit also adds the ability to rotate credentials, i.e. accept more
than one credential during a credential change.

state0 (no auth):

    <interserver_http_credentials />

state1 (auth+allow_empty migration):

    <interserver_http_credentials>
        <user>admin</user>
        <password>222</password>
        <allow_empty>true</allow_empty>
    </interserver_http_credentials>

state2 (auth+new admin password migration):

    <interserver_http_credentials>
        <user>admin</user>
        <password>333</password>
        <users>
            <admin>222</admin>
        </users>
    </interserver_http_credentials>
This commit is contained in:
John Skopis 2020-08-26 08:36:58 +00:00
parent 7f85ae7fa7
commit 9bcc5a6175
9 changed files with 254 additions and 51 deletions

View File

@ -47,6 +47,7 @@
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Interpreters/InterserverCredentials.h>
#include <Access/AccessControlManager.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/System/attachSystemTables.h>
@ -620,16 +621,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
}
if (config().has("interserver_http_credentials"))
{
String user = config().getString("interserver_http_credentials.user", "");
String password = config().getString("interserver_http_credentials.password", "");
if (user.empty())
throw Exception("Configuration parameter interserver_http_credentials user can't be empty", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
global_context->setInterserverCredentials(user, password);
}
LOG_DEBUG(log, "Initiaializing InterserverCredentials.");
global_context->updateInterserverCredentials(config());
if (config().has("macros"))
global_context->setMacros(std::make_unique<Macros>(config(), "macros", log));
@ -673,6 +666,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config);
global_context->updateStorageConfiguration(*config);
global_context->updateInterserverCredentials(*config);
},
/* already_loaded = */ true);

View File

@ -46,6 +46,7 @@
#include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/InterserverIOHandler.h>
#include <Interpreters/SystemLog.h>
@ -314,9 +315,9 @@ struct ContextShared
String interserver_io_host; /// The host name by which this server is available for other servers.
UInt16 interserver_io_port = 0; /// and port.
String interserver_io_user;
String interserver_io_password;
String interserver_scheme; /// http or https
mutable std::mutex interserver_io_credentials_mutex;
std::shared_ptr<BaseInterserverCredentials> interserver_io_credentials;
String path; /// Path to the data directory, with a slash at the end.
String flags_path; /// Path to the directory with some control flags for server maintenance.
@ -1615,6 +1616,42 @@ bool Context::hasAuxiliaryZooKeeper(const String & name) const
return getConfigRef().has("auxiliary_zookeepers." + name);
}
std::shared_ptr<BaseInterserverCredentials> Context::getInterserverCredential()
{
std::lock_guard lock(shared->interserver_io_credentials_mutex);
return shared->interserver_io_credentials;
}
void Context::setInterserverCredentials(std::shared_ptr<BaseInterserverCredentials> credentials)
{
std::lock_guard lock(shared->interserver_io_credentials_mutex);
shared->interserver_io_credentials = credentials;
}
void Context::updateInterserverCredentials(const Poco::Util::AbstractConfiguration & config)
{
std::shared_ptr<BaseInterserverCredentials> interserver_credentials = nullptr;
if (config.has("interserver_http_credentials"))
{
interserver_credentials = ConfigInterserverCredentials::make(config, "interserver_http_credentials");
}
else
{
interserver_credentials = NullInterserverCredentials::make();
}
global_context->setInterserverCredentials(interserver_credentials);
}
std::pair<String, String> Context::getInterserverCredentials() const
{
std::lock_guard lock(shared->interserver_io_credentials_mutex);
auto & credentials = shared->interserver_io_credentials;
return { credentials->getUser(), credentials->getPassword() };
}
void Context::setInterserverIOAddress(const String & host, UInt16 port)
{
shared->interserver_io_host = host;
@ -1630,17 +1667,6 @@ std::pair<String, UInt16> Context::getInterserverIOAddress() const
return { shared->interserver_io_host, shared->interserver_io_port };
}
void Context::setInterserverCredentials(const String & user_, const String & password)
{
shared->interserver_io_user = user_;
shared->interserver_io_password = password;
}
std::pair<String, String> Context::getInterserverCredentials() const
{
return { shared->interserver_io_user, shared->interserver_io_password };
}
void Context::setInterserverScheme(const String & scheme)
{
shared->interserver_scheme = scheme;

View File

@ -62,6 +62,7 @@ class AccessRightsElements;
class EmbeddedDictionaries;
class ExternalDictionariesLoader;
class ExternalModelsLoader;
class BaseInterserverCredentials;
class InterserverIOHandler;
class BackgroundSchedulePool;
class MergeList;
@ -440,7 +441,8 @@ public:
std::pair<String, UInt16> getInterserverIOAddress() const;
/// Credentials which server will use to communicate with others
void setInterserverCredentials(const String & user, const String & password);
void updateInterserverCredentials(const Poco::Util::AbstractConfiguration & config);
std::shared_ptr<BaseInterserverCredentials> getInterserverCredential();
std::pair<String, String> getInterserverCredentials() const;
/// Interserver requests scheme (http or https)
@ -695,6 +697,8 @@ private:
/// If the password is not set, the password will not be checked
void setUserImpl(const String & name, const std::optional<String> & password, const Poco::Net::SocketAddress & address);
void setInterserverCredentials(std::shared_ptr<BaseInterserverCredentials> credentials);
};

View File

@ -0,0 +1,61 @@
#include <Interpreters/InterserverCredentials.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int WRONG_PASSWORD;
}
std::shared_ptr<ConfigInterserverCredentials>
ConfigInterserverCredentials::make(const Poco::Util::AbstractConfiguration & config, const std::string root_tag)
{
const auto user = config.getString(root_tag + ".user", "");
const auto password = config.getString(root_tag + ".password", "");
if (user.empty())
throw Exception("Configuration parameter interserver_http_credentials user can't be empty", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
auto store = makeCredentialStore(user, password, config, root_tag);
return std::make_shared<ConfigInterserverCredentials>(user, password, store);
}
ConfigInterserverCredentials::Store ConfigInterserverCredentials::makeCredentialStore(
const std::string current_user_,
const std::string current_password_,
const Poco::Util::AbstractConfiguration & config,
const std::string root_tag)
{
Store store;
store.insert({{current_user_, current_password_}, true});
if (config.has(root_tag + ".allow_empty") && config.getBool(root_tag + ".allow_empty"))
{
/// Allow empty credential to support migrating from no auth
store.insert({{"", ""}, true});
}
Poco::Util::AbstractConfiguration::Keys users;
config.keys(root_tag + ".users", users);
for (const auto & user : users)
{
LOG_DEBUG(&Poco::Logger::get("InterserverCredentials"), "Adding credential for {}", user);
const auto password = config.getString(root_tag + ".users." + user);
store.insert({{user, password}, true});
}
return store;
}
bool ConfigInterserverCredentials::isValidUser(const std::pair<std::string, std::string> credentials)
{
const auto & valid = store.find(credentials);
if (valid == store.end())
throw Exception("Incorrect user or password in HTTP basic authentication: " + credentials.first, ErrorCodes::WRONG_PASSWORD);
return true;
}
}

View File

@ -0,0 +1,94 @@
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/Exception.h>
#include <common/logger_useful.h>
namespace DB
{
/// InterserverCredentials holds credentials for server (store) and client
/// credentials (current_*). The container is constructed through `make` and a
/// shared_ptr is captured inside Context.
class BaseInterserverCredentials
{
public:
BaseInterserverCredentials(std::string current_user_, std::string current_password_)
: current_user(current_user_), current_password(current_password_)
{ }
virtual ~BaseInterserverCredentials() { }
/// isValidUser returns true or throws WRONG_PASSWORD
virtual bool isValidUser(const std::pair<std::string, std::string> credentials) = 0;
std::string getUser() { return current_user; }
std::string getPassword() { return current_password; }
protected:
std::string current_user;
std::string current_password;
};
/// NullInterserverCredentials are used when authentication is not configured
class NullInterserverCredentials : public virtual BaseInterserverCredentials
{
public:
NullInterserverCredentials(const NullInterserverCredentials &) = delete;
NullInterserverCredentials() : BaseInterserverCredentials("", "") { }
~NullInterserverCredentials() override { }
static std::shared_ptr<NullInterserverCredentials> make() { return std::make_shared<NullInterserverCredentials>(); }
bool isValidUser(const std::pair<std::string, std::string> credentials) override
{
std::ignore = credentials;
return true;
}
};
/// ConfigInterserverCredentials implements authentication using a Store, which
/// is configured, e.g.
/// <interserver_http_credentials>
/// <user>admin</user>
/// <password>222</password>
/// <!-- To support mix of un/authenticated clients -->
/// <!-- <allow_empty>true</allow_empty> -->
/// <users>
/// <!-- Allow authentication using previous passwords during rotation -->
/// <admin>111</admin>
/// </users>
/// </interserver_http_credentials>
class ConfigInterserverCredentials : public virtual BaseInterserverCredentials
{
public:
using Store = std::map<std::pair<std::string, std::string>, bool>;
ConfigInterserverCredentials(const ConfigInterserverCredentials &) = delete;
static std::shared_ptr<ConfigInterserverCredentials> make(const Poco::Util::AbstractConfiguration & config, const std::string root_tag);
~ConfigInterserverCredentials() override { }
ConfigInterserverCredentials(const std::string current_user_, const std::string current_password_, const Store & store_)
: BaseInterserverCredentials(current_user_, current_password_), store(std::move(store_))
{
}
bool isValidUser(const std::pair<std::string, std::string> credentials) override;
private:
Store store;
static Store makeCredentialStore(
const std::string current_user_,
const std::string current_password_,
const Poco::Util::AbstractConfiguration & config,
const std::string root_tag);
};
}

View File

@ -99,6 +99,7 @@ SRCS(
InterpreterSystemQuery.cpp
InterpreterUseQuery.cpp
InterpreterWatchQuery.cpp
InterserverCredentials.cpp
JoinSwitcher.cpp
JoinToSubqueryTransformVisitor.cpp
JoinedTables.cpp

View File

@ -20,35 +20,25 @@ namespace DB
namespace ErrorCodes
{
extern const int ABORTED;
extern const int NOT_IMPLEMENTED;
extern const int TOO_MANY_SIMULTANEOUS_QUERIES;
extern const int WRONG_PASSWORD;
}
std::pair<String, bool> InterserverIOHTTPHandler::checkAuthentication(Poco::Net::HTTPServerRequest & request) const
bool InterserverIOHTTPHandler::checkAuthentication(Poco::Net::HTTPServerRequest & request) const
{
const auto & config = server.config();
auto creds = server.context().getInterserverCredential();
if (!request.hasCredentials())
return creds->isValidUser(std::make_pair(default_user, default_password));
if (config.has("interserver_http_credentials.user"))
{
if (!request.hasCredentials())
return {"Server requires HTTP Basic authentication, but client doesn't provide it", false};
String scheme, info;
request.getCredentials(scheme, info);
String scheme, info;
request.getCredentials(scheme, info);
if (scheme != "Basic")
return {"Server requires HTTP Basic authentication but client provides another method", false};
if (scheme != "Basic")
throw Exception("Server requires HTTP Basic authentication but client provides another method", ErrorCodes::NOT_IMPLEMENTED);
String user = config.getString("interserver_http_credentials.user");
String password = config.getString("interserver_http_credentials.password", "");
Poco::Net::HTTPBasicCredentials credentials(info);
if (std::make_pair(user, password) != std::make_pair(credentials.getUsername(), credentials.getPassword()))
return {"Incorrect user or password in HTTP Basic authentication", false};
}
else if (request.hasCredentials())
{
return {"Client requires HTTP Basic authentication, but server doesn't provide it", false};
}
return {"", true};
Poco::Net::HTTPBasicCredentials credentials(info);
return creds->isValidUser(std::make_pair(credentials.getUsername(), credentials.getPassword()));
}
void InterserverIOHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, Output & used_output)
@ -95,7 +85,7 @@ void InterserverIOHTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & requ
try
{
if (auto [message, success] = checkAuthentication(request); success)
if (checkAuthentication(request))
{
processQuery(request, response, used_output);
LOG_DEBUG(log, "Done processing query");
@ -104,12 +94,21 @@ void InterserverIOHTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & requ
{
response.setStatusAndReason(Poco::Net::HTTPServerResponse::HTTP_UNAUTHORIZED);
if (!response.sent())
writeString(message, *used_output.out);
writeString("Unauthorized.", *used_output.out);
LOG_WARNING(log, "Query processing failed request: '{}' authentication failed", request.getURI());
}
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::WRONG_PASSWORD)
{
response.setStatusAndReason(Poco::Net::HTTPServerResponse::HTTP_UNAUTHORIZED);
if (!response.sent())
writeString("Unauthorized.", *used_output.out);
LOG_WARNING(log, "Query processing failed request: '{}' authentication failed", request.getURI());
return;
}
if (e.code() == ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES)
return;

View File

@ -1,9 +1,11 @@
#pragma once
#include <memory>
#include <string>
#include <Poco/Logger.h>
#include <Poco/Net/HTTPRequestHandler.h>
#include <Common/CurrentMetrics.h>
#include <Interpreters/InterserverCredentials.h>
namespace CurrentMetrics
@ -41,7 +43,9 @@ private:
void processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, Output & used_output);
std::pair<String, bool> checkAuthentication(Poco::Net::HTTPServerRequest & request) const;
bool checkAuthentication(Poco::Net::HTTPServerRequest & request) const;
const std::string default_user = "";
const std::string default_password = "";
};
}

View File

@ -9,7 +9,6 @@ def _fill_nodes(nodes, shard):
node.query(
'''
CREATE DATABASE test;
CREATE TABLE test_table(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}', date, id, 8192);
'''.format(shard=shard, replica=node.name))
@ -135,6 +134,7 @@ def credentials_and_no_credentials_cluster():
def test_credentials_and_no_credentials(credentials_and_no_credentials_cluster):
# Initial state: node7 requires auth; node8 open
node7.query("insert into test_table values ('2017-06-21', 111, 0)")
time.sleep(1)
@ -144,5 +144,25 @@ def test_credentials_and_no_credentials(credentials_and_no_credentials_cluster):
node8.query("insert into test_table values ('2017-06-22', 222, 1)")
time.sleep(1)
assert node7.query("SELECT id FROM test_table order by id") == '111\n'
assert node7.query("SELECT id FROM test_table order by id") == '111\n222\n'
assert node8.query("SELECT id FROM test_table order by id") == '222\n'
allow_empty = """
<yandex>
<interserver_http_port>9009</interserver_http_port>
<interserver_http_credentials>
<user>admin</user>
<password>222</password>
<allow_empty>true</allow_empty>
</interserver_http_credentials>
</yandex>
"""
# change state: Flip node7 to mixed auth/non-auth (allow node8)
node7.replace_config("/etc/clickhouse-server/config.d/credentials1.xml",
allow_empty)
node7.query("insert into test_table values ('2017-06-22', 333, 1)")
node8.query("DETACH TABLE test_table")
node8.query("ATTACH TABLE test_table")
time.sleep(3)
assert node8.query("SELECT id FROM test_table order by id") == '111\n222\n333\n'