mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
minor improvements, add test
This commit is contained in:
parent
b324889241
commit
9e9fa043ca
@ -44,6 +44,7 @@
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
|
||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/Exception.h>
|
||||
@ -1059,3 +1060,49 @@ String BaseDaemon::getStoredBinaryHash() const
|
||||
{
|
||||
return stored_binary_hash;
|
||||
}
|
||||
|
||||
void BaseDaemon::loadServerUUID(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
|
||||
{
|
||||
DB::UUID uuid;
|
||||
DB::ReadBufferFromFile in(server_uuid_file);
|
||||
DB::readUUIDText(uuid, in);
|
||||
DB::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(), DB::getCurrentExceptionMessage(true));
|
||||
}
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
DB::UUID new_uuid = DB::UUIDHelpers::generateV4();
|
||||
auto uuid_str = DB::toString(new_uuid);
|
||||
DB::WriteBufferFromFile out(server_uuid_file);
|
||||
out.write(uuid_str.data(), uuid_str.size());
|
||||
out.sync();
|
||||
out.finalize();
|
||||
server_uuid = new_uuid;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
throw Poco::Exception(
|
||||
"Caught Exception " + DB::getCurrentExceptionMessage(true) + " while writing the Server UUID file "
|
||||
+ server_uuid_file.string());
|
||||
}
|
||||
}
|
||||
|
||||
DB::UUID BaseDaemon::getServerUUID() const
|
||||
{
|
||||
return server_uuid;
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <iostream>
|
||||
#include <memory>
|
||||
#include <functional>
|
||||
#include <filesystem>
|
||||
#include <optional>
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
@ -24,6 +25,7 @@
|
||||
#include <Common/StatusFile.h>
|
||||
#include <loggers/Loggers.h>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
/// \brief Base class for applications that can run as daemons.
|
||||
///
|
||||
@ -124,6 +126,9 @@ public:
|
||||
/// Hash of the binary for integrity checks.
|
||||
String getStoredBinaryHash() const;
|
||||
|
||||
void loadServerUUID(const fs::path & server_uuid_file, Poco::Logger * log);
|
||||
DB::UUID getServerUUID() const;
|
||||
|
||||
protected:
|
||||
virtual void logRevision() const;
|
||||
|
||||
@ -179,6 +184,8 @@ protected:
|
||||
|
||||
bool should_setup_watchdog = false;
|
||||
char * argv0 = nullptr;
|
||||
|
||||
DB::UUID server_uuid = DB::UUIDHelpers::Nil;
|
||||
};
|
||||
|
||||
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Common/SymbolIndex.h>
|
||||
#include <Common/StackTrace.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/getServerUUID.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 = getServerUUID();
|
||||
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));
|
||||
|
@ -326,6 +326,8 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
}
|
||||
}
|
||||
|
||||
loadServerUUID(path + "/uuid", log);
|
||||
|
||||
const Settings & settings = global_context->getSettingsRef();
|
||||
|
||||
GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 100));
|
||||
|
@ -13,7 +13,6 @@
|
||||
#include <Poco/Net/HTTPServer.h>
|
||||
#include <Poco/Net/NetException.h>
|
||||
#include <Poco/Util/HelpFormatter.h>
|
||||
#include <Poco/UUIDGenerator.h>
|
||||
#include <Poco/Environment.h>
|
||||
#include <common/scope_guard.h>
|
||||
#include <common/defines.h>
|
||||
@ -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,40 +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};
|
||||
|
||||
|
||||
/// Write a uuid file containing a unique uuid if the file doesn't already exist during server start.
|
||||
{
|
||||
fs::path server_uuid_file = fs::path(path) / "uuid";
|
||||
|
||||
if (!fs::exists(server_uuid_file))
|
||||
{
|
||||
try
|
||||
{
|
||||
/// Note: Poco::UUIDGenerator().createRandom() uses /dev/random and can be expensive. But since
|
||||
/// it's only going to be generated once (i.e if the uuid file doesn't exist), it's probably fine.
|
||||
auto uuid_str = Poco::UUIDGenerator().createRandom().toString();
|
||||
WriteBufferFromFile out(server_uuid_file.string());
|
||||
out.write(uuid_str.data(), uuid_str.size());
|
||||
out.sync();
|
||||
out.finalize();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
throw Poco::Exception(
|
||||
"Caught Exception " + getCurrentExceptionMessage(false) + " while writing the Server UUID file "
|
||||
+ server_uuid_file.string());
|
||||
}
|
||||
LOG_INFO(log, "Server UUID file {} containing a unique UUID has been written.\n", server_uuid_file.string());
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Server UUID file {} already exists, will keep it.\n", server_uuid_file.string());
|
||||
}
|
||||
}
|
||||
loadServerUUID(path / "uuid", log);
|
||||
|
||||
/// Try to increase limit on number of open files.
|
||||
{
|
||||
@ -758,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())
|
||||
@ -770,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);
|
||||
}
|
||||
@ -779,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"))
|
||||
@ -984,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);
|
||||
|
||||
@ -1120,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
|
||||
{
|
||||
|
12
src/Common/getServerUUID.cpp
Normal file
12
src/Common/getServerUUID.cpp
Normal file
@ -0,0 +1,12 @@
|
||||
#include <Common/getServerUUID.h>
|
||||
#include <daemon/BaseDaemon.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
DB::UUID getServerUUID()
|
||||
{
|
||||
const auto * daemon = dynamic_cast<const BaseDaemon *>(&Poco::Util::Application::instance());
|
||||
if (daemon)
|
||||
return daemon->getServerUUID();
|
||||
else
|
||||
return DB::UUIDHelpers::Nil;
|
||||
}
|
5
src/Common/getServerUUID.h
Normal file
5
src/Common/getServerUUID.h
Normal file
@ -0,0 +1,5 @@
|
||||
#pragma once
|
||||
#include <Core/Types.h>
|
||||
|
||||
/// Returns persistent UUID of current clickhouse-server or clickhouse-keeper instance.
|
||||
DB::UUID getServerUUID();
|
@ -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);
|
||||
|
58
src/Functions/serverUUID.cpp
Normal file
58
src/Functions/serverUUID.cpp
Normal file
@ -0,0 +1,58 @@
|
||||
#include <Common/getServerUUID.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(), getServerUUID());
|
||||
}
|
||||
|
||||
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; }
|
||||
|
||||
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>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user