Send settings from server to client

This commit is contained in:
Michael Kolupaev 2024-10-29 08:31:44 +00:00
parent 0fecf98042
commit e894f17f8a
12 changed files with 156 additions and 3 deletions

View File

@ -476,6 +476,17 @@ void Client::connect()
connection_parameters.timeouts, server_name, server_version_major, server_version_minor, server_version_patch, server_revision);
config().setString("host", connection_parameters.host);
config().setInt("port", connection_parameters.port);
/// Apply setting changes received from server, but with lower priority than settings
/// changed from command line.
SettingsChanges settings_from_server = assert_cast<Connection &>(*connection).settingsFromServer();
const Settings & settings = global_context->getSettingsRef();
std::erase_if(settings_from_server, [&](const SettingChange & change)
{
return settings.isChanged(change.name);
});
global_context->applySettingsChanges(settings_from_server);
break;
}
catch (const Exception & e)

View File

@ -5,6 +5,7 @@
#include <Core/Settings.h>
#include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedWriteBuffer.h>
#include <IO/LimitReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
@ -74,6 +75,7 @@ namespace ErrorCodes
extern const int SUPPORT_IS_DISABLED;
extern const int BAD_ARGUMENTS;
extern const int EMPTY_DATA_PASSED;
extern const int CANNOT_READ_ALL_DATA;
}
Connection::~Connection()
@ -575,6 +577,22 @@ void Connection::receiveHello(const Poco::Timespan & handshake_timeout)
readIntBinary(read_nonce, *in);
nonce.emplace(read_nonce);
}
if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_HELLO_EXTRAS)
{
UInt64 extras_size;
readVarUInt(extras_size, *in);
LimitReadBuffer extras(*in, extras_size, /*throw_exception_*/ false, /*exact_limit_*/ extras_size);
if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_SETTINGS)
{
Settings settings;
settings.read(extras, SettingsWriteFormat::STRINGS_WITH_FLAGS);
settings_from_server = settings.changes();
}
if (!extras.eof())
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected size of Hello message extras: expected {}, got {}", extras.count(), extras_size);
}
}
else if (packet_type == Protocol::Server::Exception)
receiveException()->rethrow();
@ -600,6 +618,12 @@ const String & Connection::getDefaultDatabase() const
return default_database;
}
const SettingsChanges & Connection::settingsFromServer() const
{
chassert(connected);
return settings_from_server;
}
const String & Connection::getDescription(bool with_extra) const /// NOLINT
{
if (with_extra)
@ -833,9 +857,23 @@ void Connection::sendQuery(
/// Per query settings.
if (settings)
{
std::optional<Settings> modified_settings;
const Settings * settings_to_send = settings;
if (!settings_from_server.empty())
{
/// Don't send settings that we got from the server in the first place.
modified_settings.emplace(*settings);
for (const SettingChange & change : settings_from_server)
{
if (settings->get(change.name) == change.value)
modified_settings->setDefaultValue(change.name);
}
settings_to_send = &*modified_settings;
}
auto settings_format = (server_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsWriteFormat::STRINGS_WITH_FLAGS
: SettingsWriteFormat::BINARY;
settings->write(*out, settings_format);
settings_to_send->write(*out, settings_format);
}
else
writeStringBinary("" /* empty string is a marker of the end of settings */, *out);

View File

@ -89,6 +89,8 @@ public:
const String & getServerTimezone(const ConnectionTimeouts & timeouts) override;
const String & getServerDisplayName(const ConnectionTimeouts & timeouts) override;
const SettingsChanges & settingsFromServer() const;
/// For log and exception messages.
const String & getDescription(bool with_extra = false) const override; /// NOLINT
const String & getHost() const;
@ -213,6 +215,7 @@ private:
UInt64 server_parallel_replicas_protocol_version = 0;
String server_timezone;
String server_display_name;
SettingsChanges settings_from_server;
std::unique_ptr<Poco::Net::StreamSocket> socket;
std::shared_ptr<ReadBufferFromPocoSocketChunked> in;

View File

@ -3,6 +3,8 @@
namespace DB
{
/// In the names below, "REVISION" and "PROTOCOL_VERSION" are synonyms.
static constexpr auto DBMS_MIN_REVISION_WITH_CLIENT_INFO = 54032;
static constexpr auto DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE = 54058;
static constexpr auto DBMS_MIN_REVISION_WITH_QUOTA_KEY_IN_CLIENT_INFO = 54060;
@ -90,6 +92,10 @@ static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS = 54470;
static constexpr auto DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL = 54471;
static constexpr auto DBMS_MIN_REVISION_WITH_SERVER_HELLO_EXTRAS = 54472;
static constexpr auto DBMS_MIN_REVISION_WITH_SERVER_SETTINGS = 54472;
/// Version of ClickHouse TCP protocol.
///
/// Should be incremented manually on protocol changes.
@ -97,6 +103,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCO
/// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION,
/// later is just a number for server version (one number instead of commit SHA)
/// for simplicity (sometimes it may be more convenient in some use cases).
static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54471;
static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54472;
}

View File

@ -192,6 +192,7 @@ namespace DB
DECLARE(UInt64, parts_killer_pool_size, 128, "Threads for cleanup of shared merge tree outdated threads. Only available in ClickHouse Cloud", 0) \
DECLARE(UInt64, keeper_multiread_batch_size, 10'000, "Maximum size of batch for MultiRead request to [Zoo]Keeper that support batching. If set to 0, batching is disabled. Available only in ClickHouse Cloud.", 0) \
DECLARE(Bool, use_legacy_mongodb_integration, true, "Use the legacy MongoDB integration implementation. Note: it's highly recommended to set this option to false, since legacy implementation will be removed in the future. Please submit any issues you encounter with the new implementation.", 0) \
DECLARE(Bool, send_settings_to_client, true, "Send user settings from server configuration to clients (in the server Hello message).", 0) \
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below

View File

@ -6284,6 +6284,11 @@ void Settings::read(ReadBuffer & in, SettingsWriteFormat format)
impl->read(in, format);
}
void Settings::writeEmpty(WriteBuffer & out)
{
BaseSettingsHelpers::writeString("", out);
}
void Settings::addToProgramOptions(boost::program_options::options_description & options)
{
addProgramOptions(*impl, options);

View File

@ -139,6 +139,8 @@ struct Settings
void write(WriteBuffer & out, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT) const;
void read(ReadBuffer & in, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT);
/// Equivalent to Settings().write(out, <any format>) but faster.
static void writeEmpty(WriteBuffer & out);
void addToProgramOptions(boost::program_options::options_description & options);
void addToProgramOptions(std::string_view setting_name, boost::program_options::options_description & options);

View File

@ -9,7 +9,7 @@ namespace DB
{
/** Writes the data to a string.
* Note: before using the resulting string, destroy this object.
* Note: before using the resulting string, destroy this object or call finalize().
*/
using WriteBufferFromString = WriteBufferFromVector<std::string>;

View File

@ -105,6 +105,7 @@ namespace Setting
namespace ServerSetting
{
extern const ServerSettingsBool validate_tcp_client_information;
extern const ServerSettingsBool send_settings_to_client;
}
}
@ -1752,6 +1753,24 @@ void TCPHandler::sendHello()
nonce.emplace(thread_local_rng());
writeIntBinary(nonce.value(), *out);
}
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_HELLO_EXTRAS)
{
/// Write newly added fields as one length-prefixed blob to allow proxies to skip them
/// without parsing. Needed only for Hello message because it can't use chunked protocol.
WriteBufferFromOwnString extras;
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_SETTINGS)
{
if (is_interserver_mode || !Context::getGlobalContextInstance()->getServerSettings()[ServerSetting::send_settings_to_client])
Settings::writeEmpty(extras); // send empty list of setting changes
else
session->sessionContext()->getSettingsRef().write(extras, SettingsWriteFormat::STRINGS_WITH_FLAGS);
}
writeStringBinary(extras.str(), *out);
}
out->next();
}

