Fix updating of the access rights after changing of the readonly setting. Remove non-const function Context::getSettingsRef().

This commit is contained in:
Vitaly Baranov 2020-03-13 17:50:26 +03:00
parent 5eb79c2303
commit 4af36ee5a4
16 changed files with 40 additions and 37 deletions

View File

@ -225,11 +225,11 @@ private:
context.setQueryParameters(query_parameters);
/// settings and limits could be specified in config file, but passed settings has higher priority
for (auto && setting : context.getSettingsRef())
for (const auto & setting : context.getSettingsRef())
{
const String & name = setting.getName().toString();
if (config().has(name) && !setting.isChanged())
setting.setValue(config().getString(name));
context.setSetting(name, config().getString(name));
}
/// Set path for format schema files
@ -1736,8 +1736,8 @@ public:
("server_logs_file", po::value<std::string>(), "put server logs into specified file")
;
context.makeGlobalContext();
context.getSettingsRef().addProgramOptions(main_description);
Settings cmd_settings;
cmd_settings.addProgramOptions(main_description);
/// Commandline options related to external tables.
po::options_description external_description = createOptionsDescription("External tables options", terminal_width);
@ -1805,6 +1805,9 @@ public:
}
}
context.makeGlobalContext();
context.setSettings(cmd_settings);
/// Copy settings-related program options to config.
/// TODO: Is this code necessary?
for (const auto & setting : context.getSettingsRef())

View File

@ -216,7 +216,7 @@ void ClusterCopier::reloadTaskDescription()
/// Setup settings
task_cluster->reloadSettings(*config);
context.getSettingsRef() = task_cluster->settings_common;
context.setSettings(task_cluster->settings_common);
task_cluster_current_config = config;
task_descprtion_current_stat = stat;
@ -964,8 +964,8 @@ PartitionTaskStatus ClusterCopier::processPartitionTaskImpl(const ConnectionTime
{
Context local_context = context;
// Use pull (i.e. readonly) settings, but fetch data from destination servers
local_context.getSettingsRef() = task_cluster->settings_pull;
local_context.getSettingsRef().skip_unavailable_shards = true;
local_context.setSettings(task_cluster->settings_pull);
local_context.setSetting("skip_unavailable_shards", true);
Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in);
count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0;
@ -1053,10 +1053,10 @@ PartitionTaskStatus ClusterCopier::processPartitionTaskImpl(const ConnectionTime
{
/// Custom INSERT SELECT implementation
Context context_select = context;
context_select.getSettingsRef() = task_cluster->settings_pull;
context_select.setSettings(task_cluster->settings_pull);
Context context_insert = context;
context_insert.getSettingsRef() = task_cluster->settings_push;
context_insert.setSettings(task_cluster->settings_push);
BlockInputStreamPtr input;
BlockOutputStreamPtr output;

View File

@ -21,6 +21,7 @@
#include <Common/ThreadStatus.h>
#include <Common/config_version.h>
#include <Common/quoteString.h>
#include <Common/SettingsChanges.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/UseSSL.h>
@ -92,7 +93,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
void LocalServer::applyCmdSettings()
{
context->getSettingsRef().copyChangesFrom(cmd_settings);
context->applySettingsChanges(cmd_settings.changes());
}
/// If path is specified and not empty, will try to setup server environment and load existing metadata

View File

@ -439,13 +439,13 @@ void HTTPHandler::processQuery(
/// In theory if initially readonly = 0, the client can change any setting and then set readonly
/// to some other value.
auto & settings = context.getSettingsRef();
const auto & settings = context.getSettingsRef();
/// Only readonly queries are allowed for HTTP GET requests.
if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET)
{
if (settings.readonly == 0)
settings.readonly = 2;
context.setSetting("readonly", 2);
}
bool has_external_data = startsWith(request.getContentType(), "multipart/form-data");

View File

@ -527,7 +527,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Load global settings from default_profile and system_profile.
global_context->setDefaultProfiles(config());
Settings & settings = global_context->getSettingsRef();
const Settings & settings = global_context->getSettingsRef();
/// Size of cache for marks (index of MergeTree family of tables). It is mandatory.
size_t mark_cache_size = config().getUInt64("mark_cache_size");

View File

@ -950,11 +950,11 @@ void TCPHandler::receiveUnexpectedQuery()
readStringBinary(skip_string, *in);
ClientInfo & skip_client_info = query_context->getClientInfo();
ClientInfo skip_client_info;
if (client_revision >= DBMS_MIN_REVISION_WITH_CLIENT_INFO)
skip_client_info.read(*in, client_revision);
Settings & skip_settings = query_context->getSettingsRef();
Settings skip_settings;
auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS
: SettingsBinaryFormat::OLD;
skip_settings.deserialize(*in, settings_format);

View File

@ -165,7 +165,7 @@ void SettingMaxThreads::set(const Field & x)
if (x.getType() == Field::Types::String)
set(get<const String &>(x));
else
set(safeGet<UInt64>(x));
set(applyVisitor(FieldVisitorConvertToNumber<UInt64>(), x));
}
void SettingMaxThreads::set(const String & x)
@ -246,7 +246,7 @@ void SettingTimespan<io_unit>::set(const Field & x)
if (x.getType() == Field::Types::String)
set(get<const String &>(x));
else
set(safeGet<UInt64>(x));
set(applyVisitor(FieldVisitorConvertToNumber<UInt64>(), x));
}
template <SettingTimespanIO io_unit>

View File

@ -42,7 +42,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
views_context = std::make_unique<Context>(context);
// Do not deduplicate insertions into MV if the main insertion is Ok
if (disable_deduplication_for_children)
views_context->getSettingsRef().insert_deduplicate = false;
views_context->setSetting("insert_deduplicate", false);
}
for (const auto & database_table : dependencies)

