Check for directory owner

This commit is contained in:
alesapin 2021-05-12 17:05:44 +03:00
parent ed89af22b2
commit 43ee9f0a3a
3 changed files with 96 additions and 9 deletions

View File

@ -1,5 +1,7 @@
#include "Keeper.h"
#include <sys/stat.h>
#include <pwd.h>
#include <Common/ClickHouseRevision.h>
#include <Server/ProtocolServerAdapter.h>
#include <Common/DNSResolver.h>
@ -15,6 +17,7 @@
#include <Poco/Version.h>
#include <Poco/Environment.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <filesystem>
#include <IO/UseSSL.h>
#if !defined(ARCADIA_BUILD)
@ -31,6 +34,11 @@
# include <Server/KeeperTCPHandlerFactory.h>
#endif
#if defined(OS_LINUX)
# include <unistd.h>
# include <sys/syscall.h>
#endif
int mainEntryClickHouseKeeper(int argc, char ** argv)
{
DB::Keeper app;
@ -54,14 +62,9 @@ namespace ErrorCodes
{
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int SUPPORT_IS_DISABLED;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
extern const int SYSTEM_ERROR;
extern const int FAILED_TO_GETPWUID;
extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA;
extern const int NETWORK_ERROR;
extern const int CORRUPTED_DATA;
extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA;
extern const int FAILED_TO_GETPWUID;
}
namespace
@ -135,6 +138,26 @@ Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port
#endif
}
std::string getUserName(uid_t user_id)
{
/// Try to convert user id into user name.
auto buffer_size = sysconf(_SC_GETPW_R_SIZE_MAX);
if (buffer_size <= 0)
buffer_size = 1024;
std::string buffer;
buffer.reserve(buffer_size);
struct passwd passwd_entry;
struct passwd * result = nullptr;
const auto error = getpwuid_r(user_id, &passwd_entry, buffer.data(), buffer_size, &result);
if (error)
throwFromErrno("Failed to find user name for " + toString(user_id), ErrorCodes::FAILED_TO_GETPWUID, error);
else if (result)
return result->pw_name;
return toString(user_id);
}
}
Poco::Net::SocketAddress Keeper::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const
@ -271,6 +294,39 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
if (!config().has("keeper_server"))
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Keeper configuration (<keeper_server> section) not found in config");
std::string path;
if (config().has("keeper_server.storage_path"))
path = config().getString("keeper_server.storage_path");
else if (config().has("keeper_server.log_storage_path"))
path = config().getString("keeper_server.log_storage_path");
else if (config().has("keeper_server.snapshot_storage_path"))
path = config().getString("keeper_server.snapshot_storage_path");
else
path = std::filesystem::path{DBMS_DEFAULT_PATH} / "coordination/logs";
/// Check that the process user id matches the owner of the data.
const auto effective_user_id = geteuid();
struct stat statbuf;
if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid)
{
const auto effective_user = getUserName(effective_user_id);
const auto data_owner = getUserName(statbuf.st_uid);
std::string message = "Effective user of the process (" + effective_user +
") does not match the owner of the data (" + data_owner + ").";
if (effective_user_id == 0)
{
message += " Run under 'sudo -u " + data_owner + "'.";
throw Exception(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA);
}
else
{
LOG_WARNING(log, message);
}
}
const Settings & settings = global_context->getSettingsRef();
GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 500));

View File

@ -14,6 +14,7 @@
#include <chrono>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <string>
#include <filesystem>
#include <Poco/Util/Application.h>
namespace DB
@ -59,6 +60,18 @@ void setSSLParams(nuraft::asio_service::options & asio_opts)
}
#endif
std::string getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config)
{
/// the most specialized path
if (config.has("keeper_server.snapshot_storage_path"))
return config.getString("keeper_server.snapshot_storage_path");
if (config.has("keeper_server.storage_path"))
return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots";
return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots";
}
}
KeeperServer::KeeperServer(
@ -71,7 +84,7 @@ KeeperServer::KeeperServer(
, coordination_settings(coordination_settings_)
, state_machine(nuraft::cs_new<KeeperStateMachine>(
responses_queue_, snapshots_queue_,
config.getString("keeper_server.snapshot_storage_path", config.getString("path", DBMS_DEFAULT_PATH) + "coordination/snapshots"),
getSnapshotsPathFromConfig(config),
coordination_settings))
, state_manager(nuraft::cs_new<KeeperStateManager>(server_id, "keeper_server", config, coordination_settings))
, log(&Poco::Logger::get("KeeperServer"))

View File

@ -1,5 +1,6 @@
#include <Coordination/KeeperStateManager.h>
#include <Common/Exception.h>
#include <filesystem>
namespace DB
{
@ -9,6 +10,23 @@ namespace ErrorCodes
extern const int RAFT_ERROR;
}
namespace
{
std::string getLogsPathFromConfig(const std::string & config_prefix, const Poco::Util::AbstractConfiguration & config)
{
/// the most specialized path
if (config.has(config_prefix + ".log_storage_path"))
return config.getString(config_prefix + ".log_storage_path");
if (config.has(config_prefix + ".storage_path"))
return std::filesystem::path{config.getString(config_prefix + ".storage_path")} / "logs";
return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs";
}
}
KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path)
: my_server_id(server_id_)
, my_port(port)
@ -28,7 +46,7 @@ KeeperStateManager::KeeperStateManager(
: my_server_id(my_server_id_)
, secure(config.getBool(config_prefix + ".raft_configuration.secure", false))
, log_store(nuraft::cs_new<KeeperLogStore>(
config.getString(config_prefix + ".log_storage_path", config.getString("path", DBMS_DEFAULT_PATH) + "coordination/logs"),
getLogsPathFromConfig(config_prefix, config),
coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync))
, cluster_config(nuraft::cs_new<nuraft::cluster_config>())
{