提交 8277e9d8 编写于 作者: V Vitaly Baranov

Refactoring of Settings based on new template class SettingsCollection.

Add new class SettingsConstraints.
上级 b24c6974
......@@ -816,8 +816,7 @@ private:
{
if (!old_settings)
old_settings.emplace(context.getSettingsRef());
for (const auto & change : settings_ast.as<ASTSetQuery>()->changes)
context.setSetting(change.name, change.value);
context.applySettingsChanges(settings_ast.as<ASTSetQuery>()->changes);
};
const auto * insert = parsed_query->as<ASTInsertQuery>();
if (insert && insert->settings_ast)
......@@ -847,7 +846,7 @@ private:
if (change.name == "profile")
current_profile = change.value.safeGet<String>();
else
context.setSetting(change.name, change.value);
context.applySettingChange(change);
}
}
......
......@@ -497,8 +497,7 @@ void HTTPHandler::processQuery(
settings.readonly = 2;
}
auto readonly_before_query = settings.readonly;
SettingsChanges settings_changes;
for (auto it = params.begin(); it != params.end(); ++it)
{
if (it->first == "database")
......@@ -515,21 +514,13 @@ void HTTPHandler::processQuery(
else
{
/// All other query parameters are treated as settings.
String value;
/// Setting is skipped if value wasn't changed.
if (!settings.tryGet(it->first, value) || it->second != value)
{
if (readonly_before_query == 1)
throw Exception("Cannot override setting (" + it->first + ") in readonly mode", ErrorCodes::READONLY);
if (readonly_before_query && it->first == "readonly")
throw Exception("Setting 'readonly' cannot be overrided in readonly mode", ErrorCodes::READONLY);
context.setSetting(it->first, it->second);
}
settings_changes.push_back({it->first, it->second});
}
}
context.checkSettingsConstraints(settings_changes);
context.applySettingsChanges(settings_changes);
/// HTTP response compression is turned on only if the client signalled that they support it
/// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on.
used_output.out->setCompression(client_supports_http_compression && settings.enable_http_compression);
......
......@@ -401,7 +401,7 @@ void Connection::sendQuery(
if (settings)
settings->serialize(*out);
else
writeStringBinary("", *out);
writeStringBinary("" /* empty string is a marker of the end of settings */, *out);
writeVarUInt(stage, *out);
writeVarUInt(static_cast<bool>(compression), *out);
......
#pragma once
#include <Core/Field.h>
namespace DB
{
struct SettingChange
{
String name;
Field value;
};
using SettingsChanges = std::vector<SettingChange>;
}
#include "Settings.h"
#include <Poco/Util/AbstractConfiguration.h>
#include <Core/Field.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnArray.h>
#include <Common/typeid_cast.h>
#include <string.h>
......@@ -14,94 +11,13 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
extern const int THERE_IS_NO_PROFILE;
extern const int NO_ELEMENTS_IN_CONFIG;
}
/// Set the configuration by name.
void Settings::set(const String & name, const Field & value)
{
#define TRY_SET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) NAME.set(value);
if (false) {}
APPLY_FOR_SETTINGS(TRY_SET)
else
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef TRY_SET
}
/// Set the configuration by name. Read the binary serialized value from the buffer (for interserver interaction).
void Settings::set(const String & name, ReadBuffer & buf)
{
#define TRY_SET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) NAME.set(buf);
IMPLEMENT_SETTINGS_COLLECTION(Settings, APPLY_FOR_SETTINGS)
if (false) {}
APPLY_FOR_SETTINGS(TRY_SET)
else
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef TRY_SET
}
/// Skip the binary-serialized value from the buffer.
void Settings::ignore(const String & name, ReadBuffer & buf)
{
#define TRY_IGNORE(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) decltype(NAME)(DEFAULT).set(buf);
if (false) {}
APPLY_FOR_SETTINGS(TRY_IGNORE)
else
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef TRY_IGNORE
}
/** Set the setting by name. Read the value in text form from a string (for example, from a config, or from a URL parameter).
*/
void Settings::set(const String & name, const String & value)
{
#define TRY_SET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) NAME.set(value);
if (false) {}
APPLY_FOR_SETTINGS(TRY_SET)
else
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef TRY_SET
}
String Settings::get(const String & name) const
{
#define GET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) return NAME.toString();
if (false) {}
APPLY_FOR_SETTINGS(GET)
else
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef GET
}
bool Settings::tryGet(const String & name, String & value) const
{
#define TRY_GET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) { value = NAME.toString(); return true; }
if (false) {}
APPLY_FOR_SETTINGS(TRY_GET)
else
return false;
#undef TRY_GET
}
/** 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.
......@@ -139,47 +55,6 @@ void Settings::loadSettingsFromConfig(const String & path, const Poco::Util::Abs
}
}
/// Read the settings from the buffer. They are written as a set of name-value pairs that go successively, ending with an empty `name`.
/// If the `check_readonly` flag is set, `readonly` is set in the preferences, but some changes have occurred - throw an exception.
void Settings::deserialize(ReadBuffer & buf)
{
auto before_readonly = readonly;
while (true)
{
String name;
readBinary(name, buf);
/// An empty string is the marker for the end of the settings.
if (name.empty())
break;
/// If readonly = 2, then you can change the settings, except for the readonly setting.
if (before_readonly == 0 || (before_readonly == 2 && name != "readonly"))
set(name, buf);
else
ignore(name, buf);
}
}
/// Record the changed settings to the buffer. (For example, to send to a remote server.)
void Settings::serialize(WriteBuffer & buf) const
{
#define WRITE(TYPE, NAME, DEFAULT, DESCRIPTION) \
if (NAME.changed) \
{ \
writeStringBinary(#NAME, buf); \
NAME.write(buf); \
}
APPLY_FOR_SETTINGS(WRITE)
/// An empty string is a marker for the end of the settings.
writeStringBinary("", buf);
#undef WRITE
}
void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_values_, bool changed_only)
{
/// Convert ptr and make simple check
......
......@@ -16,11 +16,11 @@ namespace DB
{
class IColumn;
class Field;
/** Settings of query execution.
*/
struct Settings
struct Settings : public SettingsCollection<Settings>
{
/// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in C++14.
/// http://en.cppreference.com/w/cpp/language/aggregate_initialization
......@@ -315,48 +315,18 @@ struct Settings
\
M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.") \
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};
APPLY_FOR_SETTINGS(DECLARE)
#undef DECLARE
/// Set setting by name.
void set(const String & name, const Field & value);
/// Set setting by name. Read value, serialized in binary form from buffer (for inter-server communication).
void set(const String & name, ReadBuffer & buf);
/// Skip value, serialized in binary form in buffer.
void ignore(const String & name, ReadBuffer & buf);
/// Set setting by name. Read value in text form from string (for example, from configuration file or from URL parameter).
void set(const String & name, const String & value);
/// Get setting by name. Converts value to String.
String get(const String & name) const;
bool tryGet(const String & name, String & value) const;
DECLARE_SETTINGS_COLLECTION(APPLY_FOR_SETTINGS)
/** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings).
* The profile can also be set using the `set` functions, like the profile setting.
*/
* The profile can also be set using the `set` functions, like the profile setting.
*/
void setProfile(const String & profile_name, const Poco::Util::AbstractConfiguration & config);
/// Load settings from configuration file, at "path" prefix in configuration.
void loadSettingsFromConfig(const String & path, const Poco::Util::AbstractConfiguration & config);
/// Read settings from buffer. They are serialized as list of contiguous name-value pairs, finished with empty name.
/// If readonly=1 is set, ignore read settings.
void deserialize(ReadBuffer & buf);
/// Write changed settings to buffer. (For example, to be sent to remote server.)
void serialize(WriteBuffer & buf) const;
/// Dumps profile events to two columns of type Array(String)
void dumpToArrayColumns(IColumn * column_names, IColumn * column_values, bool changed_only = true);
};
}
......@@ -25,6 +25,7 @@ namespace ErrorCodes
extern const int UNKNOWN_LOG_LEVEL;
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_SETTING;
}
template <typename IntType>
......@@ -33,6 +34,12 @@ String SettingInt<IntType>::toString() const
return DB::toString(value);
}
template <typename IntType>
Field SettingInt<IntType>::toField() const
{
return value;
}
template <typename IntType>
void SettingInt<IntType>::set(IntType x)
{
......@@ -43,7 +50,10 @@ void SettingInt<IntType>::set(IntType x)
template <typename IntType>
void SettingInt<IntType>::set(const Field & x)
{
set(applyVisitor(FieldVisitorConvertToNumber<IntType>(), x));
if (x.getType() == Field::Types::String)
set(get<const String &>(x));
else
set(applyVisitor(FieldVisitorConvertToNumber<IntType>(), x));
}
template <typename IntType>
......@@ -53,17 +63,17 @@ void SettingInt<IntType>::set(const String & x)
}
template <typename IntType>
void SettingInt<IntType>::set(ReadBuffer & buf)
void SettingInt<IntType>::serialize(WriteBuffer & buf) const
{
IntType x = 0;
readVarT(x, buf);
set(x);
writeVarT(value, buf);
}
template <typename IntType>
void SettingInt<IntType>::write(WriteBuffer & buf) const
void SettingInt<IntType>::deserialize(ReadBuffer & buf)
{
writeVarT(value, buf);
IntType x = 0;
readVarT(x, buf);
set(x);
}
template struct SettingInt<UInt64>;
......@@ -76,6 +86,11 @@ String SettingMaxThreads::toString() const
return DB::toString(value);
}
Field SettingMaxThreads::toField() const
{
return is_auto ? 0 : value;
}
void SettingMaxThreads::set(UInt64 x)
{
value = x ? x : getAutoValue();
......@@ -86,7 +101,7 @@ void SettingMaxThreads::set(UInt64 x)
void SettingMaxThreads::set(const Field & x)
{
if (x.getType() == Field::Types::String)
set(safeGet<const String &>(x));
set(get<const String &>(x));
else
set(safeGet<UInt64>(x));
}
......@@ -99,16 +114,16 @@ void SettingMaxThreads::set(const String & x)
set(parse<UInt64>(x));
}
void SettingMaxThreads::set(ReadBuffer & buf)
void SettingMaxThreads::serialize(WriteBuffer & buf) const
{
UInt64 x = 0;
readVarUInt(x, buf);
set(x);
writeVarUInt(is_auto ? 0 : value, buf);
}
void SettingMaxThreads::write(WriteBuffer & buf) const
void SettingMaxThreads::deserialize(ReadBuffer & buf)
{
writeVarUInt(is_auto ? 0 : value, buf);
UInt64 x = 0;
readVarUInt(x, buf);
set(x);
}
void SettingMaxThreads::setAuto()
......@@ -135,6 +150,11 @@ String SettingSeconds::toString() const
return DB::toString(totalSeconds());
}
Field SettingSeconds::toField() const
{
return totalSeconds();
}
void SettingSeconds::set(const Poco::Timespan & x)
{
value = x;
......@@ -148,7 +168,10 @@ void SettingSeconds::set(UInt64 x)
void SettingSeconds::set(const Field & x)
{
set(safeGet<UInt64>(x));
if (x.getType() == Field::Types::String)
set(get<const String &>(x));
else
set(safeGet<UInt64>(x));
}
void SettingSeconds::set(const String & x)
......@@ -156,16 +179,16 @@ void SettingSeconds::set(const String & x)
set(parse<UInt64>(x));
}
void SettingSeconds::set(ReadBuffer & buf)
void SettingSeconds::serialize(WriteBuffer & buf) const
{
UInt64 x = 0;
readVarUInt(x, buf);
set(x);
writeVarUInt(value.totalSeconds(), buf);
}
void SettingSeconds::write(WriteBuffer & buf) const
void SettingSeconds::deserialize(ReadBuffer & buf)
{
writeVarUInt(value.totalSeconds(), buf);
UInt64 x = 0;
readVarUInt(x, buf);
set(x);
}
......@@ -174,6 +197,11 @@ String SettingMilliseconds::toString() const
return DB::toString(totalMilliseconds());
}
Field SettingMilliseconds::toField() const
{
return totalMilliseconds();
}
void SettingMilliseconds::set(const Poco::Timespan & x)
{
value = x;
......@@ -187,7 +215,10 @@ void SettingMilliseconds::set(UInt64 x)
void SettingMilliseconds::set(const Field & x)
{
set(safeGet<UInt64>(x));
if (x.getType() == Field::Types::String)
set(get<const String &>(x));
else
set(safeGet<UInt64>(x));
}
void SettingMilliseconds::set(const String & x)
......@@ -195,16 +226,16 @@ void SettingMilliseconds::set(const String & x)
set(parse<UInt64>(x));
}
void SettingMilliseconds::set(ReadBuffer & buf)
void SettingMilliseconds::serialize(WriteBuffer & buf) const
{
UInt64 x = 0;
readVarUInt(x, buf);
set(x);
writeVarUInt(value.totalMilliseconds(), buf);
}
void SettingMilliseconds::write(WriteBuffer & buf) const
void SettingMilliseconds::deserialize(ReadBuffer & buf)
{
writeVarUInt(value.totalMilliseconds(), buf);
UInt64 x = 0;
readVarUInt(x, buf);
set(x);
}
......@@ -213,6 +244,11 @@ String SettingFloat::toString() const
return DB::toString(value);
}
Field SettingFloat::toField() const
{
return value;
}
void SettingFloat::set(float x)
{
value = x;
......@@ -221,7 +257,10 @@ void SettingFloat::set(float x)
void SettingFloat::set(const Field & x)
{
set(applyVisitor(FieldVisitorConvertToNumber<float>(), x));
if (x.getType() == Field::Types::String)
set(get<const String &>(x));
else
set(applyVisitor(FieldVisitorConvertToNumber<float>(), x));
}
void SettingFloat::set(const String & x)
......@@ -229,16 +268,16 @@ void SettingFloat::set(const String & x)
set(parse<float>(x));
}
void SettingFloat::set(ReadBuffer & buf)
void SettingFloat::serialize(WriteBuffer & buf) const
{
String x;
readBinary(x, buf);
set(x);
writeBinary(toString(), buf);
}
void SettingFloat::write(WriteBuffer & buf) const
void SettingFloat::deserialize(ReadBuffer & buf)
{
writeBinary(toString(), buf);
String x;
readBinary(x, buf);
set(x);
}
......@@ -261,6 +300,11 @@ String SettingLoadBalancing::toString() const
return strings[static_cast<size_t>(value)];
}
Field SettingLoadBalancing::toField() const
{
return toString();
}
void SettingLoadBalancing::set(LoadBalancing x)
{
value = x;
......@@ -277,16 +321,16 @@ void SettingLoadBalancing::set(const String & x)
set(getLoadBalancing(x));
}
void SettingLoadBalancing::set(ReadBuffer & buf)
void SettingLoadBalancing::serialize(WriteBuffer & buf) const
{
String x;
readBinary(x, buf);
set(x);
writeBinary(toString(), buf);
}
void SettingLoadBalancing::write(WriteBuffer & buf) const
void SettingLoadBalancing::deserialize(ReadBuffer & buf)
{
writeBinary(toString(), buf);
String s;
readBinary(s, buf);
set(s);
}
......@@ -308,6 +352,11 @@ String SettingJoinStrictness::toString() const
return strings[static_cast<size_t>(value)];
}
Field SettingJoinStrictness::toField() const
{
return toString();
}
void SettingJoinStrictness::set(JoinStrictness x)
{
value = x;
......@@ -324,16 +373,16 @@ void SettingJoinStrictness::set(const String & x)
set(getJoinStrictness(x));
}
void SettingJoinStrictness::set(ReadBuffer & buf)
void SettingJoinStrictness::serialize(WriteBuffer & buf) const
{
String x;
readBinary(x, buf);
set(x);
writeBinary(toString(), buf);
}
void SettingJoinStrictness::write(WriteBuffer & buf) const
void SettingJoinStrictness::deserialize(ReadBuffer & buf)
{
writeBinary(toString(), buf);
String s;
readBinary(s, buf);
set(s);
}
......@@ -360,6 +409,11 @@ String SettingTotalsMode::toString() const
__builtin_unreachable();
}
Field SettingTotalsMode::toField() const
{
return toString();
}
void SettingTotalsMode::set(TotalsMode x)
{
value = x;
......@@ -376,19 +430,20 @@ void SettingTotalsMode::set(const String & x)
set(getTotalsMode(x));
}
void SettingTotalsMode::set(ReadBuffer & buf)
void SettingTotalsMode::serialize(WriteBuffer & buf) const
{
String x;
readBinary(x, buf);
set(x);
writeBinary(toString(), buf);
}
void SettingTotalsMode::write(WriteBuffer & buf) const
void SettingTotalsMode::deserialize(ReadBuffer & buf)
{
writeBinary(toString(), buf);
String s;
readBinary(s, buf);
set(s);
}
template <bool enable_mode_any>
OverflowMode SettingOverflowMode<enable_mode_any>::getOverflowModeForGroupBy(const String & s)
{
......@@ -421,6 +476,12 @@ String SettingOverflowMode<enable_mode_any>::toString() const
return strings[static_cast<size_t>(value)];
}
template <bool enable_mode_any>
Field SettingOverflowMode<enable_mode_any>::toField() const
{
return toString();
}
template <bool enable_mode_any>
void SettingOverflowMode<enable_mode_any>::set(OverflowMode x)
{
......@@ -441,19 +502,20 @@ void SettingOverflowMode<enable_mode_any>::set(const String & x)
}
template <bool enable_mode_any>
void SettingOverflowMode<enable_mode_any>::set(ReadBuffer & buf)
void SettingOverflowMode<enable_mode_any>::serialize(WriteBuffer & buf) const
{
String x;
readBinary(x, buf);
set(x);
writeBinary(toString(), buf);
}
template <bool enable_mode_any>
void SettingOverflowMode<enable_mode_any>::write(WriteBuffer & buf) const
void SettingOverflowMode<enable_mode_any>::deserialize(ReadBuffer & buf)
{
writeBinary(toString(), buf);
String s;
readBinary(s, buf);
set(s);
}
template struct SettingOverflowMode<false>;
template struct SettingOverflowMode<true>;
......@@ -476,6 +538,11 @@ String SettingDistributedProductMode::toString() const
return strings[static_cast<size_t>(value)];
}
Field SettingDistributedProductMode::toField() const
{
return toString();
}
void SettingDistributedProductMode::set(DistributedProductMode x)
{
value = x;
......@@ -492,16 +559,16 @@ void SettingDistributedProductMode::set(const String & x)
set(getDistributedProductMode(x));
}
void SettingDistributedProductMode::set(ReadBuffer & buf)
void SettingDistributedProductMode::serialize(WriteBuffer & buf) const
{
String x;
readBinary(x, buf);
set(x);
writeBinary(toString(), buf);
}
void SettingDistributedProductMode::write(WriteBuffer & buf) const
void SettingDistributedProductMode::deserialize(ReadBuffer & buf)
{
writeBinary(toString(), buf);
String s;
readBinary(s, buf);
set(s);
}
......@@ -510,6 +577,11 @@ String SettingString::toString() const
return value;
}
Field SettingString::toField() const
{
return value;
}
void SettingString::set(const String & x)
{
value = x;
......@@ -521,16 +593,16 @@ void SettingString::set(const Field & x)
set(safeGet<const String &>(x));
}
void SettingString::set(ReadBuffer & buf)
void SettingString::serialize(WriteBuffer & buf) const
{
String x;
readBinary(x, buf);
set(x);
writeBinary(value, buf);
}
void SettingString::write(WriteBuffer & buf) const
void SettingString::deserialize(ReadBuffer & buf)
{
writeBinary(value, buf);
String s;
readBinary(s, buf);
set(s);
}
......@@ -539,6 +611,11 @@ String SettingChar::toString() const
return String(1, value);
}
Field SettingChar::toField() const
{
return toString();
}
void SettingChar::set(char x)
{
value = x;
......@@ -559,16 +636,16 @@ void SettingChar::set(const Field & x)
set(s);
}
void SettingChar::set(ReadBuffer & buf)
void SettingChar::serialize(WriteBuffer & buf) const
{
String s;
readBinary(s, buf);
set(s);
writeBinary(toString(), buf);
}
void SettingChar::write(WriteBuffer & buf) const
void SettingChar::deserialize(ReadBuffer & buf)
{
writeBinary(toString(), buf);
String s;
readBinary(s, buf);
set(s);
}
......@@ -588,6 +665,11 @@ String SettingDateTimeInputFormat::toString() const
return strings[static_cast<size_t>(value)];
}
Field SettingDateTimeInputFormat::toField() const
{
return toString();
}
void SettingDateTimeInputFormat::set(Value x)
{
value = x;
......@@ -604,16 +686,16 @@ void SettingDateTimeInputFormat::set(const String & x)
set(getValue(x));
}
void SettingDateTimeInputFormat::set(ReadBuffer & buf)
void SettingDateTimeInputFormat::serialize(WriteBuffer & buf) const
{
String x;
readBinary(x, buf);
set(x);
writeBinary(toString(), buf);
}
void SettingDateTimeInputFormat::write(WriteBuffer & buf) const
void SettingDateTimeInputFormat::deserialize(ReadBuffer & buf)
{
writeBinary(toString(), buf);
String s;
readBinary(s, buf);
set(s);
}
......@@ -635,6 +717,11 @@ String SettingLogsLevel::toString() const
return strings[static_cast<size_t>(value)];
}
Field SettingLogsLevel::toField() const
{
return toString();
}
void SettingLogsLevel::set(Value x)
{
value = x;
......@@ -651,16 +738,33 @@ void SettingLogsLevel::set(const String & x)
set(getValue(x));
}
void SettingLogsLevel::set(ReadBuffer & buf)
void SettingLogsLevel::serialize(WriteBuffer & buf) const
{
String x;
readBinary(x, buf);
set(x);
writeBinary(toString(), buf);
}
void SettingLogsLevel::write(WriteBuffer & buf) const
void SettingLogsLevel::deserialize(ReadBuffer & buf)
{
writeBinary(toString(), buf);
String s;
readBinary(s, buf);
set(s);
}
namespace details
{
String SettingsCollectionUtils::deserializeName(ReadBuffer & buf)
{
String name;
readBinary(name, buf);
return name;
}
void SettingsCollectionUtils::serializeName(const StringRef & name, WriteBuffer & buf) { writeBinary(name, buf); }
void SettingsCollectionUtils::throwNameNotFound(const StringRef & name)
{
throw Exception("Unknown setting " + name.toString(), ErrorCodes::UNKNOWN_SETTING);
}
}
}
......@@ -3,8 +3,12 @@
#include <Poco/Timespan.h>
#include <DataStreams/SizeLimits.h>
#include <Formats/FormatSettings.h>
#include <common/StringRef.h>
#include <Common/SettingsChanges.h>
#include <Compression/CompressionInfo.h>
#include <Core/Types.h>
#include <ext/singleton.h>
#include <unordered_map>
namespace DB
......@@ -36,8 +40,8 @@ struct SettingInt
/// Serialize to a test string.
String toString() const;
/// Serialize to binary stream suitable for transfer over network.
void write(WriteBuffer & buf) const;
/// Converts to a field.
Field toField() const;
void set(IntType x);
......@@ -47,8 +51,11 @@ struct SettingInt
/// Read from text string.
void set(const String & x);
/// Serialize to binary stream suitable for transfer over network.
void serialize(WriteBuffer & buf) const;
/// Read from binary stream.
void set(ReadBuffer & buf);
void deserialize(ReadBuffer & buf);
};
using SettingUInt64 = SettingInt<UInt64>;
......@@ -72,13 +79,14 @@ struct SettingMaxThreads
SettingMaxThreads & operator= (UInt64 x) { set(x); return *this; }
String toString() const;
Field toField() const;
void set(UInt64 x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
void setAuto();
UInt64 getAutoValue() const;
......@@ -101,15 +109,16 @@ struct SettingSeconds
Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); }
String toString() const;
Field toField() const;
void set(const Poco::Timespan & x);
void set(UInt64 x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -126,13 +135,15 @@ struct SettingMilliseconds
Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); }
String toString() const;
Field toField() const;
void set(const Poco::Timespan & x);
void set(UInt64 x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -147,13 +158,14 @@ struct SettingFloat
SettingFloat & operator= (float x) { set(x); return *this; }
String toString() const;
Field toField() const;
void set(float x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -185,13 +197,14 @@ struct SettingLoadBalancing
static LoadBalancing getLoadBalancing(const String & s);
String toString() const;
Field toField() const;
void set(LoadBalancing x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -216,13 +229,14 @@ struct SettingJoinStrictness
static JoinStrictness getJoinStrictness(const String & s);
String toString() const;
Field toField() const;
void set(JoinStrictness x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -251,13 +265,14 @@ struct SettingTotalsMode
static TotalsMode getTotalsMode(const String & s);
String toString() const;
Field toField() const;
void set(TotalsMode x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -276,13 +291,14 @@ struct SettingOverflowMode
static OverflowMode getOverflowMode(const String & s);
String toString() const;
Field toField() const;
void set(OverflowMode x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
/// The setting for executing distributed subqueries inside IN or JOIN sections.
......@@ -307,12 +323,14 @@ struct SettingDistributedProductMode
static DistributedProductMode getDistributedProductMode(const String & s);
String toString() const;
Field toField() const;
void set(DistributedProductMode x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -327,12 +345,13 @@ struct SettingString
SettingString & operator= (const String & x) { set(x); return *this; }
String toString() const;
Field toField() const;
void set(const String & x);
void set(const Field & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -348,13 +367,14 @@ public:
SettingChar & operator= (char x) { set(x); return *this; }
String toString() const;
Field toField() const;
void set(char x);
void set(const String & x);
void set(const Field & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -373,12 +393,14 @@ struct SettingDateTimeInputFormat
static Value getValue(const String & s);
String toString() const;
Field toField() const;
void set(Value x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
......@@ -408,12 +430,351 @@ public:
static Value getValue(const String & s);
String toString() const;
Field toField() const;
void set(Value x);
void set(const Field & x);
void set(const String & x);
void set(ReadBuffer & buf);
void write(WriteBuffer & buf) const;
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
namespace details
{
struct SettingsCollectionUtils
{
static void serializeName(const StringRef & name, WriteBuffer & buf);
static String deserializeName(ReadBuffer & buf);
static void throwNameNotFound(const StringRef & name);
};
}
/** Template class to define collections of settings.
* Example of usage:
*
* mysettings.h:
* struct MySettings : public SettingsCollection<MySettings>
* {
* # define APPLY_FOR_MYSETTINGS(M) \
* M(SettingUInt64, a, 100, "Description of a") \
* M(SettingFloat, f, 3.11, "Description of f") \
* M(SettingString, s, "default", "Description of s")
*
* DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS)
* };
*
* mysettings.cpp:
* IMPLEMENT_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS)
*/
template <typename T>
class SettingsCollection
{
private:
using Type = T;
using GetStringFunction = String (*)(const Type &);
using GetFieldFunction = Field (*)(const Type &);
using SetStringFunction = void (*)(Type &, const String &);
using SetFieldFunction = void (*)(Type &, const Field &);
using SerializeFunction = void (*)(const Type &, WriteBuffer & buf);
using DeserializeFunction = void (*)(Type &, ReadBuffer & buf);
struct MemberInfo
{
size_t offset_of_changed;
StringRef name;
StringRef description;
GetStringFunction get_string;
GetFieldFunction get_field;
SetStringFunction set_string;
SetFieldFunction set_field;
SerializeFunction serialize;
DeserializeFunction deserialize;
bool isChanged(const Type & collection) const { return *reinterpret_cast<const bool*>(reinterpret_cast<const UInt8*>(&collection) + offset_of_changed); }
};
using MemberInfos = std::vector<MemberInfo>;
class Layout
{
public:
size_t size() const { return infos.size(); }
const MemberInfo & operator[](size_t index) const { return infos[index]; }
static constexpr size_t npos = static_cast<size_t>(-1);
size_t find(const StringRef & name) const
{
auto map_it = by_name_map.find(name);
if (map_it == by_name_map.end())
return npos;
else
return map_it->second;
}
size_t findStrict(const StringRef & name) const
{
auto map_it = by_name_map.find(name);
if (map_it == by_name_map.end())
details::SettingsCollectionUtils::throwNameNotFound(name);
return map_it->second;
}
void addMemberInfo(MemberInfo && info)
{
size_t index = infos.size();
infos.emplace_back(info);
by_name_map.emplace(infos.back().name, index);
}
private:
MemberInfos infos;
std::unordered_map<StringRef, size_t> by_name_map;
};
static const Layout & getLayout();
Type & castThis() { return static_cast<Type &>(*this); }
const Type & castThis() const { return static_cast<const Type &>(*this); }
public:
/// Provides access to a setting.
class reference
{
public:
const StringRef & getName() const { return member.name; }
const StringRef & getDescription() const { return member.description; }
bool isChanged() const { return member.isChanged(collection); }
Field getValue() const { return member.get_field(collection); }
String getValueAsString() const { return member.get_string(collection); }
void setValue(const Field & value) { member.set_field(collection, value); }
void setValue(const String & value) { member.set_string(collection, value); }
reference(Type & collection_, const MemberInfo & member_) : collection(collection_), member(member_) {}
private:
Type & collection;
const MemberInfo & member;
};
/// Provides read-only access to a setting.
class const_reference
{
public:
const StringRef & getName() const { return member.name; }
const StringRef & getDescription() const { return member.description; }
bool isChanged() const { return member.isChanged(collection); }
Field getValue() const { return member.get_field(collection); }
String getValueAsString() const { return member.get_string(collection); }
const_reference(const Type & collection_, const MemberInfo & member_) : collection(collection_), member(member_) {}
private:
const Type & collection;
const MemberInfo & member;
};
/// Returns number of settings.
size_t size() const { return getLayout().size(); }
/// Returns a setting by its zero-based index.
reference operator [](size_t index) { return reference(castThis(), getLayout()[index]); }
const_reference operator [](size_t index) const { return const_reference(castThis(), getLayout()[index]); }
/// Returns a setting by its name. Throws an exception if there is not setting with such name.
reference operator [](const String & name)
{
const Layout & layout = getLayout();
return reference(castThis(), layout[layout.findStrict(name)]);
}
const_reference operator [](const String & name) const
{
const Layout & layout = getLayout();
return const_reference(castThis(), layout[layout.findStrict(name)]);
}
/// Finds a setting by name and returns its index. Returns npos if not found.
size_t find(const String & name) const { return getLayout().find(name); }
static constexpr size_t npos = static_cast<size_t>(-1);
/// Finds a setting by name and returns its index. Throws an exception if not found.
size_t findStrict(const String & name) const { return getLayout().findStrict(name); }
/// Sets setting by name.
void set(const String & name, const Field & value) { (*this)[name].setValue(value); }
/// Sets setting by name. Read value in text form from string (for example, from configuration file or from URL parameter).
void set(const String & name, const String & value) { (*this)[name].setValue(value); }
/// Returns the value of a setting.
Field get(const String & name) const { return (*this)[name].getValue(); }
/// Returns the value of a setting converted to string.
String getAsString(const String & name) const { return (*this)[name].getValueAsString(); }
/// Returns the value of a setting.
bool tryGet(const String & name, Field & value) const
{
const Layout & layout = getLayout();
size_t index = layout.find(name);
if (index == npos)
return false;
value = layout[index].get_field(castThis());
return true;
}
/// Returns the value of a setting converted to string.
bool tryGet(const String & name, String & value) const
{
const Layout & layout = getLayout();
size_t index = layout.find(name);
if (index == npos)
return false;
value = layout[index].get_string(castThis());
return true;
}
/// Compares two collections of settings.
bool operator ==(const Type & rhs) const
{
const Layout & layout = getLayout();
for (size_t i = 0; i != layout.size(); ++i)
{
const auto & member = layout[i];
bool left_changed = member.isChanged(castThis());
bool right_changed = member.isChanged(rhs);
if (left_changed || right_changed)
{
if (left_changed != right_changed)
return false;
if (member.get_field(castThis()) != member.get_field(rhs))
return false;
}
}
return true;
}
bool operator !=(const Type & rhs) const
{
return !(*this == rhs);
}
/// Gathers all changed values (e.g. for applying them later to another collection of settings).
SettingsChanges changes() const
{
SettingsChanges found_changes;
const Layout & layout = getLayout();
for (size_t i = 0; i != layout.size(); ++i)
{
const auto & member = layout[i];
if (member.isChanged(castThis()))
found_changes.emplace_back(member.name.toString(), member.get_field(castThis()));
}
return found_changes;
}
/// Applies changes to the settings.
void applyChange(const SettingChange & change)
{
set(change.name, change.value);
}
void applyChanges(const SettingsChanges & changes)
{
for (const SettingChange & change : changes)
applyChange(change);
}
void copyChangesFrom(const Type & src)
{
const Layout & layout = getLayout();
for (size_t i = 0; i != layout.size(); ++i)
{
const auto & member = layout[i];
if (member.isChanged(src))
member.set_field(castThis(), member.get_field(src));
}
}
void copyChangesTo(Type & dest) const
{
dest.copyChangesFrom(castThis());
}
/// Writes the settings to buffer (e.g. to be sent to remote server).
/// Only changed settings are written. They are written as list of contiguous name-value pairs,
/// finished with empty name.
void serialize(WriteBuffer & buf) const
{
const Layout & layout = getLayout();
for (size_t i = 0; i != layout.size(); ++i)
{
const auto & member = layout[i];
if (member.isChanged(castThis()))
{
details::SettingsCollectionUtils::serializeName(member.name, buf);
member.serialize(castThis(), buf);
}
}
details::SettingsCollectionUtils::serializeName(StringRef{} /* empty string is a marker of the end of settings */, buf);
}
/// Reads the settings from buffer.
void deserialize(ReadBuffer & buf)
{
const Layout & layout = getLayout();
while (true)
{
String name = details::SettingsCollectionUtils::deserializeName(buf);
if (name.empty() /* empty string is a marker of the end of settings */)
break; \
const auto & member = layout[layout.findStrict(name)];
member.deserialize(castThis(), buf); \
}
}
};
#define DECLARE_SETTINGS_COLLECTION(APPLY_MACRO) \
APPLY_MACRO(DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_)
#define IMPLEMENT_SETTINGS_COLLECTION(CLASS_NAME, APPLY_MACRO) \
template<> \
const SettingsCollection<CLASS_NAME>::Layout & SettingsCollection<CLASS_NAME>::getLayout() \
{ \
struct Functions \
{ \
APPLY_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \
}; \
static const SettingsCollection<Type>::Layout single_instance = [] \
{ \
SettingsCollection<Type>::Layout layout; \
APPLY_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \
return layout; \
}(); \
return single_instance; \
}
#define DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};
#define IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \
static String NAME##_getString(const Type & collection) { return collection.NAME.toString(); } \
static Field NAME##_getField(const Type & collection) { return collection.NAME.toField(); } \
static void NAME##_setString(Type & collection, const String & value) { collection.NAME.set(value); } \
static void NAME##_setField(Type & collection, const Field & value) { collection.NAME.set(value); } \
static void NAME##_serialize(const Type & collection, WriteBuffer & buf) { collection.NAME.serialize(buf); } \
static void NAME##_deserialize(Type & collection, ReadBuffer & buf) { collection.NAME.deserialize(buf); }
#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \
static_assert(std::is_same_v<decltype(std::declval<TYPE>().changed), bool>); \
layout.addMemberInfo({offsetof(Type, NAME.changed), \
StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), \
&Functions::NAME##_getString, &Functions::NAME##_getField, \
&Functions::NAME##_setString, &Functions::NAME##_setField, \
&Functions::NAME##_serialize, &Functions::NAME##_deserialize });
}
......@@ -36,6 +36,7 @@
#include <Interpreters/Cluster.h>
#include <Interpreters/InterserverIOHandler.h>
#include <Interpreters/Compiler.h>
#include <Interpreters/SettingsConstraints.h>
#include <Interpreters/SystemLog.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/QueryThreadLog.h>
......@@ -1027,27 +1028,53 @@ void Context::setSettings(const Settings & settings_)
}
void Context::setSetting(const String & name, const Field & value)
void Context::setSetting(const String & name, const String & value)
{
auto lock = getLock();
if (name == "profile")
{
auto lock = getLock();
settings.setProfile(value.safeGet<String>(), *shared->users_config);
settings.setProfile(value, *shared->users_config);
return;
}
else
settings.set(name, value);
settings.set(name, value);
}
void Context::setSetting(const String & name, const std::string & value)
void Context::setSetting(const String & name, const Field & value)
{
auto lock = getLock();
if (name == "profile")
{
auto lock = getLock();
settings.setProfile(value, *shared->users_config);
settings.setProfile(value.safeGet<String>(), *shared->users_config);
return;
}
else
settings.set(name, value);
settings.set(name, value);
}
void Context::applySettingChange(const SettingChange & change)
{
setSetting(change.name, change.value);
}
void Context::applySettingsChanges(const SettingsChanges & changes)
{
auto lock = getLock();
for (const SettingChange & change : changes)
applySettingChange(change);
}
void Context::checkSettingsConstraints(const SettingChange & change)
{
SettingsConstraints::check(settings, change);
}
void Context::checkSettingsConstraints(const SettingsChanges & changes)
{
SettingsConstraints::check(settings, changes);
}
......
......@@ -263,11 +263,15 @@ public:
Settings getSettings() const;
void setSettings(const Settings & settings_);
/// Set a setting by name.
/// Set settings by name.
void setSetting(const String & name, const String & value);
void setSetting(const String & name, const Field & value);
void applySettingChange(const SettingChange & change);
void applySettingsChanges(const SettingsChanges & changes);
/// Set a setting by name. Read the value in text form from a string (for example, from a config, or from a URL parameter).
void setSetting(const String & name, const std::string & value);
/// Checks the constraints.
void checkSettingsConstraints(const SettingChange & change);
void checkSettingsConstraints(const SettingsChanges & changes);
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
const ExternalDictionaries & getExternalDictionaries() const;
......
#include <Parsers/ASTSetQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Common/typeid_cast.h>
#include <Common/FieldVisitors.h>
namespace DB
{
namespace ErrorCodes
{
extern const int READONLY;
extern const int QUERY_IS_PROHIBITED;
}
BlockIO InterpreterSetQuery::execute()
{
const auto & ast = query_ptr->as<ASTSetQuery &>();
checkAccess(ast);
Context & target = context.getSessionContext();
for (const auto & change : ast.changes)
target.setSetting(change.name, change.value);
context.checkSettingsConstraints(ast.changes);
context.getSessionContext().applySettingsChanges(ast.changes);
return {};
}
void InterpreterSetQuery::checkAccess(const ASTSetQuery & ast)
{
/** The `readonly` value is understood as follows:
* 0 - everything allowed.
* 1 - only read queries can be made; you can not change the settings.
* 2 - You can only do read queries and you can change the settings, except for the `readonly` setting.
*/
const Settings & settings = context.getSettingsRef();
auto readonly = settings.readonly;
auto allow_ddl = settings.allow_ddl;
for (const auto & change : ast.changes)
{
String value;
/// Setting isn't checked if value wasn't changed.
if (!settings.tryGet(change.name, value) || applyVisitor(FieldVisitorToString(), change.value) != value)
{
if (!allow_ddl && change.name == "allow_ddl")
throw Exception("Cannot modify 'allow_ddl' setting when DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
if (readonly == 1)
throw Exception("Cannot execute SET query in readonly mode", ErrorCodes::READONLY);
if (readonly > 1 && change.name == "readonly")
throw Exception("Cannot modify 'readonly' setting in readonly mode", ErrorCodes::READONLY);
}
}
}
void InterpreterSetQuery::executeForCurrentContext()
{
const auto & ast = query_ptr->as<ASTSetQuery &>();
checkAccess(ast);
for (const auto & change : ast.changes)
context.setSetting(change.name, change.value);
context.checkSettingsConstraints(ast.changes);
context.applySettingsChanges(ast.changes);
}
}
......@@ -23,8 +23,6 @@ public:
*/
BlockIO execute() override;
void checkAccess(const ASTSetQuery & ast);
/** Set setting for current context (query context).
* It is used for interpretation of SETTINGS clause in SELECT query.
*/
......
#include <Interpreters/SettingsConstraints.h>
#include <Core/Settings.h>
namespace DB
{
namespace ErrorCodes
{
extern const int READONLY;
extern const int QUERY_IS_PROHIBITED;
extern const int NO_ELEMENTS_IN_CONFIG;
}
namespace
{
thread_local Settings temp_settings;
void checkImpl(const Settings & current_settings, size_t index)
{
const auto & new_setting = temp_settings[index];
Field new_value = new_setting.getValue();
const auto & current_setting = current_settings[index];
Field current_value = current_setting.getValue();
/// Setting isn't checked if value wasn't changed.
if (current_value == new_value)
return;
const StringRef & name = new_setting.getName();
if (!current_settings.allow_ddl && name == "allow_ddl")
throw Exception("Cannot modify 'allow_ddl' setting when DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
/** The `readonly` value is understood as follows:
* 0 - everything allowed.
* 1 - only read queries can be made; you can not change the settings.
* 2 - You can only do read queries and you can change the settings, except for the `readonly` setting.
*/
if (current_settings.readonly == 1)
throw Exception("Cannot modify '" + name.toString() + "' setting in readonly mode", ErrorCodes::READONLY);
if (current_settings.readonly > 1 && name == "readonly")
throw Exception("Cannot modify 'readonly' setting in readonly mode", ErrorCodes::READONLY);
}
}
void SettingsConstraints::check(const Settings & current_settings, const SettingChange & change)
{
size_t index = current_settings.find(change.name);
if (index == Settings::npos)
return;
// We store `change.value` to `temp_settings` to ensure it's converted to the correct type.
temp_settings[index].setValue(change.value);
checkImpl(current_settings, index);
}
void SettingsConstraints::check(const Settings & current_settings, const SettingsChanges & changes)
{
for (const auto & change : changes)
check(current_settings, change);
}
}
#pragma once
#include <Common/SettingsChanges.h>
namespace DB
{
struct Settings;
/** Checks if specified changes of settings are allowed or not.
* If the changes are not allowed (i.e. violates some constraints) this class throws an exception.
* This class checks that we are not in the read-only mode.
* If a setting cannot be change due to the read-only mode this class throws an exception.
* The value of `readonly` value is understood as follows:
* 0 - everything allowed.
* 1 - only read queries can be made; you can not change the settings.
* 2 - you can only do read queries and you can change the settings, except for the `readonly` setting.
*/
class SettingsConstraints
{
public:
static void check(const Settings & current_settings, const SettingChange & change);
static void check(const Settings & current_settings, const SettingsChanges & changes);
};
}
#pragma once
#include <Core/Field.h>
#include <Common/FieldVisitors.h>
#include <Common/SettingsChanges.h>
#include <Parsers/IAST.h>
......@@ -16,14 +16,7 @@ class ASTSetQuery : public IAST
public:
bool is_standalone = true; /// If false, this AST is a part of another query, such as SELECT.
struct Change
{
String name;
Field value;
};
using Changes = std::vector<Change>;
Changes changes;
SettingsChanges changes;
/** Get the text that identifies this element. */
String getID(char) const override { return "Set"; }
......@@ -35,7 +28,7 @@ public:
if (is_standalone)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SET " << (settings.hilite ? hilite_none : "");
for (ASTSetQuery::Changes::const_iterator it = changes.begin(); it != changes.end(); ++it)
for (auto it = changes.begin(); it != changes.end(); ++it)
{
if (it != changes.begin())
settings.ostr << ", ";
......
......@@ -13,7 +13,7 @@ namespace DB
/// Parse `name = value`.
static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos, Expected & expected)
static bool parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected)
{
ParserIdentifier name_p;
ParserLiteral value_p;
......@@ -31,6 +31,7 @@ static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos,
if (!value_p.parse(pos, value, expected))
return false;
String change_name;
getIdentifierName(name, change.name);
change.value = value->as<ASTLiteral &>().value;
......@@ -50,14 +51,14 @@ bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
}
ASTSetQuery::Changes changes;
SettingsChanges changes;
while (true)
{
if (!changes.empty() && !s_comma.ignore(pos))
break;
changes.push_back(ASTSetQuery::Change());
changes.push_back(SettingChange{});
if (!parseNameValuePair(changes.back(), pos, expected))
return false;
......@@ -67,7 +68,7 @@ bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
node = query;
query->is_standalone = !parse_only_internals;
query->changes = changes;
query->changes = std::move(changes);
return true;
}
......
......@@ -11,24 +11,25 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_SETTING;
}
IMPLEMENT_SETTINGS_COLLECTION(KafkaSettings, APPLY_FOR_KAFKA_SETTINGS)
void KafkaSettings::loadFromQuery(ASTStorage & storage_def)
{
if (storage_def.settings)
{
for (const ASTSetQuery::Change & setting : storage_def.settings->changes)
try
{
#define SET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (setting.name == #NAME) NAME.set(setting.value);
if (false) {}
APPLY_FOR_KAFKA_SETTINGS(SET)
applyChanges(storage_def.settings->changes);
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
throw Exception(e.message() + " for storage " + storage_def.engine->name, ErrorCodes::BAD_ARGUMENTS);
else
throw Exception(
"Unknown setting " + setting.name + " for storage " + storage_def.engine->name,
ErrorCodes::BAD_ARGUMENTS);
#undef SET
e.rethrow();
}
}
else
......
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
#include <Core/Defines.h>
#include <Core/Types.h>
#include <Core/SettingsCommon.h>
......@@ -14,7 +11,7 @@ class ASTStorage;
/** Settings for the Kafka engine.
* Could be loaded from a CREATE TABLE query (SETTINGS clause).
*/
struct KafkaSettings
struct KafkaSettings : public SettingsCollection<KafkaSettings>
{
#define APPLY_FOR_KAFKA_SETTINGS(M) \
......@@ -28,14 +25,8 @@ struct KafkaSettings
M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \
M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block")
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};
DECLARE_SETTINGS_COLLECTION(APPLY_FOR_KAFKA_SETTINGS)
APPLY_FOR_KAFKA_SETTINGS(DECLARE)
#undef DECLARE
public:
void loadFromQuery(ASTStorage & storage_def);
};
......
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTFunction.h>
......@@ -12,8 +13,11 @@ namespace ErrorCodes
{
extern const int INVALID_CONFIG_PARAMETER;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_SETTING;
}
IMPLEMENT_SETTINGS_COLLECTION(MergeTreeSettings, APPLY_FOR_MERGE_TREE_SETTINGS)
void MergeTreeSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
{
if (!config.has(config_elem))
......@@ -22,18 +26,17 @@ void MergeTreeSettings::loadFromConfig(const String & config_elem, const Poco::U
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_elem, config_keys);
for (const String & key : config_keys)
try
{
String value = config.getString(config_elem + "." + key);
#define SET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (key == #NAME) NAME.set(value);
if (false) {}
APPLY_FOR_MERGE_TREE_SETTINGS(SET)
for (const String & key : config_keys)
set(key, config.getString(config_elem + "." + key));
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
throw Exception(e.message() + " in MergeTree config", ErrorCodes::INVALID_CONFIG_PARAMETER);
else
throw Exception("Unknown MergeTree setting " + key + " in config", ErrorCodes::INVALID_CONFIG_PARAMETER);
#undef SET
e.rethrow();
}
}
......@@ -41,18 +44,16 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def)
{
if (storage_def.settings)
{
for (const ASTSetQuery::Change & setting : storage_def.settings->changes)
try
{
#define SET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (setting.name == #NAME) NAME.set(setting.value);
if (false) {}
APPLY_FOR_MERGE_TREE_SETTINGS(SET)
applyChanges(storage_def.settings->changes);
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
throw Exception(e.message() + " for storage " + storage_def.engine->name, ErrorCodes::BAD_ARGUMENTS);
else
throw Exception(
"Unknown setting " + setting.name + " for storage " + storage_def.engine->name,
ErrorCodes::BAD_ARGUMENTS);
#undef SET
e.rethrow();
}
}
else
......@@ -62,13 +63,13 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def)
storage_def.set(storage_def.settings, settings_ast);
}
ASTSetQuery::Changes & changes = storage_def.settings->changes;
SettingsChanges & changes = storage_def.settings->changes;
#define ADD_IF_ABSENT(NAME) \
if (std::find_if(changes.begin(), changes.end(), \
[](const ASTSetQuery::Change & c) { return c.name == #NAME; }) \
[](const SettingChange & c) { return c.name == #NAME; }) \
== changes.end()) \
changes.push_back(ASTSetQuery::Change{#NAME, NAME.value});
changes.push_back(SettingChange{#NAME, NAME.value});
APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(ADD_IF_ABSENT)
#undef ADD_IF_ABSENT
......
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
#include <Core/Defines.h>
#include <Core/Types.h>
#include <Core/SettingsCommon.h>
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
}
namespace DB
{
......@@ -14,7 +21,7 @@ class ASTStorage;
/** Settings for the MergeTree family of engines.
* Could be loaded from config or from a CREATE TABLE query (SETTINGS clause).
*/
struct MergeTreeSettings
struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
{
#define APPLY_FOR_MERGE_TREE_SETTINGS(M) \
......@@ -168,18 +175,12 @@ struct MergeTreeSettings
/** Minimal time in seconds, when merge with TTL can be repeated */ \
M(SettingInt64, merge_with_ttl_timeout, 3600 * 24, "")
DECLARE_SETTINGS_COLLECTION(APPLY_FOR_MERGE_TREE_SETTINGS)
/// Settings that should not change after the creation of a table.
#define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \
M(index_granularity)
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};
APPLY_FOR_MERGE_TREE_SETTINGS(DECLARE)
#undef DECLARE
public:
void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
/// NOTE: will rewrite the AST to add immutable settings.
......
......@@ -139,7 +139,7 @@ void registerStorageJoin(StorageFactory & factory)
if (args.storage_def && args.storage_def->settings)
{
for (const ASTSetQuery::Change & setting : args.storage_def->settings->changes)
for (const auto & setting : args.storage_def->settings->changes)
{
if (setting.name == "join_use_nulls")
join_use_nulls.set(setting.value);
......
......@@ -4,11 +4,11 @@
"value": "4611686018427387904"
value
value
Cannot execute SET query in readonly mode.
Cannot modify 'output_format_json_quote_64bit_integers' setting in readonly mode
"name": "value",
"value": "9223372036854775808"
"name": "value",
"value": 9223372036854775808
value
value
Cannot override setting
Cannot modify 'output_format_json_quote_64bit_integers' setting in readonly mode
......@@ -6,13 +6,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=0 | grep value
$CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=1 | grep value
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.'
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.'
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode'
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" | grep value
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" | grep value
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?session_id=readonly&session_timeout=3600" -d 'SET readonly = 1'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o 'value\|Cannot override setting'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" 2>&1 | grep -o 'value\|Cannot override setting'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode.'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode'
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册