mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-20 14:42:02 +00:00
Merge branch 'master' into hlee-s390x-LZ4
This commit is contained in:
commit
f7c6830433
@ -1,3 +1,19 @@
|
||||
if (_CLICKHOUSE_TOOLCHAIN_FILE_LOADED)
|
||||
# During first run of cmake the toolchain file will be loaded twice,
|
||||
# - /usr/share/cmake-3.23/Modules/CMakeDetermineSystem.cmake
|
||||
# - /bld/CMakeFiles/3.23.2/CMakeSystem.cmake
|
||||
#
|
||||
# But once you already have non-empty cmake cache it will be loaded only
|
||||
# once:
|
||||
# - /bld/CMakeFiles/3.23.2/CMakeSystem.cmake
|
||||
#
|
||||
# This has no harm except for double load of toolchain will add
|
||||
# --gcc-toolchain multiple times that will not allow ccache to reuse the
|
||||
# cache.
|
||||
return()
|
||||
endif()
|
||||
set (_CLICKHOUSE_TOOLCHAIN_FILE_LOADED ON)
|
||||
|
||||
set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY)
|
||||
|
||||
set (CMAKE_SYSTEM_NAME "Linux")
|
||||
|
@ -3329,6 +3329,15 @@ Read more about [memory overcommit](memory-overcommit.md).
|
||||
|
||||
Default value: `1GiB`.
|
||||
|
||||
## compatibility {#compatibility}
|
||||
|
||||
This setting changes other settings according to provided ClickHouse version.
|
||||
If a behaviour in ClickHouse was changed by using a different default value for some setting, this compatibility setting allows you to use default values from previous versions for all the settings that were not set by the user.
|
||||
|
||||
This setting takes ClickHouse version number as a string, like `21.3`, `21.8`. Empty value means that this setting is disabled.
|
||||
|
||||
Disabled by default.
|
||||
|
||||
# Format settings {#format-settings}
|
||||
|
||||
## input_format_skip_unknown_fields {#input_format_skip_unknown_fields}
|
||||
|
@ -45,7 +45,7 @@ void LRUFileCache::initialize()
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
return;
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -841,7 +841,11 @@ void LRUFileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_l
|
||||
/// cache_base_path / key_prefix / key / offset
|
||||
|
||||
if (!files.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache already initialized");
|
||||
throw Exception(
|
||||
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
|
||||
"Cache initialization is partially made. "
|
||||
"This can be a result of a failed first attempt to initialize cache. "
|
||||
"Please, check log for error messages");
|
||||
|
||||
fs::directory_iterator key_prefix_it{cache_base_path};
|
||||
for (; key_prefix_it != fs::directory_iterator(); ++key_prefix_it)
|
||||
|
@ -43,9 +43,16 @@ class BaseSettings : public TTraits::Data
|
||||
{
|
||||
using CustomSettingMap = std::unordered_map<std::string_view, std::pair<std::shared_ptr<const String>, SettingFieldCustom>>;
|
||||
public:
|
||||
BaseSettings() = default;
|
||||
BaseSettings(const BaseSettings &) = default;
|
||||
BaseSettings(BaseSettings &&) noexcept = default;
|
||||
BaseSettings & operator=(const BaseSettings &) = default;
|
||||
BaseSettings & operator=(BaseSettings &&) noexcept = default;
|
||||
virtual ~BaseSettings() = default;
|
||||
|
||||
using Traits = TTraits;
|
||||
|
||||
void set(std::string_view name, const Field & value);
|
||||
virtual void set(std::string_view name, const Field & value);
|
||||
Field get(std::string_view name) const;
|
||||
|
||||
void setString(std::string_view name, const String & value);
|
||||
@ -62,6 +69,8 @@ public:
|
||||
|
||||
/// Resets all the settings to their default values.
|
||||
void resetToDefault();
|
||||
/// Resets specified setting to its default value.
|
||||
void resetToDefault(std::string_view name);
|
||||
|
||||
bool has(std::string_view name) const { return hasBuiltin(name) || hasCustom(name); }
|
||||
static bool hasBuiltin(std::string_view name);
|
||||
@ -315,6 +324,14 @@ void BaseSettings<TTraits>::resetToDefault()
|
||||
custom_settings_map.clear();
|
||||
}
|
||||
|
||||
template <typename TTraits>
|
||||
void BaseSettings<TTraits>::resetToDefault(std::string_view name)
|
||||
{
|
||||
const auto & accessor = Traits::Accessor::instance();
|
||||
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
|
||||
accessor.resetValueToDefault(*this, index);
|
||||
}
|
||||
|
||||
template <typename TTraits>
|
||||
bool BaseSettings<TTraits>::hasBuiltin(std::string_view name)
|
||||
{
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include "Settings.h"
|
||||
|
||||
#include <Core/SettingsChangesHistory.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
@ -145,6 +146,53 @@ std::vector<String> Settings::getAllRegisteredNames() const
|
||||
return all_settings;
|
||||
}
|
||||
|
||||
void Settings::set(std::string_view name, const Field & value)
|
||||
{
|
||||
BaseSettings::set(name, value);
|
||||
|
||||
if (name == "compatibility")
|
||||
applyCompatibilitySetting();
|
||||
/// If we change setting that was changed by compatibility setting before
|
||||
/// we should remove it from settings_changed_by_compatibility_setting,
|
||||
/// otherwise the next time we will change compatibility setting
|
||||
/// this setting will be changed too (and we don't want it).
|
||||
else if (settings_changed_by_compatibility_setting.contains(name))
|
||||
settings_changed_by_compatibility_setting.erase(name);
|
||||
}
|
||||
|
||||
void Settings::applyCompatibilitySetting()
|
||||
{
|
||||
/// First, revert all changes applied by previous compatibility setting
|
||||
for (const auto & setting_name : settings_changed_by_compatibility_setting)
|
||||
resetToDefault(setting_name);
|
||||
|
||||
settings_changed_by_compatibility_setting.clear();
|
||||
String compatibility = getString("compatibility");
|
||||
/// If setting value is empty, we don't need to change settings
|
||||
if (compatibility.empty())
|
||||
return;
|
||||
|
||||
ClickHouseVersion version(compatibility);
|
||||
/// 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)
|
||||
{
|
||||
/// 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(FormatFactorySettingsTraits, FORMAT_FACTORY_SETTINGS)
|
||||
|
||||
}
|
||||
|
@ -35,6 +35,10 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
*
|
||||
* `flags` can be either 0 or IMPORTANT.
|
||||
* A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions.
|
||||
*
|
||||
* When adding new settings that control some backward incompatible changes or when changing some settings values,
|
||||
* consider adding them to settings changes history in SettingsChangesHistory.h for special `compatibility` setting
|
||||
* to work correctly.
|
||||
*/
|
||||
|
||||
#define COMMON_SETTINGS(M) \
|
||||
@ -601,6 +605,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
M(Bool, allow_deprecated_database_ordinary, false, "Allow to create databases with deprecated Ordinary engine", 0) \
|
||||
M(Bool, allow_deprecated_syntax_for_merge_tree, false, "Allow to create *MergeTree tables with deprecated engine definition syntax", 0) \
|
||||
\
|
||||
M(String, compatibility, "", "Changes other settings according to provided ClickHouse version. If we know that we changed some behaviour in ClickHouse by changing some settings in some version, this compatibility setting will control these settings", 0) \
|
||||
\
|
||||
/** Experimental functions */ \
|
||||
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \
|
||||
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \
|
||||
@ -827,6 +833,13 @@ struct Settings : public BaseSettings<SettingsTraits>, public IHints<2, Settings
|
||||
void addProgramOption(boost::program_options::options_description & options, const SettingFieldRef & field);
|
||||
|
||||
void addProgramOptionAsMultitoken(boost::program_options::options_description & options, const SettingFieldRef & field);
|
||||
|
||||
void set(std::string_view name, const Field & value) override;
|
||||
|
||||
private:
|
||||
void applyCompatibilitySetting();
|
||||
|
||||
std::unordered_set<std::string_view> settings_changed_by_compatibility_setting;
|
||||
};
|
||||
|
||||
/*
|
||||
|
113
src/Core/SettingsChangesHistory.h
Normal file
113
src/Core/SettingsChangesHistory.h
Normal file
@ -0,0 +1,113 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
class ClickHouseVersion
|
||||
{
|
||||
public:
|
||||
ClickHouseVersion(const String & version)
|
||||
{
|
||||
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;
|
||||
if (!tryParse(component, split_element))
|
||||
throw Exception{ErrorCodes::BAD_ARGUMENTS, "Cannot parse ClickHouse version here: {}", version};
|
||||
components.push_back(component);
|
||||
}
|
||||
}
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
namespace SettingsChangesHistory
|
||||
{
|
||||
struct SettingChange
|
||||
{
|
||||
String name;
|
||||
Field previous_value;
|
||||
Field new_value;
|
||||
String reason;
|
||||
};
|
||||
|
||||
using SettingsChanges = std::vector<SettingChange>;
|
||||
}
|
||||
|
||||
/// History of settings changes that controls some backward incompatible changes
|
||||
/// 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)
|
||||
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
|
||||
{
|
||||
{"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"},
|
||||
{"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 omitted 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"}}},
|
||||
};
|
||||
|
||||
}
|
@ -5,6 +5,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeDecimalBase.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -13,6 +14,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int DECIMAL_OVERFLOW;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/// Implements Decimal(P, S), where P is precision, S is scale.
|
||||
@ -58,7 +60,7 @@ inline const DataTypeDecimal<T> * checkDecimal(const IDataType & data_type)
|
||||
return typeid_cast<const DataTypeDecimal<T> *>(&data_type);
|
||||
}
|
||||
|
||||
inline UInt32 getDecimalScale(const IDataType & data_type, UInt32 default_value = std::numeric_limits<UInt32>::max())
|
||||
inline UInt32 getDecimalScale(const IDataType & data_type)
|
||||
{
|
||||
if (const auto * decimal_type = checkDecimal<Decimal32>(data_type))
|
||||
return decimal_type->getScale();
|
||||
@ -68,7 +70,10 @@ inline UInt32 getDecimalScale(const IDataType & data_type, UInt32 default_value
|
||||
return decimal_type->getScale();
|
||||
if (const auto * decimal_type = checkDecimal<Decimal256>(data_type))
|
||||
return decimal_type->getScale();
|
||||
return default_value;
|
||||
if (const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(&data_type))
|
||||
return date_time_type->getScale();
|
||||
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal scale from type {}", data_type.getName());
|
||||
}
|
||||
|
||||
inline UInt32 getDecimalPrecision(const IDataType & data_type)
|
||||
@ -81,7 +86,10 @@ inline UInt32 getDecimalPrecision(const IDataType & data_type)
|
||||
return decimal_type->getPrecision();
|
||||
if (const auto * decimal_type = checkDecimal<Decimal256>(data_type))
|
||||
return decimal_type->getPrecision();
|
||||
return 0;
|
||||
if (const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(&data_type))
|
||||
return date_time_type->getPrecision();
|
||||
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal precision from type {}", data_type.getName());
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -554,7 +554,11 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
|
||||
UInt32 max_scale = 0;
|
||||
for (const auto & type : types)
|
||||
{
|
||||
UInt32 scale = getDecimalScale(*type, 0);
|
||||
auto type_id = type->getTypeId();
|
||||
if (type_id != TypeIndex::Decimal32 && type_id != TypeIndex::Decimal64 && type_id != TypeIndex::Decimal128)
|
||||
continue;
|
||||
|
||||
UInt32 scale = getDecimalScale(*type);
|
||||
if (scale > max_scale)
|
||||
max_scale = scale;
|
||||
}
|
||||
|
@ -23,20 +23,6 @@ ActionLocksManager::ActionLocksManager(ContextPtr context_) : WithContext(contex
|
||||
{
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
inline void forEachTable(F && f, ContextPtr context)
|
||||
{
|
||||
for (auto & elem : DatabaseCatalog::instance().getDatabases())
|
||||
for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
if (auto table = iterator->table())
|
||||
f(table);
|
||||
}
|
||||
|
||||
void ActionLocksManager::add(StorageActionBlockType action_type, ContextPtr context_)
|
||||
{
|
||||
forEachTable([&](const StoragePtr & table) { add(table, action_type); }, context_);
|
||||
}
|
||||
|
||||
void ActionLocksManager::add(const StorageID & table_id, StorageActionBlockType action_type)
|
||||
{
|
||||
if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()))
|
||||
@ -54,14 +40,6 @@ void ActionLocksManager::add(const StoragePtr & table, StorageActionBlockType ac
|
||||
}
|
||||
}
|
||||
|
||||
void ActionLocksManager::remove(StorageActionBlockType action_type)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
for (auto & storage_elem : storage_locks)
|
||||
storage_elem.second.erase(action_type);
|
||||
}
|
||||
|
||||
void ActionLocksManager::remove(const StorageID & table_id, StorageActionBlockType action_type)
|
||||
{
|
||||
if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()))
|
||||
|
@ -20,14 +20,10 @@ class ActionLocksManager : WithContext
|
||||
public:
|
||||
explicit ActionLocksManager(ContextPtr context);
|
||||
|
||||
/// Adds new locks for each table
|
||||
void add(StorageActionBlockType action_type, ContextPtr context);
|
||||
/// Add new lock for a table if it has not been already added
|
||||
void add(const StorageID & table_id, StorageActionBlockType action_type);
|
||||
void add(const StoragePtr & table, StorageActionBlockType action_type);
|
||||
|
||||
/// Remove locks for all tables
|
||||
void remove(StorageActionBlockType action_type);
|
||||
/// Removes a lock for a table if it exists
|
||||
void remove(const StorageID & table_id, StorageActionBlockType action_type);
|
||||
void remove(const StoragePtr & table, StorageActionBlockType action_type);
|
||||
|
@ -146,14 +146,14 @@ namespace
|
||||
struct QueryASTSettings
|
||||
{
|
||||
bool graph = false;
|
||||
bool rewrite = false;
|
||||
bool optimize = false;
|
||||
|
||||
constexpr static char name[] = "AST";
|
||||
|
||||
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
|
||||
{
|
||||
{"graph", graph},
|
||||
{"rewrite", rewrite}
|
||||
{"optimize", optimize}
|
||||
};
|
||||
};
|
||||
|
||||
@ -280,7 +280,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
|
||||
case ASTExplainQuery::ParsedAST:
|
||||
{
|
||||
auto settings = checkAndGetSettings<QueryASTSettings>(ast.getSettings());
|
||||
if (settings.rewrite)
|
||||
if (settings.optimize)
|
||||
{
|
||||
ExplainAnalyzedSyntaxVisitor::Data data(getContext());
|
||||
ExplainAnalyzedSyntaxVisitor(data).visit(query);
|
||||
|
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");
|
||||
|
@ -81,18 +81,23 @@ EOF""",
|
||||
["bash", "-c", "rm /etc/clickhouse-server/users.d/user_c.xml"]
|
||||
)
|
||||
|
||||
expected_error = "no user with such name"
|
||||
expected_errors = ["no user with such name", "not found in user directories"]
|
||||
while True:
|
||||
out, err = instance.query_and_get_answer_with_error("SELECT 1", user="C")
|
||||
if expected_error in err:
|
||||
logging.debug(f"Got error '{expected_error}' just as expected")
|
||||
found_error = [
|
||||
expected_error
|
||||
for expected_error in expected_errors
|
||||
if (expected_error in err)
|
||||
]
|
||||
if found_error:
|
||||
logging.debug(f"Got error '{found_error}' just as expected")
|
||||
break
|
||||
if out == "1\n":
|
||||
logging.debug(f"Got output '1', retrying...")
|
||||
time.sleep(0.5)
|
||||
continue
|
||||
raise Exception(
|
||||
f"Expected either output '1' or error '{expected_error}', got output={out} and error={err}"
|
||||
f"Expected either output '1' or one of errors '{expected_errors}', got output={out} and error={err}"
|
||||
)
|
||||
|
||||
assert instance.query("SELECT name FROM system.users WHERE name='C'") == ""
|
||||
|
@ -0,0 +1,19 @@
|
||||
allow_settings_after_format_in_insert
|
||||
22.3
|
||||
1
|
||||
1
|
||||
22.4
|
||||
0
|
||||
22.5
|
||||
0
|
||||
async_socket_for_remote
|
||||
21.2
|
||||
1
|
||||
21.3
|
||||
0
|
||||
21.4
|
||||
0
|
||||
21.5
|
||||
1
|
||||
21.6
|
||||
1
|
28
tests/queries/0_stateless/02324_compatibility_setting.sh
Executable file
28
tests/queries/0_stateless/02324_compatibility_setting.sh
Executable file
@ -0,0 +1,28 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
echo "allow_settings_after_format_in_insert"
|
||||
echo "22.3"
|
||||
$CLICKHOUSE_CLIENT --compatibility=22.3 -q "select value from system.settings where name='allow_settings_after_format_in_insert'"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&compatibility=22.3" -d "select value from system.settings where name='allow_settings_after_format_in_insert'"
|
||||
echo "22.4"
|
||||
$CLICKHOUSE_CLIENT --compatibility=22.4 -q "select value from system.settings where name='allow_settings_after_format_in_insert'"
|
||||
echo "22.5"
|
||||
$CLICKHOUSE_CLIENT --compatibility=22.5 -q "select value from system.settings where name='allow_settings_after_format_in_insert'"
|
||||
|
||||
|
||||
echo "async_socket_for_remote"
|
||||
echo "21.2"
|
||||
$CLICKHOUSE_CLIENT --compatibility=21.2 -q "select value from system.settings where name='async_socket_for_remote'"
|
||||
echo "21.3"
|
||||
$CLICKHOUSE_CLIENT --compatibility=21.3 -q "select value from system.settings where name='async_socket_for_remote'"
|
||||
echo "21.4"
|
||||
$CLICKHOUSE_CLIENT --compatibility=21.4 -q "select value from system.settings where name='async_socket_for_remote'"
|
||||
echo "21.5"
|
||||
$CLICKHOUSE_CLIENT --compatibility=21.5 -q "select value from system.settings where name='async_socket_for_remote'"
|
||||
echo "21.6"
|
||||
$CLICKHOUSE_CLIENT --compatibility=21.6 -q "select value from system.settings where name='async_socket_for_remote'"
|
||||
|
@ -0,0 +1,8 @@
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
13
tests/queries/0_stateless/02325_compatibility_setting_2.sql
Normal file
13
tests/queries/0_stateless/02325_compatibility_setting_2.sql
Normal file
@ -0,0 +1,13 @@
|
||||
select value from system.settings where name='allow_settings_after_format_in_insert';
|
||||
select value from system.settings where name='allow_settings_after_format_in_insert' settings compatibility='22.3';
|
||||
select value from system.settings where name='allow_settings_after_format_in_insert';
|
||||
set compatibility = '22.3';
|
||||
select value from system.settings where name='allow_settings_after_format_in_insert';
|
||||
set compatibility = '22.4';
|
||||
select value from system.settings where name='allow_settings_after_format_in_insert';
|
||||
set allow_settings_after_format_in_insert=1;
|
||||
select value from system.settings where name='allow_settings_after_format_in_insert';
|
||||
set compatibility = '22.4';
|
||||
select value from system.settings where name='allow_settings_after_format_in_insert';
|
||||
set compatibility = '22.3';
|
||||
select value from system.settings where name='allow_settings_after_format_in_insert';
|
@ -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'
|
@ -0,0 +1,53 @@
|
||||
-- { echoOn }
|
||||
EXPLAIN AST optimize=0 SELECT * FROM numbers(0);
|
||||
SelectWithUnionQuery (children 1)
|
||||
ExpressionList (children 1)
|
||||
SelectQuery (children 2)
|
||||
ExpressionList (children 1)
|
||||
Asterisk
|
||||
TablesInSelectQuery (children 1)
|
||||
TablesInSelectQueryElement (children 1)
|
||||
TableExpression (children 1)
|
||||
Function numbers (children 1)
|
||||
ExpressionList (children 1)
|
||||
Literal UInt64_0
|
||||
EXPLAIN AST optimize=1 SELECT * FROM numbers(0);
|
||||
SelectWithUnionQuery (children 1)
|
||||
ExpressionList (children 1)
|
||||
SelectQuery (children 2)
|
||||
ExpressionList (children 1)
|
||||
Identifier number
|
||||
TablesInSelectQuery (children 1)
|
||||
TablesInSelectQueryElement (children 1)
|
||||
TableExpression (children 1)
|
||||
Function numbers (children 1)
|
||||
ExpressionList (children 1)
|
||||
Literal UInt64_0
|
||||
EXPLAIN AST optimize=0 SELECT countDistinct(number) FROM numbers(0);
|
||||
SelectWithUnionQuery (children 1)
|
||||
ExpressionList (children 1)
|
||||
SelectQuery (children 2)
|
||||
ExpressionList (children 1)
|
||||
Function countDistinct (children 1)
|
||||
ExpressionList (children 1)
|
||||
Identifier number
|
||||
TablesInSelectQuery (children 1)
|
||||
TablesInSelectQueryElement (children 1)
|
||||
TableExpression (children 1)
|
||||
Function numbers (children 1)
|
||||
ExpressionList (children 1)
|
||||
Literal UInt64_0
|
||||
EXPLAIN AST optimize=1 SELECT countDistinct(number) FROM numbers(0);
|
||||
SelectWithUnionQuery (children 1)
|
||||
ExpressionList (children 1)
|
||||
SelectQuery (children 2)
|
||||
ExpressionList (children 1)
|
||||
Function uniqExact (children 1)
|
||||
ExpressionList (children 1)
|
||||
Identifier number
|
||||
TablesInSelectQuery (children 1)
|
||||
TablesInSelectQueryElement (children 1)
|
||||
TableExpression (children 1)
|
||||
Function numbers (children 1)
|
||||
ExpressionList (children 1)
|
||||
Literal UInt64_0
|
6
tests/queries/0_stateless/02353_explain_ast_optimize.sql
Normal file
6
tests/queries/0_stateless/02353_explain_ast_optimize.sql
Normal file
@ -0,0 +1,6 @@
|
||||
-- { echoOn }
|
||||
EXPLAIN AST optimize=0 SELECT * FROM numbers(0);
|
||||
EXPLAIN AST optimize=1 SELECT * FROM numbers(0);
|
||||
EXPLAIN AST optimize=0 SELECT countDistinct(number) FROM numbers(0);
|
||||
EXPLAIN AST optimize=1 SELECT countDistinct(number) FROM numbers(0);
|
||||
-- { echoOff }
|
@ -1,25 +0,0 @@
|
||||
-- { echoOn }
|
||||
EXPLAIN AST rewrite=0 SELECT * FROM numbers(0);
|
||||
SelectWithUnionQuery (children 1)
|
||||
ExpressionList (children 1)
|
||||
SelectQuery (children 2)
|
||||
ExpressionList (children 1)
|
||||
Asterisk
|
||||
TablesInSelectQuery (children 1)
|
||||
TablesInSelectQueryElement (children 1)
|
||||
TableExpression (children 1)
|
||||
Function numbers (children 1)
|
||||
ExpressionList (children 1)
|
||||
Literal UInt64_0
|
||||
EXPLAIN AST rewrite=1 SELECT * FROM numbers(0);
|
||||
SelectWithUnionQuery (children 1)
|
||||
ExpressionList (children 1)
|
||||
SelectQuery (children 2)
|
||||
ExpressionList (children 1)
|
||||
Identifier number
|
||||
TablesInSelectQuery (children 1)
|
||||
TablesInSelectQueryElement (children 1)
|
||||
TableExpression (children 1)
|
||||
Function numbers (children 1)
|
||||
ExpressionList (children 1)
|
||||
Literal UInt64_0
|
@ -1,4 +0,0 @@
|
||||
-- { echoOn }
|
||||
EXPLAIN AST rewrite=0 SELECT * FROM numbers(0);
|
||||
EXPLAIN AST rewrite=1 SELECT * FROM numbers(0);
|
||||
-- { echoOff }
|
@ -0,0 +1,2 @@
|
||||
2022-01-24 02:30:00.008122000
|
||||
1
|
@ -0,0 +1,15 @@
|
||||
create table dat (blockNum Decimal(10,0), eventTimestamp DateTime64(9)) Engine=MergeTree() primary key eventTimestamp;
|
||||
insert into dat values (1, '2022-01-24 02:30:00.008122000');
|
||||
|
||||
CREATE DICTIONARY datDictionary
|
||||
(
|
||||
`blockNum` Decimal(10, 0),
|
||||
`eventTimestamp` DateTime64(9)
|
||||
)
|
||||
PRIMARY KEY blockNum
|
||||
SOURCE(CLICKHOUSE(TABLE 'dat'))
|
||||
LIFETIME(MIN 0 MAX 1000)
|
||||
LAYOUT(FLAT());
|
||||
|
||||
select (select eventTimestamp from datDictionary);
|
||||
select count(*) from dat where eventTimestamp >= (select eventTimestamp from datDictionary);
|
Loading…
Reference in New Issue
Block a user