Add compatibility options for MergeTree settings.

This commit is contained in:
Nikolai Kochetov 2024-10-03 13:09:19 +00:00
parent e389858f2c
commit 74de91f864
9 changed files with 158 additions and 9 deletions

View File

@ -1125,9 +1125,6 @@ try
/// We need to reload server settings because config could be updated via zookeeper.
server_settings.loadSettingsFromConfig(config());
/// NOTE: Do sanity checks after we loaded all possible substitutions (for the configuration) from ZK
sanityChecks(*this);
#if defined(OS_LINUX)
std::string executable_path = getExecutablePath();
@ -2019,6 +2016,11 @@ try
if (!filesystem_caches_path.empty())
global_context->setFilesystemCachesPath(filesystem_caches_path);
/// NOTE: Do sanity checks after we loaded all possible substitutions (for the configuration) from ZK
/// Additionally, making the check after the default profile is initialized.
/// It is important to initialize MergeTreeSettings after Settings, to support compatibility for MergeTreeSettings.
sanityChecks(*this);
/// Check sanity of MergeTreeSettings on server startup
{
size_t background_pool_tasks = global_context->getMergeMutateExecutor()->getMaxTasksCount();

View File

@ -3,6 +3,7 @@
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <boost/algorithm/string.hpp>
#include "Core/SettingsEnums.h"
namespace DB
{
@ -541,15 +542,40 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
},
};
const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> & getSettingsChangesHistory()
static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory::SettingsChanges>> merge_tree_settings_changes_history_initializer =
{
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history;
{"24.12",
{
}
},
{"24.11",
{
}
},
{"24.10",
{
}
},
{"24.9",
{
}
},
{"24.8",
{
{"deduplicate_merge_projection_mode", "drop", "throw", "Do not allow to create inconsistent projection"}
}
},
};
static std::once_flag initialized_flag;
std::call_once(initialized_flag, []()
static void initSettingsChangesHistory(
std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> & settings_changes_history,
std::once_flag & initialized_flag,
std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory::SettingsChanges>> & initializer
)
{
std::call_once(initialized_flag, [&]()
{
for (const auto & setting_change : settings_changes_history_initializer)
for (const auto & setting_change : initializer)
{
/// Disallow duplicate keys in the settings changes history. Example:
/// {"21.2", {{"some_setting_1", false, true, "[...]"}}},
@ -562,7 +588,24 @@ const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> & get
settings_changes_history[setting_change.first] = setting_change.second;
}
});
}
const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> & getSettingsChangesHistory()
{
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history;
static std::once_flag initialized_flag;
initSettingsChangesHistory(settings_changes_history, initialized_flag, settings_changes_history_initializer);
return settings_changes_history;
}
const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> & getMergeTreeSettingsChangesHistory()
{
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> merge_tree_settings_changes_history;
static std::once_flag initialized_flag;
initSettingsChangesHistory(merge_tree_settings_changes_history, initialized_flag, merge_tree_settings_changes_history_initializer);
return merge_tree_settings_changes_history;
}
}

View File

@ -39,5 +39,6 @@ namespace SettingsChangesHistory
}
const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> & getSettingsChangesHistory();
const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> & getMergeTreeSettingsChangesHistory();
}

View File

