mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-15 12:14:18 +00:00
Merge pull request #70840 from Algunenano/settings4
More setting structs to pImpl and reuse code
This commit is contained in:
commit
91223fc4eb
@ -1,5 +1,6 @@
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteIntText.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||
@ -64,14 +65,13 @@ namespace ErrorCodes
|
||||
M(UInt64, log_slow_connection_operation_threshold_ms, 1000, "Log message if a certain operation took too long inside a single connection", 0)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
|
||||
|
||||
struct CoordinationSettingsImpl : public BaseSettings<CoordinationSettingsTraits>
|
||||
{
|
||||
void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
||||
};
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
|
||||
|
||||
void CoordinationSettingsImpl::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
if (!config.has(config_elem))
|
||||
@ -99,7 +99,7 @@ void CoordinationSettingsImpl::loadFromConfig(const String & config_elem, const
|
||||
}
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
CoordinationSettings##TYPE NAME = &CoordinationSettings##Impl ::NAME;
|
||||
CoordinationSettings##TYPE NAME = &CoordinationSettingsImpl ::NAME;
|
||||
|
||||
namespace CoordinationSetting
|
||||
{
|
||||
@ -119,18 +119,7 @@ CoordinationSettings::CoordinationSettings(const CoordinationSettings & settings
|
||||
|
||||
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)
|
||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
||||
|
||||
void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
|
@ -25,19 +25,51 @@ class WriteBuffer;
|
||||
* Example of usage:
|
||||
*
|
||||
* mysettings.h:
|
||||
* #include <Core/BaseSettingsFwdMacros.h>
|
||||
* #include <Core/SettingsFields.h>
|
||||
*
|
||||
* #define MY_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
* M(CLASS_NAME, Float) \
|
||||
* M(CLASS_NAME, String) \
|
||||
* M(CLASS_NAME, UInt64)
|
||||
*
|
||||
* MY_SETTINGS_SUPPORTED_TYPES(MySettings, DECLARE_SETTING_TRAIT)
|
||||
*
|
||||
* struct MySettings
|
||||
* {
|
||||
* MySettings();
|
||||
* ~MySettings();
|
||||
*
|
||||
* MY_SETTINGS_SUPPORTED_TYPES(MySettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
* private:
|
||||
* std::unique_ptr<MySettingsImpl> impl;
|
||||
* };
|
||||
*
|
||||
* mysettings.cpp:
|
||||
* #include <Core/BaseSettings.h>
|
||||
* #include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
*
|
||||
* #define APPLY_FOR_MYSETTINGS(M) \
|
||||
* M(UInt64, a, 100, "Description of a", 0) \
|
||||
* M(Float, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \
|
||||
* M(String, s, "default", "Description of s", 0)
|
||||
*
|
||||
* DECLARE_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS)
|
||||
|
||||
* struct MySettings : public BaseSettings<MySettingsTraits>
|
||||
* IMPLEMENT_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS)
|
||||
*
|
||||
* struct MySettingsImpl : public BaseSettings<MySettingsTraits>
|
||||
* {
|
||||
* };
|
||||
*
|
||||
* mysettings.cpp:
|
||||
* IMPLEMENT_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS)
|
||||
* #define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) MySettings##TYPE NAME = &MySettings##Impl ::NAME;
|
||||
*
|
||||
* namespace MySetting
|
||||
* {
|
||||
* APPLY_FOR_MYSETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
* }
|
||||
* #undef INITIALIZE_SETTING_EXTERN
|
||||
*
|
||||
* MY_SETTINGS_SUPPORTED_TYPES(MySettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
*/
|
||||
template <class TTraits>
|
||||
class BaseSettings : public TTraits::Data
|
||||
|
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; \
|
||||
}
|
@ -8,11 +8,9 @@ namespace DB
|
||||
* User-specified file format settings for File and URL engines.
|
||||
*/
|
||||
DECLARE_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS)
|
||||
|
||||
struct FormatFactorySettingsImpl : public BaseSettings<FormatFactorySettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS)
|
||||
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <IO/MMappedFileCache.h>
|
||||
#include <IO/UncompressedCache.h>
|
||||
@ -193,14 +194,13 @@ namespace DB
|
||||
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
||||
|
||||
struct ServerSettingsImpl : public BaseSettings<ServerSettingsTraits>
|
||||
{
|
||||
void loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
};
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
||||
|
||||
void ServerSettingsImpl::loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
// settings which can be loaded from the the default profile, see also MAKE_DEPRECATED_BY_SERVER_CONFIG in src/Core/Settings.h
|
||||
@ -231,7 +231,7 @@ void ServerSettingsImpl::loadSettingsFromConfig(const Poco::Util::AbstractConfig
|
||||
}
|
||||
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) ServerSettings##TYPE NAME = &ServerSettings##Impl ::NAME;
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) ServerSettings##TYPE NAME = &ServerSettingsImpl ::NAME;
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
@ -250,18 +250,7 @@ ServerSettings::ServerSettings(const ServerSettings & settings) : impl(std::make
|
||||
|
||||
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)
|
||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
||||
|
||||
void ServerSettings::set(std::string_view name, const Field & value)
|
||||
{
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/BaseSettingsProgramOptions.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/Settings.h>
|
||||
@ -5899,7 +5900,7 @@ Allow writing simple SELECT queries without the leading SELECT keyword, which ma
|
||||
OBSOLETE_FORMAT_SETTINGS(M, ALIAS) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SettingsTraits, LIST_OF_SETTINGS)
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS)
|
||||
|
||||
/** Settings of query execution.
|
||||
* These settings go to users.xml.
|
||||
@ -5933,9 +5934,6 @@ private:
|
||||
std::unordered_set<std::string_view> settings_changed_by_compatibility_setting;
|
||||
};
|
||||
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS)
|
||||
|
||||
/** Set the settings from the profile (in the server configuration, many settings can be listed in one profile).
|
||||
* The profile can also be set using the `set` functions, like the `profile` setting.
|
||||
*/
|
||||
@ -6083,7 +6081,7 @@ void SettingsImpl::applyCompatibilitySetting(const String & compatibility_value)
|
||||
}
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
Settings ## TYPE NAME = & Settings ## Impl :: NAME;
|
||||
Settings ## TYPE NAME = & SettingsImpl :: NAME;
|
||||
|
||||
namespace Setting
|
||||
{
|
||||
@ -6117,18 +6115,7 @@ bool Settings::operator==(const Settings & other) const
|
||||
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)
|
||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
namespace DatabaseReplicatedSetting
|
||||
{
|
||||
extern const DatabaseReplicatedSettingsString collection_name;
|
||||
extern const DatabaseReplicatedSettingsFloat max_broken_tables_ratio;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NO_ZOOKEEPER;
|
||||
@ -141,8 +147,8 @@ DatabaseReplicated::DatabaseReplicated(
|
||||
if (zookeeper_path.front() != '/')
|
||||
zookeeper_path = "/" + zookeeper_path;
|
||||
|
||||
if (!db_settings.collection_name.value.empty())
|
||||
fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef());
|
||||
if (!db_settings[DatabaseReplicatedSetting::collection_name].value.empty())
|
||||
fillClusterAuthInfo(db_settings[DatabaseReplicatedSetting::collection_name].value, context_->getConfigRef());
|
||||
|
||||
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 to_db_name = getDatabaseName() + BROKEN_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);
|
||||
if (!tables_to_detach.empty())
|
||||
{
|
||||
|
@ -1,17 +1,60 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Databases/DatabaseReplicatedSettings.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
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)
|
||||
IMPLEMENT_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
|
||||
|
||||
struct DatabaseReplicatedSettingsImpl : public BaseSettings<DatabaseReplicatedSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
DatabaseReplicatedSettings##TYPE NAME = &DatabaseReplicatedSettingsImpl ::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) noexcept
|
||||
: 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)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
impl->applyChanges(storage_def.settings->changes);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -1,27 +1,35 @@
|
||||
#pragma once
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTStorage;
|
||||
struct DatabaseReplicatedSettingsImpl;
|
||||
|
||||
#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) \
|
||||
/// List of available types supported in ReplicatedSettings object
|
||||
#define DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, Float) \
|
||||
M(CLASS_NAME, String) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(DatabaseReplicatedSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
|
||||
|
||||
|
||||
struct DatabaseReplicatedSettings : public BaseSettings<DatabaseReplicatedSettingsTraits>
|
||||
struct DatabaseReplicatedSettings
|
||||
{
|
||||
DatabaseReplicatedSettings();
|
||||
DatabaseReplicatedSettings(const DatabaseReplicatedSettings & settings);
|
||||
DatabaseReplicatedSettings(DatabaseReplicatedSettings && settings) noexcept;
|
||||
~DatabaseReplicatedSettings();
|
||||
|
||||
DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(DatabaseReplicatedSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -63,8 +71,8 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread()
|
||||
break;
|
||||
}
|
||||
|
||||
if (database->db_settings.max_retries_before_automatic_recovery &&
|
||||
database->db_settings.max_retries_before_automatic_recovery <= subsequent_errors_count)
|
||||
if (database->db_settings[DatabaseReplicatedSetting::max_retries_before_automatic_recovery]
|
||||
&& database->db_settings[DatabaseReplicatedSetting::max_retries_before_automatic_recovery] <= subsequent_errors_count)
|
||||
{
|
||||
String current_task_name;
|
||||
{
|
||||
@ -155,7 +163,7 @@ void DatabaseReplicatedDDLWorker::initializeReplication()
|
||||
|
||||
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_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)
|
||||
{
|
||||
@ -306,7 +314,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
|
||||
UInt32 our_log_ptr = getLogPointer();
|
||||
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, "
|
||||
"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.
|
||||
LOG_TRACE(log, "Waiting for initiator {} to commit or rollback entry {}", initiator_name, entry_path);
|
||||
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;
|
||||
|
||||
while (!wait_committed_or_failed->tryWait(wait_time_ms))
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
@ -37,6 +38,11 @@ namespace Setting
|
||||
extern const SettingsUInt64 postgresql_connection_attempt_timeout;
|
||||
}
|
||||
|
||||
namespace MaterializedPostgreSQLSetting
|
||||
{
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_tables_list;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -362,7 +368,7 @@ void DatabaseMaterializedPostgreSQL::attachTable(ContextPtr context_, const Stri
|
||||
|
||||
try
|
||||
{
|
||||
auto tables_to_replicate = settings->materialized_postgresql_tables_list.value;
|
||||
auto tables_to_replicate = (*settings)[MaterializedPostgreSQLSetting::materialized_postgresql_tables_list].value;
|
||||
if (tables_to_replicate.empty())
|
||||
tables_to_replicate = getFormattedTablesList();
|
||||
|
||||
|
@ -5,7 +5,6 @@
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <Storages/PostgreSQL/PostgreSQLReplicationHandler.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
@ -18,6 +17,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct MaterializedPostgreSQLSettings;
|
||||
class PostgreSQLConnection;
|
||||
using PostgreSQLConnectionPtr = std::shared_ptr<PostgreSQLConnection>;
|
||||
|
||||
|
@ -1,8 +1,57 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Storages/MaterializedView/RefreshSettings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
#define LIST_OF_REFRESH_SETTINGS(M, ALIAS) \
|
||||
M(Int64, refresh_retries, 2, "How many times to retry refresh query if it fails. If all attempts fail, wait for the next refresh time according to schedule. 0 to disable retries. -1 for infinite retries.", 0) \
|
||||
M(UInt64, refresh_retry_initial_backoff_ms, 100, "Delay before the first retry if refresh query fails (if refresh_retries setting is not zero). Each subsequent retry doubles the delay, up to refresh_retry_max_backoff_ms.", 0) \
|
||||
M(UInt64, refresh_retry_max_backoff_ms, 60'000, "Limit on the exponential growth of delay between refresh attempts, if they keep failing and refresh_retries is positive.", 0) \
|
||||
M(Bool, all_replicas, /* do not change or existing tables will break */ false, "If the materialized view is in a Replicated database, and APPEND is enabled, this flag controls whether all replicas or one replica will refresh.", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RefreshSettingsTraits, LIST_OF_REFRESH_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(RefreshSettingsTraits, LIST_OF_REFRESH_SETTINGS)
|
||||
|
||||
struct RefreshSettingsImpl : public BaseSettings<RefreshSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) RefreshSettings##TYPE NAME = &RefreshSettingsImpl ::NAME;
|
||||
|
||||
namespace RefreshSetting
|
||||
{
|
||||
LIST_OF_REFRESH_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
RefreshSettings::RefreshSettings() : impl(std::make_unique<RefreshSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
RefreshSettings::RefreshSettings(const RefreshSettings & settings) : impl(std::make_unique<RefreshSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
RefreshSettings::RefreshSettings(RefreshSettings && settings) noexcept
|
||||
: impl(std::make_unique<RefreshSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
RefreshSettings::~RefreshSettings() = default;
|
||||
|
||||
RefreshSettings & RefreshSettings::operator=(const RefreshSettings & other)
|
||||
{
|
||||
*impl = *other.impl;
|
||||
return *this;
|
||||
}
|
||||
|
||||
REFRESH_SETTINGS_SUPPORTED_TYPES(RefreshSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void RefreshSettings::applyChanges(const SettingsChanges & changes)
|
||||
{
|
||||
impl->applyChanges(changes);
|
||||
}
|
||||
}
|
||||
|
@ -1,18 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct RefreshSettingsImpl;
|
||||
class SettingsChanges;
|
||||
|
||||
#define LIST_OF_REFRESH_SETTINGS(M, ALIAS) \
|
||||
M(Int64, refresh_retries, 2, "How many times to retry refresh query if it fails. If all attempts fail, wait for the next refresh time according to schedule. 0 to disable retries. -1 for infinite retries.", 0) \
|
||||
M(UInt64, refresh_retry_initial_backoff_ms, 100, "Delay before the first retry if refresh query fails (if refresh_retries setting is not zero). Each subsequent retry doubles the delay, up to refresh_retry_max_backoff_ms.", 0) \
|
||||
M(UInt64, refresh_retry_max_backoff_ms, 60'000, "Limit on the exponential growth of delay between refresh attempts, if they keep failing and refresh_retries is positive.", 0) \
|
||||
M(Bool, all_replicas, /* do not change or existing tables will break */ false, "If the materialized view is in a Replicated database, and APPEND is enabled, this flag controls whether all replicas or one replica will refresh.", 0) \
|
||||
/// List of available types supported in RabbitMQSettings object
|
||||
#define REFRESH_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, Int64) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RefreshSettingsTraits, LIST_OF_REFRESH_SETTINGS)
|
||||
REFRESH_SETTINGS_SUPPORTED_TYPES(RefreshSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
struct RefreshSettings : public BaseSettings<RefreshSettingsTraits> {};
|
||||
struct RefreshSettings
|
||||
{
|
||||
RefreshSettings();
|
||||
RefreshSettings(const RefreshSettings & settings);
|
||||
RefreshSettings(RefreshSettings && settings) noexcept;
|
||||
~RefreshSettings();
|
||||
|
||||
RefreshSettings & operator=(const RefreshSettings & other);
|
||||
|
||||
REFRESH_SETTINGS_SUPPORTED_TYPES(RefreshSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void applyChanges(const SettingsChanges & changes);
|
||||
|
||||
private:
|
||||
std::unique_ptr<RefreshSettingsImpl> impl;
|
||||
};
|
||||
}
|
||||
|
@ -38,6 +38,14 @@ namespace ServerSetting
|
||||
extern const ServerSettingsString default_replica_path;
|
||||
}
|
||||
|
||||
namespace RefreshSetting
|
||||
{
|
||||
extern const RefreshSettingsBool all_replicas;
|
||||
extern const RefreshSettingsInt64 refresh_retries;
|
||||
extern const RefreshSettingsUInt64 refresh_retry_initial_backoff_ms;
|
||||
extern const RefreshSettingsUInt64 refresh_retry_max_backoff_ms;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -187,7 +195,7 @@ void RefreshTask::checkAlterIsPossible(const DB::ASTRefreshStrategy & new_strate
|
||||
RefreshSettings s;
|
||||
if (new_strategy.settings)
|
||||
s.applyChanges(new_strategy.settings->changes);
|
||||
if (s.all_replicas != refresh_settings.all_replicas)
|
||||
if (s[RefreshSetting::all_replicas] != refresh_settings[RefreshSetting::all_replicas])
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Altering setting 'all_replicas' is not supported.");
|
||||
if (new_strategy.append != refresh_append)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Adding or removing APPEND is not supported.");
|
||||
@ -456,7 +464,7 @@ void RefreshTask::refreshTask()
|
||||
else
|
||||
{
|
||||
error_message = getCurrentExceptionMessage(true);
|
||||
LOG_ERROR(log, "{}: Refresh failed (attempt {}/{}): {}", view->getStorageID().getFullTableName(), start_znode.attempt_number, refresh_settings.refresh_retries + 1, error_message);
|
||||
LOG_ERROR(log, "{}: Refresh failed (attempt {}/{}): {}", view->getStorageID().getFullTableName(), start_znode.attempt_number, refresh_settings[RefreshSetting::refresh_retries] + 1, error_message);
|
||||
}
|
||||
}
|
||||
|
||||
@ -676,10 +684,10 @@ static std::chrono::milliseconds backoff(Int64 retry_idx, const RefreshSettings
|
||||
UInt64 delay_ms;
|
||||
UInt64 multiplier = UInt64(1) << std::min(retry_idx, Int64(62));
|
||||
/// Overflow check: a*b <= c iff a <= c/b iff a <= floor(c/b).
|
||||
if (refresh_settings.refresh_retry_initial_backoff_ms <= refresh_settings.refresh_retry_max_backoff_ms / multiplier)
|
||||
delay_ms = refresh_settings.refresh_retry_initial_backoff_ms * multiplier;
|
||||
if (refresh_settings[RefreshSetting::refresh_retry_initial_backoff_ms] <= refresh_settings[RefreshSetting::refresh_retry_max_backoff_ms] / multiplier)
|
||||
delay_ms = refresh_settings[RefreshSetting::refresh_retry_initial_backoff_ms] * multiplier;
|
||||
else
|
||||
delay_ms = refresh_settings.refresh_retry_max_backoff_ms;
|
||||
delay_ms = refresh_settings[RefreshSetting::refresh_retry_max_backoff_ms];
|
||||
return std::chrono::milliseconds(delay_ms);
|
||||
}
|
||||
|
||||
@ -687,7 +695,7 @@ std::tuple<std::chrono::system_clock::time_point, std::chrono::sys_seconds, Refr
|
||||
RefreshTask::determineNextRefreshTime(std::chrono::sys_seconds now)
|
||||
{
|
||||
auto znode = coordination.root_znode;
|
||||
if (refresh_settings.refresh_retries >= 0 && znode.attempt_number > refresh_settings.refresh_retries)
|
||||
if (refresh_settings[RefreshSetting::refresh_retries] >= 0 && znode.attempt_number > refresh_settings[RefreshSetting::refresh_retries])
|
||||
{
|
||||
/// Skip to the next scheduled refresh, as if a refresh succeeded.
|
||||
znode.last_completed_timeslot = refresh_schedule.timeslotForCompletedRefresh(znode.last_completed_timeslot, znode.last_attempt_time, znode.last_attempt_time, false);
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/BaseSettingsProgramOptions.h>
|
||||
#include <Core/MergeSelectorAlgorithm.h>
|
||||
#include <Core/SettingsChangesHistory.h>
|
||||
@ -486,8 +487,7 @@ void MergeTreeColumnSettings::validate(const SettingsChanges & changes)
|
||||
}
|
||||
}
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
MergeTreeSettings ## TYPE NAME = & MergeTreeSettings ## Impl :: NAME;
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) MergeTreeSettings##TYPE NAME = &MergeTreeSettingsImpl ::NAME;
|
||||
|
||||
namespace MergeTreeSetting
|
||||
{
|
||||
@ -511,18 +511,7 @@ MergeTreeSettings::MergeTreeSettings(MergeTreeSettings && settings) noexcept
|
||||
|
||||
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)
|
||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
||||
|
||||
bool MergeTreeSettings::has(std::string_view name) const
|
||||
{
|
||||
|
@ -41,6 +41,11 @@ namespace Setting
|
||||
extern const SettingsBool cloud_mode;
|
||||
}
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueMode mode;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
UInt64 getCurrentTime()
|
||||
@ -227,7 +232,7 @@ ObjectStorageQueueTableMetadata ObjectStorageQueueMetadata::syncWithKeeper(
|
||||
|
||||
std::vector<std::string> metadata_paths;
|
||||
size_t buckets_num = 0;
|
||||
if (settings.mode == ObjectStorageQueueMode::ORDERED)
|
||||
if (settings[ObjectStorageQueueSetting::mode] == ObjectStorageQueueMode::ORDERED)
|
||||
{
|
||||
buckets_num = getBucketsNum(table_metadata);
|
||||
if (buckets_num == 0)
|
||||
|
@ -7,7 +7,6 @@
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
|
||||
@ -17,6 +16,7 @@ namespace Poco { class Logger; }
|
||||
namespace DB
|
||||
{
|
||||
class StorageObjectStorageQueue;
|
||||
struct ObjectStorageQueueSettings;
|
||||
struct ObjectStorageQueueTableMetadata;
|
||||
struct StorageInMemoryMetadata;
|
||||
using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,8 +1,10 @@
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -13,8 +15,70 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
#define OBJECT_STORAGE_QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
||||
M(ObjectStorageQueueMode, mode, ObjectStorageQueueMode::ORDERED, \
|
||||
"With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer." \
|
||||
"With ordered mode, only the max name of the successfully consumed file stored.", \
|
||||
0) \
|
||||
M(ObjectStorageQueueAction, after_processing, ObjectStorageQueueAction::KEEP, "Delete or keep file in after successful processing", 0) \
|
||||
M(String, keeper_path, "", "Zookeeper node path", 0) \
|
||||
M(UInt32, loading_retries, 10, "Retry loading up to specified number of times", 0) \
|
||||
M(UInt32, processing_threads_num, 1, "Number of processing threads", 0) \
|
||||
M(UInt32, enable_logging_to_queue_log, 1, "Enable logging to system table system.(s3/azure_)queue_log", 0) \
|
||||
M(String, last_processed_path, "", "For Ordered mode. Files that have lexicographically smaller file name are considered already processed", 0) \
|
||||
M(UInt32, tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \
|
||||
M(UInt32, polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \
|
||||
M(UInt32, polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \
|
||||
M(UInt32, polling_backoff_ms, 1000, "Polling backoff", 0) \
|
||||
M(UInt32, tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \
|
||||
M(UInt32, cleanup_interval_min_ms, 60000, "For unordered mode. Polling backoff min for cleanup", 0) \
|
||||
M(UInt32, cleanup_interval_max_ms, 60000, "For unordered mode. Polling backoff max for cleanup", 0) \
|
||||
M(UInt32, buckets, 0, "Number of buckets for Ordered mode parallel processing", 0) \
|
||||
M(UInt32, max_processed_files_before_commit, 100, "Number of files which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processed_rows_before_commit, 0, "Number of rows which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processed_bytes_before_commit, 0, "Number of bytes which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processing_time_sec_before_commit, 0, "Timeout in seconds after which to commit files committed to keeper", 0) \
|
||||
|
||||
#define LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS(M, ALIAS) \
|
||||
OBJECT_STORAGE_QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(ObjectStorageQueueSettingsTraits, LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(ObjectStorageQueueSettingsTraits, LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS)
|
||||
|
||||
struct ObjectStorageQueueSettingsImpl : public BaseSettings<ObjectStorageQueueSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
ObjectStorageQueueSettings##TYPE NAME = &ObjectStorageQueueSettingsImpl ::NAME;
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
ObjectStorageQueueSettings::ObjectStorageQueueSettings() : impl(std::make_unique<ObjectStorageQueueSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
ObjectStorageQueueSettings::ObjectStorageQueueSettings(const ObjectStorageQueueSettings & settings)
|
||||
: impl(std::make_unique<ObjectStorageQueueSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
ObjectStorageQueueSettings::ObjectStorageQueueSettings(ObjectStorageQueueSettings && settings) noexcept
|
||||
: impl(std::make_unique<ObjectStorageQueueSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
ObjectStorageQueueSettings::~ObjectStorageQueueSettings() = default;
|
||||
|
||||
OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(ObjectStorageQueueSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
|
||||
void ObjectStorageQueueSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
@ -42,7 +106,7 @@ void ObjectStorageQueueSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
for (const auto & setting : ignore_settings)
|
||||
settings_changes.removeSetting(setting);
|
||||
|
||||
applyChanges(settings_changes);
|
||||
impl->applyChanges(settings_changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
|
@ -1,51 +1,59 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
struct ObjectStorageQueueSettingsImpl;
|
||||
|
||||
/// List of available types supported in ObjectStorageQueueSettings object
|
||||
#define OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, ArrowCompression) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, CapnProtoEnumComparingMode) \
|
||||
M(CLASS_NAME, Char) \
|
||||
M(CLASS_NAME, DateTimeInputFormat) \
|
||||
M(CLASS_NAME, DateTimeOutputFormat) \
|
||||
M(CLASS_NAME, DateTimeOverflowBehavior) \
|
||||
M(CLASS_NAME, Double) \
|
||||
M(CLASS_NAME, EscapingRule) \
|
||||
M(CLASS_NAME, Float) \
|
||||
M(CLASS_NAME, IdentifierQuotingRule) \
|
||||
M(CLASS_NAME, IdentifierQuotingStyle) \
|
||||
M(CLASS_NAME, Int64) \
|
||||
M(CLASS_NAME, IntervalOutputFormat) \
|
||||
M(CLASS_NAME, MsgPackUUIDRepresentation) \
|
||||
M(CLASS_NAME, ObjectStorageQueueAction) \
|
||||
M(CLASS_NAME, ObjectStorageQueueMode) \
|
||||
M(CLASS_NAME, ORCCompression) \
|
||||
M(CLASS_NAME, ParquetCompression) \
|
||||
M(CLASS_NAME, ParquetVersion) \
|
||||
M(CLASS_NAME, SchemaInferenceMode) \
|
||||
M(CLASS_NAME, String) \
|
||||
M(CLASS_NAME, UInt32) \
|
||||
M(CLASS_NAME, UInt64) \
|
||||
M(CLASS_NAME, UInt64Auto) \
|
||||
M(CLASS_NAME, URI)
|
||||
|
||||
#define OBJECT_STORAGE_QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
||||
M(ObjectStorageQueueMode, \
|
||||
mode, \
|
||||
ObjectStorageQueueMode::ORDERED, \
|
||||
"With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer." \
|
||||
"With ordered mode, only the max name of the successfully consumed file stored.", \
|
||||
0) \
|
||||
M(ObjectStorageQueueAction, after_processing, ObjectStorageQueueAction::KEEP, "Delete or keep file in after successful processing", 0) \
|
||||
M(String, keeper_path, "", "Zookeeper node path", 0) \
|
||||
M(UInt32, loading_retries, 10, "Retry loading up to specified number of times", 0) \
|
||||
M(UInt32, processing_threads_num, 1, "Number of processing threads", 0) \
|
||||
M(UInt32, enable_logging_to_queue_log, 1, "Enable logging to system table system.(s3/azure_)queue_log", 0) \
|
||||
M(String, last_processed_path, "", "For Ordered mode. Files that have lexicographically smaller file name are considered already processed", 0) \
|
||||
M(UInt32, tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \
|
||||
M(UInt32, polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \
|
||||
M(UInt32, polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \
|
||||
M(UInt32, polling_backoff_ms, 1000, "Polling backoff", 0) \
|
||||
M(UInt32, tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \
|
||||
M(UInt32, cleanup_interval_min_ms, 60000, "For unordered mode. Polling backoff min for cleanup", 0) \
|
||||
M(UInt32, cleanup_interval_max_ms, 60000, "For unordered mode. Polling backoff max for cleanup", 0) \
|
||||
M(UInt32, buckets, 0, "Number of buckets for Ordered mode parallel processing", 0) \
|
||||
M(UInt32, max_processed_files_before_commit, 100, "Number of files which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processed_rows_before_commit, 0, "Number of rows which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processed_bytes_before_commit, 0, "Number of bytes which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processing_time_sec_before_commit, 0, "Timeout in seconds after which to commit files committed to keeper", 0) \
|
||||
OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(ObjectStorageQueueSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
#define LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS(M, ALIAS) \
|
||||
OBJECT_STORAGE_QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(ObjectStorageQueueSettingsTraits, LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS)
|
||||
|
||||
|
||||
struct ObjectStorageQueueSettings : public BaseSettings<ObjectStorageQueueSettingsTraits>
|
||||
struct ObjectStorageQueueSettings
|
||||
{
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
};
|
||||
ObjectStorageQueueSettings();
|
||||
ObjectStorageQueueSettings(const ObjectStorageQueueSettings & settings);
|
||||
ObjectStorageQueueSettings(ObjectStorageQueueSettings && settings) noexcept;
|
||||
~ObjectStorageQueueSettings();
|
||||
|
||||
OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(ObjectStorageQueueSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
|
||||
private:
|
||||
std::unique_ptr<ObjectStorageQueueSettingsImpl> impl;
|
||||
};
|
||||
}
|
||||
|
@ -23,6 +23,15 @@ namespace Setting
|
||||
extern const SettingsMaxThreads max_parsing_threads;
|
||||
}
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueAction after_processing;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_bytes_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_files_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_rows_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processing_time_sec_before_commit;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
|
@ -1,12 +1,13 @@
|
||||
#pragma once
|
||||
#include "config.h"
|
||||
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Interpreters/ObjectStorageQueueLog.h>
|
||||
#include <Processors/ISource.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorageSource.h>
|
||||
#include <Interpreters/ObjectStorageQueueLog.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
@ -12,6 +12,19 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueAction after_processing;
|
||||
extern const ObjectStorageQueueSettingsUInt32 buckets;
|
||||
extern const ObjectStorageQueueSettingsString last_processed_path;
|
||||
extern const ObjectStorageQueueSettingsUInt32 loading_retries;
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueMode mode;
|
||||
extern const ObjectStorageQueueSettingsUInt32 processing_threads_num;
|
||||
extern const ObjectStorageQueueSettingsUInt32 tracked_files_limit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 tracked_file_ttl_sec;
|
||||
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int METADATA_MISMATCH;
|
||||
@ -43,19 +56,19 @@ ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata(
|
||||
const std::string & format_)
|
||||
: format_name(format_)
|
||||
, columns(columns_.toString())
|
||||
, after_processing(engine_settings.after_processing.toString())
|
||||
, mode(engine_settings.mode.toString())
|
||||
, tracked_files_limit(engine_settings.tracked_files_limit)
|
||||
, tracked_files_ttl_sec(engine_settings.tracked_file_ttl_sec)
|
||||
, buckets(engine_settings.buckets)
|
||||
, last_processed_path(engine_settings.last_processed_path)
|
||||
, loading_retries(engine_settings.loading_retries)
|
||||
, after_processing(engine_settings[ObjectStorageQueueSetting::after_processing].toString())
|
||||
, mode(engine_settings[ObjectStorageQueueSetting::mode].toString())
|
||||
, tracked_files_limit(engine_settings[ObjectStorageQueueSetting::tracked_files_limit])
|
||||
, tracked_files_ttl_sec(engine_settings[ObjectStorageQueueSetting::tracked_file_ttl_sec])
|
||||
, buckets(engine_settings[ObjectStorageQueueSetting::buckets])
|
||||
, last_processed_path(engine_settings[ObjectStorageQueueSetting::last_processed_path])
|
||||
, loading_retries(engine_settings[ObjectStorageQueueSetting::loading_retries])
|
||||
{
|
||||
processing_threads_num_changed = engine_settings.processing_threads_num.changed;
|
||||
if (!processing_threads_num_changed && engine_settings.processing_threads_num <= 1)
|
||||
processing_threads_num_changed = engine_settings[ObjectStorageQueueSetting::processing_threads_num].changed;
|
||||
if (!processing_threads_num_changed && engine_settings[ObjectStorageQueueSetting::processing_threads_num] <= 1)
|
||||
processing_threads_num = std::max<uint32_t>(getNumberOfCPUCoresToUse(), 16);
|
||||
else
|
||||
processing_threads_num = engine_settings.processing_threads_num;
|
||||
processing_threads_num = engine_settings[ObjectStorageQueueSetting::processing_threads_num];
|
||||
}
|
||||
|
||||
String ObjectStorageQueueTableMetadata::toString() const
|
||||
|
@ -1,15 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <base/types.h>
|
||||
#include <Poco/JSON/JSON.h>
|
||||
#include <Poco/JSON/Object.h>
|
||||
#include <base/types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ObjectStorageQueueSettings;
|
||||
class WriteBuffer;
|
||||
class ReadBuffer;
|
||||
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Storages/ObjectStorageQueue/StorageObjectStorageQueue.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
@ -39,6 +40,23 @@ namespace Setting
|
||||
extern const SettingsBool use_concurrency_control;
|
||||
}
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
extern const ObjectStorageQueueSettingsUInt32 cleanup_interval_max_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 cleanup_interval_min_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 enable_logging_to_queue_log;
|
||||
extern const ObjectStorageQueueSettingsString keeper_path;
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueMode mode;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_bytes_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_files_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_rows_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processing_time_sec_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 polling_min_timeout_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 polling_max_timeout_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 polling_backoff_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 processing_threads_num;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -56,10 +74,10 @@ namespace
|
||||
zk_path_prefix = "/";
|
||||
|
||||
std::string result_zk_path;
|
||||
if (queue_settings.keeper_path.changed)
|
||||
if (queue_settings[ObjectStorageQueueSetting::keeper_path].changed)
|
||||
{
|
||||
/// We do not add table uuid here on purpose.
|
||||
result_zk_path = fs::path(zk_path_prefix) / queue_settings.keeper_path.value;
|
||||
result_zk_path = fs::path(zk_path_prefix) / queue_settings[ObjectStorageQueueSetting::keeper_path].value;
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -73,22 +91,22 @@ namespace
|
||||
ObjectStorageQueueSettings & queue_settings,
|
||||
bool is_attach)
|
||||
{
|
||||
if (!is_attach && !queue_settings.mode.changed)
|
||||
if (!is_attach && !queue_settings[ObjectStorageQueueSetting::mode].changed)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `mode` (Unordered/Ordered) is not specified, but is required.");
|
||||
}
|
||||
/// In case !is_attach, we leave Ordered mode as default for compatibility.
|
||||
|
||||
if (!queue_settings.processing_threads_num)
|
||||
if (!queue_settings[ObjectStorageQueueSetting::processing_threads_num])
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `processing_threads_num` cannot be set to zero");
|
||||
}
|
||||
|
||||
if (queue_settings.cleanup_interval_min_ms > queue_settings.cleanup_interval_max_ms)
|
||||
if (queue_settings[ObjectStorageQueueSetting::cleanup_interval_min_ms] > queue_settings[ObjectStorageQueueSetting::cleanup_interval_max_ms])
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Setting `cleanup_interval_min_ms` ({}) must be less or equal to `cleanup_interval_max_ms` ({})",
|
||||
queue_settings.cleanup_interval_min_ms, queue_settings.cleanup_interval_max_ms);
|
||||
queue_settings[ObjectStorageQueueSetting::cleanup_interval_min_ms], queue_settings[ObjectStorageQueueSetting::cleanup_interval_max_ms]);
|
||||
}
|
||||
}
|
||||
|
||||
@ -132,19 +150,19 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_)
|
||||
, zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *queue_settings_))
|
||||
, enable_logging_to_queue_log(queue_settings_->enable_logging_to_queue_log)
|
||||
, polling_min_timeout_ms(queue_settings_->polling_min_timeout_ms)
|
||||
, polling_max_timeout_ms(queue_settings_->polling_max_timeout_ms)
|
||||
, polling_backoff_ms(queue_settings_->polling_backoff_ms)
|
||||
, enable_logging_to_queue_log((*queue_settings_)[ObjectStorageQueueSetting::enable_logging_to_queue_log])
|
||||
, polling_min_timeout_ms((*queue_settings_)[ObjectStorageQueueSetting::polling_min_timeout_ms])
|
||||
, polling_max_timeout_ms((*queue_settings_)[ObjectStorageQueueSetting::polling_max_timeout_ms])
|
||||
, polling_backoff_ms((*queue_settings_)[ObjectStorageQueueSetting::polling_backoff_ms])
|
||||
, commit_settings(CommitSettings{
|
||||
.max_processed_files_before_commit = queue_settings_->max_processed_files_before_commit,
|
||||
.max_processed_rows_before_commit = queue_settings_->max_processed_rows_before_commit,
|
||||
.max_processed_bytes_before_commit = queue_settings_->max_processed_bytes_before_commit,
|
||||
.max_processing_time_sec_before_commit = queue_settings_->max_processing_time_sec_before_commit,
|
||||
.max_processed_files_before_commit = (*queue_settings_)[ObjectStorageQueueSetting::max_processed_files_before_commit],
|
||||
.max_processed_rows_before_commit = (*queue_settings_)[ObjectStorageQueueSetting::max_processed_rows_before_commit],
|
||||
.max_processed_bytes_before_commit = (*queue_settings_)[ObjectStorageQueueSetting::max_processed_bytes_before_commit],
|
||||
.max_processing_time_sec_before_commit = (*queue_settings_)[ObjectStorageQueueSetting::max_processing_time_sec_before_commit],
|
||||
})
|
||||
, configuration{configuration_}
|
||||
, format_settings(format_settings_)
|
||||
, reschedule_processing_interval_ms(queue_settings_->polling_min_timeout_ms)
|
||||
, reschedule_processing_interval_ms((*queue_settings_)[ObjectStorageQueueSetting::polling_min_timeout_ms])
|
||||
, log(getLogger(fmt::format("Storage{}Queue ({})", configuration->getEngineName(), table_id_.getFullTableName())))
|
||||
{
|
||||
if (configuration->getPath().empty())
|
||||
@ -185,7 +203,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
|
||||
zk_path, *queue_settings_, storage_metadata.getColumns(), configuration_->format, context_, is_attach, log);
|
||||
|
||||
auto queue_metadata = std::make_unique<ObjectStorageQueueMetadata>(
|
||||
zk_path, std::move(table_metadata), queue_settings_->cleanup_interval_min_ms, queue_settings_->cleanup_interval_max_ms);
|
||||
zk_path, std::move(table_metadata), (*queue_settings_)[ObjectStorageQueueSetting::cleanup_interval_min_ms], (*queue_settings_)[ObjectStorageQueueSetting::cleanup_interval_max_ms]);
|
||||
|
||||
files_metadata = ObjectStorageQueueMetadataFactory::instance().getOrCreate(zk_path, std::move(queue_metadata));
|
||||
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSource.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -15,6 +14,7 @@
|
||||
namespace DB
|
||||
{
|
||||
class ObjectStorageQueueMetadata;
|
||||
struct ObjectStorageQueueSettings;
|
||||
|
||||
class StorageObjectStorageQueue : public IStorage, WithContext
|
||||
{
|
||||
|
@ -1,6 +1,9 @@
|
||||
#include "MaterializedPostgreSQLSettings.h"
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -15,15 +18,72 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
#define LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS(M, ALIAS) \
|
||||
M(UInt64, materialized_postgresql_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \
|
||||
M(String, materialized_postgresql_tables_list, "", "List of tables for MaterializedPostgreSQL database engine", 0) \
|
||||
M(String, materialized_postgresql_schema_list, "", "List of schemas for MaterializedPostgreSQL database engine", 0) \
|
||||
M(String, materialized_postgresql_replication_slot, "", "A user-created replication slot", 0) \
|
||||
M(String, materialized_postgresql_snapshot, "", "User provided snapshot in case he manages replication slots himself", 0) \
|
||||
M(String, materialized_postgresql_schema, "", "PostgreSQL schema", 0) \
|
||||
M(Bool, materialized_postgresql_tables_list_with_schema, false, \
|
||||
"Consider by default that if there is a dot in tables list 'name.name', " \
|
||||
"then the first name is postgres schema and second is postgres table. This setting is needed to allow table names with dots", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_min_ms, 200, "Poll backoff start point", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_max_ms, 10000, "Poll backoff max point", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_factor, 2, "Poll backoff factor", 0) \
|
||||
M(Bool, materialized_postgresql_use_unique_replication_consumer_identifier, false, "Should a unique consumer be registered for table replication", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS)
|
||||
|
||||
struct MaterializedPostgreSQLSettingsImpl : public BaseSettings<MaterializedPostgreSQLSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) MaterializedPostgreSQLSettings##TYPE NAME = &MaterializedPostgreSQLSettingsImpl ::NAME;
|
||||
|
||||
namespace MaterializedPostgreSQLSetting
|
||||
{
|
||||
LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
MaterializedPostgreSQLSettings::MaterializedPostgreSQLSettings() : impl(std::make_unique<MaterializedPostgreSQLSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
MaterializedPostgreSQLSettings::MaterializedPostgreSQLSettings(const MaterializedPostgreSQLSettings & settings)
|
||||
: impl(std::make_unique<MaterializedPostgreSQLSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
MaterializedPostgreSQLSettings::MaterializedPostgreSQLSettings(MaterializedPostgreSQLSettings && settings) noexcept
|
||||
: impl(std::make_unique<MaterializedPostgreSQLSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
MaterializedPostgreSQLSettings::~MaterializedPostgreSQLSettings() = default;
|
||||
|
||||
MATERIALIZED_POSTGRESQL_SETTINGS_SUPPORTED_TYPES(MaterializedPostgreSQLSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void MaterializedPostgreSQLSettings::applyChange(const SettingChange & change)
|
||||
{
|
||||
impl->applyChange(change);
|
||||
}
|
||||
|
||||
bool MaterializedPostgreSQLSettings::has(std::string_view name) const
|
||||
{
|
||||
return impl->has(name);
|
||||
}
|
||||
|
||||
void MaterializedPostgreSQLSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
try
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
impl->applyChanges(storage_def.settings->changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
|
@ -3,34 +3,40 @@
|
||||
#include "config.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Core/BaseSettings.h>
|
||||
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
class ASTStorage;
|
||||
struct SettingChange;
|
||||
struct MaterializedPostgreSQLSettingsImpl;
|
||||
|
||||
#define MATERIALIZED_POSTGRESQL_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, String) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
MATERIALIZED_POSTGRESQL_SETTINGS_SUPPORTED_TYPES(MaterializedPostgreSQLSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
|
||||
#define LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS(M, ALIAS) \
|
||||
M(UInt64, materialized_postgresql_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \
|
||||
M(String, materialized_postgresql_tables_list, "", "List of tables for MaterializedPostgreSQL database engine", 0) \
|
||||
M(String, materialized_postgresql_schema_list, "", "List of schemas for MaterializedPostgreSQL database engine", 0) \
|
||||
M(String, materialized_postgresql_replication_slot, "", "A user-created replication slot", 0) \
|
||||
M(String, materialized_postgresql_snapshot, "", "User provided snapshot in case he manages replication slots himself", 0) \
|
||||
M(String, materialized_postgresql_schema, "", "PostgreSQL schema", 0) \
|
||||
M(Bool, materialized_postgresql_tables_list_with_schema, false, \
|
||||
"Consider by default that if there is a dot in tables list 'name.name', " \
|
||||
"then the first name is postgres schema and second is postgres table. This setting is needed to allow table names with dots", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_min_ms, 200, "Poll backoff start point", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_max_ms, 10000, "Poll backoff max point", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_factor, 2, "Poll backoff factor", 0) \
|
||||
M(Bool, materialized_postgresql_use_unique_replication_consumer_identifier, false, "Should a unique consumer be registered for table replication", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS)
|
||||
|
||||
struct MaterializedPostgreSQLSettings : public BaseSettings<MaterializedPostgreSQLSettingsTraits>
|
||||
struct MaterializedPostgreSQLSettings
|
||||
{
|
||||
MaterializedPostgreSQLSettings();
|
||||
MaterializedPostgreSQLSettings(const MaterializedPostgreSQLSettings & settings);
|
||||
MaterializedPostgreSQLSettings(MaterializedPostgreSQLSettings && settings) noexcept;
|
||||
~MaterializedPostgreSQLSettings();
|
||||
|
||||
MATERIALIZED_POSTGRESQL_SETTINGS_SUPPORTED_TYPES(MaterializedPostgreSQLSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void applyChange(const SettingChange & change);
|
||||
bool has(std::string_view name) const;
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
|
||||
private:
|
||||
std::unique_ptr<MaterializedPostgreSQLSettingsImpl> impl;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include "PostgreSQLReplicationHandler.h"
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
@ -9,6 +8,8 @@
|
||||
#include <QueryPipeline/QueryPipeline.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
#include <Storages/PostgreSQL/PostgreSQLReplicationHandler.h>
|
||||
#include <Storages/PostgreSQL/StorageMaterializedPostgreSQL.h>
|
||||
#include <Interpreters/getTableOverride.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
@ -26,6 +27,21 @@ namespace DB
|
||||
static const auto CLEANUP_RESCHEDULE_MS = 600000 * 3; /// 30 min
|
||||
static constexpr size_t replication_slot_name_max_size = 64;
|
||||
|
||||
namespace MaterializedPostgreSQLSetting
|
||||
{
|
||||
extern const MaterializedPostgreSQLSettingsUInt64 materialized_postgresql_backoff_factor;
|
||||
extern const MaterializedPostgreSQLSettingsUInt64 materialized_postgresql_backoff_max_ms;
|
||||
extern const MaterializedPostgreSQLSettingsUInt64 materialized_postgresql_backoff_min_ms;
|
||||
extern const MaterializedPostgreSQLSettingsUInt64 materialized_postgresql_max_block_size;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_replication_slot;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_schema;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_schema_list;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_snapshot;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_tables_list;
|
||||
extern const MaterializedPostgreSQLSettingsBool materialized_postgresql_tables_list_with_schema;
|
||||
extern const MaterializedPostgreSQLSettingsBool materialized_postgresql_use_unique_replication_consumer_identifier;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -103,10 +119,10 @@ namespace
|
||||
const String & clickhouse_uuid,
|
||||
const MaterializedPostgreSQLSettings & replication_settings)
|
||||
{
|
||||
String slot_name = replication_settings.materialized_postgresql_replication_slot;
|
||||
String slot_name = replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_replication_slot];
|
||||
if (slot_name.empty())
|
||||
{
|
||||
if (replication_settings.materialized_postgresql_use_unique_replication_consumer_identifier)
|
||||
if (replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_use_unique_replication_consumer_identifier])
|
||||
slot_name = clickhouse_uuid;
|
||||
else
|
||||
slot_name = postgres_table.empty() ? postgres_database : fmt::format("{}_{}_ch_replication_slot", postgres_database, postgres_table);
|
||||
@ -131,22 +147,22 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler(
|
||||
, log(getLogger("PostgreSQLReplicationHandler"))
|
||||
, is_attach(is_attach_)
|
||||
, postgres_database(postgres_database_)
|
||||
, postgres_schema(replication_settings.materialized_postgresql_schema)
|
||||
, postgres_schema(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_schema])
|
||||
, current_database_name(clickhouse_database_)
|
||||
, connection_info(connection_info_)
|
||||
, max_block_size(replication_settings.materialized_postgresql_max_block_size)
|
||||
, max_block_size(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_max_block_size])
|
||||
, is_materialized_postgresql_database(is_materialized_postgresql_database_)
|
||||
, tables_list(replication_settings.materialized_postgresql_tables_list)
|
||||
, schema_list(replication_settings.materialized_postgresql_schema_list)
|
||||
, schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings.materialized_postgresql_tables_list_with_schema)
|
||||
, user_managed_slot(!replication_settings.materialized_postgresql_replication_slot.value.empty())
|
||||
, user_provided_snapshot(replication_settings.materialized_postgresql_snapshot)
|
||||
, tables_list(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_tables_list])
|
||||
, schema_list(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_schema_list])
|
||||
, schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_tables_list_with_schema])
|
||||
, user_managed_slot(!replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_replication_slot].value.empty())
|
||||
, user_provided_snapshot(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_snapshot])
|
||||
, replication_slot(getReplicationSlotName(postgres_database_, postgres_table_, clickhouse_uuid_, replication_settings))
|
||||
, tmp_replication_slot(replication_slot + "_tmp")
|
||||
, publication_name(getPublicationName(postgres_database_, postgres_table_))
|
||||
, reschedule_backoff_min_ms(replication_settings.materialized_postgresql_backoff_min_ms)
|
||||
, reschedule_backoff_max_ms(replication_settings.materialized_postgresql_backoff_max_ms)
|
||||
, reschedule_backoff_factor(replication_settings.materialized_postgresql_backoff_factor)
|
||||
, reschedule_backoff_min_ms(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_backoff_min_ms])
|
||||
, reschedule_backoff_max_ms(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_backoff_max_ms])
|
||||
, reschedule_backoff_factor(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_backoff_factor])
|
||||
, milliseconds_to_wait(reschedule_backoff_min_ms)
|
||||
{
|
||||
if (!schema_list.empty() && !tables_list.empty())
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include "MaterializedPostgreSQLConsumer.h"
|
||||
#include "MaterializedPostgreSQLSettings.h"
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
#include <Core/PostgreSQL/Utils.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
@ -10,6 +9,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct MaterializedPostgreSQLSettings;
|
||||
class StorageMaterializedPostgreSQL;
|
||||
struct SettingChange;
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include "StorageMaterializedPostgreSQL.h"
|
||||
#include <Storages/PostgreSQL/StorageMaterializedPostgreSQL.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Common/logger_useful.h>
|
||||
@ -46,6 +47,11 @@ namespace Setting
|
||||
extern const SettingsUInt64 postgresql_connection_attempt_timeout;
|
||||
}
|
||||
|
||||
namespace MaterializedPostgreSQLSetting
|
||||
{
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_tables_list;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -79,7 +85,7 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
setVirtuals(createVirtuals());
|
||||
|
||||
replication_settings->materialized_postgresql_tables_list = remote_table_name_;
|
||||
(*replication_settings)[MaterializedPostgreSQLSetting::materialized_postgresql_tables_list] = remote_table_name_;
|
||||
|
||||
replication_handler = std::make_unique<PostgreSQLReplicationHandler>(
|
||||
remote_database_name,
|
||||
|
@ -4,7 +4,6 @@
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include "PostgreSQLReplicationHandler.h"
|
||||
#include "MaterializedPostgreSQLSettings.h"
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
@ -17,6 +16,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct MaterializedPostgreSQLSettings;
|
||||
|
||||
/** TODO list:
|
||||
* - Actually I think we can support ddl even though logical replication does not fully support it.
|
||||
|
@ -1,8 +1,12 @@
|
||||
#include <Storages/RabbitMQ/RabbitMQSettings.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQSettings.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/NamedCollections/NamedCollections.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -12,15 +16,84 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
#define RABBITMQ_RELATED_SETTINGS(M, ALIAS) \
|
||||
M(String, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \
|
||||
M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \
|
||||
M(String, rabbitmq_format, "", "The message format.", 0) \
|
||||
M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \
|
||||
M(String, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \
|
||||
M(String, rabbitmq_schema, "", "Schema identifier (used by schema-based formats) for RabbitMQ engine", 0) \
|
||||
M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \
|
||||
M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \
|
||||
M(String, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \
|
||||
M(Bool, rabbitmq_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \
|
||||
M(Bool, rabbitmq_secure, false, "Use SSL connection", 0) \
|
||||
M(String, rabbitmq_address, "", "Address for connection", 0) \
|
||||
M(UInt64, rabbitmq_skip_broken_messages, 0, "Skip at least this number of broken messages from RabbitMQ per block", 0) \
|
||||
M(UInt64, rabbitmq_max_block_size, 0, "Number of row collected before flushing data from RabbitMQ.", 0) \
|
||||
M(UInt64, rabbitmq_flush_interval_ms, 0, "Timeout for flushing data from RabbitMQ.", 0) \
|
||||
M(String, rabbitmq_vhost, "/", "RabbitMQ vhost.", 0) \
|
||||
M(String, rabbitmq_queue_settings_list, "", "A list of rabbitmq queue settings", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_start_ms, 10, "A minimum backoff point to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_end_ms, 10000, "A maximum backoff point to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_step_ms, 100, "A backoff step to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(Bool, rabbitmq_queue_consume, false, "Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings", 0) \
|
||||
M(String, rabbitmq_username, "", "RabbitMQ username", 0) \
|
||||
M(String, rabbitmq_password, "", "RabbitMQ password", 0) \
|
||||
M(Bool, reject_unhandled_messages, false, "Allow messages to be rejected in case they cannot be processed. This also automatically implies if there is a x-deadletter-exchange queue setting added", 0) \
|
||||
M(Bool, rabbitmq_commit_on_select, false, "Commit messages when select query is made", 0) \
|
||||
M(UInt64, rabbitmq_max_rows_per_message, 1, "The maximum number of rows produced in one message for row-based formats.", 0) \
|
||||
M(StreamingHandleErrorMode, rabbitmq_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for RabbitMQ engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \
|
||||
|
||||
#define OBSOLETE_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
MAKE_OBSOLETE(M, Char, rabbitmq_row_delimiter, '\0') \
|
||||
|
||||
#define LIST_OF_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
RABBITMQ_RELATED_SETTINGS(M, ALIAS) \
|
||||
OBSOLETE_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS)
|
||||
|
||||
struct RabbitMQSettingsImpl : public BaseSettings<RabbitMQSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) RabbitMQSettings##TYPE NAME = &RabbitMQSettingsImpl ::NAME;
|
||||
|
||||
namespace RabbitMQSetting
|
||||
{
|
||||
LIST_OF_RABBITMQ_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
RabbitMQSettings::RabbitMQSettings() : impl(std::make_unique<RabbitMQSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
RabbitMQSettings::RabbitMQSettings(const RabbitMQSettings & settings) : impl(std::make_unique<RabbitMQSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
RabbitMQSettings::RabbitMQSettings(RabbitMQSettings && settings) noexcept
|
||||
: impl(std::make_unique<RabbitMQSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
RabbitMQSettings::~RabbitMQSettings() = default;
|
||||
|
||||
RABBITMQ_SETTINGS_SUPPORTED_TYPES(RabbitMQSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
|
||||
void RabbitMQSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
try
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
impl->applyChanges(storage_def.settings->changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -36,4 +109,30 @@ void RabbitMQSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
storage_def.set(storage_def.settings, settings_ast);
|
||||
}
|
||||
}
|
||||
|
||||
void RabbitMQSettings::loadFromNamedCollection(const MutableNamedCollectionPtr & named_collection)
|
||||
{
|
||||
for (const auto & setting : impl->all())
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
if (named_collection->has(setting_name))
|
||||
impl->set(setting_name, named_collection->get<String>(setting_name));
|
||||
}
|
||||
}
|
||||
|
||||
SettingsChanges RabbitMQSettings::getFormatSettings() const
|
||||
{
|
||||
SettingsChanges values;
|
||||
|
||||
for (const auto & setting : *impl)
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
|
||||
/// check for non-rabbitmq-related settings
|
||||
if (!setting_name.starts_with("rabbitmq_"))
|
||||
values.emplace_back(setting_name, setting.getValue());
|
||||
}
|
||||
|
||||
return values;
|
||||
}
|
||||
}
|
||||
|
@ -1,56 +1,60 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
#include <Common/NamedCollections/NamedCollections_fwd.h>
|
||||
#include <Common/SettingsChanges.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
class ASTStorage;
|
||||
struct RabbitMQSettingsImpl;
|
||||
|
||||
/// List of available types supported in RabbitMQSettings object
|
||||
#define RABBITMQ_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, ArrowCompression) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, CapnProtoEnumComparingMode) \
|
||||
M(CLASS_NAME, Char) \
|
||||
M(CLASS_NAME, DateTimeInputFormat) \
|
||||
M(CLASS_NAME, DateTimeOutputFormat) \
|
||||
M(CLASS_NAME, DateTimeOverflowBehavior) \
|
||||
M(CLASS_NAME, Double) \
|
||||
M(CLASS_NAME, EscapingRule) \
|
||||
M(CLASS_NAME, Float) \
|
||||
M(CLASS_NAME, IdentifierQuotingRule) \
|
||||
M(CLASS_NAME, IdentifierQuotingStyle) \
|
||||
M(CLASS_NAME, Int64) \
|
||||
M(CLASS_NAME, IntervalOutputFormat) \
|
||||
M(CLASS_NAME, MsgPackUUIDRepresentation) \
|
||||
M(CLASS_NAME, ORCCompression) \
|
||||
M(CLASS_NAME, ParquetCompression) \
|
||||
M(CLASS_NAME, ParquetVersion) \
|
||||
M(CLASS_NAME, SchemaInferenceMode) \
|
||||
M(CLASS_NAME, StreamingHandleErrorMode) \
|
||||
M(CLASS_NAME, String) \
|
||||
M(CLASS_NAME, UInt64) \
|
||||
M(CLASS_NAME, UInt64Auto) \
|
||||
M(CLASS_NAME, URI)
|
||||
|
||||
#define RABBITMQ_RELATED_SETTINGS(M, ALIAS) \
|
||||
M(String, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \
|
||||
M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \
|
||||
M(String, rabbitmq_format, "", "The message format.", 0) \
|
||||
M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \
|
||||
M(String, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \
|
||||
M(String, rabbitmq_schema, "", "Schema identifier (used by schema-based formats) for RabbitMQ engine", 0) \
|
||||
M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \
|
||||
M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \
|
||||
M(String, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \
|
||||
M(Bool, rabbitmq_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \
|
||||
M(Bool, rabbitmq_secure, false, "Use SSL connection", 0) \
|
||||
M(String, rabbitmq_address, "", "Address for connection", 0) \
|
||||
M(UInt64, rabbitmq_skip_broken_messages, 0, "Skip at least this number of broken messages from RabbitMQ per block", 0) \
|
||||
M(UInt64, rabbitmq_max_block_size, 0, "Number of row collected before flushing data from RabbitMQ.", 0) \
|
||||
M(UInt64, rabbitmq_flush_interval_ms, 0, "Timeout for flushing data from RabbitMQ.", 0) \
|
||||
M(String, rabbitmq_vhost, "/", "RabbitMQ vhost.", 0) \
|
||||
M(String, rabbitmq_queue_settings_list, "", "A list of rabbitmq queue settings", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_start_ms, 10, "A minimum backoff point to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_end_ms, 10000, "A maximum backoff point to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_step_ms, 100, "A backoff step to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(Bool, rabbitmq_queue_consume, false, "Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings", 0) \
|
||||
M(String, rabbitmq_username, "", "RabbitMQ username", 0) \
|
||||
M(String, rabbitmq_password, "", "RabbitMQ password", 0) \
|
||||
M(Bool, reject_unhandled_messages, false, "Allow messages to be rejected in case they cannot be processed. This also automatically implies if there is a x-deadletter-exchange queue setting added", 0) \
|
||||
M(Bool, rabbitmq_commit_on_select, false, "Commit messages when select query is made", 0) \
|
||||
M(UInt64, rabbitmq_max_rows_per_message, 1, "The maximum number of rows produced in one message for row-based formats.", 0) \
|
||||
M(StreamingHandleErrorMode, rabbitmq_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for RabbitMQ engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \
|
||||
RABBITMQ_SETTINGS_SUPPORTED_TYPES(RabbitMQSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
#define OBSOLETE_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
MAKE_OBSOLETE(M, Char, rabbitmq_row_delimiter, '\0') \
|
||||
|
||||
|
||||
#define LIST_OF_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
RABBITMQ_RELATED_SETTINGS(M, ALIAS) \
|
||||
OBSOLETE_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS)
|
||||
|
||||
struct RabbitMQSettings : public BaseSettings<RabbitMQSettingsTraits>
|
||||
struct RabbitMQSettings
|
||||
{
|
||||
RabbitMQSettings();
|
||||
RabbitMQSettings(const RabbitMQSettings & settings);
|
||||
RabbitMQSettings(RabbitMQSettings && settings) noexcept;
|
||||
~RabbitMQSettings();
|
||||
|
||||
RABBITMQ_SETTINGS_SUPPORTED_TYPES(RabbitMQSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
void loadFromNamedCollection(const MutableNamedCollectionPtr & named_collection);
|
||||
|
||||
SettingsChanges getFormatSettings() const;
|
||||
|
||||
private:
|
||||
std::unique_ptr<RabbitMQSettingsImpl> impl;
|
||||
};
|
||||
}
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Storages/NamedCollectionsHelpers.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQHandler.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQProducer.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQSettings.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQSource.h>
|
||||
#include <Storages/RabbitMQ/StorageRabbitMQ.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
@ -33,6 +34,8 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/setThreadName.h>
|
||||
|
||||
#include <base/range.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace Setting
|
||||
@ -43,6 +46,37 @@ namespace Setting
|
||||
extern const SettingsBool stream_like_engine_allow_direct_select;
|
||||
}
|
||||
|
||||
namespace RabbitMQSetting
|
||||
{
|
||||
extern const RabbitMQSettingsString rabbitmq_address;
|
||||
extern const RabbitMQSettingsBool rabbitmq_commit_on_select;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_empty_queue_backoff_end_ms;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_empty_queue_backoff_start_ms;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_empty_queue_backoff_step_ms;
|
||||
extern const RabbitMQSettingsString rabbitmq_exchange_name;
|
||||
extern const RabbitMQSettingsString rabbitmq_exchange_type;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_flush_interval_ms;
|
||||
extern const RabbitMQSettingsString rabbitmq_format;
|
||||
extern const RabbitMQSettingsStreamingHandleErrorMode rabbitmq_handle_error_mode;
|
||||
extern const RabbitMQSettingsString rabbitmq_host_port;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_max_block_size;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_max_rows_per_message;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_num_consumers;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_num_queues;
|
||||
extern const RabbitMQSettingsString rabbitmq_password;
|
||||
extern const RabbitMQSettingsBool rabbitmq_persistent;
|
||||
extern const RabbitMQSettingsString rabbitmq_queue_base;
|
||||
extern const RabbitMQSettingsBool rabbitmq_queue_consume;
|
||||
extern const RabbitMQSettingsString rabbitmq_queue_settings_list;
|
||||
extern const RabbitMQSettingsString rabbitmq_routing_key_list;
|
||||
extern const RabbitMQSettingsString rabbitmq_schema;
|
||||
extern const RabbitMQSettingsBool rabbitmq_secure;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_skip_broken_messages;
|
||||
extern const RabbitMQSettingsString rabbitmq_username;
|
||||
extern const RabbitMQSettingsString rabbitmq_vhost;
|
||||
extern const RabbitMQSettingsBool reject_unhandled_messages;
|
||||
}
|
||||
|
||||
static const uint32_t QUEUE_SIZE = 100000;
|
||||
static const auto MAX_FAILED_READ_ATTEMPTS = 10;
|
||||
static const auto RESCHEDULE_MS = 500;
|
||||
@ -84,26 +118,26 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, rabbitmq_settings(std::move(rabbitmq_settings_))
|
||||
, exchange_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_exchange_name))
|
||||
, format_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_format))
|
||||
, exchange_type(defineExchangeType(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_exchange_type)))
|
||||
, routing_keys(parseSettings(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_routing_key_list)))
|
||||
, schema_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_schema))
|
||||
, num_consumers(rabbitmq_settings->rabbitmq_num_consumers.value)
|
||||
, num_queues(rabbitmq_settings->rabbitmq_num_queues.value)
|
||||
, queue_base(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_queue_base))
|
||||
, queue_settings_list(parseSettings(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_queue_settings_list)))
|
||||
, max_rows_per_message(rabbitmq_settings->rabbitmq_max_rows_per_message)
|
||||
, exchange_name(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_exchange_name]))
|
||||
, format_name(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_format]))
|
||||
, exchange_type(defineExchangeType(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_exchange_type])))
|
||||
, routing_keys(parseSettings(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_routing_key_list])))
|
||||
, schema_name(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_schema]))
|
||||
, num_consumers((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_num_consumers].value)
|
||||
, num_queues((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_num_queues].value)
|
||||
, queue_base(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_queue_base]))
|
||||
, queue_settings_list(parseSettings(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_queue_settings_list])))
|
||||
, max_rows_per_message((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_max_rows_per_message])
|
||||
, log(getLogger("StorageRabbitMQ (" + table_id_.table_name + ")"))
|
||||
, persistent(rabbitmq_settings->rabbitmq_persistent.value)
|
||||
, use_user_setup(rabbitmq_settings->rabbitmq_queue_consume.value)
|
||||
, persistent((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_persistent].value)
|
||||
, use_user_setup((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_queue_consume].value)
|
||||
, hash_exchange(num_consumers > 1 || num_queues > 1)
|
||||
, semaphore(0, static_cast<int>(num_consumers))
|
||||
, unique_strbase(getRandomName())
|
||||
, queue_size(std::max(QUEUE_SIZE, static_cast<uint32_t>(getMaxBlockSize())))
|
||||
, milliseconds_to_wait(rabbitmq_settings->rabbitmq_empty_queue_backoff_start_ms)
|
||||
, milliseconds_to_wait((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_empty_queue_backoff_start_ms])
|
||||
{
|
||||
reject_unhandled_messages = rabbitmq_settings->reject_unhandled_messages
|
||||
reject_unhandled_messages = (*rabbitmq_settings)[RabbitMQSetting::reject_unhandled_messages]
|
||||
|| queue_settings_list.end() !=
|
||||
std::find_if(queue_settings_list.begin(), queue_settings_list.end(),
|
||||
[](const String & name) { return name.starts_with(deadletter_exchange_setting); });
|
||||
@ -111,11 +145,11 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
const auto & config = getContext()->getConfigRef();
|
||||
|
||||
std::pair<String, UInt16> parsed_address;
|
||||
auto setting_rabbitmq_username = rabbitmq_settings->rabbitmq_username.value;
|
||||
auto setting_rabbitmq_password = rabbitmq_settings->rabbitmq_password.value;
|
||||
auto setting_rabbitmq_username = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_username].value;
|
||||
auto setting_rabbitmq_password = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_password].value;
|
||||
String username, password;
|
||||
|
||||
if (rabbitmq_settings->rabbitmq_host_port.changed)
|
||||
if ((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_host_port].changed)
|
||||
{
|
||||
username = setting_rabbitmq_username.empty() ? config.getString("rabbitmq.username", "") : setting_rabbitmq_username;
|
||||
password = setting_rabbitmq_password.empty() ? config.getString("rabbitmq.password", "") : setting_rabbitmq_password;
|
||||
@ -124,7 +158,7 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"No username or password. They can be specified either in config or in storage settings");
|
||||
|
||||
parsed_address = parseAddress(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_host_port), 5672);
|
||||
parsed_address = parseAddress(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_host_port]), 5672);
|
||||
if (parsed_address.first.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
@ -132,7 +166,7 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
|
||||
context_->getRemoteHostFilter().checkHostAndPort(parsed_address.first, toString(parsed_address.second));
|
||||
}
|
||||
else if (!rabbitmq_settings->rabbitmq_address.changed)
|
||||
else if (!(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_address].changed)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "RabbitMQ requires either `rabbitmq_host_port` or `rabbitmq_address` setting");
|
||||
|
||||
configuration =
|
||||
@ -141,9 +175,9 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
.port = parsed_address.second,
|
||||
.username = username,
|
||||
.password = password,
|
||||
.vhost = config.getString("rabbitmq.vhost", getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_vhost)),
|
||||
.secure = rabbitmq_settings->rabbitmq_secure.value,
|
||||
.connection_string = getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_address)
|
||||
.vhost = config.getString("rabbitmq.vhost", getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_vhost])),
|
||||
.secure = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_secure].value,
|
||||
.connection_string = getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_address])
|
||||
};
|
||||
|
||||
if (configuration.secure)
|
||||
@ -156,7 +190,7 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
setVirtuals(createVirtuals(rabbitmq_settings->rabbitmq_handle_error_mode));
|
||||
setVirtuals(createVirtuals((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_handle_error_mode]));
|
||||
|
||||
rabbitmq_context = addSettings(getContext());
|
||||
rabbitmq_context->makeQueryContext();
|
||||
@ -212,6 +246,8 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
init_task->deactivate();
|
||||
}
|
||||
|
||||
StorageRabbitMQ::~StorageRabbitMQ() = default;
|
||||
|
||||
VirtualColumnsDescription StorageRabbitMQ::createVirtuals(StreamingHandleErrorMode handle_error_mode)
|
||||
{
|
||||
VirtualColumnsDescription desc;
|
||||
@ -280,8 +316,8 @@ ContextMutablePtr StorageRabbitMQ::addSettings(ContextPtr local_context) const
|
||||
auto modified_context = Context::createCopy(local_context);
|
||||
modified_context->setSetting("input_format_skip_unknown_fields", true);
|
||||
modified_context->setSetting("input_format_allow_errors_ratio", 0.);
|
||||
if (rabbitmq_settings->rabbitmq_handle_error_mode == StreamingHandleErrorMode::DEFAULT)
|
||||
modified_context->setSetting("input_format_allow_errors_num", rabbitmq_settings->rabbitmq_skip_broken_messages.value);
|
||||
if ((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_handle_error_mode] == StreamingHandleErrorMode::DEFAULT)
|
||||
modified_context->setSetting("input_format_allow_errors_num", (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_skip_broken_messages].value);
|
||||
else
|
||||
modified_context->setSetting("input_format_allow_errors_num", Field(0));
|
||||
|
||||
@ -291,14 +327,8 @@ ContextMutablePtr StorageRabbitMQ::addSettings(ContextPtr local_context) const
|
||||
if (!schema_name.empty())
|
||||
modified_context->setSetting("format_schema", schema_name);
|
||||
|
||||
for (const auto & setting : *rabbitmq_settings)
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
|
||||
/// check for non-rabbitmq-related settings
|
||||
if (!setting_name.starts_with("rabbitmq_"))
|
||||
modified_context->setSetting(setting_name, setting.getValue());
|
||||
}
|
||||
/// check for non-rabbitmq-related settings
|
||||
modified_context->applySettingsChanges(rabbitmq_settings->getFormatSettings());
|
||||
|
||||
return modified_context;
|
||||
}
|
||||
@ -396,8 +426,8 @@ void StorageRabbitMQ::deactivateTask(BackgroundSchedulePool::TaskHolder & task,
|
||||
|
||||
size_t StorageRabbitMQ::getMaxBlockSize() const
|
||||
{
|
||||
return rabbitmq_settings->rabbitmq_max_block_size.changed
|
||||
? rabbitmq_settings->rabbitmq_max_block_size.value
|
||||
return (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_max_block_size].changed
|
||||
? (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_max_block_size].value
|
||||
: (getContext()->getSettingsRef()[Setting::max_insert_block_size].value / num_consumers);
|
||||
}
|
||||
|
||||
@ -773,16 +803,16 @@ void StorageRabbitMQ::read(
|
||||
Pipes pipes;
|
||||
pipes.reserve(num_created_consumers);
|
||||
|
||||
uint64_t max_execution_time_ms = rabbitmq_settings->rabbitmq_flush_interval_ms.changed
|
||||
? rabbitmq_settings->rabbitmq_flush_interval_ms
|
||||
uint64_t max_execution_time_ms = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_flush_interval_ms].changed
|
||||
? (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_flush_interval_ms]
|
||||
: static_cast<UInt64>(getContext()->getSettingsRef()[Setting::stream_flush_interval_ms].totalMilliseconds());
|
||||
|
||||
for (size_t i = 0; i < num_created_consumers; ++i)
|
||||
{
|
||||
auto rabbit_source = std::make_shared<RabbitMQSource>(
|
||||
*this, storage_snapshot, modified_context, column_names, /* max_block_size */1,
|
||||
max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode, reject_unhandled_messages,
|
||||
/* ack_in_suffix */rabbitmq_settings->rabbitmq_commit_on_select, log);
|
||||
max_execution_time_ms, (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_handle_error_mode], reject_unhandled_messages,
|
||||
/* ack_in_suffix */(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_commit_on_select], log);
|
||||
|
||||
auto converting_dag = ActionsDAG::makeConvertingActions(
|
||||
rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(),
|
||||
@ -1041,8 +1071,8 @@ void StorageRabbitMQ::streamingToViewsFunc()
|
||||
else
|
||||
{
|
||||
/// Reschedule with backoff.
|
||||
if (milliseconds_to_wait < rabbitmq_settings->rabbitmq_empty_queue_backoff_end_ms)
|
||||
milliseconds_to_wait += rabbitmq_settings->rabbitmq_empty_queue_backoff_step_ms;
|
||||
if (milliseconds_to_wait < (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_empty_queue_backoff_end_ms])
|
||||
milliseconds_to_wait += (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_empty_queue_backoff_step_ms];
|
||||
|
||||
LOG_DEBUG(log, "Rescheduling background streaming process in {}", milliseconds_to_wait);
|
||||
streaming_task->scheduleAfter(milliseconds_to_wait);
|
||||
@ -1089,7 +1119,7 @@ void StorageRabbitMQ::streamToViewsImpl()
|
||||
break;
|
||||
}
|
||||
|
||||
milliseconds_to_wait = rabbitmq_settings->rabbitmq_empty_queue_backoff_start_ms;
|
||||
milliseconds_to_wait = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_empty_queue_backoff_start_ms];
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1110,15 +1140,15 @@ bool StorageRabbitMQ::tryStreamToViews()
|
||||
sources.reserve(num_created_consumers);
|
||||
pipes.reserve(num_created_consumers);
|
||||
|
||||
uint64_t max_execution_time_ms = rabbitmq_settings->rabbitmq_flush_interval_ms.changed
|
||||
? rabbitmq_settings->rabbitmq_flush_interval_ms
|
||||
uint64_t max_execution_time_ms = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_flush_interval_ms].changed
|
||||
? (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_flush_interval_ms]
|
||||
: static_cast<UInt64>(getContext()->getSettingsRef()[Setting::stream_flush_interval_ms].totalMilliseconds());
|
||||
|
||||
for (size_t i = 0; i < num_created_consumers; ++i)
|
||||
{
|
||||
auto source = std::make_shared<RabbitMQSource>(
|
||||
*this, storage_snapshot, rabbitmq_context, Names{}, block_size,
|
||||
max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode,
|
||||
max_execution_time_ms, (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_handle_error_mode],
|
||||
reject_unhandled_messages, /* ack_in_suffix */false, log);
|
||||
|
||||
sources.emplace_back(source);
|
||||
@ -1274,26 +1304,19 @@ void registerStorageRabbitMQ(StorageFactory & factory)
|
||||
auto rabbitmq_settings = std::make_unique<RabbitMQSettings>();
|
||||
|
||||
if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext()))
|
||||
{
|
||||
for (const auto & setting : rabbitmq_settings->all())
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
if (named_collection->has(setting_name))
|
||||
rabbitmq_settings->set(setting_name, named_collection->get<String>(setting_name));
|
||||
}
|
||||
}
|
||||
rabbitmq_settings->loadFromNamedCollection(named_collection);
|
||||
else if (!args.storage_def->settings)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "RabbitMQ engine must have settings");
|
||||
|
||||
if (args.storage_def->settings)
|
||||
rabbitmq_settings->loadFromQuery(*args.storage_def);
|
||||
|
||||
if (!rabbitmq_settings->rabbitmq_host_port.changed
|
||||
&& !rabbitmq_settings->rabbitmq_address.changed)
|
||||
if (!(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_host_port].changed
|
||||
&& !(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_address].changed)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"You must specify either `rabbitmq_host_port` or `rabbitmq_address` settings");
|
||||
|
||||
if (!rabbitmq_settings->rabbitmq_format.changed)
|
||||
if (!(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_format].changed)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `rabbitmq_format` setting");
|
||||
|
||||
return std::make_shared<StorageRabbitMQ>(args.table_id, args.getContext(), args.columns, args.comment, std::move(rabbitmq_settings), args.mode);
|
||||
|
@ -1,12 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Poco/Semaphore.h>
|
||||
#include <mutex>
|
||||
#include <atomic>
|
||||
#include <Storages/RabbitMQ/RabbitMQConsumer.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQSettings.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQConnection.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <amqpcpp/libuv.h>
|
||||
@ -16,7 +16,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct RabbitMQSettings;
|
||||
using RabbitMQConsumerPtr = std::shared_ptr<RabbitMQConsumer>;
|
||||
|
||||
class StorageRabbitMQ final: public IStorage, WithContext
|
||||
@ -30,6 +30,8 @@ public:
|
||||
std::unique_ptr<RabbitMQSettings> rabbitmq_settings_,
|
||||
LoadingStrictnessLevel mode);
|
||||
|
||||
~StorageRabbitMQ() override;
|
||||
|
||||
std::string getName() const override { return "RabbitMQ"; }
|
||||
|
||||
bool noPushingToViews() const override { return true; }
|
||||
|
@ -35,6 +35,11 @@ namespace Setting
|
||||
extern const SettingsUInt64 min_insert_block_size_rows;
|
||||
}
|
||||
|
||||
namespace RocksDBSetting
|
||||
{
|
||||
extern const RocksDBSettingsUInt64 bulk_insert_block_size;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ROCKSDB_ERROR;
|
||||
@ -91,7 +96,7 @@ EmbeddedRocksDBBulkSink::EmbeddedRocksDBBulkSink(
|
||||
|
||||
serializations = getHeader().getSerializations();
|
||||
min_block_size_rows
|
||||
= std::max(storage.getSettings().bulk_insert_block_size, getContext()->getSettingsRef()[Setting::min_insert_block_size_rows]);
|
||||
= std::max(storage.getSettings()[RocksDBSetting::bulk_insert_block_size], getContext()->getSettingsRef()[Setting::min_insert_block_size_rows]);
|
||||
|
||||
/// If max_insert_threads > 1 we may have multiple EmbeddedRocksDBBulkSink and getContext()->getCurrentQueryId() is not guarantee to
|
||||
/// to have a distinct path. Also we cannot use query id as directory name here, because it could be defined by user and not suitable
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include "RocksDBSettings.h"
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/RocksDB/RocksDBSettings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -10,17 +12,59 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(RockDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS)
|
||||
/** StorageEmbeddedRocksdb table settings
|
||||
*/
|
||||
#define LIST_OF_ROCKSDB_SETTINGS(M, ALIAS) \
|
||||
M(Bool, optimize_for_bulk_insert, true, "Table is optimized for bulk insertions (insert pipeline will create SST files and import to rocksdb database instead of writing to memtables)", 0) \
|
||||
M(UInt64, bulk_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "Size of block for bulk insert, if it's smaller than query setting min_insert_block_size_rows then it will be overridden by min_insert_block_size_rows", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RocksDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(RocksDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS)
|
||||
|
||||
struct RocksDBSettingsImpl : public BaseSettings<RocksDBSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) RocksDBSettings##TYPE NAME = &RocksDBSettingsImpl ::NAME;
|
||||
|
||||
namespace RocksDBSetting
|
||||
{
|
||||
LIST_OF_ROCKSDB_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
|
||||
void RocksDBSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr /*context*/)
|
||||
RocksDBSettings::RocksDBSettings() : impl(std::make_unique<RocksDBSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
RocksDBSettings::RocksDBSettings(const RocksDBSettings & settings) : impl(std::make_unique<RocksDBSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
RocksDBSettings::RocksDBSettings(RocksDBSettings && settings) noexcept
|
||||
: impl(std::make_unique<RocksDBSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
RocksDBSettings::~RocksDBSettings() = default;
|
||||
|
||||
ROCKSDB_SETTINGS_SUPPORTED_TYPES(RocksDBSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void RocksDBSettings::applyChanges(const SettingsChanges & changes)
|
||||
{
|
||||
impl->applyChanges(changes);
|
||||
}
|
||||
|
||||
void RocksDBSettings::loadFromQuery(const ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
try
|
||||
{
|
||||
auto changes = storage_def.settings->changes;
|
||||
applyChanges(changes);
|
||||
impl->applyChanges(changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -30,12 +74,4 @@ void RocksDBSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr /*conte
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<String> RocksDBSettings::getAllRegisteredNames() const
|
||||
{
|
||||
std::vector<String> all_settings;
|
||||
for (const auto & setting_field : all())
|
||||
all_settings.push_back(setting_field.getName());
|
||||
return all_settings;
|
||||
}
|
||||
}
|
||||
|
@ -1,39 +1,34 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <base/unit.h>
|
||||
#include <Common/NamePrompter.h>
|
||||
|
||||
|
||||
namespace Poco::Util
|
||||
{
|
||||
class AbstractConfiguration;
|
||||
}
|
||||
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
struct Settings;
|
||||
struct RocksDBSettingsImpl;
|
||||
class SettingsChanges;
|
||||
|
||||
/// List of available types supported in RocksDBSettings object
|
||||
#define ROCKSDB_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
/** StorageEmbeddedRocksdb table settings
|
||||
*/
|
||||
ROCKSDB_SETTINGS_SUPPORTED_TYPES(RocksDBSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
#define ROCKSDB_SETTINGS(M, ALIAS) \
|
||||
M(Bool, optimize_for_bulk_insert, true, "Table is optimized for bulk insertions (insert pipeline will create SST files and import to rocksdb database instead of writing to memtables)", 0) \
|
||||
M(UInt64, bulk_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "Size of block for bulk insert, if it's smaller than query setting min_insert_block_size_rows then it will be overridden by min_insert_block_size_rows", 0) \
|
||||
|
||||
#define LIST_OF_ROCKSDB_SETTINGS(M, ALIAS) ROCKSDB_SETTINGS(M, ALIAS)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RockDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS)
|
||||
|
||||
struct RocksDBSettings : public BaseSettings<RockDBSettingsTraits>, public IHints<2>
|
||||
struct RocksDBSettings
|
||||
{
|
||||
void loadFromQuery(ASTStorage & storage_def, ContextPtr context);
|
||||
std::vector<String> getAllRegisteredNames() const override;
|
||||
};
|
||||
RocksDBSettings();
|
||||
RocksDBSettings(const RocksDBSettings & settings);
|
||||
RocksDBSettings(RocksDBSettings && settings) noexcept;
|
||||
~RocksDBSettings();
|
||||
|
||||
ROCKSDB_SETTINGS_SUPPORTED_TYPES(RocksDBSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void applyChanges(const SettingsChanges & changes);
|
||||
void loadFromQuery(const ASTStorage & storage_def);
|
||||
|
||||
private:
|
||||
std::unique_ptr<RocksDBSettingsImpl> impl;
|
||||
};
|
||||
}
|
||||
|
@ -57,6 +57,11 @@ namespace Setting
|
||||
extern const SettingsBool optimize_trivial_approximate_count_query;
|
||||
}
|
||||
|
||||
namespace RocksDBSetting
|
||||
{
|
||||
extern const RocksDBSettingsBool optimize_for_bulk_insert;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -663,7 +668,7 @@ void ReadFromEmbeddedRocksDB::applyFilters(ActionDAGNodes added_filter_nodes)
|
||||
SinkToStoragePtr StorageEmbeddedRocksDB::write(
|
||||
const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context, bool /*async_insert*/)
|
||||
{
|
||||
if (getSettings().optimize_for_bulk_insert)
|
||||
if (getSettings()[RocksDBSetting::optimize_for_bulk_insert])
|
||||
{
|
||||
LOG_DEBUG(log, "Using bulk insert");
|
||||
return std::make_shared<EmbeddedRocksDBBulkSink>(query_context, *this, metadata_snapshot);
|
||||
@ -710,7 +715,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageEmbeddedRocksDB must require one column in primary key");
|
||||
}
|
||||
auto settings = std::make_unique<RocksDBSettings>();
|
||||
settings->loadFromQuery(*args.storage_def, args.getContext());
|
||||
settings->loadFromQuery(*args.storage_def);
|
||||
if (args.storage_def->settings)
|
||||
metadata.settings_changes = args.storage_def->settings->ptr();
|
||||
else
|
||||
@ -720,7 +725,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
/// SETTING queries. So we just add a setting with its default value.
|
||||
auto settings_changes = std::make_shared<ASTSetQuery>();
|
||||
settings_changes->is_standalone = false;
|
||||
settings_changes->changes.insertSetting("optimize_for_bulk_insert", settings->optimize_for_bulk_insert.value);
|
||||
settings_changes->changes.insertSetting("optimize_for_bulk_insert", (*settings)[RocksDBSetting::optimize_for_bulk_insert].value);
|
||||
metadata.settings_changes = settings_changes;
|
||||
}
|
||||
return std::make_shared<StorageEmbeddedRocksDB>(args.table_id, args.relative_data_path, metadata, args.mode, args.getContext(), std::move(settings), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only);
|
||||
|
@ -53,6 +53,11 @@ namespace ServerSetting
|
||||
extern const ServerSettingsUInt64 max_materialized_views_count_for_table;
|
||||
}
|
||||
|
||||
namespace RefreshSetting
|
||||
{
|
||||
extern const RefreshSettingsBool all_replicas;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -177,7 +182,7 @@ StorageMaterializedView::StorageMaterializedView(
|
||||
RefreshSettings s;
|
||||
if (query.refresh_strategy->settings)
|
||||
s.applyChanges(query.refresh_strategy->settings->changes);
|
||||
refresh_coordinated = !s.all_replicas;
|
||||
refresh_coordinated = !s[RefreshSetting::all_replicas];
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -179,9 +179,9 @@ StorageTimeSeries::StorageTimeSeries(
|
||||
StorageTimeSeries::~StorageTimeSeries() = default;
|
||||
|
||||
|
||||
TimeSeriesSettings StorageTimeSeries::getStorageSettings() const
|
||||
const TimeSeriesSettings & StorageTimeSeries::getStorageSettings() const
|
||||
{
|
||||
return *getStorageSettingsPtr();
|
||||
return *storage_settings;
|
||||
}
|
||||
|
||||
void StorageTimeSeries::startup()
|
||||
|
@ -41,8 +41,7 @@ public:
|
||||
|
||||
std::string getName() const override { return "TimeSeries"; }
|
||||
|
||||
TimeSeriesSettings getStorageSettings() const;
|
||||
TimeSeriesSettingsPtr getStorageSettingsPtr() const { return storage_settings; }
|
||||
const TimeSeriesSettings & getStorageSettings() const;
|
||||
|
||||
StorageID getTargetTableId(ViewTarget::Kind target_kind) const;
|
||||
StoragePtr getTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const;
|
||||
|
@ -31,6 +31,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsBool filter_by_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_REQUEST_PARAMETER;
|
||||
@ -160,7 +166,7 @@ namespace
|
||||
std::unordered_map<String, String> makeColumnNameByTagNameMap(const TimeSeriesSettings & storage_settings)
|
||||
{
|
||||
std::unordered_map<String, String> res;
|
||||
const Map & tags_to_columns = storage_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = storage_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -196,7 +202,7 @@ namespace
|
||||
exp_list->children.push_back(
|
||||
makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName));
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -255,7 +261,7 @@ namespace
|
||||
|
||||
/// WHERE <filter>
|
||||
if (auto where = makeASTFilterForReadingTimeSeries(label_matcher, min_timestamp_ms, max_timestamp_ms, data_table_id, tags_table_id,
|
||||
column_name_by_tag_name, time_series_settings.filter_by_min_time_and_max_time))
|
||||
column_name_by_tag_name, time_series_settings[TimeSeriesSetting::filter_by_min_time_and_max_time]))
|
||||
{
|
||||
select_query->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where));
|
||||
}
|
||||
@ -267,7 +273,7 @@ namespace
|
||||
exp_list->children.push_back(
|
||||
makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName));
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -329,7 +335,7 @@ namespace
|
||||
|
||||
/// Columns corresponding to specific tags specified in the "tags_to_columns" setting.
|
||||
std::unordered_map<String, const IColumn *> column_by_tag_name;
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -447,12 +453,12 @@ void PrometheusRemoteReadProtocol::readTimeSeries(google::protobuf::RepeatedPtrF
|
||||
out_time_series.Clear();
|
||||
|
||||
auto time_series_storage_id = time_series_storage->getStorageID();
|
||||
auto time_series_settings = time_series_storage->getStorageSettingsPtr();
|
||||
const auto & time_series_settings = time_series_storage->getStorageSettings();
|
||||
auto data_table_id = time_series_storage->getTargetTableId(ViewTarget::Data);
|
||||
auto tags_table_id = time_series_storage->getTargetTableId(ViewTarget::Tags);
|
||||
|
||||
ASTPtr select_query = buildSelectQueryForReadingTimeSeries(
|
||||
start_timestamp_ms, end_timestamp_ms, label_matcher, *time_series_settings, data_table_id, tags_table_id);
|
||||
start_timestamp_ms, end_timestamp_ms, label_matcher, time_series_settings, data_table_id, tags_table_id);
|
||||
|
||||
LOG_TRACE(log, "{}: Executing query {}",
|
||||
time_series_storage_id.getNameForLogs(), select_query);
|
||||
@ -468,7 +474,7 @@ void PrometheusRemoteReadProtocol::readTimeSeries(google::protobuf::RepeatedPtrF
|
||||
time_series_storage_id.getNameForLogs(), block.columns(), block.rows());
|
||||
|
||||
if (block)
|
||||
convertBlockToProtobuf(std::move(block), out_time_series, time_series_storage_id, *time_series_settings);
|
||||
convertBlockToProtobuf(std::move(block), out_time_series, time_series_storage_id, time_series_settings);
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "{}: {} time series read",
|
||||
|
@ -32,6 +32,13 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsBool store_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
extern const TimeSeriesSettingsBool use_all_tags_column_to_generate_id;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TIME_SERIES_TAGS;
|
||||
@ -261,7 +268,7 @@ namespace
|
||||
|
||||
/// Columns corresponding to specific tags specified in the "tags_to_columns" setting.
|
||||
std::unordered_map<String, IColumn *> columns_by_tag_name;
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -286,7 +293,7 @@ namespace
|
||||
IColumn * all_tags_names = nullptr;
|
||||
IColumn * all_tags_values = nullptr;
|
||||
IColumn::Offsets * all_tags_offsets = nullptr;
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
{
|
||||
const auto & all_tags_description = get_column_description(TimeSeriesColumnNames::AllTags);
|
||||
validator.validateColumnForTagsMap(all_tags_description);
|
||||
@ -301,7 +308,7 @@ namespace
|
||||
IColumn * max_time_column = nullptr;
|
||||
UInt32 min_time_scale = 0;
|
||||
UInt32 max_time_scale = 0;
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
const auto & min_time_description = get_column_description(TimeSeriesColumnNames::MinTime);
|
||||
const auto & max_time_description = get_column_description(TimeSeriesColumnNames::MaxTime);
|
||||
@ -336,7 +343,7 @@ namespace
|
||||
}
|
||||
else
|
||||
{
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
{
|
||||
all_tags_names->insertData(tag_name.data(), tag_name.length());
|
||||
all_tags_values->insertData(tag_value.data(), tag_value.length());
|
||||
@ -359,10 +366,10 @@ namespace
|
||||
|
||||
tags_offsets.push_back(tags_names.size());
|
||||
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
all_tags_offsets->push_back(all_tags_names->size());
|
||||
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
auto [min_time, max_time] = findMinTimeAndMaxTime(element.samples());
|
||||
min_time_column->insert(scaleTimestamp(min_time, min_time_scale));
|
||||
@ -571,9 +578,9 @@ void PrometheusRemoteWriteProtocol::writeTimeSeries(const google::protobuf::Repe
|
||||
time_series_storage_id.getNameForLogs(), time_series.size());
|
||||
|
||||
auto time_series_storage_metadata = time_series_storage->getInMemoryMetadataPtr();
|
||||
auto time_series_settings = time_series_storage->getStorageSettingsPtr();
|
||||
const auto & time_series_settings = time_series_storage->getStorageSettings();
|
||||
|
||||
auto blocks = toBlocks(time_series, getContext(), time_series_storage_id, *time_series_storage_metadata, *time_series_settings);
|
||||
auto blocks = toBlocks(time_series, getContext(), time_series_storage_id, *time_series_storage_metadata, time_series_settings);
|
||||
insertToTargetTables(std::move(blocks), *time_series_storage, getContext(), log.get());
|
||||
|
||||
LOG_TRACE(log, "{}: {} time series written",
|
||||
@ -588,9 +595,9 @@ void PrometheusRemoteWriteProtocol::writeMetricsMetadata(const google::protobuf:
|
||||
time_series_storage_id.getNameForLogs(), metrics_metadata.size());
|
||||
|
||||
auto time_series_storage_metadata = time_series_storage->getInMemoryMetadataPtr();
|
||||
auto time_series_settings = time_series_storage->getStorageSettingsPtr();
|
||||
const auto & time_series_settings = time_series_storage->getStorageSettings();
|
||||
|
||||
auto blocks = toBlocks(metrics_metadata, time_series_storage_id, *time_series_storage_metadata, *time_series_settings);
|
||||
auto blocks = toBlocks(metrics_metadata, time_series_storage_id, *time_series_storage_metadata, time_series_settings);
|
||||
insertToTargetTables(std::move(blocks), *time_series_storage, getContext(), log.get());
|
||||
|
||||
LOG_TRACE(log, "{}: {} metrics metadata written",
|
||||
|
@ -12,6 +12,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
@ -63,7 +68,7 @@ void TimeSeriesColumnsValidator::validateColumnsImpl(const ColumnsDescription &
|
||||
/// Validate columns for the "tags" table.
|
||||
validateColumnForMetricName(get_column_description(TimeSeriesColumnNames::MetricName));
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -127,7 +132,7 @@ void TimeSeriesColumnsValidator::validateTargetColumnsImpl(ViewTarget::Kind targ
|
||||
{
|
||||
validateColumnForMetricName(get_column_description(TimeSeriesColumnNames::MetricName));
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
|
@ -15,6 +15,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsBool aggregate_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsBool store_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
extern const TimeSeriesSettingsBool use_all_tags_column_to_generate_id;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCOMPATIBLE_COLUMNS;
|
||||
@ -85,7 +93,7 @@ void TimeSeriesDefinitionNormalizer::reorderColumns(ASTCreateQuery & create) con
|
||||
/// Reorder columns for the "tags" table.
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::MetricName);
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -96,7 +104,7 @@ void TimeSeriesDefinitionNormalizer::reorderColumns(ASTCreateQuery & create) con
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::Tags);
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::AllTags);
|
||||
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::MinTime);
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::MaxTime);
|
||||
@ -198,7 +206,7 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create)
|
||||
make_new_column(TimeSeriesColumnNames::MetricName, get_lc_string_type());
|
||||
}
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -221,7 +229,7 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create)
|
||||
make_new_column(TimeSeriesColumnNames::AllTags, get_string_to_string_map_type());
|
||||
}
|
||||
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
/// We use Nullable(DateTime64(3)) as the default type of the `min_time` and `max_time` columns.
|
||||
/// It's nullable because it allows the aggregation (see aggregate_min_time_and_max_time) work correctly even
|
||||
@ -284,13 +292,13 @@ ASTPtr TimeSeriesDefinitionNormalizer::chooseIDAlgorithm(const ASTColumnDeclarat
|
||||
ASTs arguments_for_hash_function;
|
||||
arguments_for_hash_function.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::MetricName));
|
||||
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
{
|
||||
arguments_for_hash_function.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::AllTags));
|
||||
}
|
||||
else
|
||||
{
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -424,7 +432,7 @@ void TimeSeriesDefinitionNormalizer::setInnerEngineByDefault(ViewTarget::Kind in
|
||||
case ViewTarget::Tags:
|
||||
{
|
||||
String engine_name;
|
||||
if (time_series_settings.aggregate_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::aggregate_min_time_and_max_time])
|
||||
engine_name = "AggregatingMergeTree";
|
||||
else
|
||||
engine_name = "ReplacingMergeTree";
|
||||
@ -441,7 +449,7 @@ void TimeSeriesDefinitionNormalizer::setInnerEngineByDefault(ViewTarget::Kind in
|
||||
order_by_list.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::MetricName));
|
||||
order_by_list.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::ID));
|
||||
|
||||
if (time_series_settings.store_min_time_and_max_time && !time_series_settings.aggregate_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time] && !time_series_settings[TimeSeriesSetting::aggregate_min_time_and_max_time])
|
||||
{
|
||||
order_by_list.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::MinTime));
|
||||
order_by_list.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::MaxTime));
|
||||
|
@ -22,6 +22,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsBool aggregate_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsBool store_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
extern const TimeSeriesSettingsBool use_all_tags_column_to_generate_id;
|
||||
}
|
||||
|
||||
TimeSeriesInnerTablesCreator::TimeSeriesInnerTablesCreator(ContextPtr context_,
|
||||
StorageID time_series_storage_id_,
|
||||
std::reference_wrapper<const ColumnsDescription> time_series_columns_,
|
||||
@ -70,7 +78,7 @@ ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription
|
||||
columns.add(time_series_columns.get(TimeSeriesColumnNames::MetricName));
|
||||
|
||||
/// Columns corresponding to specific tags specified in the "tags_to_columns" setting.
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -82,7 +90,7 @@ ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription
|
||||
columns.add(time_series_columns.get(TimeSeriesColumnNames::Tags));
|
||||
|
||||
/// Column "all_tags".
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
{
|
||||
ColumnDescription all_tags_column = time_series_columns.get(TimeSeriesColumnNames::AllTags);
|
||||
/// Column "all_tags" is here only to calculate the identifier of a time series for the "id" column, so it can be ephemeral.
|
||||
@ -96,11 +104,11 @@ ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription
|
||||
}
|
||||
|
||||
/// Columns "min_time" and "max_time".
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
auto min_time_column = time_series_columns.get(TimeSeriesColumnNames::MinTime);
|
||||
auto max_time_column = time_series_columns.get(TimeSeriesColumnNames::MaxTime);
|
||||
if (time_series_settings.aggregate_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::aggregate_min_time_and_max_time])
|
||||
{
|
||||
AggregateFunctionProperties properties;
|
||||
auto min_function = AggregateFunctionFactory::instance().get("min", NullsAction::EMPTY, {min_time_column.type}, {}, properties);
|
||||
|
@ -1,7 +1,8 @@
|
||||
#include <Storages/TimeSeries/TimeSeriesSettings.h>
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/TimeSeries/TimeSeriesSettings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,15 +13,53 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
#define LIST_OF_TIME_SERIES_SETTINGS(M, ALIAS) \
|
||||
M(Map, tags_to_columns, Map{}, "Map specifying which tags should be put to separate columns of the 'tags' table. Syntax: {'tag1': 'column1', 'tag2' : column2, ...}", 0) \
|
||||
M(Bool, use_all_tags_column_to_generate_id, true, "When generating an expression to calculate an identifier of a time series, this flag enables using the 'all_tags' column in that calculation. The 'all_tags' is a virtual column containing all tags except the metric name", 0) \
|
||||
M(Bool, store_min_time_and_max_time, true, "If set to true then the table will store 'min_time' and 'max_time' for each time series", 0) \
|
||||
M(Bool, aggregate_min_time_and_max_time, true, "When creating an inner target 'tags' table, this flag enables using 'SimpleAggregateFunction(min, Nullable(DateTime64(3)))' instead of just 'Nullable(DateTime64(3))' as the type of the 'min_time' column, and the same for the 'max_time' column", 0) \
|
||||
M(Bool, filter_by_min_time_and_max_time, true, "If set to true then the table will use the 'min_time' and 'max_time' columns for filtering time series", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS)
|
||||
|
||||
struct TimeSeriesSettingsImpl : public BaseSettings<TimeSeriesSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) TimeSeriesSettings##TYPE NAME = &TimeSeriesSettingsImpl ::NAME;
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
LIST_OF_TIME_SERIES_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
TimeSeriesSettings::TimeSeriesSettings() : impl(std::make_unique<TimeSeriesSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
TimeSeriesSettings::TimeSeriesSettings(const TimeSeriesSettings & settings) : impl(std::make_unique<TimeSeriesSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
TimeSeriesSettings::TimeSeriesSettings(TimeSeriesSettings && settings) noexcept
|
||||
: impl(std::make_unique<TimeSeriesSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
TimeSeriesSettings::~TimeSeriesSettings() = default;
|
||||
|
||||
TIMESERIES_SETTINGS_SUPPORTED_TYPES(TimeSeriesSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void TimeSeriesSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
try
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
impl->applyChanges(storage_def.settings->changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
|
@ -1,29 +1,36 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
struct TimeSeriesSettingsImpl;
|
||||
|
||||
#define LIST_OF_TIME_SERIES_SETTINGS(M, ALIAS) \
|
||||
M(Map, tags_to_columns, Map{}, "Map specifying which tags should be put to separate columns of the 'tags' table. Syntax: {'tag1': 'column1', 'tag2' : column2, ...}", 0) \
|
||||
M(Bool, use_all_tags_column_to_generate_id, true, "When generating an expression to calculate an identifier of a time series, this flag enables using the 'all_tags' column in that calculation. The 'all_tags' is a virtual column containing all tags except the metric name", 0) \
|
||||
M(Bool, store_min_time_and_max_time, true, "If set to true then the table will store 'min_time' and 'max_time' for each time series", 0) \
|
||||
M(Bool, aggregate_min_time_and_max_time, true, "When creating an inner target 'tags' table, this flag enables using 'SimpleAggregateFunction(min, Nullable(DateTime64(3)))' instead of just 'Nullable(DateTime64(3))' as the type of the 'min_time' column, and the same for the 'max_time' column", 0) \
|
||||
M(Bool, filter_by_min_time_and_max_time, true, "If set to true then the table will use the 'min_time' and 'max_time' columns for filtering time series", 0) \
|
||||
/// List of available types supported in TimeSeriesSettings object
|
||||
#define TIMESERIES_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, Map)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS)
|
||||
TIMESERIES_SETTINGS_SUPPORTED_TYPES(TimeSeriesSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
/// Settings for the TimeSeries table engine.
|
||||
/// Could be loaded from a CREATE TABLE query (SETTINGS clause). For example:
|
||||
/// CREATE TABLE mytable ENGINE = TimeSeries() SETTINGS tags_to_columns = {'job':'job', 'instance':'instance'} DATA ENGINE = ReplicatedMergeTree('zkpath', 'replica'), ...
|
||||
struct TimeSeriesSettings : public BaseSettings<TimeSeriesSettingsTraits>
|
||||
struct TimeSeriesSettings
|
||||
{
|
||||
TimeSeriesSettings();
|
||||
TimeSeriesSettings(const TimeSeriesSettings & settings);
|
||||
TimeSeriesSettings(TimeSeriesSettings && settings) noexcept;
|
||||
~TimeSeriesSettings();
|
||||
|
||||
TIMESERIES_SETTINGS_SUPPORTED_TYPES(TimeSeriesSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
|
||||
private:
|
||||
std::unique_ptr<TimeSeriesSettingsImpl> impl;
|
||||
};
|
||||
|
||||
using TimeSeriesSettingsPtr = std::shared_ptr<const TimeSeriesSettings>;
|
||||
|
||||
}
|
||||
|
115
utils/check-style/check-settings-style
Executable file
115
utils/check-style/check-settings-style
Executable file
@ -0,0 +1,115 @@
|
||||
#!/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/Storages/TimeSeries/TimeSeriesSettings.cpp
|
||||
$ROOT_PATH/src/Storages/RocksDB/RocksDBSettings.cpp
|
||||
$ROOT_PATH/src/Storages/RabbitMQ/RabbitMQSettings.cpp
|
||||
$ROOT_PATH/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp
|
||||
$ROOT_PATH/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp
|
||||
$ROOT_PATH/src/Storages/MaterializedView/RefreshSettings.cpp
|
||||
$ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h"
|
||||
|
||||
for settings_file in ${ALL_DECLARATION_FILES};
|
||||
do
|
||||
if ! [ -f "${settings_file}" ]; then
|
||||
echo "File '${settings_file}' does not exist."
|
||||
fi
|
||||
done
|
||||
|
||||
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}
|
||||
cat $ROOT_PATH/src/Storages/TimeSeries/TimeSeriesSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " TimeSeriesSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/RocksDB/RocksDBSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " RocksDBSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/RabbitMQ/RabbitMQSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " RabbitMQSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " MaterializedPostgreSQLSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " ObjectStorageQueueSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/MaterializedView/RefreshSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " RefreshSettings" 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/TimeSeriesSettings//g' \
|
||||
-e 's/RabbitMQSettings//g' \
|
||||
-e 's/RocksDBSettings//g' \
|
||||
-e 's/MaterializedPostgreSQLSettings//g' \
|
||||
-e 's/ObjectStorageQueueSettings//g' \
|
||||
-e 's/RefreshSettings//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 RabbitMQSettings" \
|
||||
-e "^\s**extern const RocksDBSettings" \
|
||||
-e "^\s**extern const MaterializedPostgreSQLSettings" \
|
||||
-e "^\s**extern const ObjectStorageQueueSettings" \
|
||||
-e "^\s**extern const RefreshSettings" \
|
||||
-e "^\s**extern const TimeSeriesSettings" \
|
||||
-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/TimeSeriesSettings//g' \
|
||||
-e 's/RabbitMQSettings//g' \
|
||||
-e 's/RefreshSettings//g' \
|
||||
-e 's/RocksDBSettings//g' \
|
||||
-e 's/MaterializedPostgreSQLSettings//g' \
|
||||
-e 's/ObjectStorageQueueSettings//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"
|
||||
|
||||
# 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/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}
|
||||
bash $ROOT_PATH/utils/check-style/check-settings-style
|
||||
|
||||
# Unused/Undefined/Duplicates ErrorCodes/ProfileEvents/CurrentMetrics
|
||||
declare -A EXTERN_TYPES
|
||||
|
Loading…
Reference in New Issue
Block a user