mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Add system table settings_changes
This commit is contained in:
parent
559d507f5a
commit
80b5bc5236
@ -146,7 +146,7 @@ std::vector<String> Settings::getAllRegisteredNames() const
|
||||
return all_settings;
|
||||
}
|
||||
|
||||
void Settings::set(const std::string_view & name, const Field & value)
|
||||
void Settings::set(std::string_view name, const Field & value)
|
||||
{
|
||||
BaseSettings::set(name, value);
|
||||
|
||||
@ -173,14 +173,23 @@ void Settings::applyCompatibilitySetting()
|
||||
return;
|
||||
|
||||
ClickHouseVersion version(compatibility);
|
||||
for (const auto & [setting_name, history] : settings_changes_history)
|
||||
/// Iterate through ClickHouse version in descending order and apply reversed
|
||||
/// changes for each version that is higher that version from compatibility setting
|
||||
for (auto it = settings_changes_history.rbegin(); it != settings_changes_history.rend(); ++it)
|
||||
{
|
||||
/// If this setting was changed manually, we don't change it
|
||||
if (isChanged(setting_name))
|
||||
continue;
|
||||
if (version >= it->first)
|
||||
break;
|
||||
|
||||
BaseSettings::set(setting_name, history.getValueForVersion(version));
|
||||
settings_changed_by_compatibility_setting.insert(setting_name);
|
||||
/// Apply reversed changes from this version.
|
||||
for (const auto & change : it->second)
|
||||
{
|
||||
/// If this setting was changed manually, we don't change it
|
||||
if (isChanged(change.name) && !settings_changed_by_compatibility_setting.contains(change.name))
|
||||
continue;
|
||||
|
||||
BaseSettings::set(change.name, change.previous_value);
|
||||
settings_changed_by_compatibility_setting.insert(change.name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -832,7 +832,7 @@ struct Settings : public BaseSettings<SettingsTraits>, public IHints<2, Settings
|
||||
|
||||
void addProgramOptionAsMultitoken(boost::program_options::options_description & options, const SettingFieldRef & field);
|
||||
|
||||
void set(const std::string_view & name, const Field & value) override;
|
||||
void set(std::string_view name, const Field & value) override;
|
||||
|
||||
private:
|
||||
void applyCompatibilitySetting();
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <unordered_map>
|
||||
@ -21,6 +22,9 @@ public:
|
||||
Strings split;
|
||||
boost::split(split, version, [](char c){ return c == '.'; });
|
||||
components.reserve(split.size());
|
||||
if (split.empty())
|
||||
throw Exception{ErrorCodes::BAD_ARGUMENTS, "Cannot parse ClickHouse version here: {}", version};
|
||||
|
||||
for (const auto & split_element : split)
|
||||
{
|
||||
size_t component;
|
||||
@ -32,80 +36,77 @@ public:
|
||||
|
||||
ClickHouseVersion(const char * version) : ClickHouseVersion(String(version)) {}
|
||||
|
||||
String toString() const
|
||||
{
|
||||
String version = std::to_string(components[0]);
|
||||
for (size_t i = 1; i < components.size(); ++i)
|
||||
version += "." + std::to_string(components[i]);
|
||||
|
||||
return version;
|
||||
}
|
||||
|
||||
bool operator<(const ClickHouseVersion & other) const
|
||||
{
|
||||
return components < other.components;
|
||||
}
|
||||
|
||||
bool operator>=(const ClickHouseVersion & other) const
|
||||
{
|
||||
return components >= other.components;
|
||||
}
|
||||
|
||||
private:
|
||||
std::vector<size_t> components;
|
||||
};
|
||||
|
||||
struct SettingChangesHistory
|
||||
namespace SettingsChangesHistory
|
||||
{
|
||||
struct Change
|
||||
struct SettingChange
|
||||
{
|
||||
Change(const Field & value_, const ClickHouseVersion & version_) : value(value_), version(version_) {}
|
||||
|
||||
Field value;
|
||||
ClickHouseVersion version;
|
||||
String name;
|
||||
Field previous_value;
|
||||
Field new_value;
|
||||
String reason;
|
||||
};
|
||||
|
||||
SettingChangesHistory(const Field & initial_value_, const std::vector<Change> & changes_) : initial_value(initial_value_), changes(changes_) {}
|
||||
|
||||
Field getValueForVersion(const ClickHouseVersion & version) const
|
||||
{
|
||||
Field value = initial_value;
|
||||
for (const auto & change : changes)
|
||||
{
|
||||
if (version < change.version)
|
||||
return value;
|
||||
value = change.value;
|
||||
}
|
||||
return value;
|
||||
}
|
||||
|
||||
Field initial_value;
|
||||
std::vector<Change> changes;
|
||||
};
|
||||
using SettingsChanges = std::vector<SettingChange>;
|
||||
}
|
||||
|
||||
/// History of settings changes that controls some backward incompatible changes
|
||||
/// across all ClickHouse versions. It maps setting name to special struct
|
||||
/// SettingChangesHistory {initial_value, {{changed_value_1, version1}, {changed_value_2, version_2}, ...}}
|
||||
/// across all ClickHouse versions. It maps ClickHouse version to settings changes that were done
|
||||
/// in this version. Settings changes is a vector of structs {setting_name, previous_value, new_value}
|
||||
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
|
||||
const std::unordered_map<String, SettingChangesHistory> settings_changes_history =
|
||||
std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
|
||||
{
|
||||
{"enable_positional_arguments", {false, {{true, "22.7"}}}},
|
||||
{"output_format_json_named_tuples_as_objects", {false, {{true, "22.6"}}}},
|
||||
{"memory_overcommit_ratio_denominator", {0, {{1073741824, "22.5"}}}},
|
||||
{"memory_overcommit_ratio_denominator_for_user", {0, {{1073741824, "22.5"}}}},
|
||||
{"allow_settings_after_format_in_insert", {true, {{false, "22.4"}}}},
|
||||
{"cast_ipv4_ipv6_default_on_conversion_error", {true, {{false, "22.3"}}}},
|
||||
{"input_format_ipv4_default_on_conversion_error", {true, {{false, "22.3"}}}},
|
||||
{"input_format_ipv6_default_on_conversion_error", {true, {{false, "22.3"}}}},
|
||||
{"stream_like_engine_allow_direct_select", {true, {{false, "21.12"}}}},
|
||||
{"output_format_decimal_trailing_zeros", {true, {{false, "21.9"}}}},
|
||||
{"use_hedged_requests", {false, {{true, "21.9"}}}},
|
||||
{"legacy_column_name_of_tuple_literal", {true, {{false, "21.7"}}}},
|
||||
{"async_socket_for_remote", {true, {{false, "21.3"}, {true, "21.5"}}}},
|
||||
{"optimize_normalize_count_variants", {false, {{true, "21.3"}}}},
|
||||
{"normalize_function_names", {false, {{true, "21.3"}}}},
|
||||
{"enable_global_with_statement", {false, {{true, "21.2"}}}},
|
||||
{"insert_quorum_parallel", {false, {{true, "21.1"}}}},
|
||||
{"input_format_null_as_default", {false, {{true, "21.1"}}}},
|
||||
{"optimize_on_insert", {false, {{true, "21.1"}}}},
|
||||
{"use_compact_format_in_distributed_parts_names", {false, {{true, "21.1"}}}},
|
||||
{"format_regexp_escaping_rule", {"Escaped", {{"Raw", "20.10"}}}},
|
||||
{"show_table_uuid_in_table_create_query_if_not_nil", {true, {{false, "20.7"}}}},
|
||||
{"input_format_with_names_use_header", {false, {{true, "20.5"}}}},
|
||||
{"allow_suspicious_codecs", {true, {{false, "20.5"}}}},
|
||||
{"validate_polygons", {false, {{true, "20.4"}}}},
|
||||
{"enable_scalar_subquery_optimization", {false, {{true, "19.18"}}}},
|
||||
{"any_join_distinct_right_table_keys", {true, {{false, "19.14"}}}},
|
||||
{"input_format_defaults_for_omitted_fields", {false, {{true, "19.12"}}}},
|
||||
{"max_partitions_per_insert_block", {0, {{100, "19.5"}}}},
|
||||
{"enable_optimize_predicate_expression", {0, {{1, "18.12.17"}}}},
|
||||
{"22.7", {{"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}}},
|
||||
{"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}}},
|
||||
{"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"},
|
||||
{"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}},
|
||||
{"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}},
|
||||
{"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}},
|
||||
{"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}},
|
||||
{"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"},
|
||||
{"use_hedged_requests", false, true, "Enable Hedged Requests feature bu default"}}},
|
||||
{"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}},
|
||||
{"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}},
|
||||
{"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"},
|
||||
{"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"},
|
||||
{"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}},
|
||||
{"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}},
|
||||
{"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"},
|
||||
{"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"},
|
||||
{"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"},
|
||||
{"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}},
|
||||
{"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}},
|
||||
{"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}},
|
||||
{"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"},
|
||||
{"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}},
|
||||
{"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}},
|
||||
{"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}},
|
||||
{"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}},
|
||||
{"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for ommitted fields for some input formats, because it should be the expected behaviour"}}},
|
||||
{"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}},
|
||||
{"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}},
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
37
src/Storages/System/StorageSystemSettingsChanges.cpp
Normal file
37
src/Storages/System/StorageSystemSettingsChanges.cpp
Normal file
@ -0,0 +1,37 @@
|
||||
#include <Storages/System/StorageSystemSettingsChanges.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Core/SettingsChangesHistory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
NamesAndTypesList StorageSystemSettingsChanges::getNamesAndTypes()
|
||||
{
|
||||
return {
|
||||
{"version", std::make_shared<DataTypeString>()},
|
||||
{"changes",
|
||||
std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(
|
||||
DataTypes{
|
||||
std::make_shared<DataTypeString>(),
|
||||
std::make_shared<DataTypeString>(),
|
||||
std::make_shared<DataTypeString>(),
|
||||
std::make_shared<DataTypeString>()},
|
||||
Names{"name", "previous_value", "new_value", "reason"}))},
|
||||
};
|
||||
}
|
||||
|
||||
void StorageSystemSettingsChanges::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const
|
||||
{
|
||||
for (auto it = settings_changes_history.rbegin(); it != settings_changes_history.rend(); ++it)
|
||||
{
|
||||
res_columns[0]->insert(it->first.toString());
|
||||
Array changes;
|
||||
for (const auto & change : it->second)
|
||||
changes.push_back(Tuple{change.name, toString(change.previous_value), toString(change.new_value), change.reason});
|
||||
res_columns[1]->insert(changes);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
28
src/Storages/System/StorageSystemSettingsChanges.h
Normal file
28
src/Storages/System/StorageSystemSettingsChanges.h
Normal file
@ -0,0 +1,28 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
|
||||
/** Implements system table "settings_changes", which allows to get information
|
||||
* about the settings changes through different ClickHouse versions.
|
||||
*/
|
||||
class StorageSystemSettingsChanges final : public IStorageSystemOneBlock<StorageSystemSettingsChanges>
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "SystemSettingsChanges"; }
|
||||
|
||||
static NamesAndTypesList getNamesAndTypes();
|
||||
|
||||
protected:
|
||||
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||
|
||||
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;
|
||||
};
|
||||
|
||||
}
|
@ -39,6 +39,7 @@
|
||||
#include <Storages/System/StorageSystemReplicationQueue.h>
|
||||
#include <Storages/System/StorageSystemDistributionQueue.h>
|
||||
#include <Storages/System/StorageSystemSettings.h>
|
||||
#include <Storages/System/StorageSystemSettingsChanges.h>
|
||||
#include <Storages/System/StorageSystemMergeTreeSettings.h>
|
||||
#include <Storages/System/StorageSystemTableEngines.h>
|
||||
#include <Storages/System/StorageSystemTableFunctions.h>
|
||||
@ -100,6 +101,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database)
|
||||
attach<StorageSystemFunctions>(context, system_database, "functions");
|
||||
attach<StorageSystemEvents>(context, system_database, "events");
|
||||
attach<StorageSystemSettings>(context, system_database, "settings");
|
||||
attach<StorageSystemSettingsChanges>(context, system_database, "settings_changes");
|
||||
attach<SystemMergeTreeSettings<false>>(context, system_database, "merge_tree_settings");
|
||||
attach<SystemMergeTreeSettings<true>>(context, system_database, "replicated_merge_tree_settings");
|
||||
attach<StorageSystemBuildOptions>(context, system_database, "build_options");
|
||||
|
@ -0,0 +1,3 @@
|
||||
version String
|
||||
changes Array(Tuple(name String, previous_value String, new_value String, reason String))
|
||||
22.5 [('memory_overcommit_ratio_denominator','0','1073741824','Enable memory overcommit feature by default'),('memory_overcommit_ratio_denominator_for_user','0','1073741824','Enable memory overcommit feature by default')]
|
@ -0,0 +1,2 @@
|
||||
DESC system.settings_changes;
|
||||
SELECT * FROM system.settings_changes WHERE version = '22.5'
|
Loading…
Reference in New Issue
Block a user