mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-14 19:45:11 +00:00
Add compatibility options for MergeTree settings.
This commit is contained in:
parent
e389858f2c
commit
74de91f864
@ -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();
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -39,5 +39,6 @@ namespace SettingsChangesHistory
|
||||
}
|
||||
|
||||
const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> & getSettingsChangesHistory();
|
||||
const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> & getMergeTreeSettingsChangesHistory();
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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";
|
||||
|
@ -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);
|
||||
|
||||
|
@ -0,0 +1,7 @@
|
||||
<clickhouse>
|
||||
<profiles>
|
||||
<default>
|
||||
<compatibility>24.7</compatibility>
|
||||
</default>
|
||||
</profiles>
|
||||
</clickhouse>
|
@ -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;")
|
Loading…
Reference in New Issue
Block a user