mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
DatabaseReplicatedSettings to pImpl and reuse code
This commit is contained in:
parent
eb4a74d741
commit
52c40e6276
@ -1,5 +1,6 @@
|
|||||||
#include <Coordination/CoordinationSettings.h>
|
#include <Coordination/CoordinationSettings.h>
|
||||||
#include <Core/BaseSettings.h>
|
#include <Core/BaseSettings.h>
|
||||||
|
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <IO/WriteIntText.h>
|
#include <IO/WriteIntText.h>
|
||||||
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||||
@ -119,18 +120,7 @@ CoordinationSettings::CoordinationSettings(const CoordinationSettings & settings
|
|||||||
|
|
||||||
CoordinationSettings::~CoordinationSettings() = default;
|
CoordinationSettings::~CoordinationSettings() = default;
|
||||||
|
|
||||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
|
||||||
const SettingField##TYPE & CoordinationSettings::operator[](CLASS_NAME##TYPE t) const \
|
|
||||||
{ \
|
|
||||||
return impl.get()->*t; \
|
|
||||||
} \
|
|
||||||
SettingField##TYPE & CoordinationSettings::operator[](CLASS_NAME##TYPE t) \
|
|
||||||
{ \
|
|
||||||
return impl.get()->*t; \
|
|
||||||
}
|
|
||||||
|
|
||||||
COORDINATION_SETTINGS_SUPPORTED_TYPES(CoordinationSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
COORDINATION_SETTINGS_SUPPORTED_TYPES(CoordinationSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
|
||||||
|
|
||||||
void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
||||||
{
|
{
|
||||||
|
11
src/Core/BaseSettingsFwdMacrosImpl.h
Normal file
11
src/Core/BaseSettingsFwdMacrosImpl.h
Normal file
@ -0,0 +1,11 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
||||||
|
const SettingField##TYPE & CLASS_NAME::operator[](CLASS_NAME##TYPE t) const \
|
||||||
|
{ \
|
||||||
|
return impl.get()->*t; \
|
||||||
|
} \
|
||||||
|
SettingField##TYPE & CLASS_NAME::operator[](CLASS_NAME##TYPE t) \
|
||||||
|
{ \
|
||||||
|
return impl.get()->*t; \
|
||||||
|
}
|
@ -1,5 +1,6 @@
|
|||||||
#include <Core/BackgroundSchedulePool.h>
|
#include <Core/BackgroundSchedulePool.h>
|
||||||
#include <Core/BaseSettings.h>
|
#include <Core/BaseSettings.h>
|
||||||
|
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||||
#include <Core/ServerSettings.h>
|
#include <Core/ServerSettings.h>
|
||||||
#include <IO/MMappedFileCache.h>
|
#include <IO/MMappedFileCache.h>
|
||||||
#include <IO/UncompressedCache.h>
|
#include <IO/UncompressedCache.h>
|
||||||
@ -250,18 +251,7 @@ ServerSettings::ServerSettings(const ServerSettings & settings) : impl(std::make
|
|||||||
|
|
||||||
ServerSettings::~ServerSettings() = default;
|
ServerSettings::~ServerSettings() = default;
|
||||||
|
|
||||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
|
||||||
const SettingField##TYPE & ServerSettings::operator[](CLASS_NAME##TYPE t) const \
|
|
||||||
{ \
|
|
||||||
return impl.get()->*t; \
|
|
||||||
} \
|
|
||||||
SettingField##TYPE & ServerSettings::operator[](CLASS_NAME##TYPE t) \
|
|
||||||
{ \
|
|
||||||
return impl.get()->*t; \
|
|
||||||
}
|
|
||||||
|
|
||||||
SERVER_SETTINGS_SUPPORTED_TYPES(ServerSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
SERVER_SETTINGS_SUPPORTED_TYPES(ServerSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
|
||||||
|
|
||||||
void ServerSettings::set(std::string_view name, const Field & value)
|
void ServerSettings::set(std::string_view name, const Field & value)
|
||||||
{
|
{
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <Columns/ColumnMap.h>
|
#include <Columns/ColumnMap.h>
|
||||||
#include <Core/BaseSettings.h>
|
#include <Core/BaseSettings.h>
|
||||||
#include <Core/BaseSettingsFwdMacros.h>
|
#include <Core/BaseSettingsFwdMacros.h>
|
||||||
|
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||||
#include <Core/BaseSettingsProgramOptions.h>
|
#include <Core/BaseSettingsProgramOptions.h>
|
||||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||||
#include <Core/Settings.h>
|
#include <Core/Settings.h>
|
||||||
@ -6111,18 +6112,7 @@ bool Settings::operator==(const Settings & other) const
|
|||||||
return *impl == *other.impl;
|
return *impl == *other.impl;
|
||||||
}
|
}
|
||||||
|
|
||||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
|
||||||
const SettingField##TYPE & Settings::operator[](CLASS_NAME##TYPE t) const \
|
|
||||||
{ \
|
|
||||||
return impl.get()->*t; \
|
|
||||||
} \
|
|
||||||
SettingField##TYPE & Settings::operator[](CLASS_NAME##TYPE t) \
|
|
||||||
{ \
|
|
||||||
return impl.get()->*t; \
|
|
||||||
}
|
|
||||||
|
|
||||||
COMMON_SETTINGS_SUPPORTED_TYPES(Settings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
COMMON_SETTINGS_SUPPORTED_TYPES(Settings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
|
||||||
|
|
||||||
bool Settings::has(std::string_view name) const
|
bool Settings::has(std::string_view name) const
|
||||||
{
|
{
|
||||||
|
@ -64,6 +64,12 @@ namespace ServerSetting
|
|||||||
extern const ServerSettingsUInt32 max_database_replicated_create_table_thread_pool_size;
|
extern const ServerSettingsUInt32 max_database_replicated_create_table_thread_pool_size;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
namespace DatabaseReplicatedSetting
|
||||||
|
{
|
||||||
|
extern const DatabaseReplicatedSettingsString collection_name;
|
||||||
|
extern const DatabaseReplicatedSettingsFloat max_broken_tables_ratio;
|
||||||
|
}
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NO_ZOOKEEPER;
|
extern const int NO_ZOOKEEPER;
|
||||||
@ -141,8 +147,8 @@ DatabaseReplicated::DatabaseReplicated(
|
|||||||
if (zookeeper_path.front() != '/')
|
if (zookeeper_path.front() != '/')
|
||||||
zookeeper_path = "/" + zookeeper_path;
|
zookeeper_path = "/" + zookeeper_path;
|
||||||
|
|
||||||
if (!db_settings.collection_name.value.empty())
|
if (!db_settings[DatabaseReplicatedSetting::collection_name].value.empty())
|
||||||
fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef());
|
fillClusterAuthInfo(db_settings[DatabaseReplicatedSetting::collection_name].value, context_->getConfigRef());
|
||||||
|
|
||||||
replica_group_name = context_->getConfigRef().getString("replica_group_name", "");
|
replica_group_name = context_->getConfigRef().getString("replica_group_name", "");
|
||||||
|
|
||||||
@ -1220,7 +1226,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
|||||||
String db_name = getDatabaseName();
|
String db_name = getDatabaseName();
|
||||||
String to_db_name = getDatabaseName() + BROKEN_TABLES_SUFFIX;
|
String to_db_name = getDatabaseName() + BROKEN_TABLES_SUFFIX;
|
||||||
String to_db_name_replicated = getDatabaseName() + BROKEN_REPLICATED_TABLES_SUFFIX;
|
String to_db_name_replicated = getDatabaseName() + BROKEN_REPLICATED_TABLES_SUFFIX;
|
||||||
if (total_tables * db_settings.max_broken_tables_ratio < tables_to_detach.size())
|
if (total_tables * db_settings[DatabaseReplicatedSetting::max_broken_tables_ratio] < tables_to_detach.size())
|
||||||
throw Exception(ErrorCodes::DATABASE_REPLICATION_FAILED, "Too many tables to recreate: {} of {}", tables_to_detach.size(), total_tables);
|
throw Exception(ErrorCodes::DATABASE_REPLICATION_FAILED, "Too many tables to recreate: {} of {}", tables_to_detach.size(), total_tables);
|
||||||
if (!tables_to_detach.empty())
|
if (!tables_to_detach.empty())
|
||||||
{
|
{
|
||||||
|
@ -1,17 +1,61 @@
|
|||||||
|
#include <Core/BaseSettings.h>
|
||||||
|
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||||
#include <Databases/DatabaseReplicatedSettings.h>
|
#include <Databases/DatabaseReplicatedSettings.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
#define LIST_OF_DATABASE_REPLICATED_SETTINGS(M, ALIAS) \
|
||||||
|
M(Float, max_broken_tables_ratio, 1, "Do not recover replica automatically if the ratio of staled tables to all tables is greater", 0) \
|
||||||
|
M(UInt64, max_replication_lag_to_enqueue, 50, "Replica will throw exception on attempt to execute query if its replication lag greater", 0) \
|
||||||
|
M(UInt64, wait_entry_commited_timeout_sec, 3600, "Replicas will try to cancel query if timeout exceed, but initiator host has not executed it yet", 0) \
|
||||||
|
M(String, collection_name, "", "A name of a collection defined in server's config where all info for cluster authentication is defined", 0) \
|
||||||
|
M(Bool, check_consistency, true, "Check consistency of local metadata and metadata in Keeper, do replica recovery on inconsistency", 0) \
|
||||||
|
M(UInt64, max_retries_before_automatic_recovery, 100, "Max number of attempts to execute a queue entry before marking replica as lost recovering it from snapshot (0 means infinite)", 0) \
|
||||||
|
|
||||||
|
DECLARE_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
|
||||||
|
|
||||||
|
struct DatabaseReplicatedSettingsImpl : public BaseSettings<DatabaseReplicatedSettingsTraits>
|
||||||
|
{
|
||||||
|
};
|
||||||
|
|
||||||
IMPLEMENT_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
|
IMPLEMENT_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
|
||||||
|
|
||||||
|
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||||
|
DatabaseReplicatedSettings##TYPE NAME = &DatabaseReplicatedSettings##Impl ::NAME;
|
||||||
|
|
||||||
|
namespace DatabaseReplicatedSetting
|
||||||
|
{
|
||||||
|
LIST_OF_DATABASE_REPLICATED_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||||
|
}
|
||||||
|
|
||||||
|
#undef INITIALIZE_SETTING_EXTERN
|
||||||
|
|
||||||
|
DatabaseReplicatedSettings::DatabaseReplicatedSettings() : impl(std::make_unique<DatabaseReplicatedSettingsImpl>())
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
DatabaseReplicatedSettings::DatabaseReplicatedSettings(const DatabaseReplicatedSettings & settings)
|
||||||
|
: impl(std::make_unique<DatabaseReplicatedSettingsImpl>(*settings.impl))
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
DatabaseReplicatedSettings::DatabaseReplicatedSettings(DatabaseReplicatedSettings && settings)
|
||||||
|
: impl(std::make_unique<DatabaseReplicatedSettingsImpl>(std::move(*settings.impl)))
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
DatabaseReplicatedSettings::~DatabaseReplicatedSettings() = default;
|
||||||
|
|
||||||
|
DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(DatabaseReplicatedSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||||
|
|
||||||
void DatabaseReplicatedSettings::loadFromQuery(ASTStorage & storage_def)
|
void DatabaseReplicatedSettings::loadFromQuery(ASTStorage & storage_def)
|
||||||
{
|
{
|
||||||
if (storage_def.settings)
|
if (storage_def.settings)
|
||||||
{
|
{
|
||||||
applyChanges(storage_def.settings->changes);
|
impl->applyChanges(storage_def.settings->changes);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,27 +1,35 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
#include <Core/Defines.h>
|
|
||||||
#include <Core/BaseSettings.h>
|
#include <Core/BaseSettingsFwdMacros.h>
|
||||||
|
#include <Core/SettingsFields.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class ASTStorage;
|
class ASTStorage;
|
||||||
|
struct DatabaseReplicatedSettingsImpl;
|
||||||
|
|
||||||
#define LIST_OF_DATABASE_REPLICATED_SETTINGS(M, ALIAS) \
|
/// List of available types supported in ServerSettings object
|
||||||
M(Float, max_broken_tables_ratio, 1, "Do not recover replica automatically if the ratio of staled tables to all tables is greater", 0) \
|
#define DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||||
M(UInt64, max_replication_lag_to_enqueue, 50, "Replica will throw exception on attempt to execute query if its replication lag greater", 0) \
|
M(CLASS_NAME, Bool) \
|
||||||
M(UInt64, wait_entry_commited_timeout_sec, 3600, "Replicas will try to cancel query if timeout exceed, but initiator host has not executed it yet", 0) \
|
M(CLASS_NAME, Float) \
|
||||||
M(String, collection_name, "", "A name of a collection defined in server's config where all info for cluster authentication is defined", 0) \
|
M(CLASS_NAME, String) \
|
||||||
M(Bool, check_consistency, true, "Check consistency of local metadata and metadata in Keeper, do replica recovery on inconsistency", 0) \
|
M(CLASS_NAME, UInt64)
|
||||||
M(UInt64, max_retries_before_automatic_recovery, 100, "Max number of attempts to execute a queue entry before marking replica as lost recovering it from snapshot (0 means infinite)", 0) \
|
|
||||||
|
|
||||||
|
DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(DatabaseReplicatedSettings, DECLARE_SETTING_TRAIT)
|
||||||
|
|
||||||
DECLARE_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
|
struct DatabaseReplicatedSettings
|
||||||
|
|
||||||
|
|
||||||
struct DatabaseReplicatedSettings : public BaseSettings<DatabaseReplicatedSettingsTraits>
|
|
||||||
{
|
{
|
||||||
|
DatabaseReplicatedSettings();
|
||||||
|
DatabaseReplicatedSettings(const DatabaseReplicatedSettings & settings);
|
||||||
|
DatabaseReplicatedSettings(DatabaseReplicatedSettings && settings);
|
||||||
|
~DatabaseReplicatedSettings();
|
||||||
|
|
||||||
|
DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(DatabaseReplicatedSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||||
|
|
||||||
void loadFromQuery(ASTStorage & storage_def);
|
void loadFromQuery(ASTStorage & storage_def);
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::unique_ptr<DatabaseReplicatedSettingsImpl> impl;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -18,6 +18,14 @@ namespace Setting
|
|||||||
extern const SettingsUInt64 database_replicated_initial_query_timeout_sec;
|
extern const SettingsUInt64 database_replicated_initial_query_timeout_sec;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
namespace DatabaseReplicatedSetting
|
||||||
|
{
|
||||||
|
extern const DatabaseReplicatedSettingsBool check_consistency;
|
||||||
|
extern const DatabaseReplicatedSettingsUInt64 max_replication_lag_to_enqueue;
|
||||||
|
extern const DatabaseReplicatedSettingsUInt64 max_retries_before_automatic_recovery;
|
||||||
|
extern const DatabaseReplicatedSettingsUInt64 wait_entry_commited_timeout_sec;
|
||||||
|
}
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
@ -63,8 +71,8 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread()
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (database->db_settings.max_retries_before_automatic_recovery &&
|
if (database->db_settings[DatabaseReplicatedSetting::max_retries_before_automatic_recovery]
|
||||||
database->db_settings.max_retries_before_automatic_recovery <= subsequent_errors_count)
|
&& database->db_settings[DatabaseReplicatedSetting::max_retries_before_automatic_recovery] <= subsequent_errors_count)
|
||||||
{
|
{
|
||||||
String current_task_name;
|
String current_task_name;
|
||||||
{
|
{
|
||||||
@ -155,7 +163,7 @@ void DatabaseReplicatedDDLWorker::initializeReplication()
|
|||||||
|
|
||||||
bool is_new_replica = our_log_ptr == 0;
|
bool is_new_replica = our_log_ptr == 0;
|
||||||
bool lost_according_to_log_ptr = our_log_ptr + logs_to_keep < max_log_ptr;
|
bool lost_according_to_log_ptr = our_log_ptr + logs_to_keep < max_log_ptr;
|
||||||
bool lost_according_to_digest = database->db_settings.check_consistency && local_digest != digest;
|
bool lost_according_to_digest = database->db_settings[DatabaseReplicatedSetting::check_consistency] && local_digest != digest;
|
||||||
|
|
||||||
if (is_new_replica || lost_according_to_log_ptr || lost_according_to_digest)
|
if (is_new_replica || lost_according_to_log_ptr || lost_according_to_digest)
|
||||||
{
|
{
|
||||||
@ -306,7 +314,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
|
|||||||
UInt32 our_log_ptr = getLogPointer();
|
UInt32 our_log_ptr = getLogPointer();
|
||||||
UInt32 max_log_ptr = parse<UInt32>(zookeeper->get(database->zookeeper_path + "/max_log_ptr"));
|
UInt32 max_log_ptr = parse<UInt32>(zookeeper->get(database->zookeeper_path + "/max_log_ptr"));
|
||||||
|
|
||||||
if (our_log_ptr + database->db_settings.max_replication_lag_to_enqueue < max_log_ptr)
|
if (our_log_ptr + database->db_settings[DatabaseReplicatedSetting::max_replication_lag_to_enqueue] < max_log_ptr)
|
||||||
throw Exception(ErrorCodes::NOT_A_LEADER, "Cannot enqueue query on this replica, "
|
throw Exception(ErrorCodes::NOT_A_LEADER, "Cannot enqueue query on this replica, "
|
||||||
"because it has replication lag of {} queries. Try other replica.", max_log_ptr - our_log_ptr);
|
"because it has replication lag of {} queries. Try other replica.", max_log_ptr - our_log_ptr);
|
||||||
|
|
||||||
@ -399,7 +407,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
|
|||||||
/// Query is not committed yet. We cannot just skip it and execute next one, because reordering may break replication.
|
/// Query is not committed yet. We cannot just skip it and execute next one, because reordering may break replication.
|
||||||
LOG_TRACE(log, "Waiting for initiator {} to commit or rollback entry {}", initiator_name, entry_path);
|
LOG_TRACE(log, "Waiting for initiator {} to commit or rollback entry {}", initiator_name, entry_path);
|
||||||
constexpr size_t wait_time_ms = 1000;
|
constexpr size_t wait_time_ms = 1000;
|
||||||
size_t max_iterations = database->db_settings.wait_entry_commited_timeout_sec;
|
size_t max_iterations = database->db_settings[DatabaseReplicatedSetting::wait_entry_commited_timeout_sec];
|
||||||
size_t iteration = 0;
|
size_t iteration = 0;
|
||||||
|
|
||||||
while (!wait_committed_or_failed->tryWait(wait_time_ms))
|
while (!wait_committed_or_failed->tryWait(wait_time_ms))
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Core/BaseSettings.h>
|
#include <Core/BaseSettings.h>
|
||||||
|
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||||
#include <Core/BaseSettingsProgramOptions.h>
|
#include <Core/BaseSettingsProgramOptions.h>
|
||||||
#include <Core/MergeSelectorAlgorithm.h>
|
#include <Core/MergeSelectorAlgorithm.h>
|
||||||
#include <Core/SettingsChangesHistory.h>
|
#include <Core/SettingsChangesHistory.h>
|
||||||
@ -511,18 +512,7 @@ MergeTreeSettings::MergeTreeSettings(MergeTreeSettings && settings) noexcept
|
|||||||
|
|
||||||
MergeTreeSettings::~MergeTreeSettings() = default;
|
MergeTreeSettings::~MergeTreeSettings() = default;
|
||||||
|
|
||||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
|
||||||
const SettingField##TYPE & MergeTreeSettings::operator[](CLASS_NAME##TYPE t) const \
|
|
||||||
{ \
|
|
||||||
return impl.get()->*t; \
|
|
||||||
} \
|
|
||||||
SettingField##TYPE & MergeTreeSettings::operator[](CLASS_NAME##TYPE t) \
|
|
||||||
{ \
|
|
||||||
return impl.get()->*t; \
|
|
||||||
}
|
|
||||||
|
|
||||||
MERGETREE_SETTINGS_SUPPORTED_TYPES(MergeTreeSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
MERGETREE_SETTINGS_SUPPORTED_TYPES(MergeTreeSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
|
||||||
|
|
||||||
bool MergeTreeSettings::has(std::string_view name) const
|
bool MergeTreeSettings::has(std::string_view name) const
|
||||||
{
|
{
|
||||||
|
77
utils/check-style/check-settings-style
Executable file
77
utils/check-style/check-settings-style
Executable file
@ -0,0 +1,77 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
# Fast check of all the setting struct usages
|
||||||
|
# The linker does not complain about incorrect extern usage, so we need to make sure the style checker handles
|
||||||
|
|
||||||
|
LC_ALL="en_US.UTF-8"
|
||||||
|
ROOT_PATH=$(git rev-parse --show-toplevel)
|
||||||
|
|
||||||
|
# Duplicated or incorrect setting declarations
|
||||||
|
SETTINGS_FILE=$(mktemp)
|
||||||
|
ALL_DECLARATION_FILES="
|
||||||
|
$ROOT_PATH/src/Core/Settings.cpp
|
||||||
|
$ROOT_PATH/src/Core/ServerSettings.cpp
|
||||||
|
$ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp
|
||||||
|
$ROOT_PATH/src/Coordination/CoordinationSettings.cpp
|
||||||
|
$ROOT_PATH/src/Databases/DatabaseReplicatedSettings.cpp
|
||||||
|
$ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h"
|
||||||
|
|
||||||
|
cat $ROOT_PATH/src/Core/Settings.cpp $ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " Settings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq > ${SETTINGS_FILE}
|
||||||
|
cat $ROOT_PATH/src/Core/ServerSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " ServerSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||||
|
cat $ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " MergeTreeSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||||
|
cat $ROOT_PATH/src/Coordination/CoordinationSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " CoordinationSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||||
|
cat $ROOT_PATH/src/Databases/DatabaseReplicatedSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " DatabaseReplicatedSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||||
|
|
||||||
|
# Check that if there are duplicated settings (declared in different objects) they all have the same type (it's simpler to validate style with that assert)
|
||||||
|
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||||
|
sed -e 's/CoordinationSettings//g' \
|
||||||
|
-e 's/DatabaseReplicatedSettings//g' \
|
||||||
|
-e 's/MergeTreeSettings//g' \
|
||||||
|
-e 's/ServerSettings//g' \
|
||||||
|
-e 's/Settings//g' | \
|
||||||
|
sort | uniq | awk '{ print $1 }' | uniq -d);
|
||||||
|
do
|
||||||
|
echo "# Found multiple definitions of setting ${setting} with different types: "
|
||||||
|
grep --line-number " ${setting}," ${ALL_DECLARATION_FILES} | awk '{print " > " $0 }'
|
||||||
|
done
|
||||||
|
|
||||||
|
# We append all uses of extern found in implementation files to validate them in a single pass and avoid reading the same files over and over
|
||||||
|
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | \
|
||||||
|
xargs grep -e "^\s*extern const Settings" \
|
||||||
|
-e "^\s**extern const ServerSettings" \
|
||||||
|
-e "^\s**extern const MergeTreeSettings" \
|
||||||
|
-e "^\s**extern const DatabaseReplicatedSettings" \
|
||||||
|
-e "^\s**extern const CoordinationSettings" -T | \
|
||||||
|
awk '{print substr($5, 0, length($5) -1) " " $4 " " substr($1, 0, length($1) - 1)}' >> ${SETTINGS_FILE}
|
||||||
|
|
||||||
|
# Duplicate extern declarations for settings
|
||||||
|
awk '{if (seen[$0]++) print $3 " -> " $1 ;}' ${SETTINGS_FILE} | while read line;
|
||||||
|
do
|
||||||
|
echo "# Found duplicated setting declaration in: $line"
|
||||||
|
done
|
||||||
|
|
||||||
|
# Find missing declarations (obsolete settings being used)
|
||||||
|
# Note that SettingsDeclaration are first in the file
|
||||||
|
# Disabled for now pending fixing the code
|
||||||
|
#awk '{print $1 " " $3}' ${SETTINGS_FILE} | awk '{if (!seen[$1]++) print $0}' | grep -v SettingsDeclaration | while read setting;
|
||||||
|
#do
|
||||||
|
# echo "Could not find setting (maybe obsolete but used?) $setting"
|
||||||
|
#done
|
||||||
|
|
||||||
|
# Look for settings declared with multiple types
|
||||||
|
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||||
|
sed -e 's/MergeTreeSettings//g' \
|
||||||
|
-e 's/ServerSettings//g' \
|
||||||
|
-e 's/CoordinationSettings//g' \
|
||||||
|
-e 's/DatabaseReplicatedSettings//g' \
|
||||||
|
-e 's/Settings//g' | \
|
||||||
|
sort | uniq | awk '{ print $1 }' | sort | uniq -d);
|
||||||
|
do
|
||||||
|
expected=$(grep "^$setting " ${SETTINGS_FILE} | grep SettingsDeclaration | awk '{ print $2 }')
|
||||||
|
grep "^$setting " ${SETTINGS_FILE} | grep -v " $expected" | awk '{ print $3 " found setting " $1 " with type " $2 }' | while read line;
|
||||||
|
do
|
||||||
|
echo "# In $line but it should be ${expected/$'\n'/ }"
|
||||||
|
done
|
||||||
|
done
|
||||||
|
|
||||||
|
rm ${SETTINGS_FILE}
|
@ -53,69 +53,7 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/n
|
|||||||
find -L $ROOT_PATH -type l 2>/dev/null | grep -v contrib && echo "^ Broken symlinks found"
|
find -L $ROOT_PATH -type l 2>/dev/null | grep -v contrib && echo "^ Broken symlinks found"
|
||||||
|
|
||||||
# Duplicated or incorrect setting declarations
|
# Duplicated or incorrect setting declarations
|
||||||
SETTINGS_FILE=$(mktemp)
|
bash $ROOT_PATH/utils/check-style/check-settings-style
|
||||||
ALL_DECLARATION_FILES="
|
|
||||||
$ROOT_PATH/src/Core/Settings.cpp
|
|
||||||
$ROOT_PATH/src/Core/ServerSettings.cpp
|
|
||||||
$ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp
|
|
||||||
$ROOT_PATH/src/Coordination/CoordinationSettings.cpp
|
|
||||||
$ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h"
|
|
||||||
|
|
||||||
cat $ROOT_PATH/src/Core/Settings.cpp $ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " Settings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq > ${SETTINGS_FILE}
|
|
||||||
cat $ROOT_PATH/src/Core/ServerSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " ServerSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
|
||||||
cat $ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " MergeTreeSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
|
||||||
cat $ROOT_PATH/src/Coordination/CoordinationSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " CoordinationSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
|
||||||
|
|
||||||
# Check that if there are duplicated settings (declared in different objects) they all have the same type (it's simpler to validate style with that assert)
|
|
||||||
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
|
||||||
sed -e 's/CoordinationSettings//g' \
|
|
||||||
-e 's/MergeTreeSettings//g' \
|
|
||||||
-e 's/ServerSettings//g' \
|
|
||||||
-e 's/Settings//g' | \
|
|
||||||
sort | uniq | awk '{ print $1 }' | uniq -d);
|
|
||||||
do
|
|
||||||
echo "# Found multiple definitions of setting ${setting} with different types: "
|
|
||||||
grep --line-number " ${setting}," ${ALL_DECLARATION_FILES} | awk '{print " > " $0 }'
|
|
||||||
done
|
|
||||||
|
|
||||||
# We append all uses of extern found in implementation files to validate them in a single pass and avoid reading the same files over and over
|
|
||||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | \
|
|
||||||
xargs grep -e "^\s*extern const Settings" \
|
|
||||||
-e "^\s**extern const ServerSettings" \
|
|
||||||
-e "^\s**extern const MergeTreeSettings" \
|
|
||||||
-e "^\s**extern const CoordinationSettings" -T | \
|
|
||||||
awk '{print substr($5, 0, length($5) -1) " " $4 " " substr($1, 0, length($1) - 1)}' >> ${SETTINGS_FILE}
|
|
||||||
|
|
||||||
# Duplicate extern declarations for settings
|
|
||||||
awk '{if (seen[$0]++) print $3 " -> " $1 ;}' ${SETTINGS_FILE} | while read line;
|
|
||||||
do
|
|
||||||
echo "# Found duplicated setting declaration in: $line"
|
|
||||||
done
|
|
||||||
|
|
||||||
# Find missing declarations (obsolete settings being used)
|
|
||||||
# Note that SettingsDeclaration are first in the file
|
|
||||||
# Disabled for now pending fixing the code
|
|
||||||
#awk '{print $1 " " $3}' ${SETTINGS_FILE} | awk '{if (!seen[$1]++) print $0}' | grep -v SettingsDeclaration | while read setting;
|
|
||||||
#do
|
|
||||||
# echo "Could not find setting (maybe obsolete but used?) $setting"
|
|
||||||
#done
|
|
||||||
|
|
||||||
# Look for settings declared with multiple types
|
|
||||||
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
|
||||||
sed -e 's/MergeTreeSettings//g' \
|
|
||||||
-e 's/ServerSettings//g' \
|
|
||||||
-e 's/CoordinationSettings//g' \
|
|
||||||
-e 's/Settings//g' | \
|
|
||||||
sort | uniq | awk '{ print $1 }' | sort | uniq -d);
|
|
||||||
do
|
|
||||||
expected=$(grep "^$setting " ${SETTINGS_FILE} | grep SettingsDeclaration | awk '{ print $2 }')
|
|
||||||
grep "^$setting " ${SETTINGS_FILE} | grep -v " $expected" | awk '{ print $3 " found setting " $1 " with type " $2 }' | while read line;
|
|
||||||
do
|
|
||||||
echo "# In $line but it should be ${expected/$'\n'/ }"
|
|
||||||
done
|
|
||||||
done
|
|
||||||
|
|
||||||
rm ${SETTINGS_FILE}
|
|
||||||
|
|
||||||
# Unused/Undefined/Duplicates ErrorCodes/ProfileEvents/CurrentMetrics
|
# Unused/Undefined/Duplicates ErrorCodes/ProfileEvents/CurrentMetrics
|
||||||
declare -A EXTERN_TYPES
|
declare -A EXTERN_TYPES
|
||||||
|
Loading…
Reference in New Issue
Block a user