Merge pull request #27755 from ClickHouse/ncb/server_uuid

Merging  #20089
This commit is contained in:
tavplubix 2021-08-19 14:59:18 +03:00 committed by GitHub
commit 9ef45d92c2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 182 additions and 16 deletions

View File

@ -12,6 +12,7 @@
#include <Common/SymbolIndex.h>
#include <Common/StackTrace.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Core/ServerUUID.h>
#if !defined(ARCADIA_BUILD)
# include "Common/config_version.h"
@ -38,6 +39,13 @@ void setExtras()
if (!anonymize)
sentry_set_extra("server_name", sentry_value_new_string(getFQDNOrHostName().c_str()));
DB::UUID server_uuid = DB::ServerUUID::get();
if (server_uuid != DB::UUIDHelpers::Nil)
{
std::string server_uuid_str = DB::toString(server_uuid);
sentry_set_extra("server_uuid", sentry_value_new_string(server_uuid_str.c_str()));
}
sentry_set_tag("version", VERSION_STRING);
sentry_set_extra("version_githash", sentry_value_new_string(VERSION_GITHASH));
sentry_set_extra("version_describe", sentry_value_new_string(VERSION_DESCRIBE));

View File

@ -17,6 +17,7 @@
#include <Poco/Version.h>
#include <Poco/Environment.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Core/ServerUUID.h>
#include <filesystem>
#include <IO/UseSSL.h>
@ -326,6 +327,8 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
}
}
DB::ServerUUID::load(path + "/uuid", log);
const Settings & settings = global_context->getSettingsRef();
GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 100));

View File

@ -39,6 +39,7 @@
#include <Common/getMappedArea.h>
#include <Common/remapExecutable.h>
#include <Common/TLDListsHolder.h>
#include <Core/ServerUUID.h>
#include <IO/HTTPCommon.h>
#include <IO/ReadHelpers.h>
#include <IO/UseSSL.h>
@ -80,7 +81,6 @@
#include <Server/HTTP/HTTPServer.h>
#include <filesystem>
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
# include "Common/config_version.h"
@ -147,7 +147,6 @@ static bool jemallocOptionEnabled(const char *name)
static bool jemallocOptionEnabled(const char *) { return 0; }
#endif
int mainEntryClickHouseServer(int argc, char ** argv)
{
DB::Server app;
@ -668,13 +667,14 @@ if (ThreadFuzzer::instance().isEffective())
global_context->setRemoteHostFilter(config());
std::string path = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
fs::path path = path_str;
std::string default_database = config().getString("default_database", "default");
/// 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)
if (stat(path_str.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);
@ -691,9 +691,11 @@ if (ThreadFuzzer::instance().isEffective())
}
}
global_context->setPath(path);
global_context->setPath(path_str);
StatusFile status{path + "status", StatusFile::write_full_info};
StatusFile status{path / "status", StatusFile::write_full_info};
DB::ServerUUID::load(path / "uuid", log);
/// Try to increase limit on number of open files.
{
@ -727,7 +729,7 @@ if (ThreadFuzzer::instance().isEffective())
/// Storage with temporary data for processing of heavy queries.
{
std::string tmp_path = config().getString("tmp_path", path + "tmp/");
std::string tmp_path = config().getString("tmp_path", path / "tmp/");
std::string tmp_policy = config().getString("tmp_policy", "");
const VolumePtr & volume = global_context->setTemporaryStorage(tmp_path, tmp_policy);
for (const DiskPtr & disk : volume->getDisks())
@ -739,7 +741,7 @@ if (ThreadFuzzer::instance().isEffective())
* Examples: do repair of local data; clone all replicated tables from replica.
*/
{
auto flags_path = fs::path(path) / "flags/";
auto flags_path = path / "flags/";
fs::create_directories(flags_path);
global_context->setFlagsPath(flags_path);
}
@ -748,29 +750,29 @@ if (ThreadFuzzer::instance().isEffective())
*/
{
std::string user_files_path = config().getString("user_files_path", fs::path(path) / "user_files/");
std::string user_files_path = config().getString("user_files_path", path / "user_files/");
global_context->setUserFilesPath(user_files_path);
fs::create_directories(user_files_path);
}
{
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", fs::path(path) / "dictionaries_lib/");
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", path / "dictionaries_lib/");
global_context->setDictionariesLibPath(dictionaries_lib_path);
fs::create_directories(dictionaries_lib_path);
}
/// top_level_domains_lists
{
const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/");
const std::string & top_level_domains_path = config().getString("top_level_domains_path", path / "top_level_domains/");
TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config());
}
{
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
fs::create_directories(path / "data/");
fs::create_directories(path / "metadata/");
/// Directory with metadata of tables, which was marked as dropped by Atomic database
fs::create_directories(fs::path(path) / "metadata_dropped/");
fs::create_directories(path / "metadata_dropped/");
}
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
@ -953,7 +955,7 @@ if (ThreadFuzzer::instance().isEffective())
#endif
/// Set path for format schema files
fs::path format_schema_path(config().getString("format_schema_path", fs::path(path) / "format_schemas/"));
fs::path format_schema_path(config().getString("format_schema_path", path / "format_schemas/"));
global_context->setFormatSchemaPath(format_schema_path);
fs::create_directories(format_schema_path);
@ -1089,7 +1091,7 @@ if (ThreadFuzzer::instance().isEffective())
/// system logs may copy global context.
global_context->setCurrentDatabaseNameInGlobalContext(default_database);
LOG_INFO(log, "Loading metadata from {}", path);
LOG_INFO(log, "Loading metadata from {}", path_str);
try
{

56
src/Core/ServerUUID.cpp Normal file
View File

@ -0,0 +1,56 @@
#include <Core/ServerUUID.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_CREATE_FILE;
}
void ServerUUID::load(const fs::path & server_uuid_file, Poco::Logger * log)
{
/// Write a uuid file containing a unique uuid if the file doesn't already exist during server start.
if (fs::exists(server_uuid_file))
{
try
{
UUID uuid;
ReadBufferFromFile in(server_uuid_file);
readUUIDText(uuid, in);
assertEOF(in);
server_uuid = uuid;
return;
}
catch (...)
{
/// As for now it's ok to just overwrite it, because persistency in not essential.
LOG_ERROR(log, "Cannot read server UUID from file {}: {}. Will overwrite it",
server_uuid_file.string(), getCurrentExceptionMessage(true));
}
}
try
{
UUID new_uuid = UUIDHelpers::generateV4();
auto uuid_str = toString(new_uuid);
WriteBufferFromFile out(server_uuid_file);
out.write(uuid_str.data(), uuid_str.size());
out.sync();
out.finalize();
server_uuid = new_uuid;
}
catch (...)
{
throw Exception(ErrorCodes::CANNOT_CREATE_FILE, "Caught Exception {} while writing the Server UUID file {}",
getCurrentExceptionMessage(false), server_uuid_file.string());
}
}
}