@ -238,6 +238,7 @@ namespace Setting
extern const SettingsBool use_page_cache_for_disks_without_file_cache;
extern const SettingsUInt64 use_structure_from_insertion_table_in_table_functions;
extern const SettingsString workload;
extern const SettingsString compatibility;
}
namespace ErrorCodes
@ -4648,6 +4649,7 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const
{
const auto & config = shared->getConfigRefWithLock(lock);
MergeTreeSettings mt_settings;
mt_settings.applyCompatibilitySetting((*settings)[Setting::compatibility]);
mt_settings.loadFromConfig("merge_tree", config);
shared->merge_tree_settings.emplace(mt_settings);
}
@ -4663,6 +4665,7 @@ const MergeTreeSettings & Context::getReplicatedMergeTreeSettings() const
{
const auto & config = shared->getConfigRefWithLock(lock);
MergeTreeSettings mt_settings;
mt_settings.applyCompatibilitySetting((*settings)[Setting::compatibility]);
mt_settings.loadFromConfig("merge_tree", config);
mt_settings.loadFromConfig("replicated_merge_tree", config);
shared->replicated_merge_tree_settings.emplace(mt_settings);

View File

@ -10,6 +10,7 @@
#include <Common/Exception.h>
#include <Common/logger_useful.h>
#include <Core/Settings.h>
#include <Core/SettingsChangesHistory.h>
namespace DB
@ -119,6 +120,31 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte
#undef ADD_IF_ABSENT
}
void MergeTreeSettings::applyCompatibilitySetting(const String & compatibility_value)
{
/// If setting value is empty, we don't need to change settings
if (compatibility_value.empty())
return;
ClickHouseVersion version(compatibility_value);
const auto & settings_changes_history = getMergeTreeSettingsChangesHistory();
/// 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 (version >= it->first)
break;
/// Apply reversed changes from this version.
for (const auto & change : it->second)
{
/// In case the alias is being used (e.g. use enable_analyzer) we must change the original setting
auto final_name = MergeTreeSettingsTraits::resolveName(change.name);
BaseSettings::set(final_name, change.previous_value);
}
}
}
bool MergeTreeSettings::isReadonlySetting(const String & name)
{
return name == "index_granularity" || name == "index_granularity_bytes" || name == "enable_mixed_granularity_parts";

View File

@ -275,6 +275,8 @@ struct MergeTreeSettings : public BaseSettings<MergeTreeSettingsTraits>, public
/// NOTE: will rewrite the AST to add immutable settings.
void loadFromQuery(ASTStorage & storage_def, ContextPtr context, bool is_attach);
void applyCompatibilitySetting(const String & compatibility_value);
static bool isReadonlySetting(const String & name);
static bool isPartFormatSetting(const String & name);

View File

@ -0,0 +1,7 @@
<clickhouse>
<profiles>
<default>
<compatibility>24.7</compatibility>
</default>
</profiles>
</clickhouse>

View File

@ -0,0 +1,65 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance("node1", with_zookeeper=True)
node_with_compatibility = cluster.add_instance("node2", with_zookeeper=True, user_configs=["configs/compatibility.xml"])
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_check_projections_compatibility(started_cluster):
create_with_invalid_projection = """
CREATE TABLE tp (type Int32, eventcnt UInt64, PROJECTION p (select sum(eventcnt), type group by type))
engine = {} order by type;
"""
create_no_projection = """
CREATE TABLE tp (type Int32, eventcnt UInt64)
engine = {} order by type;
"""
alter_add_projection = """
ALTER TABLE tp ADD PROJECTION p (select sum(eventcnt), type group by type);
"""
# Create with invalid projection is not supported by default
assert "Projection is fully supported" in node.query_and_get_error(create_with_invalid_projection.format("ReplacingMergeTree"))
assert "Projection is fully supported" in node.query_and_get_error(create_with_invalid_projection.format("ReplicatedReplacingMergeTree('/tables/tp', '0')"))
# Adding invalid projection is not supported by default
node.query(create_no_projection.format("ReplacingMergeTree"))
assert "Projection is fully supported" in node.query_and_get_error(alter_add_projection)
node.query("drop table tp;")
node.query(create_no_projection.format("ReplicatedReplacingMergeTree('/tables/tp', '0')"))
assert "Projection is fully supported" in node.query_and_get_error(alter_add_projection)
node.query("drop table tp;")
# Create with invalid projection is supported with compatibility
node_with_compatibility.query(create_with_invalid_projection.format("ReplacingMergeTree"))
node_with_compatibility.query("drop table tp;")
node_with_compatibility.query(create_with_invalid_projection.format("ReplicatedReplacingMergeTree('/tables/tp2', '0')"))
node_with_compatibility.query("drop table tp;")
# Adding invalid projection is supported with compatibility
node_with_compatibility.query(create_no_projection.format("ReplacingMergeTree"))
node_with_compatibility.query(alter_add_projection)
node_with_compatibility.query("drop table tp;")
node_with_compatibility.query(create_no_projection.format("ReplicatedReplacingMergeTree('/tables/tp3', '0')"))
node_with_compatibility.query(alter_add_projection)
node_with_compatibility.query("drop table tp;")