mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
Fix updating of the access rights after changing of the readonly
setting. Remove non-const function Context::getSettingsRef().
This commit is contained in:
parent
5eb79c2303
commit
4af36ee5a4
@ -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())
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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");
|
||||
|
@ -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");
|
||||
|
@ -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);
|
||||
|
@ -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>
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
|
@ -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_;
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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)};
|
||||
|
||||
|
@ -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"],
|
||||
|
Loading…
Reference in New Issue
Block a user