26
src/Core/ServerUUID.h Normal file
View File

@ -0,0 +1,26 @@
#pragma once
#include <Core/UUID.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace Poco
{
class Logger;
}
namespace DB
{
class ServerUUID
{
inline static UUID server_uuid = UUIDHelpers::Nil;
public:
/// Returns persistent UUID of current clickhouse-server or clickhouse-keeper instance.
static UUID get() { return server_uuid; }
/// Loads server UUID from file or creates new one. Should be called on daemon startup.
static void load(const fs::path & server_uuid_file, Poco::Logger * log);
};
}

View File

@ -78,6 +78,7 @@ void registerFunctionPartitionId(FunctionFactory & factory);
void registerFunctionIsIPAddressContainedIn(FunctionFactory &);
void registerFunctionQueryID(FunctionFactory & factory);
void registerFunctionInitialQueryID(FunctionFactory & factory);
void registerFunctionServerUUID(FunctionFactory &);
#if USE_ICU
void registerFunctionConvertCharset(FunctionFactory &);
@ -156,6 +157,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
registerFunctionIsIPAddressContainedIn(factory);
registerFunctionQueryID(factory);
registerFunctionInitialQueryID(factory);
registerFunctionServerUUID(factory);
#if USE_ICU
registerFunctionConvertCharset(factory);

View File

@ -0,0 +1,60 @@
#include <Core/ServerUUID.h>
#include <DataTypes/DataTypeUUID.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
namespace DB
{
namespace
{
class FunctionServerUUID : public IFunction
{
public:
static constexpr auto name = "serverUUID";
static FunctionPtr create(ContextPtr context)
{
return std::make_shared<FunctionServerUUID>(context->isDistributed(), ServerUUID::get());
}
explicit FunctionServerUUID(bool is_distributed_, UUID server_uuid_)
: is_distributed(is_distributed_), server_uuid(server_uuid_)
{
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes &) const override { return std::make_shared<DataTypeUUID>(); }
bool isDeterministic() const override { return false; }
bool isDeterministicInScopeOfQuery() const override { return true; }
bool isSuitableForConstantFolding() const override { return !is_distributed; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
{
return DataTypeUUID().createColumnConst(input_rows_count, server_uuid);
}
private:
bool is_distributed;
const UUID server_uuid;
};
}
void registerFunctionServerUUID(FunctionFactory & factory)
{
factory.registerFunction<FunctionServerUUID>();
}
}

View File

@ -305,3 +305,12 @@ def test_startup_without_zk(started_cluster):
main_node.query("EXCHANGE TABLES startup.rmt AND startup.m")
assert main_node.query("SELECT (*,).1 FROM startup.m") == "42\n"
def test_server_uuid(started_cluster):
uuid1 = main_node.query("select serverUUID()")
uuid2 = dummy_node.query("select serverUUID()")
assert uuid1 != uuid2
main_node.restart_clickhouse()
uuid1_after_restart = main_node.query("select serverUUID()")
assert uuid1 == uuid1_after_restart