View File

@ -74,7 +74,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
/// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication).
context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {});
/// Processors are not supported here yet.
context.getSettingsRef().experimental_use_processors = false;
context.setSetting("experimental_use_processors", false);
/// Query context is needed because some code in executeQuery function may assume it exists.
/// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock.
context.makeQueryContext();

View File

@ -936,9 +936,9 @@ Settings Context::getSettings() const
void Context::setSettings(const Settings & settings_)
{
auto lock = getLock();
bool old_readonly = settings.readonly;
bool old_allow_ddl = settings.allow_ddl;
bool old_allow_introspection_functions = settings.allow_introspection_functions;
auto old_readonly = settings.readonly;
auto old_allow_ddl = settings.allow_ddl;
auto old_allow_introspection_functions = settings.allow_introspection_functions;
settings = settings_;

View File

@ -427,7 +427,6 @@ public:
}
const Settings & getSettingsRef() const { return settings; }
Settings & getSettingsRef() { return settings; }
void setProgressCallback(ProgressCallback callback);
/// Used in InterpreterSelectQuery to pass it to the IBlockInputStream.

View File

@ -164,8 +164,8 @@ bool isStorageTouchedByMutations(
return true;
}
context_copy.getSettingsRef().max_streams_to_max_threads_ratio = 1;
context_copy.getSettingsRef().max_threads = 1;
context_copy.setSetting("max_streams_to_max_threads_ratio", 1);
context_copy.setSetting("max_threads", 1);
ASTPtr select_query = prepareQueryAffectedAST(commands);

View File

@ -1170,9 +1170,7 @@ TestResult check(const TestEntry & entry)
database->attachTable("visits_all", storage_distributed_visits);
database->attachTable("hits_all", storage_distributed_hits);
context.setCurrentDatabase("test");
auto & settings = context.getSettingsRef();
settings.distributed_product_mode = entry.mode;
context.setSetting("distributed_product_mode", entry.mode);
/// Parse and process the incoming query.
DB::ASTPtr ast_input;

View File

@ -958,8 +958,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
auto storage_from_source_part = StorageFromMergeTreeDataPart::create(source_part);
auto context_for_reading = context;
context_for_reading.getSettingsRef().max_streams_to_max_threads_ratio = 1;
context_for_reading.getSettingsRef().max_threads = 1;
context_for_reading.setSetting("max_streams_to_max_threads_ratio", 1);
context_for_reading.setSetting("max_threads", 1);
MutationCommands commands_for_part;
for (const auto & command : commands)

View File

@ -186,7 +186,7 @@ Pipes StorageMerge::read(
* since there is no certainty that it works when one of table is MergeTree and other is not.
*/
auto modified_context = std::make_shared<Context>(context);
modified_context->getSettingsRef().optimize_move_to_prewhere = false;
modified_context->setSetting("optimize_move_to_prewhere", false);
/// What will be result structure depending on query processed stage in source tables?
Block header = getQueryHeader(column_names, query_info, context, processed_stage);
@ -300,8 +300,8 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer
modified_query_info.query->as<ASTSelectQuery>()->replaceDatabaseAndTable(source_database, table_name);
/// Maximum permissible parallelism is streams_num
modified_context->getSettingsRef().max_threads = UInt64(streams_num);
modified_context->getSettingsRef().max_streams_to_max_threads_ratio = 1;
modified_context->setSetting("max_threads", streams_num);
modified_context->setSetting("max_streams_to_max_threads_ratio", 1);
InterpreterSelectQuery interpreter{modified_query_info.query, *modified_context, SelectQueryOptions(processed_stage)};

View File

@ -29,12 +29,14 @@ def cluster():
cluster.shutdown()
def run_query(instance, query, stdin=None, settings=None):
def run_query(instance, query, data=None, settings=None):
# type: (ClickHouseInstance, str, object, dict) -> str
logging.info("Running query '{}'...".format(query))
# use http to force parsing on server
result = instance.http_query(query, data=stdin, params=settings)
if not data:
data = " " # make POST request
result = instance.http_query(query, data=data, params=settings)
logging.info("Query finished")
return result
@ -64,7 +66,7 @@ def test_select(cluster):
'test_subject', schema, {'value': x}
)
buf.write(message)
stdin = buf.getvalue()
data = buf.getvalue()
instance = cluster.instances["dummy"] # type: ClickHouseInstance
schema_registry_url = "http://{}:{}".format(
@ -74,7 +76,7 @@ def test_select(cluster):
run_query(instance, "create table avro_data(value Int64) engine = Memory()")
settings = {'format_avro_schema_registry_url': schema_registry_url}
run_query(instance, "insert into avro_data format AvroConfluent", stdin, settings)
run_query(instance, "insert into avro_data format AvroConfluent", data, settings)
stdout = run_query(instance, "select * from avro_data")
assert list(map(str.split, stdout.splitlines())) == [
["0"],