View File

@ -0,0 +1,21 @@
<clickhouse>
<profiles>
<default>
<output_format_json_quote_64bit_integers>0</output_format_json_quote_64bit_integers>
</default>
<second_profile>
<output_format_json_quote_64bit_integers>1</output_format_json_quote_64bit_integers>
<output_format_json_array_of_rows>1</output_format_json_array_of_rows>
</second_profile>
</profiles>
<users>
<default>
<password></password>
<profile>default</profile>
</default>
<second_user>
<password></password>
<profile>second_profile</profile>
</second_user>
</users>
</clickhouse>

View File

@ -0,0 +1,47 @@
import pytest
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=[],
user_configs=[
"configs/users.d/users.xml",
],
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_settings_from_server(started_cluster):
# Setting changed by server (default user).
res = node.query("select 42::UInt64 as x format JSON")
assert '"x": 42' in res, "should be unquoted"
# Setting changed by server to a different value (other user).
res = node.query("select 42::UInt64 as x format JSON", user='second_user')
assert '"x": "42"' in res, "should be quoted"
# Setting not changed by server (default user).
res = node.query("select 42::UInt64 as x format JSONEachRow")
assert '[' not in res, "should not be formatted as a JSON array"
# Setting changed by server (other user).
res = node.query("select 42::UInt64 as x format JSONEachRow", user='second_user')
assert '[' in res, "should be formatted as a JSON array"
# Setting changed by server but changed back by the query.
res = node.query("select 42::UInt64 as x settings output_format_json_array_of_rows=0 format JSONEachRow", user='second_user')
assert '[' not in res, "should not be formatted as a JSON array"
# Setting changed by server but changed back client command line.
res = node.query("select 42::UInt64 as x format JSONEachRow", user='second_user', settings={"output_format_json_array_of_rows": "0"})
assert '[' not in res, "should not be formatted as a JSON array"