mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Replace all Context references with std::weak_ptr (#22297)
* Replace all Context references with std::weak_ptr * Fix shared context captured by value * Fix build * Fix Context with named sessions * Fix copy context * Fix gcc build * Merge with master and fix build * Fix gcc-9 build
This commit is contained in:
parent
def996b508
commit
495c6e03aa
@ -200,8 +200,8 @@ int IBridge::main(const std::vector<std::string> & /*args*/)
|
||||
http_params->setKeepAliveTimeout(keep_alive_timeout);
|
||||
|
||||
auto shared_context = Context::createShared();
|
||||
Context context(Context::createGlobal(shared_context.get()));
|
||||
context.makeGlobalContext();
|
||||
auto context = Context::createGlobal(shared_context.get());
|
||||
context->makeGlobalContext();
|
||||
|
||||
if (config().has("query_masking_rules"))
|
||||
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
|
||||
|
@ -2,10 +2,11 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||
#include <Poco/Util/ServerApplication.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <daemon/BaseDaemon.h>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Util/ServerApplication.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -29,9 +30,9 @@ protected:
|
||||
|
||||
int main(const std::vector<std::string> & args) override;
|
||||
|
||||
virtual const std::string bridgeName() const = 0;
|
||||
virtual std::string bridgeName() const = 0;
|
||||
|
||||
virtual HandlerFactoryPtr getHandlerFactoryPtr(Context & context) const = 0;
|
||||
virtual HandlerFactoryPtr getHandlerFactoryPtr(ContextPtr context) const = 0;
|
||||
|
||||
size_t keep_alive_timeout;
|
||||
|
||||
|
@ -95,8 +95,8 @@ public:
|
||||
comparison_info_total.emplace_back(std::make_shared<Stats>());
|
||||
}
|
||||
|
||||
global_context.makeGlobalContext();
|
||||
global_context.setSettings(settings);
|
||||
global_context->makeGlobalContext();
|
||||
global_context->setSettings(settings);
|
||||
|
||||
std::cerr << std::fixed << std::setprecision(3);
|
||||
|
||||
@ -159,7 +159,7 @@ private:
|
||||
bool print_stacktrace;
|
||||
const Settings & settings;
|
||||
SharedContextHolder shared_context;
|
||||
Context global_context;
|
||||
ContextPtr global_context;
|
||||
QueryProcessingStage::Enum query_processing_stage;
|
||||
|
||||
/// Don't execute new queries after timelimit or SIGINT or exception
|
||||
|
@ -191,7 +191,7 @@ private:
|
||||
bool has_vertical_output_suffix = false; /// Is \G present at the end of the query string?
|
||||
|
||||
SharedContextHolder shared_context = Context::createShared();
|
||||
Context context = Context::createGlobal(shared_context.get());
|
||||
ContextPtr context = Context::createGlobal(shared_context.get());
|
||||
|
||||
/// Buffer that reads from stdin in batch mode.
|
||||
ReadBufferFromFileDescriptor std_in {STDIN_FILENO};
|
||||
@ -274,20 +274,20 @@ private:
|
||||
|
||||
configReadClient(config(), home_path);
|
||||
|
||||
context.setApplicationType(Context::ApplicationType::CLIENT);
|
||||
context.setQueryParameters(query_parameters);
|
||||
context->setApplicationType(Context::ApplicationType::CLIENT);
|
||||
context->setQueryParameters(query_parameters);
|
||||
|
||||
/// settings and limits could be specified in config file, but passed settings has higher priority
|
||||
for (const auto & setting : context.getSettingsRef().allUnchanged())
|
||||
for (const auto & setting : context->getSettingsRef().allUnchanged())
|
||||
{
|
||||
const auto & name = setting.getName();
|
||||
if (config().has(name))
|
||||
context.setSetting(name, config().getString(name));
|
||||
context->setSetting(name, config().getString(name));
|
||||
}
|
||||
|
||||
/// Set path for format schema files
|
||||
if (config().has("format_schema_path"))
|
||||
context.setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString());
|
||||
context->setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString());
|
||||
|
||||
/// Initialize query_id_formats if any
|
||||
if (config().has("query_id_formats"))
|
||||
@ -538,15 +538,15 @@ private:
|
||||
else
|
||||
format = config().getString("format", is_interactive ? "PrettyCompact" : "TabSeparated");
|
||||
|
||||
format_max_block_size = config().getInt("format_max_block_size", context.getSettingsRef().max_block_size);
|
||||
format_max_block_size = config().getInt("format_max_block_size", context->getSettingsRef().max_block_size);
|
||||
|
||||
insert_format = "Values";
|
||||
|
||||
/// Setting value from cmd arg overrides one from config
|
||||
if (context.getSettingsRef().max_insert_block_size.changed)
|
||||
insert_format_max_block_size = context.getSettingsRef().max_insert_block_size;
|
||||
if (context->getSettingsRef().max_insert_block_size.changed)
|
||||
insert_format_max_block_size = context->getSettingsRef().max_insert_block_size;
|
||||
else
|
||||
insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size);
|
||||
insert_format_max_block_size = config().getInt("insert_format_max_block_size", context->getSettingsRef().max_insert_block_size);
|
||||
|
||||
if (!is_interactive)
|
||||
{
|
||||
@ -555,7 +555,7 @@ private:
|
||||
ignore_error = config().getBool("ignore-error", false);
|
||||
}
|
||||
|
||||
ClientInfo & client_info = context.getClientInfo();
|
||||
ClientInfo & client_info = context->getClientInfo();
|
||||
client_info.setInitialQuery();
|
||||
client_info.quota_key = config().getString("quota_key", "");
|
||||
|
||||
@ -563,7 +563,7 @@ private:
|
||||
|
||||
/// Initialize DateLUT here to avoid counting time spent here as query execution time.
|
||||
const auto local_tz = DateLUT::instance().getTimeZone();
|
||||
if (!context.getSettingsRef().use_client_time_zone)
|
||||
if (!context->getSettingsRef().use_client_time_zone)
|
||||
{
|
||||
const auto & time_zone = connection->getServerTimezone(connection_parameters.timeouts);
|
||||
if (!time_zone.empty())
|
||||
@ -738,7 +738,7 @@ private:
|
||||
{
|
||||
auto query_id = config().getString("query_id", "");
|
||||
if (!query_id.empty())
|
||||
context.setCurrentQueryId(query_id);
|
||||
context->setCurrentQueryId(query_id);
|
||||
|
||||
nonInteractive();
|
||||
|
||||
@ -1038,7 +1038,7 @@ private:
|
||||
{
|
||||
Tokens tokens(this_query_begin, all_queries_end);
|
||||
IParser::Pos token_iterator(tokens,
|
||||
context.getSettingsRef().max_parser_depth);
|
||||
context->getSettingsRef().max_parser_depth);
|
||||
if (!token_iterator.isValid())
|
||||
{
|
||||
break;
|
||||
@ -1087,7 +1087,7 @@ private:
|
||||
if (ignore_error)
|
||||
{
|
||||
Tokens tokens(this_query_begin, all_queries_end);
|
||||
IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth);
|
||||
IParser::Pos token_iterator(tokens, context->getSettingsRef().max_parser_depth);
|
||||
while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid())
|
||||
++token_iterator;
|
||||
this_query_begin = token_iterator->end;
|
||||
@ -1133,7 +1133,7 @@ private:
|
||||
// beneficial so that we see proper trailing comments in "echo" and
|
||||
// server log.
|
||||
adjustQueryEnd(this_query_end, all_queries_end,
|
||||
context.getSettingsRef().max_parser_depth);
|
||||
context->getSettingsRef().max_parser_depth);
|
||||
|
||||
// full_query is the query + inline INSERT data + trailing comments
|
||||
// (the latter is our best guess for now).
|
||||
@ -1173,7 +1173,7 @@ private:
|
||||
{
|
||||
this_query_end = insert_ast->end;
|
||||
adjustQueryEnd(this_query_end, all_queries_end,
|
||||
context.getSettingsRef().max_parser_depth);
|
||||
context->getSettingsRef().max_parser_depth);
|
||||
}
|
||||
|
||||
// Now we know for sure where the query ends.
|
||||
@ -1290,7 +1290,7 @@ private:
|
||||
// Prints changed settings to stderr. Useful for debugging fuzzing failures.
|
||||
void printChangedSettings() const
|
||||
{
|
||||
const auto & changes = context.getSettingsRef().changes();
|
||||
const auto & changes = context->getSettingsRef().changes();
|
||||
if (!changes.empty())
|
||||
{
|
||||
fmt::print(stderr, "Changed settings: ");
|
||||
@ -1590,11 +1590,11 @@ private:
|
||||
if (is_interactive)
|
||||
{
|
||||
// Generate a new query_id
|
||||
context.setCurrentQueryId("");
|
||||
context->setCurrentQueryId("");
|
||||
for (const auto & query_id_format : query_id_formats)
|
||||
{
|
||||
writeString(query_id_format.first, std_out);
|
||||
writeString(fmt::format(query_id_format.second, fmt::arg("query_id", context.getCurrentQueryId())), std_out);
|
||||
writeString(fmt::format(query_id_format.second, fmt::arg("query_id", context->getCurrentQueryId())), std_out);
|
||||
writeChar('\n', std_out);
|
||||
std_out.next();
|
||||
}
|
||||
@ -1610,12 +1610,12 @@ private:
|
||||
{
|
||||
/// Temporarily apply query settings to context.
|
||||
std::optional<Settings> old_settings;
|
||||
SCOPE_EXIT_SAFE({ if (old_settings) context.setSettings(*old_settings); });
|
||||
SCOPE_EXIT_SAFE({ if (old_settings) context->setSettings(*old_settings); });
|
||||
auto apply_query_settings = [&](const IAST & settings_ast)
|
||||
{
|
||||
if (!old_settings)
|
||||
old_settings.emplace(context.getSettingsRef());
|
||||
context.applySettingsChanges(settings_ast.as<ASTSetQuery>()->changes);
|
||||
old_settings.emplace(context->getSettingsRef());
|
||||
context->applySettingsChanges(settings_ast.as<ASTSetQuery>()->changes);
|
||||
};
|
||||
const auto * insert = parsed_query->as<ASTInsertQuery>();
|
||||
if (insert && insert->settings_ast)
|
||||
@ -1653,7 +1653,7 @@ private:
|
||||
if (change.name == "profile")
|
||||
current_profile = change.value.safeGet<String>();
|
||||
else
|
||||
context.applySettingChange(change);
|
||||
context->applySettingChange(change);
|
||||
}
|
||||
}
|
||||
|
||||
@ -1725,10 +1725,10 @@ private:
|
||||
connection->sendQuery(
|
||||
connection_parameters.timeouts,
|
||||
query_to_send,
|
||||
context.getCurrentQueryId(),
|
||||
context->getCurrentQueryId(),
|
||||
query_processing_stage,
|
||||
&context.getSettingsRef(),
|
||||
&context.getClientInfo(),
|
||||
&context->getSettingsRef(),
|
||||
&context->getClientInfo(),
|
||||
true);
|
||||
|
||||
sendExternalTables();
|
||||
@ -1766,10 +1766,10 @@ private:
|
||||
connection->sendQuery(
|
||||
connection_parameters.timeouts,
|
||||
query_to_send,
|
||||
context.getCurrentQueryId(),
|
||||
context->getCurrentQueryId(),
|
||||
query_processing_stage,
|
||||
&context.getSettingsRef(),
|
||||
&context.getClientInfo(),
|
||||
&context->getSettingsRef(),
|
||||
&context->getClientInfo(),
|
||||
true);
|
||||
|
||||
sendExternalTables();
|
||||
@ -1792,7 +1792,7 @@ private:
|
||||
ParserQuery parser(end);
|
||||
ASTPtr res;
|
||||
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = context->getSettingsRef();
|
||||
size_t max_length = 0;
|
||||
if (!allow_multi_statements)
|
||||
max_length = settings.max_query_size;
|
||||
@ -1880,7 +1880,7 @@ private:
|
||||
current_format = insert->format;
|
||||
}
|
||||
|
||||
BlockInputStreamPtr block_input = context.getInputFormat(
|
||||
BlockInputStreamPtr block_input = context->getInputFormat(
|
||||
current_format, buf, sample, insert_format_max_block_size);
|
||||
|
||||
if (columns_description.hasDefaults())
|
||||
@ -2204,9 +2204,9 @@ private:
|
||||
|
||||
/// It is not clear how to write progress with parallel formatting. It may increase code complexity significantly.
|
||||
if (!need_render_progress)
|
||||
block_out_stream = context.getOutputStreamParallelIfPossible(current_format, *out_buf, block);
|
||||
block_out_stream = context->getOutputStreamParallelIfPossible(current_format, *out_buf, block);
|
||||
else
|
||||
block_out_stream = context.getOutputStream(current_format, *out_buf, block);
|
||||
block_out_stream = context->getOutputStream(current_format, *out_buf, block);
|
||||
|
||||
block_out_stream->writePrefix();
|
||||
}
|
||||
@ -2710,12 +2710,12 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
context.makeGlobalContext();
|
||||
context.setSettings(cmd_settings);
|
||||
context->makeGlobalContext();
|
||||
context->setSettings(cmd_settings);
|
||||
|
||||
/// Copy settings-related program options to config.
|
||||
/// TODO: Is this code necessary?
|
||||
for (const auto & setting : context.getSettingsRef().all())
|
||||
for (const auto & setting : context->getSettingsRef().all())
|
||||
{
|
||||
const auto & name = setting.getName();
|
||||
if (options.count(name))
|
||||
@ -2807,7 +2807,7 @@ public:
|
||||
{
|
||||
std::string traceparent = options["opentelemetry-traceparent"].as<std::string>();
|
||||
std::string error;
|
||||
if (!context.getClientInfo().client_trace_context.parseTraceparentHeader(
|
||||
if (!context->getClientInfo().client_trace_context.parseTraceparentHeader(
|
||||
traceparent, error))
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
@ -2818,7 +2818,7 @@ public:
|
||||
|
||||
if (options.count("opentelemetry-tracestate"))
|
||||
{
|
||||
context.getClientInfo().client_trace_context.tracestate =
|
||||
context->getClientInfo().client_trace_context.tracestate =
|
||||
options["opentelemetry-tracestate"].as<std::string>();
|
||||
}
|
||||
|
||||
|
@ -22,7 +22,7 @@ namespace ErrorCodes
|
||||
|
||||
void ClusterCopier::init()
|
||||
{
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
auto zookeeper = getContext()->getZooKeeper();
|
||||
|
||||
task_description_watch_callback = [this] (const Coordination::WatchResponse & response)
|
||||
{
|
||||
@ -39,14 +39,14 @@ void ClusterCopier::init()
|
||||
task_cluster_initial_config = task_cluster_current_config;
|
||||
|
||||
task_cluster->loadTasks(*task_cluster_initial_config);
|
||||
context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix);
|
||||
getContext()->setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix);
|
||||
|
||||
/// Set up shards and their priority
|
||||
task_cluster->random_engine.seed(task_cluster->random_device());
|
||||
for (auto & task_table : task_cluster->table_tasks)
|
||||
{
|
||||
task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name);
|
||||
task_table.cluster_push = context.getCluster(task_table.cluster_push_name);
|
||||
task_table.cluster_pull = getContext()->getCluster(task_table.cluster_pull_name);
|
||||
task_table.cluster_push = getContext()->getCluster(task_table.cluster_push_name);
|
||||
task_table.initShards(task_cluster->random_engine);
|
||||
}
|
||||
|
||||
@ -206,7 +206,7 @@ void ClusterCopier::uploadTaskDescription(const std::string & task_path, const s
|
||||
if (task_config_str.empty())
|
||||
return;
|
||||
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
auto zookeeper = getContext()->getZooKeeper();
|
||||
|
||||
zookeeper->createAncestors(local_task_description_path);
|
||||
auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent);
|
||||
@ -219,7 +219,7 @@ void ClusterCopier::uploadTaskDescription(const std::string & task_path, const s
|
||||
|
||||
void ClusterCopier::reloadTaskDescription()
|
||||
{
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
auto zookeeper = getContext()->getZooKeeper();
|
||||
task_description_watch_zookeeper = zookeeper;
|
||||
|
||||
String task_config_str;
|
||||
@ -235,7 +235,7 @@ void ClusterCopier::reloadTaskDescription()
|
||||
|
||||
/// Setup settings
|
||||
task_cluster->reloadSettings(*config);
|
||||
context.setSettings(task_cluster->settings_common);
|
||||
getContext()->setSettings(task_cluster->settings_common);
|
||||
|
||||
task_cluster_current_config = config;
|
||||
task_description_current_stat = stat;
|
||||
@ -440,7 +440,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons
|
||||
{
|
||||
LOG_DEBUG(log, "Check that all shards processed partition {} piece {} successfully", partition_name, piece_number);
|
||||
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
auto zookeeper = getContext()->getZooKeeper();
|
||||
|
||||
/// Collect all shards that contain partition piece number piece_number.
|
||||
Strings piece_status_paths;
|
||||
@ -532,7 +532,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
|
||||
|
||||
LOG_DEBUG(log, "Try to move {} to destination table", partition_name);
|
||||
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
auto zookeeper = getContext()->getZooKeeper();
|
||||
|
||||
const auto current_partition_attach_is_active = task_table.getPartitionAttachIsActivePath(partition_name);
|
||||
const auto current_partition_attach_is_done = task_table.getPartitionAttachIsDonePath(partition_name);
|
||||
@ -1095,7 +1095,7 @@ TaskStatus ClusterCopier::tryCreateDestinationTable(const ConnectionTimeouts & t
|
||||
= rewriteCreateQueryStorage(task_shard->current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast);
|
||||
auto & create = create_query_push_ast->as<ASTCreateQuery &>();
|
||||
create.if_not_exists = true;
|
||||
InterpreterCreateQuery::prepareOnClusterQuery(create, context, task_table.cluster_push_name);
|
||||
InterpreterCreateQuery::prepareOnClusterQuery(create, getContext(), task_table.cluster_push_name);
|
||||
String query = queryToString(create_query_push_ast);
|
||||
|
||||
LOG_DEBUG(log, "Create destination tables. Query: {}", query);
|
||||
@ -1211,7 +1211,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
|
||||
auto split_table_for_current_piece = task_shard.list_of_split_tables_on_shard[current_piece_number];
|
||||
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
auto zookeeper = getContext()->getZooKeeper();
|
||||
|
||||
const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath();
|
||||
const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath();
|
||||
@ -1262,7 +1262,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
|
||||
ParserQuery p_query(query.data() + query.size());
|
||||
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
return parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||
};
|
||||
|
||||
@ -1366,10 +1366,10 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
ASTPtr query_select_ast = get_select_query(split_table_for_current_piece, "count()", /*enable_splitting*/ true);
|
||||
UInt64 count;
|
||||
{
|
||||
Context local_context = context;
|
||||
auto local_context = Context::createCopy(context);
|
||||
// Use pull (i.e. readonly) settings, but fetch data from destination servers
|
||||
local_context.setSettings(task_cluster->settings_pull);
|
||||
local_context.setSetting("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().getInputStream());
|
||||
count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0;
|
||||
@ -1468,7 +1468,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " VALUES ";
|
||||
|
||||
ParserQuery p_query(query.data() + query.size());
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
query_insert_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||
|
||||
LOG_DEBUG(log, "Executing INSERT query: {}", query);
|
||||
@ -1476,18 +1476,18 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
|
||||
try
|
||||
{
|
||||
std::unique_ptr<Context> context_select = std::make_unique<Context>(context);
|
||||
auto context_select = Context::createCopy(context);
|
||||
context_select->setSettings(task_cluster->settings_pull);
|
||||
|
||||
std::unique_ptr<Context> context_insert = std::make_unique<Context>(context);
|
||||
auto context_insert = Context::createCopy(context);
|
||||
context_insert->setSettings(task_cluster->settings_push);
|
||||
|
||||
/// Custom INSERT SELECT implementation
|
||||
BlockInputStreamPtr input;
|
||||
BlockOutputStreamPtr output;
|
||||
{
|
||||
BlockIO io_select = InterpreterFactory::get(query_select_ast, *context_select)->execute();
|
||||
BlockIO io_insert = InterpreterFactory::get(query_insert_ast, *context_insert)->execute();
|
||||
BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute();
|
||||
BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute();
|
||||
|
||||
input = io_select.getInputStream();
|
||||
output = io_insert.out;
|
||||
@ -1581,7 +1581,7 @@ void ClusterCopier::dropAndCreateLocalTable(const ASTPtr & create_ast)
|
||||
const auto & create = create_ast->as<ASTCreateQuery &>();
|
||||
dropLocalTableIfExists({create.database, create.table});
|
||||
|
||||
InterpreterCreateQuery interpreter(create_ast, context);
|
||||
InterpreterCreateQuery interpreter(create_ast, getContext());
|
||||
interpreter.execute();
|
||||
}
|
||||
|
||||
@ -1592,7 +1592,7 @@ void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_na
|
||||
drop_ast->database = table_name.first;
|
||||
drop_ast->table = table_name.second;
|
||||
|
||||
InterpreterDropQuery interpreter(drop_ast, context);
|
||||
InterpreterDropQuery interpreter(drop_ast, getContext());
|
||||
interpreter.execute();
|
||||
}
|
||||
|
||||
@ -1654,8 +1654,8 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT
|
||||
|
||||
String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings & settings)
|
||||
{
|
||||
Context remote_context(context);
|
||||
remote_context.setSettings(settings);
|
||||
auto remote_context = Context::createCopy(context);
|
||||
remote_context->setSettings(settings);
|
||||
|
||||
String query = "SHOW CREATE TABLE " + getQuotedTable(table);
|
||||
Block block = getBlockWithAllStreamData(std::make_shared<RemoteBlockInputStream>(
|
||||
@ -1674,7 +1674,7 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time
|
||||
task_cluster->settings_pull);
|
||||
|
||||
ParserCreateQuery parser_create_query;
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
return parseQuery(parser_create_query, create_query_pull_str, settings.max_query_size, settings.max_parser_depth);
|
||||
}
|
||||
|
||||
@ -1703,7 +1703,7 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout
|
||||
/// Create special cluster with single shard
|
||||
String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name;
|
||||
ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster());
|
||||
context.setCluster(shard_read_cluster_name, cluster_pull_current_shard);
|
||||
getContext()->setCluster(shard_read_cluster_name, cluster_pull_current_shard);
|
||||
|
||||
auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second);
|
||||
|
||||
@ -1763,13 +1763,13 @@ std::set<String> ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti
|
||||
}
|
||||
|
||||
ParserQuery parser_query(query.data() + query.size());
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||
|
||||
LOG_DEBUG(log, "Computing destination partition set, executing query: {}", query);
|
||||
|
||||
Context local_context = context;
|
||||
local_context.setSettings(task_cluster->settings_pull);
|
||||
auto local_context = Context::createCopy(context);
|
||||
local_context->setSettings(task_cluster->settings_pull);
|
||||
Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().getInputStream());
|
||||
|
||||
if (block)
|
||||
@ -1809,11 +1809,11 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts,
|
||||
LOG_DEBUG(log, "Checking shard {} for partition {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, query);
|
||||
|
||||
ParserQuery parser_query(query.data() + query.size());
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||
|
||||
Context local_context = context;
|
||||
local_context.setSettings(task_cluster->settings_pull);
|
||||
auto local_context = Context::createCopy(context);
|
||||
local_context->setSettings(task_cluster->settings_pull);
|
||||
return InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows() != 0;
|
||||
}
|
||||
|
||||
@ -1848,11 +1848,11 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi
|
||||
LOG_DEBUG(log, "Checking shard {} for partition {} piece {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, std::to_string(current_piece_number), query);
|
||||
|
||||
ParserQuery parser_query(query.data() + query.size());
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||
|
||||
Context local_context = context;
|
||||
local_context.setSettings(task_cluster->settings_pull);
|
||||
auto local_context = Context::createCopy(context);
|
||||
local_context->setSettings(task_cluster->settings_pull);
|
||||
auto result = InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows();
|
||||
if (result != 0)
|
||||
LOG_DEBUG(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription());
|
||||
@ -1908,7 +1908,7 @@ UInt64 ClusterCopier::executeQueryOnCluster(
|
||||
/// In that case we don't have local replicas, but do it just in case
|
||||
for (UInt64 i = 0; i < num_local_replicas; ++i)
|
||||
{
|
||||
auto interpreter = InterpreterFactory::get(query_ast, context);
|
||||
auto interpreter = InterpreterFactory::get(query_ast, getContext());
|
||||
interpreter->execute();
|
||||
|
||||
if (increment_and_check_exit())
|
||||
@ -1923,8 +1923,8 @@ UInt64 ClusterCopier::executeQueryOnCluster(
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(shard_settings).getSaturated(shard_settings.max_execution_time);
|
||||
auto connections = shard.pool->getMany(timeouts, &shard_settings, pool_mode);
|
||||
|
||||
Context shard_context(context);
|
||||
shard_context.setSettings(shard_settings);
|
||||
auto shard_context = Context::createCopy(context);
|
||||
shard_context->setSettings(shard_settings);
|
||||
|
||||
for (auto & connection : connections)
|
||||
{
|
||||
|
@ -12,18 +12,17 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ClusterCopier
|
||||
class ClusterCopier : WithContext
|
||||
{
|
||||
public:
|
||||
ClusterCopier(const String & task_path_,
|
||||
const String & host_id_,
|
||||
const String & proxy_database_name_,
|
||||
Context & context_)
|
||||
:
|
||||
ContextPtr context_)
|
||||
: WithContext(context_),
|
||||
task_zookeeper_path(task_path_),
|
||||
host_id(host_id_),
|
||||
working_database_name(proxy_database_name_),
|
||||
context(context_),
|
||||
log(&Poco::Logger::get("ClusterCopier")) {}
|
||||
|
||||
void init();
|
||||
@ -36,7 +35,7 @@ public:
|
||||
/// Compute set of partitions, assume set of partitions aren't changed during the processing
|
||||
void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0);
|
||||
|
||||
void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force);
|
||||
void uploadTaskDescription(const std::string & task_path, const std::string & task_file, bool force);
|
||||
|
||||
void reloadTaskDescription();
|
||||
|
||||
@ -120,7 +119,7 @@ protected:
|
||||
/// Removes MATERIALIZED and ALIAS columns from create table query
|
||||
static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast);
|
||||
|
||||
bool tryDropPartitionPiece(ShardPartition & task_partition, const size_t current_piece_number,
|
||||
bool tryDropPartitionPiece(ShardPartition & task_partition, size_t current_piece_number,
|
||||
const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock);
|
||||
|
||||
static constexpr UInt64 max_table_tries = 3;
|
||||
@ -141,7 +140,7 @@ protected:
|
||||
|
||||
TaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts,
|
||||
ShardPartition & task_partition,
|
||||
const size_t current_piece_number,
|
||||
size_t current_piece_number,
|
||||
bool is_unprioritized_task);
|
||||
|
||||
void dropAndCreateLocalTable(const ASTPtr & create_ast);
|
||||
@ -219,7 +218,6 @@ private:
|
||||
|
||||
bool experimental_use_sample_offset{false};
|
||||
|
||||
Context & context;
|
||||
Poco::Logger * log;
|
||||
|
||||
std::chrono::milliseconds default_sleep_time{1000};
|
||||
|
@ -111,7 +111,7 @@ void ClusterCopierApp::mainImpl()
|
||||
LOG_INFO(log, "Starting clickhouse-copier (id {}, host_id {}, path {}, revision {})", process_id, host_id, process_path, ClickHouseRevision::getVersionRevision());
|
||||
|
||||
SharedContextHolder shared_context = Context::createShared();
|
||||
auto context = std::make_unique<Context>(Context::createGlobal(shared_context.get()));
|
||||
auto context = Context::createGlobal(shared_context.get());
|
||||
context->makeGlobalContext();
|
||||
SCOPE_EXIT_SAFE(context->shutdown());
|
||||
|
||||
@ -128,13 +128,13 @@ void ClusterCopierApp::mainImpl()
|
||||
registerFormats();
|
||||
|
||||
static const std::string default_database = "_local";
|
||||
DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared<DatabaseMemory>(default_database, *context));
|
||||
DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared<DatabaseMemory>(default_database, context));
|
||||
context->setCurrentDatabase(default_database);
|
||||
|
||||
/// Initialize query scope just in case.
|
||||
CurrentThread::QueryScope query_scope(*context);
|
||||
CurrentThread::QueryScope query_scope(context);
|
||||
|
||||
auto copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, *context);
|
||||
auto copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, context);
|
||||
copier->setSafeMode(is_safe_mode);
|
||||
copier->setCopyFaultProbability(copy_fault_probability);
|
||||
copier->setMoveFaultProbability(move_fault_probability);
|
||||
|
@ -102,8 +102,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
}
|
||||
|
||||
SharedContextHolder shared_context = Context::createShared();
|
||||
Context context = Context::createGlobal(shared_context.get());
|
||||
context.makeGlobalContext();
|
||||
auto context = Context::createGlobal(shared_context.get());
|
||||
context->makeGlobalContext();
|
||||
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
|
@ -16,7 +16,7 @@ namespace DB
|
||||
return std::make_unique<PingHandler>(keep_alive_timeout);
|
||||
|
||||
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)
|
||||
return std::make_unique<LibraryRequestHandler>(keep_alive_timeout, context);
|
||||
return std::make_unique<LibraryRequestHandler>(keep_alive_timeout, getContext());
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
@ -12,17 +12,17 @@ class SharedLibraryHandler;
|
||||
using SharedLibraryHandlerPtr = std::shared_ptr<SharedLibraryHandler>;
|
||||
|
||||
/// Factory for '/ping', '/' handlers.
|
||||
class LibraryBridgeHandlerFactory : public HTTPRequestHandlerFactory
|
||||
class LibraryBridgeHandlerFactory : public HTTPRequestHandlerFactory, WithContext
|
||||
{
|
||||
public:
|
||||
LibraryBridgeHandlerFactory(
|
||||
const std::string & name_,
|
||||
size_t keep_alive_timeout_,
|
||||
Context & context_)
|
||||
: log(&Poco::Logger::get(name_))
|
||||
ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get(name_))
|
||||
, name(name_)
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -32,7 +32,6 @@ private:
|
||||
Poco::Logger * log;
|
||||
std::string name;
|
||||
size_t keep_alive_timeout;
|
||||
Context & context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -131,7 +131,7 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe
|
||||
}
|
||||
|
||||
ReadBufferFromString read_block_buf(params.get("null_values"));
|
||||
auto format = FormatFactory::instance().getInput(FORMAT, read_block_buf, *sample_block, context, DEFAULT_BLOCK_SIZE);
|
||||
auto format = FormatFactory::instance().getInput(FORMAT, read_block_buf, *sample_block, getContext(), DEFAULT_BLOCK_SIZE);
|
||||
auto reader = std::make_shared<InputStreamFromInputFormat>(format);
|
||||
auto sample_block_with_nulls = reader->read();
|
||||
|
||||
@ -176,7 +176,7 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe
|
||||
const auto & sample_block = library_handler->getSampleBlock();
|
||||
auto input = library_handler->loadAll();
|
||||
|
||||
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, context);
|
||||
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, getContext());
|
||||
copyData(*input, *output);
|
||||
}
|
||||
else if (method == "loadIds")
|
||||
@ -193,7 +193,7 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe
|
||||
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
|
||||
const auto & sample_block = library_handler->getSampleBlock();
|
||||
auto input = library_handler->loadIds(ids);
|
||||
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, context);
|
||||
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, getContext());
|
||||
copyData(*input, *output);
|
||||
}
|
||||
else if (method == "loadKeys")
|
||||
@ -219,14 +219,14 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe
|
||||
}
|
||||
|
||||
auto & read_buf = request.getStream();
|
||||
auto format = FormatFactory::instance().getInput(FORMAT, read_buf, *requested_sample_block, context, DEFAULT_BLOCK_SIZE);
|
||||
auto format = FormatFactory::instance().getInput(FORMAT, read_buf, *requested_sample_block, getContext(), DEFAULT_BLOCK_SIZE);
|
||||
auto reader = std::make_shared<InputStreamFromInputFormat>(format);
|
||||
auto block = reader->read();
|
||||
|
||||
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
|
||||
const auto & sample_block = library_handler->getSampleBlock();
|
||||
auto input = library_handler->loadKeys(block.getColumns());
|
||||
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, context);
|
||||
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, getContext());
|
||||
copyData(*input, *output);
|
||||
}
|
||||
}
|
||||
|
@ -17,16 +17,16 @@ namespace DB
|
||||
/// names of dictionary attributes, sample block to parse block of null values, block of null values. Everything is
|
||||
/// passed in binary format and is urlencoded. When dictionary is cloned, a new handler is created.
|
||||
/// Each handler is unique to dictionary.
|
||||
class LibraryRequestHandler : public HTTPRequestHandler
|
||||
class LibraryRequestHandler : public HTTPRequestHandler, WithContext
|
||||
{
|
||||
public:
|
||||
|
||||
LibraryRequestHandler(
|
||||
size_t keep_alive_timeout_,
|
||||
Context & context_)
|
||||
: log(&Poco::Logger::get("LibraryRequestHandler"))
|
||||
ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get("LibraryRequestHandler"))
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -39,7 +39,6 @@ private:
|
||||
|
||||
Poco::Logger * log;
|
||||
size_t keep_alive_timeout;
|
||||
Context & context;
|
||||
};
|
||||
|
||||
|
||||
|
@ -12,12 +12,12 @@ class LibraryBridge : public IBridge
|
||||
{
|
||||
|
||||
protected:
|
||||
const std::string bridgeName() const override
|
||||
std::string bridgeName() const override
|
||||
{
|
||||
return "LibraryBridge";
|
||||
}
|
||||
|
||||
HandlerFactoryPtr getHandlerFactoryPtr(Context & context) const override
|
||||
HandlerFactoryPtr getHandlerFactoryPtr(ContextPtr context) const override
|
||||
{
|
||||
return std::make_shared<LibraryBridgeHandlerFactory>("LibraryRequestHandlerFactory-factory", keep_alive_timeout, context);
|
||||
}
|
||||
|
@ -99,9 +99,9 @@ void LocalServer::initialize(Poco::Util::Application & self)
|
||||
}
|
||||
}
|
||||
|
||||
void LocalServer::applyCmdSettings(Context & context)
|
||||
void LocalServer::applyCmdSettings(ContextPtr context)
|
||||
{
|
||||
context.applySettingsChanges(cmd_settings.changes());
|
||||
context->applySettingsChanges(cmd_settings.changes());
|
||||
}
|
||||
|
||||
/// If path is specified and not empty, will try to setup server environment and load existing metadata
|
||||
@ -176,7 +176,7 @@ void LocalServer::tryInitPath()
|
||||
}
|
||||
|
||||
|
||||
static void attachSystemTables(const Context & context)
|
||||
static void attachSystemTables(ContextPtr context)
|
||||
{
|
||||
DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(DatabaseCatalog::SYSTEM_DATABASE);
|
||||
if (!system_database)
|
||||
@ -211,7 +211,7 @@ try
|
||||
}
|
||||
|
||||
shared_context = Context::createShared();
|
||||
global_context = std::make_unique<Context>(Context::createGlobal(shared_context.get()));
|
||||
global_context = Context::createGlobal(shared_context.get());
|
||||
global_context->makeGlobalContext();
|
||||
global_context->setApplicationType(Context::ApplicationType::LOCAL);
|
||||
tryInitPath();
|
||||
@ -274,9 +274,9 @@ try
|
||||
* if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons.
|
||||
*/
|
||||
std::string default_database = config().getString("default_database", "_local");
|
||||
DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared<DatabaseMemory>(default_database, *global_context));
|
||||
DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared<DatabaseMemory>(default_database, global_context));
|
||||
global_context->setCurrentDatabase(default_database);
|
||||
applyCmdOptions(*global_context);
|
||||
applyCmdOptions(global_context);
|
||||
|
||||
if (config().has("path"))
|
||||
{
|
||||
@ -288,15 +288,15 @@ try
|
||||
LOG_DEBUG(log, "Loading metadata from {}", path);
|
||||
Poco::File(path + "data/").createDirectories();
|
||||
Poco::File(path + "metadata/").createDirectories();
|
||||
loadMetadataSystem(*global_context);
|
||||
attachSystemTables(*global_context);
|
||||
loadMetadata(*global_context);
|
||||
loadMetadataSystem(global_context);
|
||||
attachSystemTables(global_context);
|
||||
loadMetadata(global_context);
|
||||
DatabaseCatalog::instance().loadDatabases();
|
||||
LOG_DEBUG(log, "Loaded metadata.");
|
||||
}
|
||||
else if (!config().has("no-system-tables"))
|
||||
{
|
||||
attachSystemTables(*global_context);
|
||||
attachSystemTables(global_context);
|
||||
}
|
||||
|
||||
processQueries();
|
||||
@ -375,13 +375,13 @@ void LocalServer::processQueries()
|
||||
|
||||
/// we can't mutate global global_context (can lead to races, as it was already passed to some background threads)
|
||||
/// so we can't reuse it safely as a query context and need a copy here
|
||||
auto context = Context(*global_context);
|
||||
auto context = Context::createCopy(global_context);
|
||||
|
||||
context.makeSessionContext();
|
||||
context.makeQueryContext();
|
||||
context->makeSessionContext();
|
||||
context->makeQueryContext();
|
||||
|
||||
context.setUser("default", "", Poco::Net::SocketAddress{});
|
||||
context.setCurrentQueryId("");
|
||||
context->setUser("default", "", Poco::Net::SocketAddress{});
|
||||
context->setCurrentQueryId("");
|
||||
applyCmdSettings(context);
|
||||
|
||||
/// Use the same query_id (and thread group) for all queries
|
||||
@ -618,9 +618,9 @@ void LocalServer::init(int argc, char ** argv)
|
||||
argsToConfig(arguments, config(), 100);
|
||||
}
|
||||
|
||||
void LocalServer::applyCmdOptions(Context & context)
|
||||
void LocalServer::applyCmdOptions(ContextPtr context)
|
||||
{
|
||||
context.setDefaultFormat(config().getString("output-format", config().getString("format", "TSV")));
|
||||
context->setDefaultFormat(config().getString("output-format", config().getString("format", "TSV")));
|
||||
applyCmdSettings(context);
|
||||
}
|
||||
|
||||
|
@ -36,15 +36,15 @@ private:
|
||||
std::string getInitialCreateTableQuery();
|
||||
|
||||
void tryInitPath();
|
||||
void applyCmdOptions(Context & context);
|
||||
void applyCmdSettings(Context & context);
|
||||
void applyCmdOptions(ContextPtr context);
|
||||
void applyCmdSettings(ContextPtr context);
|
||||
void processQueries();
|
||||
void setupUsers();
|
||||
void cleanup();
|
||||
|
||||
protected:
|
||||
SharedContextHolder shared_context;
|
||||
std::unique_ptr<Context> global_context;
|
||||
ContextPtr global_context;
|
||||
|
||||
/// Settings specified via command line args
|
||||
Settings cmd_settings;
|
||||
|
@ -1129,8 +1129,8 @@ try
|
||||
}
|
||||
|
||||
SharedContextHolder shared_context = Context::createShared();
|
||||
Context context = Context::createGlobal(shared_context.get());
|
||||
context.makeGlobalContext();
|
||||
ContextPtr context = Context::createGlobal(shared_context.get());
|
||||
context->makeGlobalContext();
|
||||
|
||||
ReadBufferFromFileDescriptor file_in(STDIN_FILENO);
|
||||
WriteBufferFromFileDescriptor file_out(STDOUT_FILENO);
|
||||
@ -1152,7 +1152,7 @@ try
|
||||
if (!silent)
|
||||
std::cerr << "Training models\n";
|
||||
|
||||
BlockInputStreamPtr input = context.getInputFormat(input_format, file_in, header, max_block_size);
|
||||
BlockInputStreamPtr input = context->getInputFormat(input_format, file_in, header, max_block_size);
|
||||
|
||||
input->readPrefix();
|
||||
while (Block block = input->read())
|
||||
@ -1179,8 +1179,8 @@ try
|
||||
|
||||
file_in.seek(0, SEEK_SET);
|
||||
|
||||
BlockInputStreamPtr input = context.getInputFormat(input_format, file_in, header, max_block_size);
|
||||
BlockOutputStreamPtr output = context.getOutputStreamParallelIfPossible(output_format, file_out, header);
|
||||
BlockInputStreamPtr input = context->getInputFormat(input_format, file_in, header, max_block_size);
|
||||
BlockOutputStreamPtr output = context->getOutputStreamParallelIfPossible(output_format, file_out, header);
|
||||
|
||||
if (processed_rows + source_rows > limit)
|
||||
input = std::make_shared<LimitBlockInputStream>(input, limit - processed_rows, 0);
|
||||
|
@ -109,7 +109,7 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ
|
||||
|
||||
SCOPE_EXIT(SQLFreeStmt(hstmt, SQL_DROP));
|
||||
|
||||
const auto & context_settings = context.getSettingsRef();
|
||||
const auto & context_settings = getContext()->getSettingsRef();
|
||||
|
||||
/// TODO Why not do SQLColumns instead?
|
||||
std::string name = schema_name.empty() ? backQuoteIfNeed(table_name) : backQuoteIfNeed(schema_name) + "." + backQuoteIfNeed(table_name);
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#if USE_ODBC
|
||||
|
||||
# include <Interpreters/Context.h>
|
||||
# include <Interpreters/Context_fwd.h>
|
||||
# include <Server/HTTP/HTTPRequestHandler.h>
|
||||
# include <Common/config.h>
|
||||
|
||||
@ -15,11 +15,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ODBCColumnsInfoHandler : public HTTPRequestHandler
|
||||
class ODBCColumnsInfoHandler : public HTTPRequestHandler, WithContext
|
||||
{
|
||||
public:
|
||||
ODBCColumnsInfoHandler(size_t keep_alive_timeout_, Context & context_)
|
||||
: log(&Poco::Logger::get("ODBCColumnsInfoHandler")), keep_alive_timeout(keep_alive_timeout_), context(context_)
|
||||
ODBCColumnsInfoHandler(size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_), log(&Poco::Logger::get("ODBCColumnsInfoHandler")), keep_alive_timeout(keep_alive_timeout_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -28,7 +28,6 @@ public:
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
size_t keep_alive_timeout;
|
||||
Context & context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -21,26 +21,26 @@ std::unique_ptr<HTTPRequestHandler> ODBCBridgeHandlerFactory::createRequestHandl
|
||||
|
||||
if (uri.getPath() == "/columns_info")
|
||||
#if USE_ODBC
|
||||
return std::make_unique<ODBCColumnsInfoHandler>(keep_alive_timeout, context);
|
||||
return std::make_unique<ODBCColumnsInfoHandler>(keep_alive_timeout, getContext());
|
||||
#else
|
||||
return nullptr;
|
||||
#endif
|
||||
else if (uri.getPath() == "/identifier_quote")
|
||||
#if USE_ODBC
|
||||
return std::make_unique<IdentifierQuoteHandler>(keep_alive_timeout, context);
|
||||
return std::make_unique<IdentifierQuoteHandler>(keep_alive_timeout, getContext());
|
||||
#else
|
||||
return nullptr;
|
||||
#endif
|
||||
else if (uri.getPath() == "/schema_allowed")
|
||||
#if USE_ODBC
|
||||
return std::make_unique<SchemaAllowedHandler>(keep_alive_timeout, context);
|
||||
return std::make_unique<SchemaAllowedHandler>(keep_alive_timeout, getContext());
|
||||
#else
|
||||
return nullptr;
|
||||
#endif
|
||||
else if (uri.getPath() == "/write")
|
||||
return std::make_unique<ODBCHandler>(pool_map, keep_alive_timeout, context, "write");
|
||||
return std::make_unique<ODBCHandler>(pool_map, keep_alive_timeout, getContext(), "write");
|
||||
else
|
||||
return std::make_unique<ODBCHandler>(pool_map, keep_alive_timeout, context, "read");
|
||||
return std::make_unique<ODBCHandler>(pool_map, keep_alive_timeout, getContext(), "read");
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||
#include "ColumnInfoHandler.h"
|
||||
#include "IdentifierQuoteHandler.h"
|
||||
@ -20,11 +20,11 @@ namespace DB
|
||||
/** Factory for '/ping', '/', '/columns_info', '/identifier_quote', '/schema_allowed' handlers.
|
||||
* Also stores Session pools for ODBC connections
|
||||
*/
|
||||
class ODBCBridgeHandlerFactory : public HTTPRequestHandlerFactory
|
||||
class ODBCBridgeHandlerFactory : public HTTPRequestHandlerFactory, WithContext
|
||||
{
|
||||
public:
|
||||
ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, Context & context_)
|
||||
: log(&Poco::Logger::get(name_)), name(name_), keep_alive_timeout(keep_alive_timeout_), context(context_)
|
||||
ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_), log(&Poco::Logger::get(name_)), name(name_), keep_alive_timeout(keep_alive_timeout_)
|
||||
{
|
||||
pool_map = std::make_shared<ODBCHandler::PoolMap>();
|
||||
}
|
||||
@ -35,7 +35,6 @@ private:
|
||||
Poco::Logger * log;
|
||||
std::string name;
|
||||
size_t keep_alive_timeout;
|
||||
Context & context;
|
||||
std::shared_ptr<ODBCHandler::PoolMap> pool_map;
|
||||
};
|
||||
|
||||
|
@ -14,7 +14,7 @@ namespace DB
|
||||
class IdentifierQuoteHandler : public HTTPRequestHandler
|
||||
{
|
||||
public:
|
||||
IdentifierQuoteHandler(size_t keep_alive_timeout_, Context &)
|
||||
IdentifierQuoteHandler(size_t keep_alive_timeout_, ContextPtr)
|
||||
: log(&Poco::Logger::get("IdentifierQuoteHandler")), keep_alive_timeout(keep_alive_timeout_)
|
||||
{
|
||||
}
|
||||
|
@ -165,7 +165,7 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
|
||||
|
||||
auto pool = getPool(connection_string);
|
||||
auto & read_buf = request.getStream();
|
||||
auto input_format = FormatFactory::instance().getInput(format, read_buf, *sample_block, context, max_block_size);
|
||||
auto input_format = FormatFactory::instance().getInput(format, read_buf, *sample_block, getContext(), max_block_size);
|
||||
auto input_stream = std::make_shared<InputStreamFromInputFormat>(input_format);
|
||||
ODBCBlockOutputStream output_stream(pool->get(), db_name, table_name, *sample_block, quoting_style);
|
||||
copyData(*input_stream, output_stream);
|
||||
@ -176,7 +176,8 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
|
||||
std::string query = params.get("query");
|
||||
LOG_TRACE(log, "Query: {}", query);
|
||||
|
||||
BlockOutputStreamPtr writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, out, *sample_block, context);
|
||||
BlockOutputStreamPtr writer
|
||||
= FormatFactory::instance().getOutputStreamParallelIfPossible(format, out, *sample_block, getContext());
|
||||
auto pool = getPool(connection_string);
|
||||
ODBCBlockInputStream inp(pool->get(), query, *sample_block, max_block_size);
|
||||
copyData(inp, *writer);
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Server/HTTP/HTTPRequestHandler.h>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
@ -10,6 +10,10 @@
|
||||
#include <Poco/Data/SessionPool.h>
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Main handler for requests to ODBC driver
|
||||
@ -17,7 +21,7 @@ namespace DB
|
||||
* and also query in request body
|
||||
* response in RowBinary format
|
||||
*/
|
||||
class ODBCHandler : public HTTPRequestHandler
|
||||
class ODBCHandler : public HTTPRequestHandler, WithContext
|
||||
{
|
||||
public:
|
||||
using PoolPtr = std::shared_ptr<Poco::Data::SessionPool>;
|
||||
@ -25,12 +29,12 @@ public:
|
||||
|
||||
ODBCHandler(std::shared_ptr<PoolMap> pool_map_,
|
||||
size_t keep_alive_timeout_,
|
||||
Context & context_,
|
||||
ContextPtr context_,
|
||||
const String & mode_)
|
||||
: log(&Poco::Logger::get("ODBCHandler"))
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get("ODBCHandler"))
|
||||
, pool_map(pool_map_)
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
, context(context_)
|
||||
, mode(mode_)
|
||||
{
|
||||
}
|
||||
@ -42,7 +46,6 @@ private:
|
||||
|
||||
std::shared_ptr<PoolMap> pool_map;
|
||||
size_t keep_alive_timeout;
|
||||
Context & context;
|
||||
String mode;
|
||||
|
||||
static inline std::mutex mutex;
|
||||
|
@ -13,12 +13,12 @@ class ODBCBridge : public IBridge
|
||||
{
|
||||
|
||||
protected:
|
||||
const std::string bridgeName() const override
|
||||
std::string bridgeName() const override
|
||||
{
|
||||
return "ODBCBridge";
|
||||
}
|
||||
|
||||
HandlerFactoryPtr getHandlerFactoryPtr(Context & context) const override
|
||||
HandlerFactoryPtr getHandlerFactoryPtr(ContextPtr context) const override
|
||||
{
|
||||
return std::make_shared<ODBCBridgeHandlerFactory>("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context);
|
||||
}
|
||||
|
@ -15,7 +15,7 @@ class Context;
|
||||
class SchemaAllowedHandler : public HTTPRequestHandler
|
||||
{
|
||||
public:
|
||||
SchemaAllowedHandler(size_t keep_alive_timeout_, Context &)
|
||||
SchemaAllowedHandler(size_t keep_alive_timeout_, ContextPtr)
|
||||
: log(&Poco::Logger::get("SchemaAllowedHandler")), keep_alive_timeout(keep_alive_timeout_)
|
||||
{
|
||||
}
|
||||
|
@ -426,8 +426,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
* settings, available functions, data types, aggregate functions, databases, ...
|
||||
*/
|
||||
auto shared_context = Context::createShared();
|
||||
auto global_context = std::make_unique<Context>(Context::createGlobal(shared_context.get()));
|
||||
global_context_ptr = global_context.get();
|
||||
global_context = Context::createGlobal(shared_context.get());
|
||||
|
||||
global_context->makeGlobalContext();
|
||||
global_context->setApplicationType(Context::ApplicationType::SERVER);
|
||||
@ -934,7 +933,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
|
||||
* At this moment, no one could own shared part of Context.
|
||||
*/
|
||||
global_context_ptr = nullptr;
|
||||
global_context.reset();
|
||||
shared_context.reset();
|
||||
LOG_DEBUG(log, "Destroyed global context.");
|
||||
@ -948,14 +946,14 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
|
||||
try
|
||||
{
|
||||
loadMetadataSystem(*global_context);
|
||||
loadMetadataSystem(global_context);
|
||||
/// After attaching system databases we can initialize system log.
|
||||
global_context->initializeSystemLogs();
|
||||
auto & database_catalog = DatabaseCatalog::instance();
|
||||
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
|
||||
attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper);
|
||||
/// Then, load remaining databases
|
||||
loadMetadata(*global_context, default_database);
|
||||
loadMetadata(global_context, default_database);
|
||||
database_catalog.loadDatabases();
|
||||
/// After loading validate that default database exists
|
||||
database_catalog.assertDatabaseExists(default_database);
|
||||
@ -1035,7 +1033,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
else
|
||||
{
|
||||
/// Initialize a watcher periodically updating DNS cache
|
||||
dns_cache_updater = std::make_unique<DNSCacheUpdater>(*global_context, config().getInt("dns_cache_update_period", 15));
|
||||
dns_cache_updater = std::make_unique<DNSCacheUpdater>(global_context, config().getInt("dns_cache_update_period", 15));
|
||||
}
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
@ -1067,7 +1065,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
{
|
||||
/// This object will periodically calculate some metrics.
|
||||
AsynchronousMetrics async_metrics(
|
||||
*global_context, config().getUInt("asynchronous_metrics_update_period_s", 60), servers_to_start_before_tables, servers);
|
||||
global_context, config().getUInt("asynchronous_metrics_update_period_s", 60), servers_to_start_before_tables, servers);
|
||||
attachSystemTablesAsync(*DatabaseCatalog::instance().getSystemDatabase(), async_metrics);
|
||||
|
||||
for (const auto & listen_host : listen_hosts)
|
||||
@ -1330,7 +1328,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
int pool_size = config().getInt("distributed_ddl.pool_size", 1);
|
||||
if (pool_size < 1)
|
||||
throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
global_context->setDDLWorker(std::make_unique<DDLWorker>(pool_size, ddl_zookeeper_path, *global_context, &config(),
|
||||
global_context->setDDLWorker(std::make_unique<DDLWorker>(pool_size, ddl_zookeeper_path, global_context, &config(),
|
||||
"distributed_ddl", "DDLWorker", &CurrentMetrics::MaxDDLEntryID));
|
||||
}
|
||||
|
||||
|
@ -40,9 +40,9 @@ public:
|
||||
return BaseDaemon::logger();
|
||||
}
|
||||
|
||||
Context & context() const override
|
||||
ContextPtr context() const override
|
||||
{
|
||||
return *global_context_ptr;
|
||||
return global_context;
|
||||
}
|
||||
|
||||
bool isCancelled() const override
|
||||
@ -64,8 +64,7 @@ protected:
|
||||
std::string getDefaultCorePath() const override;
|
||||
|
||||
private:
|
||||
Context * global_context_ptr = nullptr;
|
||||
|
||||
ContextPtr global_context;
|
||||
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;
|
||||
|
||||
using CreateServerFunc = std::function<void(UInt16)>;
|
||||
|
@ -121,7 +121,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
|
||||
is_case_insensitive = true;
|
||||
}
|
||||
|
||||
const Context * query_context = nullptr;
|
||||
ContextPtr query_context;
|
||||
if (CurrentThread::isInitialized())
|
||||
query_context = CurrentThread::get().getQueryContext();
|
||||
|
||||
|
@ -146,7 +146,7 @@ void LinearModelData::predict(
|
||||
const ColumnsWithTypeAndName & arguments,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
const Context & context) const
|
||||
ContextPtr context) const
|
||||
{
|
||||
gradient_computer->predict(container, arguments, offset, limit, weights, bias, context);
|
||||
}
|
||||
@ -453,7 +453,7 @@ void LogisticRegression::predict(
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
const Context & /*context*/) const
|
||||
ContextPtr /*context*/) const
|
||||
{
|
||||
size_t rows_num = arguments.front().column->size();
|
||||
|
||||
@ -521,7 +521,7 @@ void LinearRegression::predict(
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
const Context & /*context*/) const
|
||||
ContextPtr /*context*/) const
|
||||
{
|
||||
if (weights.size() + 1 != arguments.size())
|
||||
{
|
||||
|
@ -3,10 +3,10 @@
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include "IAggregateFunction.h"
|
||||
|
||||
namespace DB
|
||||
@ -44,7 +44,7 @@ public:
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
const Context & context) const = 0;
|
||||
ContextPtr context) const = 0;
|
||||
};
|
||||
|
||||
|
||||
@ -69,7 +69,7 @@ public:
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
const Context & context) const override;
|
||||
ContextPtr context) const override;
|
||||
};
|
||||
|
||||
|
||||
@ -94,7 +94,7 @@ public:
|
||||
size_t limit,
|
||||
const std::vector<Float64> & weights,
|
||||
Float64 bias,
|
||||
const Context & context) const override;
|
||||
ContextPtr context) const override;
|
||||
};
|
||||
|
||||
|
||||
@ -264,7 +264,7 @@ public:
|
||||
const ColumnsWithTypeAndName & arguments,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
const Context & context) const;
|
||||
ContextPtr context) const;
|
||||
|
||||
void returnWeights(IColumn & to) const;
|
||||
private:
|
||||
@ -365,7 +365,7 @@ public:
|
||||
const ColumnsWithTypeAndName & arguments,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
const Context & context) const override
|
||||
ContextPtr context) const override
|
||||
{
|
||||
if (arguments.size() != param_num + 1)
|
||||
throw Exception(
|
||||
|
@ -1,21 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <common/types.h>
|
||||
|
||||
#include <cstddef>
|
||||
#include <memory>
|
||||
#include <type_traits>
|
||||
#include <vector>
|
||||
|
||||
#include <common/types.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
@ -122,7 +124,7 @@ public:
|
||||
const ColumnsWithTypeAndName & /*arguments*/,
|
||||
size_t /*offset*/,
|
||||
size_t /*limit*/,
|
||||
const Context & /*context*/) const
|
||||
ContextPtr /*context*/) const
|
||||
{
|
||||
throw Exception("Method predictValues is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
@ -14,7 +14,7 @@ namespace DB
|
||||
|
||||
/// Common base class for XDBC and Library bridge helpers.
|
||||
/// Contains helper methods to check/start bridge sync.
|
||||
class IBridgeHelper
|
||||
class IBridgeHelper: protected WithContext
|
||||
{
|
||||
|
||||
public:
|
||||
@ -27,6 +27,7 @@ public:
|
||||
static const inline std::string PING_METHOD = Poco::Net::HTTPRequest::HTTP_GET;
|
||||
static const inline std::string MAIN_METHOD = Poco::Net::HTTPRequest::HTTP_POST;
|
||||
|
||||
explicit IBridgeHelper(ContextPtr context_) : WithContext(context_) {}
|
||||
virtual ~IBridgeHelper() = default;
|
||||
|
||||
void startBridgeSync() const;
|
||||
@ -38,9 +39,9 @@ public:
|
||||
|
||||
protected:
|
||||
/// clickhouse-odbc-bridge, clickhouse-library-bridge
|
||||
virtual const String serviceAlias() const = 0;
|
||||
virtual String serviceAlias() const = 0;
|
||||
|
||||
virtual const String serviceFileName() const = 0;
|
||||
virtual String serviceFileName() const = 0;
|
||||
|
||||
virtual size_t getDefaultPort() const = 0;
|
||||
|
||||
@ -48,9 +49,7 @@ protected:
|
||||
|
||||
virtual void startBridge(std::unique_ptr<ShellCommand> cmd) const = 0;
|
||||
|
||||
virtual const String configPrefix() const = 0;
|
||||
|
||||
virtual const Context & getContext() const = 0;
|
||||
virtual String configPrefix() const = 0;
|
||||
|
||||
virtual const Poco::Util::AbstractConfiguration & getConfig() const = 0;
|
||||
|
||||
|
@ -21,14 +21,14 @@ namespace DB
|
||||
{
|
||||
|
||||
LibraryBridgeHelper::LibraryBridgeHelper(
|
||||
const Context & context_,
|
||||
ContextPtr context_,
|
||||
const Block & sample_block_,
|
||||
const Field & dictionary_id_)
|
||||
: log(&Poco::Logger::get("LibraryBridgeHelper"))
|
||||
, context(context_)
|
||||
: IBridgeHelper(context_)
|
||||
, log(&Poco::Logger::get("LibraryBridgeHelper"))
|
||||
, sample_block(sample_block_)
|
||||
, config(context.getConfigRef())
|
||||
, http_timeout(context.getSettingsRef().http_receive_timeout.value.totalSeconds())
|
||||
, config(context_->getConfigRef())
|
||||
, http_timeout(context_->getSettingsRef().http_receive_timeout.value.totalSeconds())
|
||||
, dictionary_id(dictionary_id_)
|
||||
{
|
||||
bridge_port = config.getUInt("library_bridge.port", DEFAULT_PORT);
|
||||
@ -57,7 +57,7 @@ Poco::URI LibraryBridgeHelper::createBaseURI() const
|
||||
|
||||
void LibraryBridgeHelper::startBridge(std::unique_ptr<ShellCommand> cmd) const
|
||||
{
|
||||
context.addBridgeCommand(std::move(cmd));
|
||||
getContext()->addBridgeCommand(std::move(cmd));
|
||||
}
|
||||
|
||||
|
||||
@ -68,7 +68,7 @@ bool LibraryBridgeHelper::initLibrary(const std::string & library_path, const st
|
||||
|
||||
/// Sample block must contain null values
|
||||
WriteBufferFromOwnString out;
|
||||
auto output_stream = context.getOutputStream(LibraryBridgeHelper::DEFAULT_FORMAT, out, sample_block);
|
||||
auto output_stream = getContext()->getOutputStream(LibraryBridgeHelper::DEFAULT_FORMAT, out, sample_block);
|
||||
formatBlock(output_stream, sample_block);
|
||||
auto block_string = out.str();
|
||||
|
||||
@ -142,8 +142,7 @@ BlockInputStreamPtr LibraryBridgeHelper::loadKeys(const Block & requested_block)
|
||||
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [requested_block, this](std::ostream & os)
|
||||
{
|
||||
WriteBufferFromOStream out_buffer(os);
|
||||
auto output_stream = context.getOutputStream(
|
||||
LibraryBridgeHelper::DEFAULT_FORMAT, out_buffer, sample_block);
|
||||
auto output_stream = getContext()->getOutputStream(LibraryBridgeHelper::DEFAULT_FORMAT, out_buffer, sample_block);
|
||||
formatBlock(output_stream, requested_block);
|
||||
};
|
||||
return loadBase(uri, out_stream_callback);
|
||||
@ -156,7 +155,7 @@ bool LibraryBridgeHelper::executeRequest(const Poco::URI & uri, ReadWriteBufferF
|
||||
uri,
|
||||
Poco::Net::HTTPRequest::HTTP_POST,
|
||||
std::move(out_stream_callback),
|
||||
ConnectionTimeouts::getHTTPTimeouts(context));
|
||||
ConnectionTimeouts::getHTTPTimeouts(getContext()));
|
||||
|
||||
bool res;
|
||||
readBoolText(res, buf);
|
||||
@ -170,13 +169,13 @@ BlockInputStreamPtr LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWri
|
||||
uri,
|
||||
Poco::Net::HTTPRequest::HTTP_POST,
|
||||
std::move(out_stream_callback),
|
||||
ConnectionTimeouts::getHTTPTimeouts(context),
|
||||
ConnectionTimeouts::getHTTPTimeouts(getContext()),
|
||||
0,
|
||||
Poco::Net::HTTPBasicCredentials{},
|
||||
DBMS_DEFAULT_BUFFER_SIZE,
|
||||
ReadWriteBufferFromHTTP::HTTPHeaderEntries{});
|
||||
|
||||
auto input_stream = context.getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE);
|
||||
auto input_stream = getContext()->getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE);
|
||||
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(read_buf_ptr));
|
||||
}
|
||||
|
||||
|
@ -17,9 +17,9 @@ class LibraryBridgeHelper : public IBridgeHelper
|
||||
public:
|
||||
static constexpr inline size_t DEFAULT_PORT = 9012;
|
||||
|
||||
LibraryBridgeHelper(const Context & context_, const Block & sample_block, const Field & dictionary_id_);
|
||||
LibraryBridgeHelper(ContextPtr context_, const Block & sample_block, const Field & dictionary_id_);
|
||||
|
||||
bool initLibrary(const std::string & library_path, const std::string library_settings, const std::string attributes_names);
|
||||
bool initLibrary(const std::string & library_path, std::string library_settings, std::string attributes_names);
|
||||
|
||||
bool cloneLibrary(const Field & other_dictionary_id);
|
||||
|
||||
@ -31,7 +31,7 @@ public:
|
||||
|
||||
BlockInputStreamPtr loadAll();
|
||||
|
||||
BlockInputStreamPtr loadIds(const std::string ids_string);
|
||||
BlockInputStreamPtr loadIds(std::string ids_string);
|
||||
|
||||
BlockInputStreamPtr loadKeys(const Block & requested_block);
|
||||
|
||||
@ -43,17 +43,15 @@ public:
|
||||
protected:
|
||||
void startBridge(std::unique_ptr<ShellCommand> cmd) const override;
|
||||
|
||||
const String serviceAlias() const override { return "clickhouse-library-bridge"; }
|
||||
String serviceAlias() const override { return "clickhouse-library-bridge"; }
|
||||
|
||||
const String serviceFileName() const override { return serviceAlias(); }
|
||||
String serviceFileName() const override { return serviceAlias(); }
|
||||
|
||||
size_t getDefaultPort() const override { return DEFAULT_PORT; }
|
||||
|
||||
bool startBridgeManually() const override { return false; }
|
||||
|
||||
const String configPrefix() const override { return "library_bridge"; }
|
||||
|
||||
const Context & getContext() const override { return context; }
|
||||
String configPrefix() const override { return "library_bridge"; }
|
||||
|
||||
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
|
||||
|
||||
@ -76,7 +74,6 @@ private:
|
||||
Poco::URI createRequestURI(const String & method) const;
|
||||
|
||||
Poco::Logger * log;
|
||||
const Context & context;
|
||||
const Block sample_block;
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
const Poco::Timespan http_timeout;
|
||||
@ -85,4 +82,5 @@ private:
|
||||
std::string bridge_host;
|
||||
size_t bridge_port;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -35,6 +35,8 @@ class IXDBCBridgeHelper : public IBridgeHelper
|
||||
{
|
||||
|
||||
public:
|
||||
explicit IXDBCBridgeHelper(ContextPtr context_) : IBridgeHelper(context_) {}
|
||||
|
||||
virtual std::vector<std::pair<std::string, std::string>> getURLParams(const std::string & cols, UInt64 max_block_size) const = 0;
|
||||
|
||||
virtual Poco::URI getColumnsInfoURI() const = 0;
|
||||
@ -43,7 +45,7 @@ public:
|
||||
|
||||
virtual bool isSchemaAllowed() = 0;
|
||||
|
||||
virtual const String getName() const = 0;
|
||||
virtual String getName() const = 0;
|
||||
};
|
||||
|
||||
using BridgeHelperPtr = std::shared_ptr<IXDBCBridgeHelper>;
|
||||
@ -60,14 +62,14 @@ public:
|
||||
static constexpr inline auto SCHEMA_ALLOWED_HANDLER = "/schema_allowed";
|
||||
|
||||
XDBCBridgeHelper(
|
||||
const Context & global_context_,
|
||||
ContextPtr global_context_,
|
||||
const Poco::Timespan & http_timeout_,
|
||||
const std::string & connection_string_)
|
||||
: log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"))
|
||||
: IXDBCBridgeHelper(global_context_)
|
||||
, log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"))
|
||||
, connection_string(connection_string_)
|
||||
, http_timeout(http_timeout_)
|
||||
, context(global_context_)
|
||||
, config(context.getConfigRef())
|
||||
, config(global_context_->getConfigRef())
|
||||
{
|
||||
bridge_host = config.getString(BridgeHelperMixin::configPrefix() + ".host", DEFAULT_HOST);
|
||||
bridge_port = config.getUInt(BridgeHelperMixin::configPrefix() + ".port", DEFAULT_PORT);
|
||||
@ -77,18 +79,16 @@ public:
|
||||
protected:
|
||||
auto getConnectionString() const { return connection_string; }
|
||||
|
||||
const String getName() const override { return BridgeHelperMixin::getName(); }
|
||||
String getName() const override { return BridgeHelperMixin::getName(); }
|
||||
|
||||
size_t getDefaultPort() const override { return DEFAULT_PORT; }
|
||||
|
||||
const String serviceAlias() const override { return BridgeHelperMixin::serviceAlias(); }
|
||||
String serviceAlias() const override { return BridgeHelperMixin::serviceAlias(); }
|
||||
|
||||
/// Same for odbc and jdbc
|
||||
const String serviceFileName() const override { return "clickhouse-odbc-bridge"; }
|
||||
String serviceFileName() const override { return "clickhouse-odbc-bridge"; }
|
||||
|
||||
const String configPrefix() const override { return BridgeHelperMixin::configPrefix(); }
|
||||
|
||||
const Context & getContext() const override { return context; }
|
||||
String configPrefix() const override { return BridgeHelperMixin::configPrefix(); }
|
||||
|
||||
const Poco::Timespan & getHTTPTimeout() const override { return http_timeout; }
|
||||
|
||||
@ -109,7 +109,7 @@ protected:
|
||||
|
||||
void startBridge(std::unique_ptr<ShellCommand> cmd) const override
|
||||
{
|
||||
context.addBridgeCommand(std::move(cmd));
|
||||
getContext()->addBridgeCommand(std::move(cmd));
|
||||
}
|
||||
|
||||
|
||||
@ -122,7 +122,6 @@ private:
|
||||
std::string bridge_host;
|
||||
size_t bridge_port;
|
||||
|
||||
const Context & context;
|
||||
const Configuration & config;
|
||||
|
||||
std::optional<IdentifierQuotingStyle> quote_style;
|
||||
@ -160,8 +159,7 @@ protected:
|
||||
uri.setPath(SCHEMA_ALLOWED_HANDLER);
|
||||
uri.addQueryParameter("connection_string", getConnectionString());
|
||||
|
||||
ReadWriteBufferFromHTTP buf(
|
||||
uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context));
|
||||
ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(getContext()));
|
||||
|
||||
bool res;
|
||||
readBoolText(res, buf);
|
||||
@ -181,12 +179,14 @@ protected:
|
||||
uri.setPath(IDENTIFIER_QUOTE_HANDLER);
|
||||
uri.addQueryParameter("connection_string", getConnectionString());
|
||||
|
||||
ReadWriteBufferFromHTTP buf(
|
||||
uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context));
|
||||
ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(getContext()));
|
||||
|
||||
std::string character;
|
||||
readStringBinary(character, buf);
|
||||
if (character.length() > 1)
|
||||
throw Exception("Failed to parse quoting style from '" + character + "' for service " + BridgeHelperMixin::serviceAlias(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(
|
||||
"Failed to parse quoting style from '" + character + "' for service " + BridgeHelperMixin::serviceAlias(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else if (character.length() == 0)
|
||||
quote_style = IdentifierQuotingStyle::None;
|
||||
else if (character[0] == '`')
|
||||
@ -206,17 +206,17 @@ struct JDBCBridgeMixin
|
||||
{
|
||||
static constexpr inline auto DEFAULT_PORT = 9019;
|
||||
|
||||
static const String configPrefix()
|
||||
static String configPrefix()
|
||||
{
|
||||
return "jdbc_bridge";
|
||||
}
|
||||
|
||||
static const String serviceAlias()
|
||||
static String serviceAlias()
|
||||
{
|
||||
return "clickhouse-jdbc-bridge";
|
||||
}
|
||||
|
||||
static const String getName()
|
||||
static String getName()
|
||||
{
|
||||
return "JDBC";
|
||||
}
|
||||
@ -237,17 +237,17 @@ struct ODBCBridgeMixin
|
||||
{
|
||||
static constexpr inline auto DEFAULT_PORT = 9018;
|
||||
|
||||
static const String configPrefix()
|
||||
static String configPrefix()
|
||||
{
|
||||
return "odbc_bridge";
|
||||
}
|
||||
|
||||
static const String serviceAlias()
|
||||
static String serviceAlias()
|
||||
{
|
||||
return "clickhouse-odbc-bridge";
|
||||
}
|
||||
|
||||
static const String getName()
|
||||
static String getName()
|
||||
{
|
||||
return "ODBC";
|
||||
}
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include <IO/ReadBufferFromPocoSocket.h>
|
||||
|
||||
#include <Interpreters/TablesStatus.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
#include <Compression/ICompressionCodec.h>
|
||||
|
||||
@ -52,8 +53,6 @@ class Connection;
|
||||
using ConnectionPtr = std::shared_ptr<Connection>;
|
||||
using Connections = std::vector<ConnectionPtr>;
|
||||
|
||||
using Scalars = std::map<String, Block>;
|
||||
|
||||
|
||||
/// Packet that could be received from server.
|
||||
struct Packet
|
||||
@ -111,7 +110,7 @@ public:
|
||||
setDescription();
|
||||
}
|
||||
|
||||
virtual ~Connection() {}
|
||||
virtual ~Connection() = default;
|
||||
|
||||
/// Set throttler of network traffic. One throttler could be used for multiple connections to limit total traffic.
|
||||
void setThrottler(const ThrottlerPtr & throttler_)
|
||||
|
@ -162,7 +162,7 @@ MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr colum
|
||||
return res;
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnAggregateFunction::predictValues(const ColumnsWithTypeAndName & arguments, const Context & context) const
|
||||
MutableColumnPtr ColumnAggregateFunction::predictValues(const ColumnsWithTypeAndName & arguments, ContextPtr context) const
|
||||
{
|
||||
MutableColumnPtr res = func->getReturnTypeToPredict()->createColumn();
|
||||
res->reserve(data.size());
|
||||
|
@ -82,7 +82,7 @@ private:
|
||||
/// Name of the type to distinguish different aggregation states.
|
||||
String type_string;
|
||||
|
||||
ColumnAggregateFunction() {}
|
||||
ColumnAggregateFunction() = default;
|
||||
|
||||
/// Create a new column that has another column as a source.
|
||||
MutablePtr createView() const;
|
||||
@ -119,7 +119,7 @@ public:
|
||||
const char * getFamilyName() const override { return "AggregateFunction"; }
|
||||
TypeIndex getDataType() const override { return TypeIndex::AggregateFunction; }
|
||||
|
||||
MutableColumnPtr predictValues(const ColumnsWithTypeAndName & arguments, const Context & context) const;
|
||||
MutableColumnPtr predictValues(const ColumnsWithTypeAndName & arguments, ContextPtr context) const;
|
||||
|
||||
size_t size() const override
|
||||
{
|
||||
|
@ -1,11 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <common/StringRef.h>
|
||||
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
#include <common/StringRef.h>
|
||||
#include <Common/ThreadStatus.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -18,7 +19,6 @@ class MemoryTracker;
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class QueryStatus;
|
||||
struct Progress;
|
||||
class InternalTextLogsQueue;
|
||||
@ -87,7 +87,7 @@ public:
|
||||
/// Initializes query with current thread as master thread in constructor, and detaches it in destructor
|
||||
struct QueryScope
|
||||
{
|
||||
explicit QueryScope(Context & query_context);
|
||||
explicit QueryScope(ContextPtr query_context);
|
||||
~QueryScope();
|
||||
|
||||
void logPeakMemoryUsage();
|
||||
@ -99,7 +99,7 @@ private:
|
||||
|
||||
/// Sets query_context for current thread group
|
||||
/// Can by used only through QueryScope
|
||||
static void attachQueryContext(Context & query_context);
|
||||
static void attachQueryContext(ContextPtr query_context);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -101,7 +101,8 @@ ThreadStatus::~ThreadStatus()
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
/// It may cause segfault if query_context was destroyed, but was not detached
|
||||
assert((!query_context && query_id.empty()) || (query_context && query_id == query_context->getCurrentQueryId()));
|
||||
auto query_context_ptr = query_context.lock();
|
||||
assert((!query_context_ptr && query_id.empty()) || (query_context_ptr && query_id == query_context_ptr->getCurrentQueryId()));
|
||||
#endif
|
||||
|
||||
if (deleter)
|
||||
|
@ -1,20 +1,20 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/StringRef.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <IO/Progress.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/OpenTelemetryTraceContext.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <common/StringRef.h>
|
||||
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
#include <IO/Progress.h>
|
||||
|
||||
#include <memory>
|
||||
#include <functional>
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <shared_mutex>
|
||||
#include <functional>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
|
||||
namespace Poco
|
||||
@ -26,7 +26,6 @@ namespace Poco
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class QueryStatus;
|
||||
class ThreadStatus;
|
||||
class QueryProfilerReal;
|
||||
@ -58,8 +57,8 @@ public:
|
||||
ProfileEvents::Counters performance_counters{VariableContext::Process};
|
||||
MemoryTracker memory_tracker{VariableContext::Process};
|
||||
|
||||
Context * query_context = nullptr;
|
||||
Context * global_context = nullptr;
|
||||
ContextWeakPtr query_context;
|
||||
ContextWeakPtr global_context;
|
||||
|
||||
InternalTextLogsQueueWeakPtr logs_queue_ptr;
|
||||
std::function<void()> fatal_error_callback;
|
||||
@ -122,9 +121,9 @@ protected:
|
||||
std::atomic<int> thread_state{ThreadState::DetachedFromQuery};
|
||||
|
||||
/// Is set once
|
||||
Context * global_context = nullptr;
|
||||
ContextWeakPtr global_context;
|
||||
/// Use it only from current thread
|
||||
Context * query_context = nullptr;
|
||||
ContextWeakPtr query_context;
|
||||
|
||||
String query_id;
|
||||
|
||||
@ -178,9 +177,9 @@ public:
|
||||
return query_id;
|
||||
}
|
||||
|
||||
const Context * getQueryContext() const
|
||||
auto getQueryContext() const
|
||||
{
|
||||
return query_context;
|
||||
return query_context.lock();
|
||||
}
|
||||
|
||||
/// Starts new query and create new thread group for it, current thread becomes master thread of the query
|
||||
@ -203,7 +202,7 @@ public:
|
||||
|
||||
/// Sets query context for current master thread and its thread group
|
||||
/// NOTE: query_context have to be alive until detachQuery() is called
|
||||
void attachQueryContext(Context & query_context);
|
||||
void attachQueryContext(ContextPtr query_context);
|
||||
|
||||
/// Update several ProfileEvents counters
|
||||
void updatePerformanceCounters();
|
||||
|
@ -5,14 +5,14 @@
|
||||
struct ContextHolder
|
||||
{
|
||||
DB::SharedContextHolder shared_context;
|
||||
DB::Context context;
|
||||
DB::ContextPtr context;
|
||||
|
||||
ContextHolder()
|
||||
: shared_context(DB::Context::createShared())
|
||||
, context(DB::Context::createGlobal(shared_context.get()))
|
||||
{
|
||||
context.makeGlobalContext();
|
||||
context.setPath("./");
|
||||
context->makeGlobalContext();
|
||||
context->setPath("./");
|
||||
}
|
||||
|
||||
ContextHolder(ContextHolder &&) = default;
|
||||
|
@ -1,15 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <list>
|
||||
#include <set>
|
||||
#include <map>
|
||||
#include <initializer_list>
|
||||
|
||||
#include <Core/BlockInfo.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <Core/ColumnsWithTypeAndName.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
|
||||
#include <initializer_list>
|
||||
#include <list>
|
||||
#include <map>
|
||||
#include <set>
|
||||
#include <vector>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -22,8 +22,6 @@ namespace DB
|
||||
* Allows to insert, remove columns in arbitrary position, to change order of columns.
|
||||
*/
|
||||
|
||||
class Context;
|
||||
|
||||
class Block
|
||||
{
|
||||
private:
|
||||
|
@ -1,9 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
|
||||
#include <common/types.h>
|
||||
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -31,11 +31,11 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
ExternalTableDataPtr BaseExternalTable::getData(const Context & context)
|
||||
ExternalTableDataPtr BaseExternalTable::getData(ContextPtr context)
|
||||
{
|
||||
initReadBuffer();
|
||||
initSampleBlock();
|
||||
auto input = context.getInputFormat(format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE);
|
||||
auto input = context->getInputFormat(format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE);
|
||||
auto stream = std::make_shared<AsynchronousBlockInputStream>(input);
|
||||
|
||||
auto data = std::make_unique<ExternalTableData>();
|
||||
@ -127,7 +127,7 @@ ExternalTable::ExternalTable(const boost::program_options::variables_map & exter
|
||||
|
||||
void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, ReadBuffer & stream)
|
||||
{
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = getContext()->getSettingsRef();
|
||||
|
||||
if (settings.http_max_multipart_form_data_size)
|
||||
read_buffer = std::make_unique<LimitReadBuffer>(
|
||||
@ -152,14 +152,14 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header,
|
||||
else
|
||||
throw Exception("Neither structure nor types have not been provided for external table " + name + ". Use fields " + name + "_structure or " + name + "_types to do so.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
ExternalTableDataPtr data = getData(context);
|
||||
ExternalTableDataPtr data = getData(getContext());
|
||||
|
||||
/// Create table
|
||||
NamesAndTypesList columns = sample_block.getNamesAndTypesList();
|
||||
auto temporary_table = TemporaryTableHolder(context, ColumnsDescription{columns}, {});
|
||||
auto temporary_table = TemporaryTableHolder(getContext(), ColumnsDescription{columns}, {});
|
||||
auto storage = temporary_table.getTable();
|
||||
context.addExternalTable(data->table_name, std::move(temporary_table));
|
||||
BlockOutputStreamPtr output = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), context);
|
||||
getContext()->addExternalTable(data->table_name, std::move(temporary_table));
|
||||
BlockOutputStreamPtr output = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext());
|
||||
|
||||
/// Write data
|
||||
data->pipe->resize(1);
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Client/Connection.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <Server/HTTP/HTMLForm.h>
|
||||
|
||||
@ -11,30 +12,21 @@
|
||||
#include <vector>
|
||||
|
||||
|
||||
namespace Poco
|
||||
namespace Poco::Net
|
||||
{
|
||||
namespace Net
|
||||
{
|
||||
class NameValueCollection;
|
||||
class MessageHeader;
|
||||
}
|
||||
class NameValueCollection;
|
||||
class MessageHeader;
|
||||
}
|
||||
|
||||
namespace boost
|
||||
namespace boost::program_options
|
||||
{
|
||||
namespace program_options
|
||||
{
|
||||
class variables_map;
|
||||
}
|
||||
class variables_map;
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
|
||||
/// The base class containing the basic information about external table and
|
||||
/// basic functions for extracting this information from text fields.
|
||||
class BaseExternalTable
|
||||
@ -56,7 +48,7 @@ public:
|
||||
virtual void initReadBuffer() {}
|
||||
|
||||
/// Get the table data - a pair (a stream with the contents of the table, the name of the table)
|
||||
ExternalTableDataPtr getData(const Context & context);
|
||||
ExternalTableDataPtr getData(ContextPtr context);
|
||||
|
||||
protected:
|
||||
/// Clear all accumulated information
|
||||
@ -88,15 +80,14 @@ public:
|
||||
/// Parsing of external table used when sending tables via http
|
||||
/// The `handlePart` function will be called for each table passed,
|
||||
/// so it's also necessary to call `clean` at the end of the `handlePart`.
|
||||
class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable
|
||||
class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable, WithContext
|
||||
{
|
||||
public:
|
||||
ExternalTablesHandler(Context & context_, const Poco::Net::NameValueCollection & params_) : context(context_), params(params_) {}
|
||||
ExternalTablesHandler(ContextPtr context_, const Poco::Net::NameValueCollection & params_) : WithContext(context_), params(params_) {}
|
||||
|
||||
void handlePart(const Poco::Net::MessageHeader & header, ReadBuffer & stream) override;
|
||||
|
||||
private:
|
||||
Context & context;
|
||||
const Poco::Net::NameValueCollection & params;
|
||||
};
|
||||
|
||||
|
@ -73,7 +73,7 @@ Native41::Native41(const String & password, const String & auth_plugin_data)
|
||||
}
|
||||
|
||||
void Native41::authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, Context & context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool, const Poco::Net::SocketAddress & address)
|
||||
{
|
||||
if (!auth_response)
|
||||
@ -86,15 +86,17 @@ void Native41::authenticate(
|
||||
|
||||
if (auth_response->empty())
|
||||
{
|
||||
context.setUser(user_name, "", address);
|
||||
context->setUser(user_name, "", address);
|
||||
return;
|
||||
}
|
||||
|
||||
if (auth_response->size() != Poco::SHA1Engine::DIGEST_SIZE)
|
||||
throw Exception("Wrong size of auth response. Expected: " + std::to_string(Poco::SHA1Engine::DIGEST_SIZE) + " bytes, received: " + std::to_string(auth_response->size()) + " bytes.",
|
||||
throw Exception(
|
||||
"Wrong size of auth response. Expected: " + std::to_string(Poco::SHA1Engine::DIGEST_SIZE)
|
||||
+ " bytes, received: " + std::to_string(auth_response->size()) + " bytes.",
|
||||
ErrorCodes::UNKNOWN_EXCEPTION);
|
||||
|
||||
auto user = context.getAccessControlManager().read<User>(user_name);
|
||||
auto user = context->getAccessControlManager().read<User>(user_name);
|
||||
|
||||
Poco::SHA1Engine::Digest double_sha1_value = user->authentication.getPasswordDoubleSHA1();
|
||||
assert(double_sha1_value.size() == Poco::SHA1Engine::DIGEST_SIZE);
|
||||
@ -109,7 +111,7 @@ void Native41::authenticate(
|
||||
{
|
||||
password_sha1[i] = digest[i] ^ static_cast<unsigned char>((*auth_response)[i]);
|
||||
}
|
||||
context.setUser(user_name, password_sha1, address);
|
||||
context->setUser(user_name, password_sha1, address);
|
||||
}
|
||||
|
||||
#if USE_SSL
|
||||
@ -134,7 +136,7 @@ Sha256Password::Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logg
|
||||
}
|
||||
|
||||
void Sha256Password::authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, Context & context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address)
|
||||
{
|
||||
if (!auth_response)
|
||||
@ -229,7 +231,7 @@ void Sha256Password::authenticate(
|
||||
password.pop_back();
|
||||
}
|
||||
|
||||
context.setUser(user_name, password, address);
|
||||
context->setUser(user_name, password, address);
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
virtual String getAuthPluginData() = 0;
|
||||
|
||||
virtual void authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, Context & context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) = 0;
|
||||
};
|
||||
|
||||
@ -49,7 +49,7 @@ public:
|
||||
String getAuthPluginData() override { return scramble; }
|
||||
|
||||
void authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, Context & context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool /* is_secure_connection */, const Poco::Net::SocketAddress & address) override;
|
||||
|
||||
private:
|
||||
@ -69,7 +69,7 @@ public:
|
||||
String getAuthPluginData() override { return scramble; }
|
||||
|
||||
void authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, Context & context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) override;
|
||||
|
||||
private:
|
||||
|
@ -797,15 +797,15 @@ namespace PGAuthentication
|
||||
class AuthenticationMethod
|
||||
{
|
||||
protected:
|
||||
void setPassword(
|
||||
static void setPassword(
|
||||
const String & user_name,
|
||||
const String & password,
|
||||
Context & context,
|
||||
ContextPtr context,
|
||||
Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address)
|
||||
{
|
||||
try {
|
||||
context.setUser(user_name, password, address);
|
||||
context->setUser(user_name, password, address);
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
@ -819,7 +819,7 @@ protected:
|
||||
public:
|
||||
virtual void authenticate(
|
||||
const String & user_name,
|
||||
Context & context,
|
||||
ContextPtr context,
|
||||
Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address) = 0;
|
||||
|
||||
@ -833,7 +833,7 @@ class NoPasswordAuth : public AuthenticationMethod
|
||||
public:
|
||||
void authenticate(
|
||||
const String & /* user_name */,
|
||||
Context & /* context */,
|
||||
ContextPtr /* context */,
|
||||
Messaging::MessageTransport & /* mt */,
|
||||
const Poco::Net::SocketAddress & /* address */) override {}
|
||||
|
||||
@ -848,7 +848,7 @@ class CleartextPasswordAuth : public AuthenticationMethod
|
||||
public:
|
||||
void authenticate(
|
||||
const String & user_name,
|
||||
Context & context,
|
||||
ContextPtr context,
|
||||
Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address) override
|
||||
{
|
||||
@ -891,11 +891,11 @@ public:
|
||||
|
||||
void authenticate(
|
||||
const String & user_name,
|
||||
Context & context,
|
||||
ContextPtr context,
|
||||
Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address)
|
||||
{
|
||||
auto user = context.getAccessControlManager().read<User>(user_name);
|
||||
auto user = context->getAccessControlManager().read<User>(user_name);
|
||||
Authentication::Type user_auth_type = user->authentication.getType();
|
||||
|
||||
if (type_to_method.find(user_auth_type) != type_to_method.end())
|
||||
|
@ -10,7 +10,7 @@ AddingDefaultBlockOutputStream::AddingDefaultBlockOutputStream(
|
||||
const BlockOutputStreamPtr & output_,
|
||||
const Block & header_,
|
||||
const ColumnsDescription & columns_,
|
||||
const Context & context_)
|
||||
ContextPtr context_)
|
||||
: output(output_), header(header_)
|
||||
{
|
||||
auto dag = addMissingDefaults(header_, output->getHeader().getNamesAndTypesList(), columns_, context_);
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
const BlockOutputStreamPtr & output_,
|
||||
const Block & header_,
|
||||
const ColumnsDescription & columns_,
|
||||
const Context & context_);
|
||||
ContextPtr context_);
|
||||
|
||||
Block getHeader() const override { return header; }
|
||||
void write(const Block & block) override;
|
||||
|
@ -130,7 +130,7 @@ static MutableColumnPtr mixColumns(const ColumnWithTypeAndName & col_read,
|
||||
AddingDefaultsBlockInputStream::AddingDefaultsBlockInputStream(
|
||||
const BlockInputStreamPtr & input,
|
||||
const ColumnsDescription & columns_,
|
||||
const Context & context_)
|
||||
ContextPtr context_)
|
||||
: columns(columns_)
|
||||
, column_defaults(columns.getDefaults())
|
||||
, context(context_)
|
||||
|
@ -7,8 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/// Adds defaults to columns using BlockDelayedDefaults bitmask attached to Block by child InputStream.
|
||||
class AddingDefaultsBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
@ -16,7 +14,7 @@ public:
|
||||
AddingDefaultsBlockInputStream(
|
||||
const BlockInputStreamPtr & input,
|
||||
const ColumnsDescription & columns_,
|
||||
const Context & context_);
|
||||
ContextPtr context_);
|
||||
|
||||
String getName() const override { return "AddingDefaults"; }
|
||||
Block getHeader() const override { return header; }
|
||||
@ -28,7 +26,7 @@ private:
|
||||
Block header;
|
||||
const ColumnsDescription columns;
|
||||
const ColumnDefaults column_defaults;
|
||||
const Context & context;
|
||||
ContextPtr context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -50,7 +50,7 @@ struct BlockIO
|
||||
}
|
||||
}
|
||||
|
||||
void onException()
|
||||
void onException() const
|
||||
{
|
||||
if (exception_callback)
|
||||
exception_callback();
|
||||
|
@ -27,7 +27,7 @@ CheckConstraintsBlockOutputStream::CheckConstraintsBlockOutputStream(
|
||||
const BlockOutputStreamPtr & output_,
|
||||
const Block & header_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const Context & context_)
|
||||
ContextPtr context_)
|
||||
: table_id(table_id_),
|
||||
output(output_),
|
||||
header(header_),
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
const BlockOutputStreamPtr & output_,
|
||||
const Block & header_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const Context & context_);
|
||||
ContextPtr context_);
|
||||
|
||||
Block getHeader() const override { return header; }
|
||||
void write(const Block & block) override;
|
||||
|
@ -2,9 +2,8 @@
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <DataStreams/BlockStreamProfileInfo.h>
|
||||
#include <DataStreams/IBlockStream_fwd.h>
|
||||
#include <DataStreams/SizeLimits.h>
|
||||
#include <DataStreams/ExecutionSpeedLimits.h>
|
||||
#include <DataStreams/SizeLimits.h>
|
||||
#include <DataStreams/StreamLocalLimits.h>
|
||||
#include <IO/Progress.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
|
@ -24,7 +24,7 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
|
||||
const ASTPtr & ast,
|
||||
ReadBuffer * input_buffer_tail_part,
|
||||
const Block & header,
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const ASTPtr & input_function)
|
||||
{
|
||||
const auto * ast_insert_query = ast->as<ASTInsertQuery>();
|
||||
@ -58,9 +58,9 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
|
||||
|
||||
input_buffer_contacenated = std::make_unique<ConcatReadBuffer>(buffers);
|
||||
|
||||
res_stream = context.getInputFormat(format, *input_buffer_contacenated, header, context.getSettings().max_insert_block_size);
|
||||
res_stream = context->getInputFormat(format, *input_buffer_contacenated, header, context->getSettings().max_insert_block_size);
|
||||
|
||||
if (context.getSettingsRef().input_format_defaults_for_omitted_fields && ast_insert_query->table_id && !input_function)
|
||||
if (context->getSettingsRef().input_format_defaults_for_omitted_fields && ast_insert_query->table_id && !input_function)
|
||||
{
|
||||
StoragePtr storage = DatabaseCatalog::instance().getTable(ast_insert_query->table_id, context);
|
||||
auto metadata_snapshot = storage->getInMemoryMetadataPtr();
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
const ASTPtr & ast,
|
||||
ReadBuffer * input_buffer_tail_part,
|
||||
const Block & header,
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const ASTPtr & input_function);
|
||||
|
||||
Block readImpl() override { return res_stream->read(); }
|
||||
|
@ -26,13 +26,13 @@ namespace DB
|
||||
PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
const StoragePtr & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const Context & context_,
|
||||
ContextPtr context_,
|
||||
const ASTPtr & query_ptr_,
|
||||
bool no_destination)
|
||||
: storage(storage_)
|
||||
: WithContext(context_)
|
||||
, storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, log(&Poco::Logger::get("PushingToViewsBlockOutputStream"))
|
||||
, context(context_)
|
||||
, query_ptr(query_ptr_)
|
||||
{
|
||||
checkStackSize();
|
||||
@ -42,12 +42,12 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
* but it's clear that here is not the best place for this functionality.
|
||||
*/
|
||||
addTableLock(
|
||||
storage->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout));
|
||||
storage->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout));
|
||||
|
||||
/// If the "root" table deduplicates blocks, there are no need to make deduplication for children
|
||||
/// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks
|
||||
bool disable_deduplication_for_children = false;
|
||||
if (!context.getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)
|
||||
if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)
|
||||
disable_deduplication_for_children = !no_destination && storage->supportsDeduplication();
|
||||
|
||||
auto table_id = storage->getStorageID();
|
||||
@ -56,8 +56,8 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
/// We need special context for materialized views insertions
|
||||
if (!dependencies.empty())
|
||||
{
|
||||
select_context = std::make_unique<Context>(context);
|
||||
insert_context = std::make_unique<Context>(context);
|
||||
select_context = Context::createCopy(context);
|
||||
insert_context = Context::createCopy(context);
|
||||
|
||||
const auto & insert_settings = insert_context->getSettingsRef();
|
||||
|
||||
@ -74,7 +74,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
|
||||
for (const auto & database_table : dependencies)
|
||||
{
|
||||
auto dependent_table = DatabaseCatalog::instance().getTable(database_table, context);
|
||||
auto dependent_table = DatabaseCatalog::instance().getTable(database_table, getContext());
|
||||
auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr();
|
||||
|
||||
ASTPtr query;
|
||||
@ -83,7 +83,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
if (auto * materialized_view = dynamic_cast<StorageMaterializedView *>(dependent_table.get()))
|
||||
{
|
||||
addTableLock(
|
||||
materialized_view->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout));
|
||||
materialized_view->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout));
|
||||
|
||||
StoragePtr inner_table = materialized_view->getTargetTable();
|
||||
auto inner_table_id = inner_table->getStorageID();
|
||||
@ -94,7 +94,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
insert->table_id = inner_table_id;
|
||||
|
||||
/// Get list of columns we get from select query.
|
||||
auto header = InterpreterSelectQuery(query, *select_context, SelectQueryOptions().analyze())
|
||||
auto header = InterpreterSelectQuery(query, select_context, SelectQueryOptions().analyze())
|
||||
.getSampleBlock();
|
||||
|
||||
/// Insert only columns returned by select.
|
||||
@ -110,16 +110,16 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
insert->columns = std::move(list);
|
||||
|
||||
ASTPtr insert_query_ptr(insert.release());
|
||||
InterpreterInsertQuery interpreter(insert_query_ptr, *insert_context);
|
||||
InterpreterInsertQuery interpreter(insert_query_ptr, insert_context);
|
||||
BlockIO io = interpreter.execute();
|
||||
out = io.out;
|
||||
}
|
||||
else if (dynamic_cast<const StorageLiveView *>(dependent_table.get()))
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(
|
||||
dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr(), true);
|
||||
dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true);
|
||||
else
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(
|
||||
dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr());
|
||||
dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr());
|
||||
|
||||
views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, 0 /* elapsed_ms */});
|
||||
}
|
||||
@ -127,7 +127,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
/// Do not push to destination table if the flag is set
|
||||
if (!no_destination)
|
||||
{
|
||||
output = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), context);
|
||||
output = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), getContext());
|
||||
replicated_output = dynamic_cast<ReplicatedMergeTreeBlockOutputStream *>(output.get());
|
||||
}
|
||||
}
|
||||
@ -155,7 +155,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block)
|
||||
|
||||
if (auto * live_view = dynamic_cast<StorageLiveView *>(storage.get()))
|
||||
{
|
||||
StorageLiveView::writeIntoLiveView(*live_view, block, context);
|
||||
StorageLiveView::writeIntoLiveView(*live_view, block, getContext());
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -166,11 +166,11 @@ void PushingToViewsBlockOutputStream::write(const Block & block)
|
||||
}
|
||||
|
||||
/// Don't process materialized views if this block is duplicate
|
||||
if (!context.getSettingsRef().deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate())
|
||||
if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate())
|
||||
return;
|
||||
|
||||
// Insert data into materialized views only after successful insert into main table
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = getContext()->getSettingsRef();
|
||||
if (settings.parallel_view_processing && views.size() > 1)
|
||||
{
|
||||
// Push to views concurrently if enabled and more than one view is attached
|
||||
@ -228,7 +228,7 @@ void PushingToViewsBlockOutputStream::writeSuffix()
|
||||
|
||||
std::exception_ptr first_exception;
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = getContext()->getSettingsRef();
|
||||
bool parallel_processing = false;
|
||||
|
||||
/// Run writeSuffix() for views in separate thread pool.
|
||||
@ -353,10 +353,9 @@ void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & vi
|
||||
/// but it will contain single block (that is INSERT-ed into main table).
|
||||
/// InterpreterSelectQuery will do processing of alias columns.
|
||||
|
||||
Context local_context = *select_context;
|
||||
local_context.addViewSource(
|
||||
StorageValues::create(
|
||||
storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals()));
|
||||
auto local_context = Context::createCopy(select_context);
|
||||
local_context->addViewSource(
|
||||
StorageValues::create(storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals()));
|
||||
select.emplace(view.query, local_context, SelectQueryOptions());
|
||||
in = std::make_shared<MaterializingBlockInputStream>(select->execute().getInputStream());
|
||||
|
||||
@ -364,7 +363,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & vi
|
||||
/// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY
|
||||
/// and two-level aggregation is triggered).
|
||||
in = std::make_shared<SquashingBlockInputStream>(
|
||||
in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes);
|
||||
in, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes);
|
||||
in = std::make_shared<ConvertingBlockInputStream>(in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name);
|
||||
}
|
||||
else
|
||||
|
@ -1,9 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
@ -14,18 +14,16 @@ namespace DB
|
||||
{
|
||||
|
||||
class ReplicatedMergeTreeBlockOutputStream;
|
||||
class Context;
|
||||
|
||||
|
||||
/** Writes data to the specified table and to all dependent materialized views.
|
||||
*/
|
||||
class PushingToViewsBlockOutputStream : public IBlockOutputStream
|
||||
class PushingToViewsBlockOutputStream : public IBlockOutputStream, WithContext
|
||||
{
|
||||
public:
|
||||
PushingToViewsBlockOutputStream(
|
||||
const StoragePtr & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const Context & context_,
|
||||
ContextPtr context_,
|
||||
const ASTPtr & query_ptr_,
|
||||
bool no_destination = false);
|
||||
|
||||
@ -43,7 +41,6 @@ private:
|
||||
ReplicatedMergeTreeBlockOutputStream * replicated_output = nullptr;
|
||||
Poco::Logger * log;
|
||||
|
||||
const Context & context;
|
||||
ASTPtr query_ptr;
|
||||
Stopwatch main_watch;
|
||||
|
||||
@ -57,8 +54,8 @@ private:
|
||||
};
|
||||
|
||||
std::vector<ViewInfo> views;
|
||||
std::unique_ptr<Context> select_context;
|
||||
std::unique_ptr<Context> insert_context;
|
||||
ContextPtr select_context;
|
||||
ContextPtr insert_context;
|
||||
|
||||
void process(const Block & block, ViewInfo & view);
|
||||
};
|
||||
|
@ -6,7 +6,7 @@ namespace DB
|
||||
|
||||
RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
Connection & connection,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: query_executor(connection, query_, header_, context_, throttler, scalars_, external_tables_, stage_)
|
||||
{
|
||||
@ -15,7 +15,7 @@ RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
|
||||
RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
std::vector<IConnectionPool::Entry> && connections,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: query_executor(std::move(connections), query_, header_, context_, throttler, scalars_, external_tables_, stage_)
|
||||
{
|
||||
@ -24,7 +24,7 @@ RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
|
||||
RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
const ConnectionPoolWithFailoverPtr & pool,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: query_executor(pool, query_, header_, context_, throttler, scalars_, external_tables_, stage_)
|
||||
{
|
||||
|
@ -25,21 +25,21 @@ public:
|
||||
/// Takes already set connection.
|
||||
RemoteBlockInputStream(
|
||||
Connection & connection,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
/// Accepts several connections already taken from pool.
|
||||
RemoteBlockInputStream(
|
||||
std::vector<IConnectionPool::Entry> && connections,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
/// Takes a pool and gets one or several connections from it.
|
||||
RemoteBlockInputStream(
|
||||
const ConnectionPoolWithFailoverPtr & pool,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
|
@ -28,41 +28,51 @@ namespace ErrorCodes
|
||||
|
||||
RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
Connection & connection,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
ThrottlerPtr throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_)
|
||||
, scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
const String & query_,
|
||||
const Block & header_,
|
||||
ContextPtr context_,
|
||||
ThrottlerPtr throttler,
|
||||
const Scalars & scalars_,
|
||||
const Tables & external_tables_,
|
||||
QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
create_connections = [this, &connection, throttler]()
|
||||
{
|
||||
return std::make_unique<MultiplexedConnections>(connection, context.getSettingsRef(), throttler);
|
||||
return std::make_unique<MultiplexedConnections>(connection, context->getSettingsRef(), throttler);
|
||||
};
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
std::vector<IConnectionPool::Entry> && connections_,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_)
|
||||
, scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
const String & query_,
|
||||
const Block & header_,
|
||||
ContextPtr context_,
|
||||
const ThrottlerPtr & throttler,
|
||||
const Scalars & scalars_,
|
||||
const Tables & external_tables_,
|
||||
QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
create_connections = [this, connections_, throttler]() mutable
|
||||
{
|
||||
return std::make_unique<MultiplexedConnections>(
|
||||
std::move(connections_), context.getSettingsRef(), throttler);
|
||||
create_connections = [this, connections_, throttler]() mutable {
|
||||
return std::make_unique<MultiplexedConnections>(std::move(connections_), context->getSettingsRef(), throttler);
|
||||
};
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
const ConnectionPoolWithFailoverPtr & pool,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_)
|
||||
, scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
const String & query_,
|
||||
const Block & header_,
|
||||
ContextPtr context_,
|
||||
const ThrottlerPtr & throttler,
|
||||
const Scalars & scalars_,
|
||||
const Tables & external_tables_,
|
||||
QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
create_connections = [this, pool, throttler]()->std::unique_ptr<IConnections>
|
||||
{
|
||||
const Settings & current_settings = context.getSettingsRef();
|
||||
const Settings & current_settings = context->getSettingsRef();
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
@ -157,7 +167,7 @@ void RemoteQueryExecutor::sendQuery()
|
||||
|
||||
connections = create_connections();
|
||||
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = context->getSettingsRef();
|
||||
if (settings.skip_unavailable_shards && 0 == connections->size())
|
||||
return;
|
||||
|
||||
@ -176,7 +186,7 @@ void RemoteQueryExecutor::sendQuery()
|
||||
was_cancelled = false;
|
||||
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings);
|
||||
ClientInfo modified_client_info = context.getClientInfo();
|
||||
ClientInfo modified_client_info = context->getClientInfo();
|
||||
modified_client_info.query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
if (CurrentThread::isInitialized())
|
||||
{
|
||||
@ -205,7 +215,7 @@ Block RemoteQueryExecutor::read()
|
||||
{
|
||||
sendQuery();
|
||||
|
||||
if (context.getSettingsRef().skip_unavailable_shards && (0 == connections->size()))
|
||||
if (context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()))
|
||||
return {};
|
||||
}
|
||||
|
||||
@ -231,7 +241,7 @@ std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext>
|
||||
{
|
||||
sendQuery();
|
||||
|
||||
if (context.getSettingsRef().skip_unavailable_shards && (0 == connections->size()))
|
||||
if (context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()))
|
||||
return Block();
|
||||
}
|
||||
|
||||
@ -363,8 +373,8 @@ std::optional<Block> RemoteQueryExecutor::processPacket(Packet packet)
|
||||
|
||||
bool RemoteQueryExecutor::setPartUUIDs(const std::vector<UUID> & uuids)
|
||||
{
|
||||
Context & query_context = const_cast<Context &>(context).getQueryContext();
|
||||
auto duplicates = query_context.getPartUUIDs()->add(uuids);
|
||||
auto query_context = context->getQueryContext();
|
||||
auto duplicates = query_context->getPartUUIDs()->add(uuids);
|
||||
|
||||
if (!duplicates.empty())
|
||||
{
|
||||
|
@ -35,21 +35,21 @@ public:
|
||||
/// Takes already set connection.
|
||||
RemoteQueryExecutor(
|
||||
Connection & connection,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
ThrottlerPtr throttler_ = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
/// Accepts several connections already taken from pool.
|
||||
RemoteQueryExecutor(
|
||||
std::vector<IConnectionPool::Entry> && connections_,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
/// Takes a pool and gets one or several connections from it.
|
||||
RemoteQueryExecutor(
|
||||
const ConnectionPoolWithFailoverPtr & pool,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
@ -108,8 +108,8 @@ private:
|
||||
std::unique_ptr<IConnections> connections;
|
||||
|
||||
const String query;
|
||||
String query_id = "";
|
||||
Context context;
|
||||
String query_id;
|
||||
ContextPtr context;
|
||||
|
||||
ProgressCallback progress_callback;
|
||||
ProfileInfoCallback profile_info_callback;
|
||||
|
@ -28,12 +28,12 @@ TTLAggregationAlgorithm::TTLAggregationAlgorithm(
|
||||
descr.arguments.push_back(header.getPositionByName(name));
|
||||
|
||||
columns_for_aggregator.resize(description.aggregate_descriptions.size());
|
||||
const Settings & settings = storage_.global_context.getSettingsRef();
|
||||
const Settings & settings = storage_.getContext()->getSettingsRef();
|
||||
|
||||
Aggregator::Params params(header, keys, aggregates,
|
||||
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0,
|
||||
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
|
||||
storage_.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
|
||||
storage_.getContext()->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
|
||||
|
||||
aggregator = std::make_unique<Aggregator>(params);
|
||||
}
|
||||
|
@ -68,8 +68,9 @@ TTLBlockInputStream::TTLBlockInputStream(
|
||||
auto default_ast = it->second.expression->clone();
|
||||
default_ast = addTypeConversionToAST(std::move(default_ast), column.type->getName());
|
||||
|
||||
auto syntax_result = TreeRewriter(storage_.global_context).analyze(default_ast, metadata_snapshot_->getColumns().getAllPhysical());
|
||||
default_expression = ExpressionAnalyzer{default_ast, syntax_result, storage_.global_context}.getActions(true);
|
||||
auto syntax_result
|
||||
= TreeRewriter(storage_.getContext()).analyze(default_ast, metadata_snapshot_->getColumns().getAllPhysical());
|
||||
default_expression = ExpressionAnalyzer{default_ast, syntax_result, storage_.getContext()}.getActions(true);
|
||||
default_column_name = default_ast->getColumnName();
|
||||
}
|
||||
|
||||
|
@ -19,9 +19,14 @@ namespace ErrorCodes
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
bool decimalCheckComparisonOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; }
|
||||
bool decimalCheckArithmeticOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; }
|
||||
bool decimalCheckComparisonOverflow(ContextPtr context)
|
||||
{
|
||||
return context->getSettingsRef().decimal_check_overflow;
|
||||
}
|
||||
bool decimalCheckArithmeticOverflow(ContextPtr context)
|
||||
{
|
||||
return context->getSettingsRef().decimal_check_overflow;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
Field DataTypeDecimalBase<T>::getDefault() const
|
||||
|
@ -1,11 +1,12 @@
|
||||
#pragma once
|
||||
#include <cmath>
|
||||
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
#include <cmath>
|
||||
#include <type_traits>
|
||||
|
||||
|
||||
@ -17,9 +18,8 @@ namespace ErrorCodes
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
class Context;
|
||||
bool decimalCheckComparisonOverflow(const Context & context);
|
||||
bool decimalCheckArithmeticOverflow(const Context & context);
|
||||
bool decimalCheckComparisonOverflow(ContextPtr context);
|
||||
bool decimalCheckArithmeticOverflow(ContextPtr context);
|
||||
|
||||
inline UInt32 leastDecimalPrecisionFor(TypeIndex int_type)
|
||||
{
|
||||
|
@ -156,7 +156,7 @@ void DataTypeFactory::registerSimpleDataTypeCustom(const String &name, SimpleCre
|
||||
|
||||
const DataTypeFactory::Value & DataTypeFactory::findCreatorByName(const String & family_name) const
|
||||
{
|
||||
const Context * query_context = nullptr;
|
||||
ContextPtr query_context;
|
||||
if (CurrentThread::isInitialized())
|
||||
query_context = CurrentThread::get().getQueryContext();
|
||||
|
||||
|
@ -35,10 +35,10 @@ public:
|
||||
UUID uuid() const override { return table()->getStorageID().uuid; }
|
||||
};
|
||||
|
||||
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, const Context & context_)
|
||||
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, ContextPtr context_)
|
||||
: DatabaseOrdinary(name_, std::move(metadata_path_), "store/", logger_name, context_)
|
||||
, path_to_table_symlinks(global_context.getPath() + "data/" + escapeForFileName(name_) + "/")
|
||||
, path_to_metadata_symlink(global_context.getPath() + "metadata/" + escapeForFileName(name_))
|
||||
, path_to_table_symlinks(getContext()->getPath() + "data/" + escapeForFileName(name_) + "/")
|
||||
, path_to_metadata_symlink(getContext()->getPath() + "metadata/" + escapeForFileName(name_))
|
||||
, db_uuid(uuid)
|
||||
{
|
||||
assert(db_uuid != UUIDHelpers::Nil);
|
||||
@ -46,7 +46,7 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, c
|
||||
tryCreateMetadataSymlink();
|
||||
}
|
||||
|
||||
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const Context & context_)
|
||||
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, ContextPtr context_)
|
||||
: DatabaseAtomic(name_, std::move(metadata_path_), uuid, "DatabaseAtomic (" + name_ + ")", context_)
|
||||
{
|
||||
}
|
||||
@ -68,7 +68,7 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const
|
||||
return tmp;
|
||||
}
|
||||
|
||||
void DatabaseAtomic::drop(const Context &)
|
||||
void DatabaseAtomic::drop(ContextPtr)
|
||||
{
|
||||
assert(tables.empty());
|
||||
try
|
||||
@ -106,13 +106,13 @@ StoragePtr DatabaseAtomic::detachTable(const String & name)
|
||||
return table;
|
||||
}
|
||||
|
||||
void DatabaseAtomic::dropTable(const Context & context, const String & table_name, bool no_delay)
|
||||
void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool no_delay)
|
||||
{
|
||||
if (auto * mv = dynamic_cast<StorageMaterializedView *>(tryGetTable(table_name, context).get()))
|
||||
if (auto * mv = dynamic_cast<StorageMaterializedView *>(tryGetTable(table_name, local_context).get()))
|
||||
{
|
||||
/// Remove the inner table (if any) to avoid deadlock
|
||||
/// (due to attempt to execute DROP from the worker thread)
|
||||
mv->dropInnerTable(no_delay, context);
|
||||
mv->dropInnerTable(no_delay, local_context);
|
||||
}
|
||||
|
||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||
@ -122,8 +122,8 @@ void DatabaseAtomic::dropTable(const Context & context, const String & table_nam
|
||||
std::unique_lock lock(mutex);
|
||||
table = getTableUnlocked(table_name, lock);
|
||||
table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID());
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
if (txn && !context.isInternalSubquery())
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
if (txn && !local_context->isInternalSubquery())
|
||||
txn->commit(); /// Commit point (a sort of) for Replicated database
|
||||
|
||||
/// NOTE: replica will be lost if server crashes before the following rename
|
||||
@ -144,7 +144,7 @@ void DatabaseAtomic::dropTable(const Context & context, const String & table_nam
|
||||
DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay);
|
||||
}
|
||||
|
||||
void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database,
|
||||
void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_name, IDatabase & to_database,
|
||||
const String & to_table_name, bool exchange, bool dictionary)
|
||||
{
|
||||
if (typeid(*this) != typeid(to_database))
|
||||
@ -152,7 +152,7 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
|
||||
if (!typeid_cast<DatabaseOrdinary *>(&to_database))
|
||||
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
/// Allow moving tables between Atomic and Ordinary (with table lock)
|
||||
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
|
||||
DatabaseOnDisk::renameTable(local_context, table_name, to_database, to_table_name, exchange, dictionary);
|
||||
return;
|
||||
}
|
||||
|
||||
@ -248,8 +248,8 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
|
||||
}
|
||||
|
||||
/// Table renaming actually begins here
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
if (txn && !context.isInternalSubquery())
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
if (txn && !local_context->isInternalSubquery())
|
||||
txn->commit(); /// Commit point (a sort of) for Replicated database
|
||||
|
||||
/// NOTE: replica will be lost if server crashes before the following rename
|
||||
@ -291,7 +291,7 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
|
||||
|
||||
void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path,
|
||||
const Context & query_context)
|
||||
ContextPtr query_context)
|
||||
{
|
||||
DetachedTables not_in_use;
|
||||
auto table_data_path = getTableDataPath(query);
|
||||
@ -309,8 +309,8 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora
|
||||
DatabaseCatalog::instance().addUUIDMapping(query.uuid);
|
||||
locked_uuid = true;
|
||||
|
||||
auto txn = query_context.getZooKeeperMetadataTransaction();
|
||||
if (txn && !query_context.isInternalSubquery())
|
||||
auto txn = query_context->getZooKeeperMetadataTransaction();
|
||||
if (txn && !query_context->isInternalSubquery())
|
||||
txn->commit(); /// Commit point (a sort of) for Replicated database
|
||||
|
||||
/// NOTE: replica will be lost if server crashes before the following renameNoReplace(...)
|
||||
@ -333,7 +333,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora
|
||||
}
|
||||
|
||||
void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path,
|
||||
const String & /*statement*/, const Context & query_context)
|
||||
const String & /*statement*/, ContextPtr query_context)
|
||||
{
|
||||
bool check_file_exists = true;
|
||||
SCOPE_EXIT({ std::error_code code; if (check_file_exists) std::filesystem::remove(table_metadata_tmp_path, code); });
|
||||
@ -344,8 +344,8 @@ void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String &
|
||||
if (table_id.uuid != actual_table_id.uuid)
|
||||
throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER);
|
||||
|
||||
auto txn = query_context.getZooKeeperMetadataTransaction();
|
||||
if (txn && !query_context.isInternalSubquery())
|
||||
auto txn = query_context->getZooKeeperMetadataTransaction();
|
||||
if (txn && !query_context->isInternalSubquery())
|
||||
txn->commit(); /// Commit point (a sort of) for Replicated database
|
||||
|
||||
/// NOTE: replica will be lost if server crashes before the following rename
|
||||
@ -409,26 +409,27 @@ void DatabaseAtomic::assertCanBeDetached(bool cleanup)
|
||||
"because some tables are still in use. Retry later.", ErrorCodes::DATABASE_NOT_EMPTY);
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const Context & context, const IDatabase::FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr
|
||||
DatabaseAtomic::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(context, filter_by_table_name);
|
||||
auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name);
|
||||
return std::make_unique<AtomicDatabaseTablesSnapshotIterator>(std::move(typeid_cast<DatabaseTablesSnapshotIterator &>(*base_iter)));
|
||||
}
|
||||
|
||||
UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const
|
||||
{
|
||||
if (auto table = tryGetTable(table_name, global_context))
|
||||
if (auto table = tryGetTable(table_name, getContext()))
|
||||
return table->getStorageID().uuid;
|
||||
return UUIDHelpers::Nil;
|
||||
}
|
||||
|
||||
void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach)
|
||||
void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach)
|
||||
{
|
||||
/// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken
|
||||
if (has_force_restore_data_flag)
|
||||
Poco::File(path_to_table_symlinks).remove(true);
|
||||
|
||||
DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag, force_attach);
|
||||
DatabaseOrdinary::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach);
|
||||
|
||||
if (has_force_restore_data_flag)
|
||||
{
|
||||
@ -449,7 +450,7 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String &
|
||||
try
|
||||
{
|
||||
String link = path_to_table_symlinks + escapeForFileName(table_name);
|
||||
Poco::File data = Poco::Path(global_context.getPath()).makeAbsolute().toString() + actual_data_path;
|
||||
Poco::File data = Poco::Path(getContext()->getPath()).makeAbsolute().toString() + actual_data_path;
|
||||
if (!if_data_path_exist || data.exists())
|
||||
data.linkTo(link, Poco::File::LINK_SYMBOLIC);
|
||||
}
|
||||
@ -509,8 +510,8 @@ void DatabaseAtomic::renameDatabase(const String & new_name)
|
||||
}
|
||||
|
||||
auto new_name_escaped = escapeForFileName(new_name);
|
||||
auto old_database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(getDatabaseName()) + ".sql";
|
||||
auto new_database_metadata_path = global_context.getPath() + "metadata/" + new_name_escaped + ".sql";
|
||||
auto old_database_metadata_path = getContext()->getPath() + "metadata/" + escapeForFileName(getDatabaseName()) + ".sql";
|
||||
auto new_database_metadata_path = getContext()->getPath() + "metadata/" + new_name_escaped + ".sql";
|
||||
renameNoReplace(old_database_metadata_path, new_database_metadata_path);
|
||||
|
||||
String old_path_to_table_symlinks;
|
||||
@ -535,9 +536,9 @@ void DatabaseAtomic::renameDatabase(const String & new_name)
|
||||
renameDictionaryInMemoryUnlocked(old_name, name);
|
||||
}
|
||||
|
||||
path_to_metadata_symlink = global_context.getPath() + "metadata/" + new_name_escaped;
|
||||
path_to_metadata_symlink = getContext()->getPath() + "metadata/" + new_name_escaped;
|
||||
old_path_to_table_symlinks = path_to_table_symlinks;
|
||||
path_to_table_symlinks = global_context.getPath() + "data/" + new_name_escaped + "/";
|
||||
path_to_table_symlinks = getContext()->getPath() + "data/" + new_name_escaped + "/";
|
||||
}
|
||||
|
||||
Poco::File(old_path_to_table_symlinks).renameTo(path_to_table_symlinks);
|
||||
|
@ -19,8 +19,8 @@ namespace DB
|
||||
class DatabaseAtomic : public DatabaseOrdinary
|
||||
{
|
||||
public:
|
||||
DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, const Context & context_);
|
||||
DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const Context & context_);
|
||||
DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, ContextPtr context_);
|
||||
DatabaseAtomic(String name_, String metadata_path_, UUID uuid, ContextPtr context_);
|
||||
|
||||
String getEngineName() const override { return "Atomic"; }
|
||||
UUID getUUID() const override { return db_uuid; }
|
||||
@ -28,14 +28,14 @@ public:
|
||||
void renameDatabase(const String & new_name) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
bool exchange,
|
||||
bool dictionary) override;
|
||||
|
||||
void dropTable(const Context & context, const String & table_name, bool no_delay) override;
|
||||
void dropTable(ContextPtr context, const String & table_name, bool no_delay) override;
|
||||
|
||||
void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override;
|
||||
StoragePtr detachTable(const String & name) override;
|
||||
@ -43,11 +43,11 @@ public:
|
||||
String getTableDataPath(const String & table_name) const override;
|
||||
String getTableDataPath(const ASTCreateQuery & query) const override;
|
||||
|
||||
void drop(const Context & /*context*/) override;
|
||||
void drop(ContextPtr /*context*/) override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
|
||||
/// Atomic database cannot be detached if there is detached table which still in use
|
||||
void assertCanBeDetached(bool cleanup) override;
|
||||
@ -61,12 +61,12 @@ public:
|
||||
void setDetachedTableNotInUseForce(const UUID & uuid);
|
||||
|
||||
protected:
|
||||
void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, const Context & query_context) override;
|
||||
void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, ContextPtr query_context) override;
|
||||
void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path, const Context & query_context) override;
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context) override;
|
||||
|
||||
void assertDetachedTableNotInUse(const UUID & uuid);
|
||||
typedef std::unordered_map<UUID, StoragePtr> DetachedTables;
|
||||
using DetachedTables = std::unordered_map<UUID, StoragePtr>;
|
||||
[[nodiscard]] DetachedTables cleanupDetachedTables();
|
||||
|
||||
void tryCreateMetadataSymlink();
|
||||
@ -74,7 +74,7 @@ protected:
|
||||
void renameDictionaryInMemoryUnlocked(const StorageID & old_name, const StorageID & new_name);
|
||||
|
||||
//TODO store path in DatabaseWithOwnTables::tables
|
||||
typedef std::unordered_map<String, String> NameToPathMap;
|
||||
using NameToPathMap = std::unordered_map<String, String>;
|
||||
NameToPathMap table_name_to_path;
|
||||
|
||||
DetachedTables detached_tables;
|
||||
|
@ -44,17 +44,16 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & context_)
|
||||
: IDatabase(name_)
|
||||
DatabaseDictionary::DatabaseDictionary(const String & name_, ContextPtr context_)
|
||||
: IDatabase(name_), WithContext(context_->getGlobalContext())
|
||||
, log(&Poco::Logger::get("DatabaseDictionary(" + database_name + ")"))
|
||||
, global_context(context_.getGlobalContext())
|
||||
{
|
||||
}
|
||||
|
||||
Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_name)
|
||||
{
|
||||
Tables tables;
|
||||
auto load_results = global_context.getExternalDictionariesLoader().getLoadResults(filter_by_name);
|
||||
auto load_results = getContext()->getExternalDictionariesLoader().getLoadResults(filter_by_name);
|
||||
String db_name = getDatabaseName();
|
||||
for (auto & load_result : load_results)
|
||||
{
|
||||
@ -65,34 +64,34 @@ Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_nam
|
||||
return tables;
|
||||
}
|
||||
|
||||
bool DatabaseDictionary::isTableExist(const String & table_name, const Context &) const
|
||||
bool DatabaseDictionary::isTableExist(const String & table_name, ContextPtr) const
|
||||
{
|
||||
return global_context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST;
|
||||
return getContext()->getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST;
|
||||
}
|
||||
|
||||
StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, const Context &) const
|
||||
StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, ContextPtr) const
|
||||
{
|
||||
auto load_result = global_context.getExternalDictionariesLoader().getLoadResult(table_name);
|
||||
auto load_result = getContext()->getExternalDictionariesLoader().getLoadResult(table_name);
|
||||
return createStorageDictionary(getDatabaseName(), load_result);
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(listTables(filter_by_table_name), getDatabaseName());
|
||||
}
|
||||
|
||||
bool DatabaseDictionary::empty() const
|
||||
{
|
||||
return !global_context.getExternalDictionariesLoader().hasObjects();
|
||||
return !getContext()->getExternalDictionariesLoader().hasObjects();
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const
|
||||
ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const
|
||||
{
|
||||
String query;
|
||||
{
|
||||
WriteBufferFromString buffer(query);
|
||||
|
||||
auto load_result = global_context.getExternalDictionariesLoader().getLoadResult(table_name);
|
||||
auto load_result = getContext()->getExternalDictionariesLoader().getLoadResult(table_name);
|
||||
if (!load_result.config)
|
||||
{
|
||||
if (throw_on_error)
|
||||
@ -106,7 +105,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, co
|
||||
buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")";
|
||||
}
|
||||
|
||||
auto settings = global_context.getSettingsRef();
|
||||
auto settings = getContext()->getSettingsRef();
|
||||
ParserCreateQuery parser;
|
||||
const char * pos = query.data();
|
||||
std::string error_message;
|
||||
@ -126,7 +125,7 @@ ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const
|
||||
WriteBufferFromString buffer(query);
|
||||
buffer << "CREATE DATABASE " << backQuoteIfNeed(getDatabaseName()) << " ENGINE = Dictionary";
|
||||
}
|
||||
auto settings = global_context.getSettingsRef();
|
||||
auto settings = getContext()->getSettingsRef();
|
||||
ParserCreateQuery parser;
|
||||
return parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth);
|
||||
}
|
||||
|
@ -1,11 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <mutex>
|
||||
#include <unordered_set>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
#include <mutex>
|
||||
#include <unordered_set>
|
||||
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
@ -19,21 +20,21 @@ namespace DB
|
||||
/* Database to store StorageDictionary tables
|
||||
* automatically creates tables for all dictionaries
|
||||
*/
|
||||
class DatabaseDictionary final : public IDatabase
|
||||
class DatabaseDictionary final : public IDatabase, WithContext
|
||||
{
|
||||
public:
|
||||
DatabaseDictionary(const String & name_, const Context & context_);
|
||||
DatabaseDictionary(const String & name_, ContextPtr context_);
|
||||
|
||||
String getEngineName() const override
|
||||
{
|
||||
return "Dictionary";
|
||||
}
|
||||
|
||||
bool isTableExist(const String & table_name, const Context & context) const override;
|
||||
bool isTableExist(const String & table_name, ContextPtr context) const override;
|
||||
|
||||
StoragePtr tryGetTable(const String & table_name, const Context & context) const override;
|
||||
StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
bool empty() const override;
|
||||
|
||||
@ -44,11 +45,10 @@ public:
|
||||
void shutdown() override;
|
||||
|
||||
protected:
|
||||
ASTPtr getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const override;
|
||||
ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr context, bool throw_on_error) const override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
const Context & global_context;
|
||||
|
||||
Tables listTables(const FilterByNameFunction & filter_by_name);
|
||||
};
|
||||
|
@ -51,7 +51,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_CREATE_DATABASE;
|
||||
}
|
||||
|
||||
DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, Context & context)
|
||||
DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context)
|
||||
{
|
||||
bool created = false;
|
||||
|
||||
@ -66,8 +66,8 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m
|
||||
|
||||
DatabasePtr impl = getImpl(create, metadata_path, context);
|
||||
|
||||
if (impl && context.hasQueryContext() && context.getSettingsRef().log_queries)
|
||||
context.getQueryContext().addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName());
|
||||
if (impl && context->hasQueryContext() && context->getSettingsRef().log_queries)
|
||||
context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName());
|
||||
|
||||
return impl;
|
||||
|
||||
@ -92,7 +92,7 @@ static inline ValueType safeGetLiteralValue(const ASTPtr &ast, const String &eng
|
||||
return ast->as<ASTLiteral>()->value.safeGet<ValueType>();
|
||||
}
|
||||
|
||||
DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, Context & context)
|
||||
DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context)
|
||||
{
|
||||
auto * engine_define = create.storage;
|
||||
const String & database_name = create.database;
|
||||
@ -145,7 +145,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
{
|
||||
auto mysql_database_settings = std::make_unique<ConnectionMySQLSettings>();
|
||||
/// Split into replicas if needed.
|
||||
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
|
||||
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
|
||||
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
|
||||
auto mysql_pool = mysqlxx::PoolWithFailover(mysql_database_name, addresses, mysql_user_name, mysql_user_password);
|
||||
|
||||
@ -209,9 +209,9 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
String shard_name = safeGetLiteralValue<String>(arguments[1], "Replicated");
|
||||
String replica_name = safeGetLiteralValue<String>(arguments[2], "Replicated");
|
||||
|
||||
zookeeper_path = context.getMacros()->expand(zookeeper_path);
|
||||
shard_name = context.getMacros()->expand(shard_name);
|
||||
replica_name = context.getMacros()->expand(replica_name);
|
||||
zookeeper_path = context->getMacros()->expand(zookeeper_path);
|
||||
shard_name = context->getMacros()->expand(shard_name);
|
||||
replica_name = context->getMacros()->expand(replica_name);
|
||||
|
||||
DatabaseReplicatedSettings database_replicated_settings{};
|
||||
if (engine_define->settings)
|
||||
@ -249,7 +249,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
use_table_cache = safeGetLiteralValue<UInt64>(engine_args[4], engine_name);
|
||||
|
||||
/// Split into replicas if needed.
|
||||
size_t max_addresses = context.getSettingsRef().glob_expansion_max_elements;
|
||||
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
|
||||
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
|
||||
|
||||
/// no connection is made here
|
||||
@ -257,8 +257,8 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
postgres_database_name,
|
||||
addresses,
|
||||
username, password,
|
||||
context.getSettingsRef().postgresql_connection_pool_size,
|
||||
context.getSettingsRef().postgresql_connection_pool_wait_timeout);
|
||||
context->getSettingsRef().postgresql_connection_pool_size,
|
||||
context->getSettingsRef().postgresql_connection_pool_wait_timeout);
|
||||
|
||||
return std::make_shared<DatabasePostgreSQL>(
|
||||
context, metadata_path, engine_define, database_name, postgres_database_name, connection_pool, use_table_cache);
|
||||
|
@ -11,9 +11,9 @@ class ASTCreateQuery;
|
||||
class DatabaseFactory
|
||||
{
|
||||
public:
|
||||
static DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, Context & context);
|
||||
static DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context);
|
||||
|
||||
static DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, Context & context);
|
||||
static DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -27,7 +27,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_)
|
||||
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, ContextPtr context_)
|
||||
: DatabaseOnDisk(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseLazy (" + name_ + ")", context_)
|
||||
, expiration_time(expiration_time_)
|
||||
{
|
||||
@ -35,11 +35,11 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_,
|
||||
|
||||
|
||||
void DatabaseLazy::loadStoredObjects(
|
||||
Context & context,
|
||||
ContextPtr local_context,
|
||||
bool /* has_force_restore_data_flag */,
|
||||
bool /*force_attach*/)
|
||||
{
|
||||
iterateMetadataFiles(context, [this](const String & file_name)
|
||||
iterateMetadataFiles(local_context, [this](const String & file_name)
|
||||
{
|
||||
const std::string table_name = file_name.substr(0, file_name.size() - 4);
|
||||
|
||||
@ -56,7 +56,7 @@ void DatabaseLazy::loadStoredObjects(
|
||||
|
||||
|
||||
void DatabaseLazy::createTable(
|
||||
const Context & context,
|
||||
ContextPtr local_context,
|
||||
const String & table_name,
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query)
|
||||
@ -64,7 +64,7 @@ void DatabaseLazy::createTable(
|
||||
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
|
||||
if (!endsWith(table->getName(), "Log"))
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
DatabaseOnDisk::createTable(context, table_name, table, query);
|
||||
DatabaseOnDisk::createTable(local_context, table_name, table, query);
|
||||
|
||||
/// DatabaseOnDisk::createTable renames file, so we need to get new metadata_modification_time.
|
||||
std::lock_guard lock(mutex);
|
||||
@ -74,16 +74,16 @@ void DatabaseLazy::createTable(
|
||||
}
|
||||
|
||||
void DatabaseLazy::dropTable(
|
||||
const Context & context,
|
||||
ContextPtr local_context,
|
||||
const String & table_name,
|
||||
bool no_delay)
|
||||
{
|
||||
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
|
||||
DatabaseOnDisk::dropTable(context, table_name, no_delay);
|
||||
DatabaseOnDisk::dropTable(local_context, table_name, no_delay);
|
||||
}
|
||||
|
||||
void DatabaseLazy::renameTable(
|
||||
const Context & context,
|
||||
ContextPtr local_context,
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
@ -91,7 +91,7 @@ void DatabaseLazy::renameTable(
|
||||
bool dictionary)
|
||||
{
|
||||
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
|
||||
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
|
||||
DatabaseOnDisk::renameTable(local_context, table_name, to_database, to_table_name, exchange, dictionary);
|
||||
}
|
||||
|
||||
|
||||
@ -105,7 +105,7 @@ time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name
|
||||
}
|
||||
|
||||
void DatabaseLazy::alterTable(
|
||||
const Context & /* context */,
|
||||
ContextPtr /* context */,
|
||||
const StorageID & /*table_id*/,
|
||||
const StorageInMemoryMetadata & /* metadata */)
|
||||
{
|
||||
@ -142,7 +142,7 @@ StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const
|
||||
return loadTable(table_name);
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
Strings filtered_tables;
|
||||
@ -233,9 +233,9 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const
|
||||
try
|
||||
{
|
||||
StoragePtr table;
|
||||
Context context_copy(global_context); /// some tables can change context, but not LogTables
|
||||
auto context_copy = Context::createCopy(context); /// some tables can change context, but not LogTables
|
||||
|
||||
auto ast = parseQueryFromMetadata(log, global_context, table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false);
|
||||
auto ast = parseQueryFromMetadata(log, getContext(), table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false);
|
||||
if (ast)
|
||||
{
|
||||
const auto & ast_create = ast->as<const ASTCreateQuery &>();
|
||||
|
@ -18,7 +18,7 @@ class Context;
|
||||
class DatabaseLazy final : public DatabaseOnDisk
|
||||
{
|
||||
public:
|
||||
DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_);
|
||||
DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, ContextPtr context_);
|
||||
|
||||
String getEngineName() const override { return "Lazy"; }
|
||||
|
||||
@ -27,22 +27,22 @@ public:
|
||||
bool canContainDistributedTables() const override { return false; }
|
||||
|
||||
void loadStoredObjects(
|
||||
Context & context,
|
||||
ContextPtr context,
|
||||
bool has_force_restore_data_flag, bool force_attach) override;
|
||||
|
||||
void createTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & table_name,
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void dropTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & table_name,
|
||||
bool no_delay) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
@ -50,21 +50,21 @@ public:
|
||||
bool dictionary) override;
|
||||
|
||||
void alterTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const StorageID & table_id,
|
||||
const StorageInMemoryMetadata & metadata) override;
|
||||
|
||||
time_t getObjectMetadataModificationTime(const String & table_name) const override;
|
||||
|
||||
bool isTableExist(const String & table_name, const Context &) const override { return isTableExist(table_name); }
|
||||
bool isTableExist(const String & table_name, ContextPtr) const override { return isTableExist(table_name); }
|
||||
bool isTableExist(const String & table_name) const;
|
||||
|
||||
StoragePtr tryGetTable(const String & table_name, const Context &) const override { return tryGetTable(table_name); }
|
||||
StoragePtr tryGetTable(const String & table_name, ContextPtr) const override { return tryGetTable(table_name); }
|
||||
StoragePtr tryGetTable(const String & table_name) const;
|
||||
|
||||
bool empty() const override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override;
|
||||
|
||||
|
@ -16,13 +16,13 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_TABLE;
|
||||
}
|
||||
|
||||
DatabaseMemory::DatabaseMemory(const String & name_, const Context & context)
|
||||
: DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")", context)
|
||||
DatabaseMemory::DatabaseMemory(const String & name_, ContextPtr context_)
|
||||
: DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")", context_)
|
||||
, data_path("data/" + escapeForFileName(database_name) + "/")
|
||||
{}
|
||||
|
||||
void DatabaseMemory::createTable(
|
||||
const Context & /*context*/,
|
||||
ContextPtr /*context*/,
|
||||
const String & table_name,
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query)
|
||||
@ -33,7 +33,7 @@ void DatabaseMemory::createTable(
|
||||
}
|
||||
|
||||
void DatabaseMemory::dropTable(
|
||||
const Context & /*context*/,
|
||||
ContextPtr /*context*/,
|
||||
const String & table_name,
|
||||
bool /*no_delay*/)
|
||||
{
|
||||
@ -67,7 +67,7 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery() const
|
||||
return create_query;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const
|
||||
ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = create_queries.find(table_name);
|
||||
@ -83,15 +83,15 @@ ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, const
|
||||
|
||||
UUID DatabaseMemory::tryGetTableUUID(const String & table_name) const
|
||||
{
|
||||
if (auto table = tryGetTable(table_name, global_context))
|
||||
if (auto table = tryGetTable(table_name, getContext()))
|
||||
return table->getStorageID().uuid;
|
||||
return UUIDHelpers::Nil;
|
||||
}
|
||||
|
||||
void DatabaseMemory::drop(const Context & context)
|
||||
void DatabaseMemory::drop(ContextPtr local_context)
|
||||
{
|
||||
/// Remove data on explicit DROP DATABASE
|
||||
std::filesystem::remove_all(context.getPath() + data_path);
|
||||
std::filesystem::remove_all(local_context->getPath() + data_path);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,22 +19,22 @@ namespace DB
|
||||
class DatabaseMemory final : public DatabaseWithOwnTablesBase
|
||||
{
|
||||
public:
|
||||
DatabaseMemory(const String & name_, const Context & context);
|
||||
DatabaseMemory(const String & name_, ContextPtr context);
|
||||
|
||||
String getEngineName() const override { return "Memory"; }
|
||||
|
||||
void createTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & table_name,
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void dropTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & table_name,
|
||||
bool no_delay) override;
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(const String & name, const Context & context, bool throw_on_error) const override;
|
||||
ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override;
|
||||
ASTPtr getCreateDatabaseQuery() const override;
|
||||
|
||||
/// DatabaseMemory allows to create tables, which store data on disk.
|
||||
@ -46,7 +46,7 @@ public:
|
||||
|
||||
UUID tryGetTableUUID(const String & table_name) const override;
|
||||
|
||||
void drop(const Context & context) override;
|
||||
void drop(ContextPtr context) override;
|
||||
|
||||
private:
|
||||
String data_path;
|
||||
|
@ -46,7 +46,7 @@ std::pair<String, StoragePtr> createTableFromAST(
|
||||
ASTCreateQuery ast_create_query,
|
||||
const String & database_name,
|
||||
const String & table_data_path_relative,
|
||||
Context & context,
|
||||
ContextPtr context,
|
||||
bool has_force_restore_data_flag)
|
||||
{
|
||||
ast_create_query.attach = true;
|
||||
@ -79,7 +79,7 @@ std::pair<String, StoragePtr> createTableFromAST(
|
||||
ast_create_query,
|
||||
table_data_path_relative,
|
||||
context,
|
||||
context.getGlobalContext(),
|
||||
context->getGlobalContext(),
|
||||
columns,
|
||||
constraints,
|
||||
has_force_restore_data_flag)
|
||||
@ -188,23 +188,23 @@ DatabaseOnDisk::DatabaseOnDisk(
|
||||
const String & metadata_path_,
|
||||
const String & data_path_,
|
||||
const String & logger,
|
||||
const Context & context)
|
||||
: DatabaseWithOwnTablesBase(name, logger, context)
|
||||
ContextPtr local_context)
|
||||
: DatabaseWithOwnTablesBase(name, logger, local_context)
|
||||
, metadata_path(metadata_path_)
|
||||
, data_path(data_path_)
|
||||
{
|
||||
Poco::File(context.getPath() + data_path).createDirectories();
|
||||
Poco::File(local_context->getPath() + data_path).createDirectories();
|
||||
Poco::File(metadata_path).createDirectories();
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOnDisk::createTable(
|
||||
const Context & context,
|
||||
ContextPtr local_context,
|
||||
const String & table_name,
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query)
|
||||
{
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = local_context->getSettingsRef();
|
||||
const auto & create = query->as<ASTCreateQuery &>();
|
||||
assert(table_name == create.table);
|
||||
|
||||
@ -221,17 +221,19 @@ void DatabaseOnDisk::createTable(
|
||||
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
||||
|
||||
if (isDictionaryExist(table_name))
|
||||
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
throw Exception(
|
||||
ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
|
||||
if (isTableExist(table_name, global_context))
|
||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
if (isTableExist(table_name, getContext()))
|
||||
throw Exception(
|
||||
ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
|
||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||
|
||||
if (create.attach_short_syntax)
|
||||
{
|
||||
/// Metadata already exists, table was detached
|
||||
removeDetachedPermanentlyFlag(context, table_name, table_metadata_path, true);
|
||||
removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, true);
|
||||
attachTable(table_name, table, getTableDataPath(create));
|
||||
return;
|
||||
}
|
||||
@ -241,7 +243,7 @@ void DatabaseOnDisk::createTable(
|
||||
|
||||
if (create.attach && Poco::File(table_metadata_path).exists())
|
||||
{
|
||||
ASTPtr ast_detached = parseQueryFromMetadata(log, context, table_metadata_path);
|
||||
ASTPtr ast_detached = parseQueryFromMetadata(log, local_context, table_metadata_path);
|
||||
auto & create_detached = ast_detached->as<ASTCreateQuery &>();
|
||||
|
||||
// either both should be Nil, either values should be equal
|
||||
@ -268,14 +270,14 @@ void DatabaseOnDisk::createTable(
|
||||
out.close();
|
||||
}
|
||||
|
||||
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, context);
|
||||
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, local_context);
|
||||
|
||||
removeDetachedPermanentlyFlag(context, table_name, table_metadata_path, false);
|
||||
removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, false);
|
||||
}
|
||||
|
||||
/// If the table was detached permanently we will have a flag file with
|
||||
/// .sql.detached extension, is not needed anymore since we attached the table back
|
||||
void DatabaseOnDisk::removeDetachedPermanentlyFlag(const Context &, const String & table_name, const String & table_metadata_path, bool) const
|
||||
void DatabaseOnDisk::removeDetachedPermanentlyFlag(ContextPtr, const String & table_name, const String & table_metadata_path, bool) const
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -293,7 +295,7 @@ void DatabaseOnDisk::removeDetachedPermanentlyFlag(const Context &, const String
|
||||
|
||||
void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path,
|
||||
const Context & /*query_context*/)
|
||||
ContextPtr /*query_context*/)
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -311,7 +313,7 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::detachTablePermanently(const Context &, const String & table_name)
|
||||
void DatabaseOnDisk::detachTablePermanently(ContextPtr, const String & table_name)
|
||||
{
|
||||
auto table = detachTable(table_name);
|
||||
|
||||
@ -327,7 +329,7 @@ void DatabaseOnDisk::detachTablePermanently(const Context &, const String & tabl
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::dropTable(const Context & context, const String & table_name, bool /*no_delay*/)
|
||||
void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_name, bool /*no_delay*/)
|
||||
{
|
||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||
String table_metadata_path_drop = table_metadata_path + drop_suffix;
|
||||
@ -349,7 +351,7 @@ void DatabaseOnDisk::dropTable(const Context & context, const String & table_nam
|
||||
table->drop();
|
||||
table->is_dropped = true;
|
||||
|
||||
Poco::File table_data_dir{context.getPath() + table_data_path_relative};
|
||||
Poco::File table_data_dir{local_context->getPath() + table_data_path_relative};
|
||||
if (table_data_dir.exists())
|
||||
table_data_dir.remove(true);
|
||||
}
|
||||
@ -387,7 +389,7 @@ void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::renameTable(
|
||||
const Context & context,
|
||||
ContextPtr local_context,
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
@ -418,15 +420,16 @@ void DatabaseOnDisk::renameTable(
|
||||
String table_metadata_path;
|
||||
ASTPtr attach_query;
|
||||
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
|
||||
StoragePtr table = tryGetTable(table_name, global_context);
|
||||
StoragePtr table = tryGetTable(table_name, getContext());
|
||||
detachTable(table_name);
|
||||
UUID prev_uuid = UUIDHelpers::Nil;
|
||||
try
|
||||
{
|
||||
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
table_lock = table->lockExclusively(
|
||||
local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout);
|
||||
|
||||
table_metadata_path = getObjectMetadataPath(table_name);
|
||||
attach_query = parseQueryFromMetadata(log, context, table_metadata_path);
|
||||
attach_query = parseQueryFromMetadata(log, local_context, table_metadata_path);
|
||||
auto & create = attach_query->as<ASTCreateQuery &>();
|
||||
create.database = to_database.getDatabaseName();
|
||||
create.table = to_table_name;
|
||||
@ -454,7 +457,7 @@ void DatabaseOnDisk::renameTable(
|
||||
}
|
||||
|
||||
/// Now table data are moved to new database, so we must add metadata and attach table to new database
|
||||
to_database.createTable(context, to_table_name, table, attach_query);
|
||||
to_database.createTable(local_context, to_table_name, table, attach_query);
|
||||
|
||||
Poco::File(table_metadata_path).remove();
|
||||
|
||||
@ -473,10 +476,10 @@ void DatabaseOnDisk::renameTable(
|
||||
|
||||
|
||||
/// It returns create table statement (even if table is detached)
|
||||
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const
|
||||
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const
|
||||
{
|
||||
ASTPtr ast;
|
||||
bool has_table = tryGetTable(table_name, global_context) != nullptr;
|
||||
bool has_table = tryGetTable(table_name, getContext()) != nullptr;
|
||||
auto table_metadata_path = getObjectMetadataPath(table_name);
|
||||
try
|
||||
{
|
||||
@ -497,11 +500,11 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
|
||||
{
|
||||
ASTPtr ast;
|
||||
|
||||
auto settings = global_context.getSettingsRef();
|
||||
auto settings = getContext()->getSettingsRef();
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql";
|
||||
ast = parseQueryFromMetadata(log, global_context, database_metadata_path, true);
|
||||
auto database_metadata_path = getContext()->getPath() + "metadata/" + escapeForFileName(database_name) + ".sql";
|
||||
ast = parseQueryFromMetadata(log, getContext(), database_metadata_path, true);
|
||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||
ast_create_query.attach = false;
|
||||
ast_create_query.database = database_name;
|
||||
@ -518,10 +521,10 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
|
||||
return ast;
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::drop(const Context & context)
|
||||
void DatabaseOnDisk::drop(ContextPtr local_context)
|
||||
{
|
||||
assert(tables.empty());
|
||||
Poco::File(context.getPath() + getDataPath()).remove(false);
|
||||
Poco::File(local_context->getPath() + getDataPath()).remove(false);
|
||||
Poco::File(getMetadataPath()).remove(false);
|
||||
}
|
||||
|
||||
@ -541,14 +544,14 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const
|
||||
void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const
|
||||
{
|
||||
auto process_tmp_drop_metadata_file = [&](const String & file_name)
|
||||
{
|
||||
assert(getUUID() == UUIDHelpers::Nil);
|
||||
static const char * tmp_drop_ext = ".sql.tmp_drop";
|
||||
const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext));
|
||||
if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists())
|
||||
if (Poco::File(local_context->getPath() + getDataPath() + '/' + object_name).exists())
|
||||
{
|
||||
Poco::File(getMetadataPath() + file_name).renameTo(getMetadataPath() + object_name + ".sql");
|
||||
LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name));
|
||||
@ -615,7 +618,12 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati
|
||||
pool.wait();
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Context & context, const String & metadata_file_path, bool throw_on_error /*= true*/, bool remove_empty /*= false*/)
|
||||
ASTPtr DatabaseOnDisk::parseQueryFromMetadata(
|
||||
Poco::Logger * logger,
|
||||
ContextPtr local_context,
|
||||
const String & metadata_file_path,
|
||||
bool throw_on_error /*= true*/,
|
||||
bool remove_empty /*= false*/)
|
||||
{
|
||||
String query;
|
||||
|
||||
@ -643,7 +651,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
auto settings = context.getSettingsRef();
|
||||
auto settings = local_context->getSettingsRef();
|
||||
ParserCreateQuery parser;
|
||||
const char * pos = query.data();
|
||||
std::string error_message;
|
||||
@ -662,7 +670,12 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte
|
||||
table_name = unescapeForFileName(table_name);
|
||||
|
||||
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER && logger)
|
||||
LOG_WARNING(logger, "File {} contains both UUID and table name. Will use name `{}` instead of `{}`", metadata_file_path, table_name, create.table);
|
||||
LOG_WARNING(
|
||||
logger,
|
||||
"File {} contains both UUID and table name. Will use name `{}` instead of `{}`",
|
||||
metadata_file_path,
|
||||
table_name,
|
||||
create.table);
|
||||
create.table = table_name;
|
||||
}
|
||||
|
||||
@ -671,7 +684,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metadata_path, bool throw_on_error) const
|
||||
{
|
||||
ASTPtr ast = parseQueryFromMetadata(log, global_context, database_metadata_path, throw_on_error);
|
||||
ASTPtr ast = parseQueryFromMetadata(log, getContext(), database_metadata_path, throw_on_error);
|
||||
|
||||
if (ast)
|
||||
{
|
||||
|
@ -16,7 +16,7 @@ std::pair<String, StoragePtr> createTableFromAST(
|
||||
ASTCreateQuery ast_create_query,
|
||||
const String & database_name,
|
||||
const String & table_data_path_relative,
|
||||
Context & context,
|
||||
ContextPtr context,
|
||||
bool has_force_restore_data_flag);
|
||||
|
||||
/** Get the string with the table definition based on the CREATE query.
|
||||
@ -33,23 +33,23 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo
|
||||
class DatabaseOnDisk : public DatabaseWithOwnTablesBase
|
||||
{
|
||||
public:
|
||||
DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context);
|
||||
DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context);
|
||||
|
||||
void createTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & table_name,
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void detachTablePermanently(const Context & context, const String & table_name) override;
|
||||
void detachTablePermanently(ContextPtr context, const String & table_name) override;
|
||||
|
||||
void dropTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & table_name,
|
||||
bool no_delay) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
@ -58,7 +58,7 @@ public:
|
||||
|
||||
ASTPtr getCreateDatabaseQuery() const override;
|
||||
|
||||
void drop(const Context & context) override;
|
||||
void drop(ContextPtr context) override;
|
||||
|
||||
String getObjectMetadataPath(const String & object_name) const override;
|
||||
|
||||
@ -69,7 +69,7 @@ public:
|
||||
String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.table); }
|
||||
String getMetadataPath() const override { return metadata_path; }
|
||||
|
||||
static ASTPtr parseQueryFromMetadata(Poco::Logger * log, const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false);
|
||||
static ASTPtr parseQueryFromMetadata(Poco::Logger * log, ContextPtr context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false);
|
||||
|
||||
/// will throw when the table we want to attach already exists (in active / detached / detached permanently form)
|
||||
void checkMetadataFilenameAvailability(const String & to_table_name) const;
|
||||
@ -82,19 +82,19 @@ protected:
|
||||
|
||||
using IteratingFunction = std::function<void(const String &)>;
|
||||
|
||||
void iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const;
|
||||
void iterateMetadataFiles(ContextPtr context, const IteratingFunction & process_metadata_file) const;
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(
|
||||
const String & table_name,
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
bool throw_on_error) const override;
|
||||
|
||||
ASTPtr getCreateQueryFromMetadata(const String & metadata_path, bool throw_on_error) const;
|
||||
|
||||
virtual void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path, const Context & query_context);
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context);
|
||||
|
||||
virtual void removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path, bool attach) const;
|
||||
virtual void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach) const;
|
||||
|
||||
const String metadata_path;
|
||||
const String data_path;
|
||||
|
@ -36,7 +36,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
||||
namespace
|
||||
{
|
||||
void tryAttachTable(
|
||||
Context & context,
|
||||
ContextPtr context,
|
||||
const ASTCreateQuery & query,
|
||||
DatabaseOrdinary & database,
|
||||
const String & database_name,
|
||||
@ -62,7 +62,7 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
void tryAttachDictionary(const ASTPtr & query, DatabaseOrdinary & database, const String & metadata_path, const Context & context)
|
||||
void tryAttachDictionary(const ASTPtr & query, DatabaseOrdinary & database, const String & metadata_path, ContextPtr context)
|
||||
{
|
||||
auto & create_query = query->as<ASTCreateQuery &>();
|
||||
assert(create_query.is_dictionary);
|
||||
@ -94,18 +94,18 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context_)
|
||||
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context_)
|
||||
: DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_)
|
||||
{
|
||||
}
|
||||
|
||||
DatabaseOrdinary::DatabaseOrdinary(
|
||||
const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context_)
|
||||
const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context_)
|
||||
: DatabaseWithDictionaries(name_, metadata_path_, data_path_, logger, context_)
|
||||
{
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool /*force_attach*/)
|
||||
void DatabaseOrdinary::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool /*force_attach*/)
|
||||
{
|
||||
/** Tables load faster if they are loaded in sorted (by name) order.
|
||||
* Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order,
|
||||
@ -117,7 +117,8 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto
|
||||
|
||||
size_t total_dictionaries = 0;
|
||||
|
||||
auto process_metadata = [&context, &file_names, &total_dictionaries, &file_names_mutex, this](const String & file_name)
|
||||
auto process_metadata = [context_weak = ContextWeakPtr(local_context), &file_names, &total_dictionaries, &file_names_mutex, this](
|
||||
const String & file_name)
|
||||
{
|
||||
fs::path path(getMetadataPath());
|
||||
fs::path file_path(file_name);
|
||||
@ -125,7 +126,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto
|
||||
|
||||
try
|
||||
{
|
||||
auto ast = parseQueryFromMetadata(log, context, full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false);
|
||||
auto ast = parseQueryFromMetadata(log, getContext(), full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false);
|
||||
if (ast)
|
||||
{
|
||||
auto * create_query = ast->as<ASTCreateQuery>();
|
||||
@ -155,7 +156,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto
|
||||
}
|
||||
};
|
||||
|
||||
iterateMetadataFiles(context, process_metadata);
|
||||
iterateMetadataFiles(local_context, process_metadata);
|
||||
|
||||
size_t total_tables = file_names.size() - total_dictionaries;
|
||||
|
||||
@ -180,7 +181,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto
|
||||
auto create_query = query->as<const ASTCreateQuery &>();
|
||||
if (create_query.is_dictionary)
|
||||
{
|
||||
tryAttachDictionary(query, *this, getMetadataPath() + name, context);
|
||||
tryAttachDictionary(query, *this, getMetadataPath() + name, local_context);
|
||||
|
||||
/// Messages, so that it's not boring to wait for the server to load for a long time.
|
||||
logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch);
|
||||
@ -195,7 +196,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto
|
||||
pool.scheduleOrThrowOnError([&]()
|
||||
{
|
||||
tryAttachTable(
|
||||
context,
|
||||
local_context,
|
||||
create_query,
|
||||
*this,
|
||||
database_name,
|
||||
@ -245,7 +246,7 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
|
||||
thread_pool.wait();
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata)
|
||||
void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata)
|
||||
{
|
||||
String table_name = table_id.table_name;
|
||||
/// Read the definition of the table and replace the necessary parts with new ones.
|
||||
@ -265,7 +266,7 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab
|
||||
statement.data() + statement.size(),
|
||||
"in file " + table_metadata_path,
|
||||
0,
|
||||
context.getSettingsRef().max_parser_depth);
|
||||
local_context->getSettingsRef().max_parser_depth);
|
||||
|
||||
applyMetadataChangesToCreateQuery(ast, metadata);
|
||||
|
||||
@ -274,15 +275,15 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab
|
||||
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
|
||||
writeString(statement, out);
|
||||
out.next();
|
||||
if (context.getSettingsRef().fsync_metadata)
|
||||
if (local_context->getSettingsRef().fsync_metadata)
|
||||
out.sync();
|
||||
out.close();
|
||||
}
|
||||
|
||||
commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, context);
|
||||
commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, local_context);
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & /*statement*/, const Context & /*query_context*/)
|
||||
void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & /*statement*/, ContextPtr /*query_context*/)
|
||||
{
|
||||
try
|
||||
{
|
||||
|
@ -14,23 +14,26 @@ namespace DB
|
||||
class DatabaseOrdinary : public DatabaseWithDictionaries
|
||||
{
|
||||
public:
|
||||
DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context);
|
||||
DatabaseOrdinary(const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context_);
|
||||
DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context);
|
||||
DatabaseOrdinary(
|
||||
const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context_);
|
||||
|
||||
String getEngineName() const override { return "Ordinary"; }
|
||||
|
||||
void loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag,
|
||||
bool force_attach) override;
|
||||
void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
|
||||
void alterTable(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const StorageID & table_id,
|
||||
const StorageInMemoryMetadata & metadata) override;
|
||||
|
||||
protected:
|
||||
virtual void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, const Context & query_context);
|
||||
virtual void commitAlterTable(
|
||||
const StorageID & table_id,
|
||||
const String & table_metadata_tmp_path,
|
||||
const String & table_metadata_path,
|
||||
const String & statement,
|
||||
ContextPtr query_context);
|
||||
|
||||
void startupTables(ThreadPool & thread_pool);
|
||||
};
|
||||
|
@ -47,12 +47,12 @@ static constexpr const char * BROKEN_TABLES_SUFFIX = "_broken_tables";
|
||||
|
||||
zkutil::ZooKeeperPtr DatabaseReplicated::getZooKeeper() const
|
||||
{
|
||||
return global_context.getZooKeeper();
|
||||
return getContext()->getZooKeeper();
|
||||
}
|
||||
|
||||
static inline String getHostID(const Context & global_context, const UUID & db_uuid)
|
||||
static inline String getHostID(ContextPtr global_context, const UUID & db_uuid)
|
||||
{
|
||||
return Cluster::Address::toString(getFQDNOrHostName(), global_context.getTCPPort()) + ':' + toString(db_uuid);
|
||||
return Cluster::Address::toString(getFQDNOrHostName(), global_context->getTCPPort()) + ':' + toString(db_uuid);
|
||||
}
|
||||
|
||||
|
||||
@ -66,7 +66,7 @@ DatabaseReplicated::DatabaseReplicated(
|
||||
const String & shard_name_,
|
||||
const String & replica_name_,
|
||||
DatabaseReplicatedSettings db_settings_,
|
||||
const Context & context_)
|
||||
ContextPtr context_)
|
||||
: DatabaseAtomic(name_, metadata_path_, uuid, "DatabaseReplicated (" + name_ + ")", context_)
|
||||
, zookeeper_path(zookeeper_path_)
|
||||
, shard_name(shard_name_)
|
||||
@ -126,7 +126,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const
|
||||
Strings hosts;
|
||||
Strings host_ids;
|
||||
|
||||
auto zookeeper = global_context.getZooKeeper();
|
||||
auto zookeeper = getContext()->getZooKeeper();
|
||||
constexpr int max_retries = 10;
|
||||
int iteration = 0;
|
||||
bool success = false;
|
||||
@ -189,22 +189,22 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const
|
||||
|
||||
String username = db_settings.cluster_username;
|
||||
String password = db_settings.cluster_password;
|
||||
UInt16 default_port = global_context.getTCPPort();
|
||||
UInt16 default_port = getContext()->getTCPPort();
|
||||
bool secure = db_settings.cluster_secure_connection;
|
||||
|
||||
return std::make_shared<Cluster>(global_context.getSettingsRef(), shards, username, password, default_port, false, secure);
|
||||
return std::make_shared<Cluster>(getContext()->getSettingsRef(), shards, username, password, default_port, false, secure);
|
||||
}
|
||||
|
||||
void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(bool force_attach)
|
||||
{
|
||||
try
|
||||
{
|
||||
if (!global_context.hasZooKeeper())
|
||||
if (!getContext()->hasZooKeeper())
|
||||
{
|
||||
throw Exception("Can't create replicated database without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
|
||||
}
|
||||
|
||||
auto current_zookeeper = global_context.getZooKeeper();
|
||||
auto current_zookeeper = getContext()->getZooKeeper();
|
||||
|
||||
if (!current_zookeeper->exists(zookeeper_path))
|
||||
{
|
||||
@ -217,7 +217,7 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(bool force_attach)
|
||||
String replica_host_id;
|
||||
if (current_zookeeper->tryGet(replica_path, replica_host_id))
|
||||
{
|
||||
String host_id = getHostID(global_context, db_uuid);
|
||||
String host_id = getHostID(getContext(), db_uuid);
|
||||
if (replica_host_id != host_id)
|
||||
throw Exception(ErrorCodes::REPLICA_IS_ALREADY_EXIST,
|
||||
"Replica {} of shard {} of replicated database at {} already exists. Replica host ID: '{}', current host ID: '{}'",
|
||||
@ -291,7 +291,7 @@ void DatabaseReplicated::createEmptyLogEntry(Coordination::Requests & ops, const
|
||||
void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPtr & current_zookeeper)
|
||||
{
|
||||
/// Write host name to replica_path, it will protect from multiple replicas with the same name
|
||||
auto host_id = getHostID(global_context, db_uuid);
|
||||
auto host_id = getHostID(getContext(), db_uuid);
|
||||
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent));
|
||||
@ -300,17 +300,17 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt
|
||||
current_zookeeper->multi(ops);
|
||||
}
|
||||
|
||||
void DatabaseReplicated::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach)
|
||||
void DatabaseReplicated::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach)
|
||||
{
|
||||
tryConnectToZooKeeperAndInitDatabase(force_attach);
|
||||
|
||||
DatabaseAtomic::loadStoredObjects(context, has_force_restore_data_flag, force_attach);
|
||||
DatabaseAtomic::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach);
|
||||
|
||||
ddl_worker = std::make_unique<DatabaseReplicatedDDLWorker>(this, global_context);
|
||||
ddl_worker = std::make_unique<DatabaseReplicatedDDLWorker>(this, getContext());
|
||||
ddl_worker->startup();
|
||||
}
|
||||
|
||||
void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & query_context) const
|
||||
void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_context) const
|
||||
{
|
||||
/// Replicas will set correct name of current database in query context (database name can be different on replicas)
|
||||
if (auto * ddl_query = dynamic_cast<ASTQueryWithTableAndOutput *>(query.get()))
|
||||
@ -343,12 +343,12 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & q
|
||||
|
||||
Macros::MacroExpansionInfo info;
|
||||
info.table_id = {getDatabaseName(), create->table, create->uuid};
|
||||
query_context.getMacros()->expand(maybe_path, info);
|
||||
query_context->getMacros()->expand(maybe_path, info);
|
||||
bool maybe_shard_macros = info.expanded_other;
|
||||
info.expanded_other = false;
|
||||
query_context.getMacros()->expand(maybe_replica, info);
|
||||
query_context->getMacros()->expand(maybe_replica, info);
|
||||
bool maybe_replica_macros = info.expanded_other;
|
||||
bool enable_functional_tests_helper = global_context.getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros");
|
||||
bool enable_functional_tests_helper = getContext()->getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros");
|
||||
|
||||
if (!enable_functional_tests_helper)
|
||||
LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments");
|
||||
@ -383,7 +383,7 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & q
|
||||
|
||||
if (auto * query_drop = query->as<ASTDropQuery>())
|
||||
{
|
||||
if (query_drop->kind == ASTDropQuery::Kind::Detach && query_context.getSettingsRef().database_replicated_always_detach_permanently)
|
||||
if (query_drop->kind == ASTDropQuery::Kind::Detach && query_context->getSettingsRef().database_replicated_always_detach_permanently)
|
||||
query_drop->permanently = true;
|
||||
if (query_drop->kind == ASTDropQuery::Kind::Detach && !query_drop->permanently)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH TABLE is not allowed for Replicated databases. "
|
||||
@ -392,12 +392,12 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & q
|
||||
}
|
||||
}
|
||||
|
||||
BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context)
|
||||
BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context)
|
||||
{
|
||||
if (is_readonly)
|
||||
throw Exception(ErrorCodes::NO_ZOOKEEPER, "Database is in readonly mode, because it cannot connect to ZooKeeper");
|
||||
|
||||
if (query_context.getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY)
|
||||
if (query_context->getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "It's not initial query. ON CLUSTER is not allowed for Replicated database.");
|
||||
|
||||
checkQueryValid(query, query_context);
|
||||
@ -413,15 +413,15 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const
|
||||
return getDistributedDDLStatus(node_path, entry, query_context, hosts_to_wait);
|
||||
}
|
||||
|
||||
static UUID getTableUUIDIfReplicated(const String & metadata, const Context & context)
|
||||
static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context)
|
||||
{
|
||||
bool looks_like_replicated = metadata.find("ReplicatedMergeTree") != std::string::npos;
|
||||
if (!looks_like_replicated)
|
||||
return UUIDHelpers::Nil;
|
||||
|
||||
ParserCreateQuery parser;
|
||||
auto size = context.getSettingsRef().max_query_size;
|
||||
auto depth = context.getSettingsRef().max_parser_depth;
|
||||
auto size = context->getSettingsRef().max_query_size;
|
||||
auto depth = context->getSettingsRef().max_parser_depth;
|
||||
ASTPtr query = parseQuery(parser, metadata, size, depth);
|
||||
const ASTCreateQuery & create = query->as<const ASTCreateQuery &>();
|
||||
if (!create.storage || !create.storage->engine)
|
||||
@ -457,7 +457,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
std::unordered_map<UUID, String> zk_replicated_id_to_name;
|
||||
for (const auto & zk_table : table_name_to_metadata)
|
||||
{
|
||||
UUID zk_replicated_id = getTableUUIDIfReplicated(zk_table.second, global_context);
|
||||
UUID zk_replicated_id = getTableUUIDIfReplicated(zk_table.second, getContext());
|
||||
if (zk_replicated_id != UUIDHelpers::Nil)
|
||||
zk_replicated_id_to_name.emplace(zk_replicated_id, zk_table.first);
|
||||
}
|
||||
@ -467,7 +467,8 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
std::vector<std::pair<String, String>> replicated_tables_to_rename;
|
||||
size_t total_tables = 0;
|
||||
std::vector<UUID> replicated_ids;
|
||||
for (auto existing_tables_it = getTablesIterator(global_context, {}); existing_tables_it->isValid(); existing_tables_it->next(), ++total_tables)
|
||||
for (auto existing_tables_it = getTablesIterator(getContext(), {}); existing_tables_it->isValid();
|
||||
existing_tables_it->next(), ++total_tables)
|
||||
{
|
||||
String name = existing_tables_it->name();
|
||||
UUID local_replicated_id = UUIDHelpers::Nil;
|
||||
@ -506,7 +507,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
/// We use Ordinary engine for destination database, because it's the only way to discard table UUID
|
||||
/// and make possible creation of new table with the same UUID.
|
||||
String query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Ordinary", backQuoteIfNeed(to_db_name));
|
||||
Context query_context = global_context;
|
||||
auto query_context = Context::createCopy(getContext());
|
||||
executeQuery(query, query_context, true);
|
||||
}
|
||||
|
||||
@ -519,12 +520,12 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
if (getDatabaseName() != db_name)
|
||||
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed, will retry");
|
||||
|
||||
auto table = tryGetTable(table_name, global_context);
|
||||
auto table = tryGetTable(table_name, getContext());
|
||||
if (isDictionaryExist(table_name))
|
||||
{
|
||||
/// We can safely drop any dictionaries because they do not store data
|
||||
LOG_DEBUG(log, "Will DROP DICTIONARY {}", backQuoteIfNeed(table_name));
|
||||
DatabaseAtomic::removeDictionary(global_context, table_name);
|
||||
DatabaseAtomic::removeDictionary(getContext(), table_name);
|
||||
++dropped_dicts;
|
||||
}
|
||||
else if (!table->storesDataOnDisk())
|
||||
@ -532,7 +533,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
LOG_DEBUG(log, "Will DROP TABLE {}, because it does not store data on disk and can be safely dropped", backQuoteIfNeed(table_name));
|
||||
dropped_tables.push_back(tryGetTableUUID(table_name));
|
||||
table->shutdown();
|
||||
DatabaseAtomic::dropTable(global_context, table_name, true);
|
||||
DatabaseAtomic::dropTable(getContext(), table_name, true);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -542,7 +543,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
assert(db_name < to_db_name);
|
||||
DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(to_db_name, to_name);
|
||||
auto to_db_ptr = DatabaseCatalog::instance().getDatabase(to_db_name);
|
||||
DatabaseAtomic::renameTable(global_context, table_name, *to_db_ptr, to_name, false, false);
|
||||
DatabaseAtomic::renameTable(getContext(), table_name, *to_db_ptr, to_name, false, false);
|
||||
++moved_tables;
|
||||
}
|
||||
}
|
||||
@ -561,7 +562,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
/// TODO Maybe we should do it in two steps: rename all tables to temporary names and then rename them to actual names?
|
||||
DDLGuardPtr table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::min(from, to));
|
||||
DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::max(from, to));
|
||||
DatabaseAtomic::renameTable(global_context, from, *this, to, false, false);
|
||||
DatabaseAtomic::renameTable(getContext(), from, *this, to, false, false);
|
||||
}
|
||||
|
||||
for (const auto & id : dropped_tables)
|
||||
@ -569,7 +570,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
|
||||
for (const auto & name_and_meta : table_name_to_metadata)
|
||||
{
|
||||
if (isTableExist(name_and_meta.first, global_context))
|
||||
if (isTableExist(name_and_meta.first, getContext()))
|
||||
{
|
||||
assert(name_and_meta.second == readMetadataFile(name_and_meta.first));
|
||||
continue;
|
||||
@ -577,11 +578,11 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
|
||||
auto query_ast = parseQueryFromMetadataInZooKeeper(name_and_meta.first, name_and_meta.second);
|
||||
|
||||
Context query_context = global_context;
|
||||
query_context.makeQueryContext();
|
||||
query_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
query_context.setCurrentDatabase(database_name);
|
||||
query_context.setCurrentQueryId(""); // generate random query_id
|
||||
auto query_context = Context::createCopy(getContext());
|
||||
query_context->makeQueryContext();
|
||||
query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
query_context->setCurrentDatabase(database_name);
|
||||
query_context->setCurrentQueryId(""); // generate random query_id
|
||||
|
||||
LOG_INFO(log, "Executing {}", serializeAST(*query_ast));
|
||||
InterpreterCreateQuery(query_ast, query_context).execute();
|
||||
@ -643,7 +644,7 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node
|
||||
{
|
||||
ParserCreateQuery parser;
|
||||
String description = "in ZooKeeper " + zookeeper_path + "/metadata/" + node_name;
|
||||
auto ast = parseQuery(parser, query, description, 0, global_context.getSettingsRef().max_parser_depth);
|
||||
auto ast = parseQuery(parser, query, description, 0, getContext()->getSettingsRef().max_parser_depth);
|
||||
|
||||
auto & create = ast->as<ASTCreateQuery &>();
|
||||
if (create.uuid == UUIDHelpers::Nil || create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER || !create.database.empty())
|
||||
@ -658,7 +659,7 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node
|
||||
return ast;
|
||||
}
|
||||
|
||||
void DatabaseReplicated::drop(const Context & context_)
|
||||
void DatabaseReplicated::drop(ContextPtr context_)
|
||||
{
|
||||
auto current_zookeeper = getZooKeeper();
|
||||
Coordination::Requests ops;
|
||||
@ -691,22 +692,22 @@ void DatabaseReplicated::shutdown()
|
||||
}
|
||||
|
||||
|
||||
void DatabaseReplicated::dropTable(const Context & context, const String & table_name, bool no_delay)
|
||||
void DatabaseReplicated::dropTable(ContextPtr local_context, const String & table_name, bool no_delay)
|
||||
{
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
assert(!ddl_worker->isCurrentlyActive() || txn || startsWith(table_name, ".inner_id."));
|
||||
if (txn && txn->isInitialQuery())
|
||||
{
|
||||
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
|
||||
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1));
|
||||
}
|
||||
DatabaseAtomic::dropTable(context, table_name, no_delay);
|
||||
DatabaseAtomic::dropTable(local_context, table_name, no_delay);
|
||||
}
|
||||
|
||||
void DatabaseReplicated::renameTable(const Context & context, const String & table_name, IDatabase & to_database,
|
||||
void DatabaseReplicated::renameTable(ContextPtr local_context, const String & table_name, IDatabase & to_database,
|
||||
const String & to_table_name, bool exchange, bool dictionary)
|
||||
{
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
assert(txn);
|
||||
|
||||
if (txn->isInitialQuery())
|
||||
@ -715,9 +716,9 @@ void DatabaseReplicated::renameTable(const Context & context, const String & tab
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Moving tables between databases is not supported for Replicated engine");
|
||||
if (table_name == to_table_name)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot rename table to itself");
|
||||
if (!isTableExist(table_name, context))
|
||||
if (!isTableExist(table_name, local_context))
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", table_name);
|
||||
if (exchange && !to_database.isTableExist(to_table_name, context))
|
||||
if (exchange && !to_database.isTableExist(to_table_name, local_context))
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", to_table_name);
|
||||
|
||||
String statement = readMetadataFile(table_name);
|
||||
@ -733,14 +734,14 @@ void DatabaseReplicated::renameTable(const Context & context, const String & tab
|
||||
txn->addOp(zkutil::makeCreateRequest(metadata_zk_path_to, statement, zkutil::CreateMode::Persistent));
|
||||
}
|
||||
|
||||
DatabaseAtomic::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
|
||||
DatabaseAtomic::renameTable(local_context, table_name, to_database, to_table_name, exchange, dictionary);
|
||||
}
|
||||
|
||||
void DatabaseReplicated::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path,
|
||||
const Context & query_context)
|
||||
ContextPtr query_context)
|
||||
{
|
||||
auto txn = query_context.getZooKeeperMetadataTransaction();
|
||||
auto txn = query_context->getZooKeeperMetadataTransaction();
|
||||
assert(!ddl_worker->isCurrentlyActive() || txn);
|
||||
if (txn && txn->isInitialQuery())
|
||||
{
|
||||
@ -754,9 +755,9 @@ void DatabaseReplicated::commitCreateTable(const ASTCreateQuery & query, const S
|
||||
|
||||
void DatabaseReplicated::commitAlterTable(const StorageID & table_id,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path,
|
||||
const String & statement, const Context & query_context)
|
||||
const String & statement, ContextPtr query_context)
|
||||
{
|
||||
auto txn = query_context.getZooKeeperMetadataTransaction();
|
||||
auto txn = query_context->getZooKeeperMetadataTransaction();
|
||||
if (txn && txn->isInitialQuery())
|
||||
{
|
||||
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_id.table_name);
|
||||
@ -765,11 +766,11 @@ void DatabaseReplicated::commitAlterTable(const StorageID & table_id,
|
||||
DatabaseAtomic::commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, query_context);
|
||||
}
|
||||
|
||||
void DatabaseReplicated::createDictionary(const Context & context,
|
||||
void DatabaseReplicated::createDictionary(ContextPtr local_context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query)
|
||||
{
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
assert(!ddl_worker->isCurrentlyActive() || txn);
|
||||
if (txn && txn->isInitialQuery())
|
||||
{
|
||||
@ -777,24 +778,24 @@ void DatabaseReplicated::createDictionary(const Context & context,
|
||||
String statement = getObjectDefinitionFromCreateQuery(query->clone());
|
||||
txn->addOp(zkutil::makeCreateRequest(metadata_zk_path, statement, zkutil::CreateMode::Persistent));
|
||||
}
|
||||
DatabaseAtomic::createDictionary(context, dictionary_name, query);
|
||||
DatabaseAtomic::createDictionary(local_context, dictionary_name, query);
|
||||
}
|
||||
|
||||
void DatabaseReplicated::removeDictionary(const Context & context, const String & dictionary_name)
|
||||
void DatabaseReplicated::removeDictionary(ContextPtr local_context, const String & dictionary_name)
|
||||
{
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
assert(!ddl_worker->isCurrentlyActive() || txn);
|
||||
if (txn && txn->isInitialQuery())
|
||||
{
|
||||
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(dictionary_name);
|
||||
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1));
|
||||
}
|
||||
DatabaseAtomic::removeDictionary(context, dictionary_name);
|
||||
DatabaseAtomic::removeDictionary(local_context, dictionary_name);
|
||||
}
|
||||
|
||||
void DatabaseReplicated::detachTablePermanently(const Context & context, const String & table_name)
|
||||
void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name)
|
||||
{
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
assert(!ddl_worker->isCurrentlyActive() || txn);
|
||||
if (txn && txn->isInitialQuery())
|
||||
{
|
||||
@ -803,12 +804,12 @@ void DatabaseReplicated::detachTablePermanently(const Context & context, const S
|
||||
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
|
||||
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1));
|
||||
}
|
||||
DatabaseAtomic::detachTablePermanently(context, table_name);
|
||||
DatabaseAtomic::detachTablePermanently(local_context, table_name);
|
||||
}
|
||||
|
||||
void DatabaseReplicated::removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path, bool attach) const
|
||||
void DatabaseReplicated::removeDetachedPermanentlyFlag(ContextPtr local_context, const String & table_name, const String & table_metadata_path, bool attach) const
|
||||
{
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
assert(!ddl_worker->isCurrentlyActive() || txn);
|
||||
if (txn && txn->isInitialQuery() && attach)
|
||||
{
|
||||
@ -816,7 +817,7 @@ void DatabaseReplicated::removeDetachedPermanentlyFlag(const Context & context,
|
||||
String statement = readMetadataFile(table_name);
|
||||
txn->addOp(zkutil::makeCreateRequest(metadata_zk_path, statement, zkutil::CreateMode::Persistent));
|
||||
}
|
||||
DatabaseAtomic::removeDetachedPermanentlyFlag(context, table_name, table_metadata_path, attach);
|
||||
DatabaseAtomic::removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, attach);
|
||||
}
|
||||
|
||||
|
||||
|
@ -24,32 +24,32 @@ public:
|
||||
DatabaseReplicated(const String & name_, const String & metadata_path_, UUID uuid,
|
||||
const String & zookeeper_path_, const String & shard_name_, const String & replica_name_,
|
||||
DatabaseReplicatedSettings db_settings_,
|
||||
const Context & context);
|
||||
ContextPtr context);
|
||||
|
||||
~DatabaseReplicated() override;
|
||||
|
||||
String getEngineName() const override { return "Replicated"; }
|
||||
|
||||
/// If current query is initial, then the following methods add metadata updating ZooKeeper operations to current ZooKeeperMetadataTransaction.
|
||||
void dropTable(const Context &, const String & table_name, bool no_delay) override;
|
||||
void renameTable(const Context & context, const String & table_name, IDatabase & to_database,
|
||||
void dropTable(ContextPtr, const String & table_name, bool no_delay) override;
|
||||
void renameTable(ContextPtr context, const String & table_name, IDatabase & to_database,
|
||||
const String & to_table_name, bool exchange, bool dictionary) override;
|
||||
void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path,
|
||||
const Context & query_context) override;
|
||||
ContextPtr query_context) override;
|
||||
void commitAlterTable(const StorageID & table_id,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path,
|
||||
const String & statement, const Context & query_context) override;
|
||||
void createDictionary(const Context & context,
|
||||
const String & statement, ContextPtr query_context) override;
|
||||
void createDictionary(ContextPtr context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query) override;
|
||||
void removeDictionary(const Context & context, const String & dictionary_name) override;
|
||||
void detachTablePermanently(const Context & context, const String & table_name) override;
|
||||
void removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path, bool attach) const override;
|
||||
void removeDictionary(ContextPtr context, const String & dictionary_name) override;
|
||||
void detachTablePermanently(ContextPtr context, const String & table_name) override;
|
||||
void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach) const override;
|
||||
|
||||
/// Try to execute DLL query on current host as initial query. If query is succeed,
|
||||
/// then it will be executed on all replicas.
|
||||
BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context);
|
||||
BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context);
|
||||
|
||||
void stopReplication();
|
||||
|
||||
@ -59,9 +59,9 @@ public:
|
||||
/// Returns cluster consisting of database replicas
|
||||
ClusterPtr getCluster() const;
|
||||
|
||||
void drop(const Context & /*context*/) override;
|
||||
void drop(ContextPtr /*context*/) override;
|
||||
|
||||
void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
void shutdown() override;
|
||||
|
||||
friend struct DatabaseReplicatedTask;
|
||||
@ -71,7 +71,7 @@ private:
|
||||
bool createDatabaseNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper);
|
||||
void createReplicaNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper);
|
||||
|
||||
void checkQueryValid(const ASTPtr & query, const Context & query_context) const;
|
||||
void checkQueryValid(const ASTPtr & query, ContextPtr query_context) const;
|
||||
|
||||
void recoverLostReplica(const ZooKeeperPtr & current_zookeeper, UInt32 our_log_ptr, UInt32 max_log_ptr);
|
||||
std::map<String, String> tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr);
|
||||
|
@ -13,7 +13,7 @@ namespace ErrorCodes
|
||||
extern const int UNFINISHED;
|
||||
}
|
||||
|
||||
DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db, const Context & context_)
|
||||
DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db, ContextPtr context_)
|
||||
: DDLWorker(/* pool_size */ 1, db->zookeeper_path + "/log", context_, nullptr, {}, fmt::format("DDLWorker({})", db->getDatabaseName()))
|
||||
, database(db)
|
||||
{
|
||||
@ -91,7 +91,7 @@ String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry)
|
||||
return node_path;
|
||||
}
|
||||
|
||||
String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entry, const Context & query_context)
|
||||
String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entry, ContextPtr query_context)
|
||||
{
|
||||
/// NOTE Possibly it would be better to execute initial query on the most up-to-date node,
|
||||
/// but it requires more complex logic around /try node.
|
||||
@ -115,7 +115,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
|
||||
task->is_initial_query = true;
|
||||
|
||||
LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name);
|
||||
UInt64 timeout = query_context.getSettingsRef().database_replicated_initial_query_timeout_sec;
|
||||
UInt64 timeout = query_context->getSettingsRef().database_replicated_initial_query_timeout_sec;
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
bool processed = wait_current_task_change.wait_for(lock, std::chrono::seconds(timeout), [&]()
|
||||
|
@ -21,11 +21,11 @@ class DatabaseReplicated;
|
||||
class DatabaseReplicatedDDLWorker : public DDLWorker
|
||||
{
|
||||
public:
|
||||
DatabaseReplicatedDDLWorker(DatabaseReplicated * db, const Context & context_);
|
||||
DatabaseReplicatedDDLWorker(DatabaseReplicated * db, ContextPtr context_);
|
||||
|
||||
String enqueueQuery(DDLLogEntry & entry) override;
|
||||
|
||||
String tryEnqueueAndExecuteEntry(DDLLogEntry & entry, const Context & query_context);
|
||||
String tryEnqueueAndExecuteEntry(DDLLogEntry & entry, ContextPtr query_context);
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
|
@ -125,9 +125,9 @@ void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_na
|
||||
detachTable(dictionary_name);
|
||||
}
|
||||
|
||||
void DatabaseWithDictionaries::createDictionary(const Context & context, const String & dictionary_name, const ASTPtr & query)
|
||||
void DatabaseWithDictionaries::createDictionary(ContextPtr local_context, const String & dictionary_name, const ASTPtr & query)
|
||||
{
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = local_context->getSettingsRef();
|
||||
|
||||
/** The code is based on the assumption that all threads share the same order of operations:
|
||||
* - create the .sql.tmp file;
|
||||
@ -151,7 +151,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
|
||||
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS,
|
||||
"Dictionary {} already exists.", dict_id.getFullNameNotQuoted());
|
||||
|
||||
if (isTableExist(dictionary_name, global_context))
|
||||
if (isTableExist(dictionary_name, getContext()))
|
||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", dict_id.getFullTableName());
|
||||
|
||||
String dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
|
||||
@ -177,9 +177,9 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
|
||||
/// Add a temporary repository containing the dictionary.
|
||||
/// We need this temp repository to try loading the dictionary before actually attaching it to the database.
|
||||
auto temp_repository = external_loader.addConfigRepository(std::make_unique<ExternalLoaderTempConfigRepository>(
|
||||
getDatabaseName(), dictionary_metadata_tmp_path, getDictionaryConfigurationFromAST(query->as<const ASTCreateQuery &>(), context)));
|
||||
getDatabaseName(), dictionary_metadata_tmp_path, getDictionaryConfigurationFromAST(query->as<const ASTCreateQuery &>(), local_context)));
|
||||
|
||||
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
|
||||
bool lazy_load = local_context->getConfigRef().getBool("dictionaries_lazy_load", true);
|
||||
if (!lazy_load)
|
||||
{
|
||||
/// load() is called here to force loading the dictionary, wait until the loading is finished,
|
||||
@ -187,15 +187,15 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
|
||||
external_loader.load(dict_id.getInternalDictionaryName());
|
||||
}
|
||||
|
||||
auto config = getDictionaryConfigurationFromAST(query->as<const ASTCreateQuery &>(), context);
|
||||
auto config = getDictionaryConfigurationFromAST(query->as<const ASTCreateQuery &>(), local_context);
|
||||
attachDictionary(dictionary_name, DictionaryAttachInfo{query, config, time(nullptr)});
|
||||
SCOPE_EXIT({
|
||||
if (!succeeded)
|
||||
detachDictionary(dictionary_name);
|
||||
});
|
||||
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
if (txn && !context.isInternalSubquery())
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
if (txn && !local_context->isInternalSubquery())
|
||||
txn->commit(); /// Commit point (a sort of) for Replicated database
|
||||
|
||||
/// If it was ATTACH query and file with dictionary metadata already exist
|
||||
@ -210,7 +210,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
|
||||
succeeded = true;
|
||||
}
|
||||
|
||||
void DatabaseWithDictionaries::removeDictionary(const Context & context, const String & dictionary_name)
|
||||
void DatabaseWithDictionaries::removeDictionary(ContextPtr local_context, const String & dictionary_name)
|
||||
{
|
||||
DictionaryAttachInfo attach_info;
|
||||
detachDictionaryImpl(dictionary_name, attach_info);
|
||||
@ -219,8 +219,8 @@ void DatabaseWithDictionaries::removeDictionary(const Context & context, const S
|
||||
{
|
||||
String dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
|
||||
|
||||
auto txn = context.getZooKeeperMetadataTransaction();
|
||||
if (txn && !context.isInternalSubquery())
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
if (txn && !local_context->isInternalSubquery())
|
||||
txn->commit(); /// Commit point (a sort of) for Replicated database
|
||||
|
||||
Poco::File(dictionary_metadata_path).remove();
|
||||
@ -335,7 +335,7 @@ void DatabaseWithDictionaries::reloadDictionaryConfig(const String & full_name)
|
||||
/// Ensure that this database is attached to ExternalLoader as a config repository.
|
||||
if (!database_as_config_repo_for_external_loader.load())
|
||||
{
|
||||
auto repository = std::make_unique<ExternalLoaderDatabaseConfigRepository>(*this, global_context);
|
||||
auto repository = std::make_unique<ExternalLoaderDatabaseConfigRepository>(*this, getContext());
|
||||
auto remove_repository_callback = external_loader.addConfigRepository(std::move(repository));
|
||||
database_as_config_repo_for_external_loader = boost::make_shared<ext::scope_guard>(std::move(remove_repository_callback));
|
||||
}
|
||||
@ -359,9 +359,9 @@ void DatabaseWithDictionaries::shutdown()
|
||||
|
||||
|
||||
DatabaseWithDictionaries::DatabaseWithDictionaries(
|
||||
const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context)
|
||||
: DatabaseOnDisk(name, metadata_path_, data_path_, logger, context)
|
||||
, external_loader(context.getExternalDictionariesLoader())
|
||||
const String & name, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context_)
|
||||
: DatabaseOnDisk(name, metadata_path_, data_path_, logger, context_)
|
||||
, external_loader(context_->getExternalDictionariesLoader())
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -17,11 +17,11 @@ public:
|
||||
|
||||
void detachDictionary(const String & dictionary_name) override;
|
||||
|
||||
void createDictionary(const Context & context,
|
||||
void createDictionary(ContextPtr context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void removeDictionary(const Context & context, const String & dictionary_name) override;
|
||||
void removeDictionary(ContextPtr context, const String & dictionary_name) override;
|
||||
|
||||
bool isDictionaryExist(const String & dictionary_name) const override;
|
||||
|
||||
@ -38,7 +38,7 @@ public:
|
||||
~DatabaseWithDictionaries() override;
|
||||
|
||||
protected:
|
||||
DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context);
|
||||
DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context);
|
||||
|
||||
ASTPtr getCreateDictionaryQueryImpl(const String & dictionary_name, bool throw_on_error) const override;
|
||||
|
||||
|
@ -20,18 +20,18 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_DATABASE;
|
||||
}
|
||||
|
||||
DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context)
|
||||
: IDatabase(name_), log(&Poco::Logger::get(logger)), global_context(context.getGlobalContext())
|
||||
DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context_)
|
||||
: IDatabase(name_), WithContext(context_->getGlobalContext()), log(&Poco::Logger::get(logger))
|
||||
{
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name, const Context &) const
|
||||
bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name, ContextPtr) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return tables.find(table_name) != tables.end();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, const Context &) const
|
||||
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, ContextPtr) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
@ -40,7 +40,7 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, con
|
||||
return {};
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!filter_by_table_name)
|
||||
|
@ -16,12 +16,12 @@ namespace DB
|
||||
class Context;
|
||||
|
||||
/// A base class for databases that manage their own list of tables.
|
||||
class DatabaseWithOwnTablesBase : public IDatabase
|
||||
class DatabaseWithOwnTablesBase : public IDatabase, protected WithContext
|
||||
{
|
||||
public:
|
||||
bool isTableExist(const String & table_name, const Context & context) const override;
|
||||
bool isTableExist(const String & table_name, ContextPtr context) const override;
|
||||
|
||||
StoragePtr tryGetTable(const String & table_name, const Context & context) const override;
|
||||
StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override;
|
||||
|
||||
bool empty() const override;
|
||||
|
||||
@ -29,7 +29,7 @@ public:
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
@ -38,9 +38,8 @@ public:
|
||||
protected:
|
||||
Tables tables;
|
||||
Poco::Logger * log;
|
||||
const Context & global_context;
|
||||
|
||||
DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context);
|
||||
DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context);
|
||||
|
||||
void attachTableUnlocked(const String & table_name, const StoragePtr & table, std::unique_lock<std::mutex> & lock);
|
||||
StoragePtr detachTableUnlocked(const String & table_name, std::unique_lock<std::mutex> & lock);
|
||||
|
@ -10,6 +10,7 @@
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
|
||||
#include <ctime>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
@ -18,7 +19,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
struct Settings;
|
||||
struct ConstraintsDescription;
|
||||
struct IndicesDescription;
|
||||
@ -153,10 +153,10 @@ public:
|
||||
|
||||
/// Load a set of existing tables.
|
||||
/// You can call only once, right after the object is created.
|
||||
virtual void loadStoredObjects(Context & /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {}
|
||||
virtual void loadStoredObjects(ContextPtr /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {}
|
||||
|
||||
/// Check the existence of the table.
|
||||
virtual bool isTableExist(const String & name, const Context & context) const = 0;
|
||||
virtual bool isTableExist(const String & name, ContextPtr context) const = 0;
|
||||
|
||||
/// Check the existence of the dictionary
|
||||
virtual bool isDictionaryExist(const String & /*name*/) const
|
||||
@ -165,7 +165,7 @@ public:
|
||||
}
|
||||
|
||||
/// Get the table for work. Return nullptr if there is no table.
|
||||
virtual StoragePtr tryGetTable(const String & name, const Context & context) const = 0;
|
||||
virtual StoragePtr tryGetTable(const String & name, ContextPtr context) const = 0;
|
||||
|
||||
virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; }
|
||||
|
||||
@ -173,7 +173,7 @@ public:
|
||||
|
||||
/// Get an iterator that allows you to pass through all the tables.
|
||||
/// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above.
|
||||
virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
|
||||
virtual DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
|
||||
|
||||
/// Get an iterator to pass through all the dictionaries.
|
||||
virtual DatabaseDictionariesIteratorPtr getDictionariesIterator([[maybe_unused]] const FilterByNameFunction & filter_by_dictionary_name = {})
|
||||
@ -186,7 +186,7 @@ public:
|
||||
|
||||
/// Add the table to the database. Record its presence in the metadata.
|
||||
virtual void createTable(
|
||||
const Context & /*context*/,
|
||||
ContextPtr /*context*/,
|
||||
const String & /*name*/,
|
||||
const StoragePtr & /*table*/,
|
||||
const ASTPtr & /*query*/)
|
||||
@ -196,7 +196,7 @@ public:
|
||||
|
||||
/// Add the dictionary to the database. Record its presence in the metadata.
|
||||
virtual void createDictionary(
|
||||
const Context & /*context*/,
|
||||
ContextPtr /*context*/,
|
||||
const String & /*dictionary_name*/,
|
||||
const ASTPtr & /*query*/)
|
||||
{
|
||||
@ -205,7 +205,7 @@ public:
|
||||
|
||||
/// Delete the table from the database, drop table and delete the metadata.
|
||||
virtual void dropTable(
|
||||
const Context & /*context*/,
|
||||
ContextPtr /*context*/,
|
||||
const String & /*name*/,
|
||||
[[maybe_unused]] bool no_delay = false)
|
||||
{
|
||||
@ -214,7 +214,7 @@ public:
|
||||
|
||||
/// Delete the dictionary from the database. Delete the metadata.
|
||||
virtual void removeDictionary(
|
||||
const Context & /*context*/,
|
||||
ContextPtr /*context*/,
|
||||
const String & /*dictionary_name*/)
|
||||
{
|
||||
throw Exception("There is no DROP DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
@ -249,14 +249,14 @@ public:
|
||||
|
||||
/// Forget about the table without deleting it's data, but rename metadata file to prevent reloading it
|
||||
/// with next restart. The database may not support this method.
|
||||
virtual void detachTablePermanently(const Context & /*context*/, const String & /*name*/)
|
||||
virtual void detachTablePermanently(ContextPtr /*context*/, const String & /*name*/)
|
||||
{
|
||||
throw Exception("There is no DETACH TABLE PERMANENTLY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// Rename the table and possibly move the table to another database.
|
||||
virtual void renameTable(
|
||||
const Context & /*context*/,
|
||||
ContextPtr /*context*/,
|
||||
const String & /*name*/,
|
||||
IDatabase & /*to_database*/,
|
||||
const String & /*to_name*/,
|
||||
@ -271,7 +271,7 @@ public:
|
||||
/// Change the table structure in metadata.
|
||||
/// You must call under the alter_lock of the corresponding table . If engine_modifier is empty, then engine does not change.
|
||||
virtual void alterTable(
|
||||
const Context & /*context*/,
|
||||
ContextPtr /*context*/,
|
||||
const StorageID & /*table_id*/,
|
||||
const StorageInMemoryMetadata & /*metadata*/)
|
||||
{
|
||||
@ -285,12 +285,12 @@ public:
|
||||
}
|
||||
|
||||
/// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata.
|
||||
ASTPtr tryGetCreateTableQuery(const String & name, const Context & context) const noexcept
|
||||
ASTPtr tryGetCreateTableQuery(const String & name, ContextPtr context) const noexcept
|
||||
{
|
||||
return getCreateTableQueryImpl(name, context, false);
|
||||
}
|
||||
|
||||
ASTPtr getCreateTableQuery(const String & name, const Context & context) const
|
||||
ASTPtr getCreateTableQuery(const String & name, ContextPtr context) const
|
||||
{
|
||||
return getCreateTableQueryImpl(name, context, true);
|
||||
}
|
||||
@ -351,12 +351,12 @@ public:
|
||||
virtual void shutdown() = 0;
|
||||
|
||||
/// Delete data and metadata stored inside the database, if exists.
|
||||
virtual void drop(const Context & /*context*/) {}
|
||||
virtual void drop(ContextPtr /*context*/) {}
|
||||
|
||||
virtual ~IDatabase() {}
|
||||
virtual ~IDatabase() = default;
|
||||
|
||||
protected:
|
||||
virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, const Context & /*context*/, bool throw_on_error) const
|
||||
virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, ContextPtr /*context*/, bool throw_on_error) const
|
||||
{
|
||||
if (throw_on_error)
|
||||
throw Exception("There is no SHOW CREATE TABLE query for Database" + getEngineName(), ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
|
@ -50,12 +50,12 @@ void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
#undef ADD_IF_ABSENT
|
||||
}
|
||||
|
||||
void ConnectionMySQLSettings::loadFromQueryContext(const Context & context)
|
||||
void ConnectionMySQLSettings::loadFromQueryContext(ContextPtr context)
|
||||
{
|
||||
if (!context.hasQueryContext())
|
||||
if (!context->hasQueryContext())
|
||||
return;
|
||||
|
||||
const Settings & settings = context.getQueryContext().getSettingsRef();
|
||||
const Settings & settings = context->getQueryContext()->getSettingsRef();
|
||||
|
||||
if (settings.mysql_datatypes_support_level.value != mysql_datatypes_support_level.value)
|
||||
set("mysql_datatypes_support_level", settings.mysql_datatypes_support_level.toString());
|
||||
|
@ -1,13 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class ASTStorage;
|
||||
|
||||
#define LIST_OF_CONNECTION_MYSQL_SETTINGS(M) \
|
||||
@ -27,7 +27,7 @@ struct ConnectionMySQLSettings : public BaseSettings<ConnectionMySQLSettingsTrai
|
||||
{
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
|
||||
void loadFromQueryContext(const Context & context);
|
||||
void loadFromQueryContext(ContextPtr context);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -46,7 +46,7 @@ static constexpr const std::chrono::seconds cleaner_sleep_time{30};
|
||||
static const std::chrono::seconds lock_acquire_timeout{10};
|
||||
|
||||
DatabaseConnectionMySQL::DatabaseConnectionMySQL(
|
||||
const Context & context,
|
||||
ContextPtr context_,
|
||||
const String & database_name_,
|
||||
const String & metadata_path_,
|
||||
const ASTStorage * database_engine_define_,
|
||||
@ -54,7 +54,7 @@ DatabaseConnectionMySQL::DatabaseConnectionMySQL(
|
||||
std::unique_ptr<ConnectionMySQLSettings> settings_,
|
||||
mysqlxx::PoolWithFailover && pool)
|
||||
: IDatabase(database_name_)
|
||||
, global_context(context.getGlobalContext())
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, metadata_path(metadata_path_)
|
||||
, database_engine_define(database_engine_define_->clone())
|
||||
, database_name_in_mysql(database_name_in_mysql_)
|
||||
@ -69,7 +69,7 @@ bool DatabaseConnectionMySQL::empty() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
fetchTablesIntoLocalCache(global_context);
|
||||
fetchTablesIntoLocalCache(getContext());
|
||||
|
||||
if (local_tables_cache.empty())
|
||||
return true;
|
||||
@ -81,12 +81,12 @@ bool DatabaseConnectionMySQL::empty() const
|
||||
return true;
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
Tables tables;
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
fetchTablesIntoLocalCache(context);
|
||||
fetchTablesIntoLocalCache(local_context);
|
||||
|
||||
for (const auto & [table_name, modify_time_and_storage] : local_tables_cache)
|
||||
if (!remove_or_detach_tables.count(table_name) && (!filter_by_table_name || filter_by_table_name(table_name)))
|
||||
@ -95,16 +95,16 @@ DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(const Conte
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(tables, database_name);
|
||||
}
|
||||
|
||||
bool DatabaseConnectionMySQL::isTableExist(const String & name, const Context & context) const
|
||||
bool DatabaseConnectionMySQL::isTableExist(const String & name, ContextPtr local_context) const
|
||||
{
|
||||
return bool(tryGetTable(name, context));
|
||||
return bool(tryGetTable(name, local_context));
|
||||
}
|
||||
|
||||
StoragePtr DatabaseConnectionMySQL::tryGetTable(const String & mysql_table_name, const Context & context) const
|
||||
StoragePtr DatabaseConnectionMySQL::tryGetTable(const String & mysql_table_name, ContextPtr local_context) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
fetchTablesIntoLocalCache(context);
|
||||
fetchTablesIntoLocalCache(local_context);
|
||||
|
||||
if (!remove_or_detach_tables.count(mysql_table_name) && local_tables_cache.find(mysql_table_name) != local_tables_cache.end())
|
||||
return local_tables_cache[mysql_table_name].second;
|
||||
@ -159,11 +159,11 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr
|
||||
return create_table_query;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseConnectionMySQL::getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const
|
||||
ASTPtr DatabaseConnectionMySQL::getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
fetchTablesIntoLocalCache(context);
|
||||
fetchTablesIntoLocalCache(local_context);
|
||||
|
||||
if (local_tables_cache.find(table_name) == local_tables_cache.end())
|
||||
{
|
||||
@ -180,7 +180,7 @@ time_t DatabaseConnectionMySQL::getObjectMetadataModificationTime(const String &
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
fetchTablesIntoLocalCache(global_context);
|
||||
fetchTablesIntoLocalCache(getContext());
|
||||
|
||||
if (local_tables_cache.find(table_name) == local_tables_cache.end())
|
||||
throw Exception("MySQL table " + database_name_in_mysql + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
@ -196,12 +196,12 @@ ASTPtr DatabaseConnectionMySQL::getCreateDatabaseQuery() const
|
||||
return create_query;
|
||||
}
|
||||
|
||||
void DatabaseConnectionMySQL::fetchTablesIntoLocalCache(const Context & context) const
|
||||
void DatabaseConnectionMySQL::fetchTablesIntoLocalCache(ContextPtr local_context) const
|
||||
{
|
||||
const auto & tables_with_modification_time = fetchTablesWithModificationTime();
|
||||
|
||||
destroyLocalCacheExtraTables(tables_with_modification_time);
|
||||
fetchLatestTablesStructureIntoCache(tables_with_modification_time, context);
|
||||
fetchLatestTablesStructureIntoCache(tables_with_modification_time, local_context);
|
||||
}
|
||||
|
||||
void DatabaseConnectionMySQL::destroyLocalCacheExtraTables(const std::map<String, UInt64> & tables_with_modification_time) const
|
||||
@ -218,7 +218,8 @@ void DatabaseConnectionMySQL::destroyLocalCacheExtraTables(const std::map<String
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache(const std::map<String, UInt64> &tables_modification_time, const Context & context) const
|
||||
void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache(
|
||||
const std::map<String, UInt64> & tables_modification_time, ContextPtr local_context) const
|
||||
{
|
||||
std::vector<String> wait_update_tables_name;
|
||||
for (const auto & table_modification_time : tables_modification_time)
|
||||
@ -230,7 +231,7 @@ void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache(const std::map
|
||||
wait_update_tables_name.emplace_back(table_modification_time.first);
|
||||
}
|
||||
|
||||
std::map<String, NamesAndTypesList> tables_and_columns = fetchTablesColumnsList(wait_update_tables_name, context);
|
||||
std::map<String, NamesAndTypesList> tables_and_columns = fetchTablesColumnsList(wait_update_tables_name, local_context);
|
||||
|
||||
for (const auto & table_and_columns : tables_and_columns)
|
||||
{
|
||||
@ -247,7 +248,7 @@ void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache(const std::map
|
||||
|
||||
local_tables_cache[table_name] = std::make_pair(table_modification_time, StorageMySQL::create(
|
||||
StorageID(database_name, table_name), std::move(mysql_pool), database_name_in_mysql, table_name,
|
||||
false, "", ColumnsDescription{columns_name_and_type}, ConstraintsDescription{}, global_context));
|
||||
false, "", ColumnsDescription{columns_name_and_type}, ConstraintsDescription{}, getContext()));
|
||||
}
|
||||
}
|
||||
|
||||
@ -282,9 +283,10 @@ std::map<String, UInt64> DatabaseConnectionMySQL::fetchTablesWithModificationTim
|
||||
return tables_with_modification_time;
|
||||
}
|
||||
|
||||
std::map<String, NamesAndTypesList> DatabaseConnectionMySQL::fetchTablesColumnsList(const std::vector<String> & tables_name, const Context & context) const
|
||||
std::map<String, NamesAndTypesList>
|
||||
DatabaseConnectionMySQL::fetchTablesColumnsList(const std::vector<String> & tables_name, ContextPtr local_context) const
|
||||
{
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & settings = local_context->getSettingsRef();
|
||||
|
||||
return DB::fetchTablesColumnsList(
|
||||
mysql_pool,
|
||||
@ -309,7 +311,7 @@ void DatabaseConnectionMySQL::shutdown()
|
||||
local_tables_cache.clear();
|
||||
}
|
||||
|
||||
void DatabaseConnectionMySQL::drop(const Context & /*context*/)
|
||||
void DatabaseConnectionMySQL::drop(ContextPtr /*context*/)
|
||||
{
|
||||
Poco::File(getMetadataPath()).remove(true);
|
||||
}
|
||||
@ -384,7 +386,7 @@ String DatabaseConnectionMySQL::getMetadataPath() const
|
||||
return metadata_path;
|
||||
}
|
||||
|
||||
void DatabaseConnectionMySQL::loadStoredObjects(Context &, bool, bool /*force_attach*/)
|
||||
void DatabaseConnectionMySQL::loadStoredObjects(ContextPtr, bool, bool /*force_attach*/)
|
||||
{
|
||||
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
@ -401,7 +403,7 @@ void DatabaseConnectionMySQL::loadStoredObjects(Context &, bool, bool /*force_at
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseConnectionMySQL::detachTablePermanently(const Context &, const String & table_name)
|
||||
void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String & table_name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
|
||||
@ -435,9 +437,9 @@ void DatabaseConnectionMySQL::detachTablePermanently(const Context &, const Stri
|
||||
table_iter->second.second->is_dropped = true;
|
||||
}
|
||||
|
||||
void DatabaseConnectionMySQL::dropTable(const Context & context, const String & table_name, bool /*no_delay*/)
|
||||
void DatabaseConnectionMySQL::dropTable(ContextPtr local_context, const String & table_name, bool /*no_delay*/)
|
||||
{
|
||||
detachTablePermanently(context, table_name);
|
||||
detachTablePermanently(local_context, table_name);
|
||||
}
|
||||
|
||||
DatabaseConnectionMySQL::~DatabaseConnectionMySQL()
|
||||
@ -462,7 +464,7 @@ DatabaseConnectionMySQL::~DatabaseConnectionMySQL()
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseConnectionMySQL::createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query)
|
||||
void DatabaseConnectionMySQL::createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query)
|
||||
{
|
||||
const auto & create = create_query->as<ASTCreateQuery>();
|
||||
|
||||
@ -473,7 +475,7 @@ void DatabaseConnectionMySQL::createTable(const Context &, const String & table_
|
||||
/// XXX: hack
|
||||
/// In order to prevent users from broken the table structure by executing attach table database_name.table_name (...)
|
||||
/// we should compare the old and new create_query to make them completely consistent
|
||||
const auto & origin_create_query = getCreateTableQuery(table_name, global_context);
|
||||
const auto & origin_create_query = getCreateTableQuery(table_name, getContext());
|
||||
origin_create_query->as<ASTCreateQuery>()->attach = true;
|
||||
|
||||
if (queryToString(origin_create_query) != queryToString(create_query))
|
||||
|
@ -31,13 +31,13 @@ enum class MySQLDataTypesSupport;
|
||||
* It doesn't make any manipulations with filesystem.
|
||||
* All tables are created by calling code after real-time pull-out structure from remote MySQL
|
||||
*/
|
||||
class DatabaseConnectionMySQL final : public IDatabase
|
||||
class DatabaseConnectionMySQL final : public IDatabase, WithContext
|
||||
{
|
||||
public:
|
||||
~DatabaseConnectionMySQL() override;
|
||||
|
||||
DatabaseConnectionMySQL(
|
||||
const Context & context,
|
||||
ContextPtr context,
|
||||
const String & database_name,
|
||||
const String & metadata_path,
|
||||
const ASTStorage * database_engine_define,
|
||||
@ -55,39 +55,38 @@ public:
|
||||
|
||||
bool empty() const override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery() const override;
|
||||
|
||||
bool isTableExist(const String & name, const Context & context) const override;
|
||||
bool isTableExist(const String & name, ContextPtr context) const override;
|
||||
|
||||
StoragePtr tryGetTable(const String & name, const Context & context) const override;
|
||||
StoragePtr tryGetTable(const String & name, ContextPtr context) const override;
|
||||
|
||||
time_t getObjectMetadataModificationTime(const String & name) const override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
void drop(const Context & /*context*/) override;
|
||||
void drop(ContextPtr /*context*/) override;
|
||||
|
||||
String getMetadataPath() const override;
|
||||
|
||||
void createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override;
|
||||
void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override;
|
||||
|
||||
void loadStoredObjects(Context &, bool, bool force_attach) override;
|
||||
void loadStoredObjects(ContextPtr, bool, bool force_attach) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
void detachTablePermanently(const Context & context, const String & table_name) override;
|
||||
void detachTablePermanently(ContextPtr context, const String & table_name) override;
|
||||
|
||||
void dropTable(const Context & context, const String & table_name, bool no_delay) override;
|
||||
void dropTable(ContextPtr context, const String & table_name, bool no_delay) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override;
|
||||
|
||||
protected:
|
||||
ASTPtr getCreateTableQueryImpl(const String & name, const Context & context, bool throw_on_error) const override;
|
||||
ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override;
|
||||
|
||||
private:
|
||||
const Context & global_context;
|
||||
String metadata_path;
|
||||
ASTPtr database_engine_define;
|
||||
String database_name_in_mysql;
|
||||
@ -107,15 +106,15 @@ private:
|
||||
|
||||
void cleanOutdatedTables();
|
||||
|
||||
void fetchTablesIntoLocalCache(const Context & context) const;
|
||||
void fetchTablesIntoLocalCache(ContextPtr context) const;
|
||||
|
||||
std::map<String, UInt64> fetchTablesWithModificationTime() const;
|
||||
|
||||
std::map<String, NamesAndTypesList> fetchTablesColumnsList(const std::vector<String> & tables_name, const Context & context) const;
|
||||
std::map<String, NamesAndTypesList> fetchTablesColumnsList(const std::vector<String> & tables_name, ContextPtr context) const;
|
||||
|
||||
void destroyLocalCacheExtraTables(const std::map<String, UInt64> & tables_with_modification_time) const;
|
||||
|
||||
void fetchLatestTablesStructureIntoCache(const std::map<String, UInt64> & tables_modification_time, const Context & context) const;
|
||||
void fetchLatestTablesStructureIntoCache(const std::map<String, UInt64> & tables_modification_time, ContextPtr context) const;
|
||||
|
||||
ThreadFromGlobalPool thread;
|
||||
};
|
||||
|
@ -26,27 +26,40 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
template<>
|
||||
template <>
|
||||
DatabaseMaterializeMySQL<DatabaseOrdinary>::DatabaseMaterializeMySQL(
|
||||
const Context & context, const String & database_name_, const String & metadata_path_, UUID /*uuid*/,
|
||||
const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, std::unique_ptr<MaterializeMySQLSettings> settings_)
|
||||
: DatabaseOrdinary(database_name_
|
||||
, metadata_path_
|
||||
, "data/" + escapeForFileName(database_name_) + "/"
|
||||
, "DatabaseMaterializeMySQL<Ordinary> (" + database_name_ + ")", context
|
||||
)
|
||||
ContextPtr context_,
|
||||
const String & database_name_,
|
||||
const String & metadata_path_,
|
||||
UUID /*uuid*/,
|
||||
const String & mysql_database_name_,
|
||||
mysqlxx::Pool && pool_,
|
||||
MySQLClient && client_,
|
||||
std::unique_ptr<MaterializeMySQLSettings> settings_)
|
||||
: DatabaseOrdinary(
|
||||
database_name_,
|
||||
metadata_path_,
|
||||
"data/" + escapeForFileName(database_name_) + "/",
|
||||
"DatabaseMaterializeMySQL<Ordinary> (" + database_name_ + ")",
|
||||
context_)
|
||||
, settings(std::move(settings_))
|
||||
, materialize_thread(context, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get())
|
||||
, materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get())
|
||||
{
|
||||
}
|
||||
|
||||
template<>
|
||||
template <>
|
||||
DatabaseMaterializeMySQL<DatabaseAtomic>::DatabaseMaterializeMySQL(
|
||||
const Context & context, const String & database_name_, const String & metadata_path_, UUID uuid,
|
||||
const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, std::unique_ptr<MaterializeMySQLSettings> settings_)
|
||||
: DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializeMySQL<Atomic> (" + database_name_ + ")", context)
|
||||
ContextPtr context_,
|
||||
const String & database_name_,
|
||||
const String & metadata_path_,
|
||||
UUID uuid,
|
||||
const String & mysql_database_name_,
|
||||
mysqlxx::Pool && pool_,
|
||||
MySQLClient && client_,
|
||||
std::unique_ptr<MaterializeMySQLSettings> settings_)
|
||||
: DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializeMySQL<Atomic> (" + database_name_ + ")", context_)
|
||||
, settings(std::move(settings_))
|
||||
, materialize_thread(context, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get())
|
||||
, materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get())
|
||||
{
|
||||
}
|
||||
|
||||
@ -79,9 +92,9 @@ void DatabaseMaterializeMySQL<Base>::setException(const std::exception_ptr & exc
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
void DatabaseMaterializeMySQL<Base>::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach)
|
||||
void DatabaseMaterializeMySQL<Base>::loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach)
|
||||
{
|
||||
Base::loadStoredObjects(context, has_force_restore_data_flag, force_attach);
|
||||
Base::loadStoredObjects(context_, has_force_restore_data_flag, force_attach);
|
||||
if (!force_attach)
|
||||
materialize_thread.assertMySQLAvailable();
|
||||
|
||||
@ -90,17 +103,17 @@ void DatabaseMaterializeMySQL<Base>::loadStoredObjects(Context & context, bool h
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
void DatabaseMaterializeMySQL<Base>::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query)
|
||||
void DatabaseMaterializeMySQL<Base>::createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query)
|
||||
{
|
||||
assertCalledFromSyncThreadOrDrop("create table");
|
||||
Base::createTable(context, name, table, query);
|
||||
Base::createTable(context_, name, table, query);
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
void DatabaseMaterializeMySQL<Base>::dropTable(const Context & context, const String & name, bool no_delay)
|
||||
void DatabaseMaterializeMySQL<Base>::dropTable(ContextPtr context_, const String & name, bool no_delay)
|
||||
{
|
||||
assertCalledFromSyncThreadOrDrop("drop table");
|
||||
Base::dropTable(context, name, no_delay);
|
||||
Base::dropTable(context_, name, no_delay);
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
@ -118,7 +131,7 @@ StoragePtr DatabaseMaterializeMySQL<Base>::detachTable(const String & name)
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
void DatabaseMaterializeMySQL<Base>::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary)
|
||||
void DatabaseMaterializeMySQL<Base>::renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary)
|
||||
{
|
||||
assertCalledFromSyncThreadOrDrop("rename table");
|
||||
|
||||
@ -131,18 +144,18 @@ void DatabaseMaterializeMySQL<Base>::renameTable(const Context & context, const
|
||||
if (to_database.getDatabaseName() != Base::getDatabaseName())
|
||||
throw Exception("Cannot rename with other database for MaterializeMySQL database.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
Base::renameTable(context, name, *this, to_name, exchange, dictionary);
|
||||
Base::renameTable(context_, name, *this, to_name, exchange, dictionary);
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
void DatabaseMaterializeMySQL<Base>::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata)
|
||||
void DatabaseMaterializeMySQL<Base>::alterTable(ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata)
|
||||
{
|
||||
assertCalledFromSyncThreadOrDrop("alter table");
|
||||
Base::alterTable(context, table_id, metadata);
|
||||
Base::alterTable(context_, table_id, metadata);
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
void DatabaseMaterializeMySQL<Base>::drop(const Context & context)
|
||||
void DatabaseMaterializeMySQL<Base>::drop(ContextPtr context_)
|
||||
{
|
||||
/// Remove metadata info
|
||||
Poco::File metadata(Base::getMetadataPath() + "/.metadata");
|
||||
@ -150,15 +163,15 @@ void DatabaseMaterializeMySQL<Base>::drop(const Context & context)
|
||||
if (metadata.exists())
|
||||
metadata.remove(false);
|
||||
|
||||
Base::drop(context);
|
||||
Base::drop(context_);
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
StoragePtr DatabaseMaterializeMySQL<Base>::tryGetTable(const String & name, const Context & context) const
|
||||
StoragePtr DatabaseMaterializeMySQL<Base>::tryGetTable(const String & name, ContextPtr context_) const
|
||||
{
|
||||
if (!MaterializeMySQLSyncThread::isMySQLSyncThread())
|
||||
{
|
||||
StoragePtr nested_storage = Base::tryGetTable(name, context);
|
||||
StoragePtr nested_storage = Base::tryGetTable(name, context_);
|
||||
|
||||
if (!nested_storage)
|
||||
return {};
|
||||
@ -166,19 +179,20 @@ StoragePtr DatabaseMaterializeMySQL<Base>::tryGetTable(const String & name, cons
|
||||
return std::make_shared<StorageMaterializeMySQL>(std::move(nested_storage), this);
|
||||
}
|
||||
|
||||
return Base::tryGetTable(name, context);
|
||||
return Base::tryGetTable(name, context_);
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
DatabaseTablesIteratorPtr DatabaseMaterializeMySQL<Base>::getTablesIterator(const Context & context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name)
|
||||
template <typename Base>
|
||||
DatabaseTablesIteratorPtr
|
||||
DatabaseMaterializeMySQL<Base>::getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
if (!MaterializeMySQLSyncThread::isMySQLSyncThread())
|
||||
{
|
||||
DatabaseTablesIteratorPtr iterator = Base::getTablesIterator(context, filter_by_table_name);
|
||||
DatabaseTablesIteratorPtr iterator = Base::getTablesIterator(context_, filter_by_table_name);
|
||||
return std::make_unique<DatabaseMaterializeTablesIterator>(std::move(iterator), this);
|
||||
}
|
||||
|
||||
return Base::getTablesIterator(context, filter_by_table_name);
|
||||
return Base::getTablesIterator(context_, filter_by_table_name);
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
|
@ -23,7 +23,7 @@ class DatabaseMaterializeMySQL : public Base
|
||||
public:
|
||||
|
||||
DatabaseMaterializeMySQL(
|
||||
const Context & context, const String & database_name_, const String & metadata_path_, UUID uuid,
|
||||
ContextPtr context, const String & database_name_, const String & metadata_path_, UUID uuid,
|
||||
const String & mysql_database_name_, mysqlxx::Pool && pool_,
|
||||
MySQLClient && client_, std::unique_ptr<MaterializeMySQLSettings> settings_);
|
||||
|
||||
@ -43,25 +43,25 @@ protected:
|
||||
public:
|
||||
String getEngineName() const override { return "MaterializeMySQL"; }
|
||||
|
||||
void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
void loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
|
||||
void createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) override;
|
||||
void createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) override;
|
||||
|
||||
void dropTable(const Context & context, const String & name, bool no_delay) override;
|
||||
void dropTable(ContextPtr context_, const String & name, bool no_delay) override;
|
||||
|
||||
void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override;
|
||||
|
||||
StoragePtr detachTable(const String & name) override;
|
||||
|
||||
void renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override;
|
||||
void renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override;
|
||||
|
||||
void alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override;
|
||||
void alterTable(ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override;
|
||||
|
||||
void drop(const Context & context) override;
|
||||
void drop(ContextPtr context_) override;
|
||||
|
||||
StoragePtr tryGetTable(const String & name, const Context & context) const override;
|
||||
StoragePtr tryGetTable(const String & name, ContextPtr context_) const override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
void assertCalledFromSyncThreadOrDrop(const char * method) const;
|
||||
|
||||
|
@ -41,30 +41,30 @@ namespace ErrorCodes
|
||||
|
||||
static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync";
|
||||
|
||||
static Context createQueryContext(const Context & global_context)
|
||||
static ContextPtr createQueryContext(ContextPtr context)
|
||||
{
|
||||
Settings new_query_settings = global_context.getSettings();
|
||||
Settings new_query_settings = context->getSettings();
|
||||
new_query_settings.insert_allow_materialized_columns = true;
|
||||
|
||||
/// To avoid call AST::format
|
||||
/// TODO: We need to implement the format function for MySQLAST
|
||||
new_query_settings.enable_global_with_statement = false;
|
||||
|
||||
Context query_context(global_context);
|
||||
query_context.setSettings(new_query_settings);
|
||||
auto query_context = Context::createCopy(context);
|
||||
query_context->setSettings(new_query_settings);
|
||||
CurrentThread::QueryScope query_scope(query_context);
|
||||
|
||||
query_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
query_context.setCurrentQueryId(""); // generate random query_id
|
||||
query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
query_context->setCurrentQueryId(""); // generate random query_id
|
||||
return query_context;
|
||||
}
|
||||
|
||||
static BlockIO tryToExecuteQuery(const String & query_to_execute, Context & query_context, const String & database, const String & comment)
|
||||
static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextPtr query_context, const String & database, const String & comment)
|
||||
{
|
||||
try
|
||||
{
|
||||
if (!database.empty())
|
||||
query_context.setCurrentDatabase(database);
|
||||
query_context->setCurrentDatabase(database);
|
||||
|
||||
return executeQuery("/*" + comment + "*/ " + query_to_execute, query_context, true);
|
||||
}
|
||||
@ -143,10 +143,19 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection)
|
||||
}
|
||||
|
||||
MaterializeMySQLSyncThread::MaterializeMySQLSyncThread(
|
||||
const Context & context, const String & database_name_, const String & mysql_database_name_,
|
||||
mysqlxx::Pool && pool_, MySQLClient && client_, MaterializeMySQLSettings * settings_)
|
||||
: log(&Poco::Logger::get("MaterializeMySQLSyncThread")), global_context(context.getGlobalContext()), database_name(database_name_)
|
||||
, mysql_database_name(mysql_database_name_), pool(std::move(pool_)), client(std::move(client_)), settings(settings_)
|
||||
ContextPtr context_,
|
||||
const String & database_name_,
|
||||
const String & mysql_database_name_,
|
||||
mysqlxx::Pool && pool_,
|
||||
MySQLClient && client_,
|
||||
MaterializeMySQLSettings * settings_)
|
||||
: WithContext(context_->getGlobalContext())
|
||||
, log(&Poco::Logger::get("MaterializeMySQLSyncThread"))
|
||||
, database_name(database_name_)
|
||||
, mysql_database_name(mysql_database_name_)
|
||||
, pool(std::move(pool_))
|
||||
, client(std::move(client_))
|
||||
, settings(settings_)
|
||||
{
|
||||
query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ") ";
|
||||
}
|
||||
@ -248,7 +257,7 @@ void MaterializeMySQLSyncThread::assertMySQLAvailable()
|
||||
}
|
||||
}
|
||||
|
||||
static inline void cleanOutdatedTables(const String & database_name, const Context & context)
|
||||
static inline void cleanOutdatedTables(const String & database_name, ContextPtr context)
|
||||
{
|
||||
String cleaning_table_name;
|
||||
try
|
||||
@ -258,7 +267,7 @@ static inline void cleanOutdatedTables(const String & database_name, const Conte
|
||||
|
||||
for (auto iterator = clean_database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
Context query_context = createQueryContext(context);
|
||||
auto query_context = createQueryContext(context);
|
||||
String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data";
|
||||
cleaning_table_name = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(iterator->name());
|
||||
tryToExecuteQuery(" DROP TABLE " + cleaning_table_name, query_context, database_name, comment);
|
||||
@ -271,7 +280,8 @@ static inline void cleanOutdatedTables(const String & database_name, const Conte
|
||||
}
|
||||
}
|
||||
|
||||
static inline BlockOutputStreamPtr getTableOutput(const String & database_name, const String & table_name, Context & query_context, bool insert_materialized = false)
|
||||
static inline BlockOutputStreamPtr
|
||||
getTableOutput(const String & database_name, const String & table_name, ContextPtr query_context, bool insert_materialized = false)
|
||||
{
|
||||
const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context);
|
||||
|
||||
@ -303,7 +313,7 @@ static inline BlockOutputStreamPtr getTableOutput(const String & database_name,
|
||||
static inline void dumpDataForTables(
|
||||
mysqlxx::Pool::Entry & connection, const std::unordered_map<String, String> & need_dumping_tables,
|
||||
const String & query_prefix, const String & database_name, const String & mysql_database_name,
|
||||
const Context & context, const std::function<bool()> & is_cancelled)
|
||||
ContextPtr context, const std::function<bool()> & is_cancelled)
|
||||
{
|
||||
auto iterator = need_dumping_tables.begin();
|
||||
for (; iterator != need_dumping_tables.end() && !is_cancelled(); ++iterator)
|
||||
@ -311,7 +321,7 @@ static inline void dumpDataForTables(
|
||||
try
|
||||
{
|
||||
const auto & table_name = iterator->first;
|
||||
Context query_context = createQueryContext(context);
|
||||
auto query_context = createQueryContext(context);
|
||||
String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data";
|
||||
tryToExecuteQuery(query_prefix + " " + iterator->second, query_context, database_name, comment); /// create table.
|
||||
|
||||
@ -376,8 +386,12 @@ bool MaterializeMySQLSyncThread::prepareSynchronized(MaterializeMetadata & metad
|
||||
|
||||
metadata.transaction(position, [&]()
|
||||
{
|
||||
cleanOutdatedTables(database_name, global_context);
|
||||
dumpDataForTables(connection, need_dumping_tables, query_prefix, database_name, mysql_database_name, global_context, [this] { return isCancelled(); });
|
||||
cleanOutdatedTables(database_name, getContext());
|
||||
dumpDataForTables(
|
||||
connection, need_dumping_tables, query_prefix, database_name, mysql_database_name, getContext(), [this]
|
||||
{
|
||||
return isCancelled();
|
||||
});
|
||||
});
|
||||
|
||||
const auto & position_message = [&]()
|
||||
@ -431,7 +445,7 @@ void MaterializeMySQLSyncThread::flushBuffersData(Buffers & buffers, Materialize
|
||||
if (buffers.data.empty())
|
||||
return;
|
||||
|
||||
metadata.transaction(client.getPosition(), [&]() { buffers.commit(global_context); });
|
||||
metadata.transaction(client.getPosition(), [&]() { buffers.commit(getContext()); });
|
||||
|
||||
const auto & position_message = [&]()
|
||||
{
|
||||
@ -664,21 +678,21 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr
|
||||
if (receive_event->type() == MYSQL_WRITE_ROWS_EVENT)
|
||||
{
|
||||
WriteRowsEvent & write_rows_event = static_cast<WriteRowsEvent &>(*receive_event);
|
||||
Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(write_rows_event.table, global_context);
|
||||
Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(write_rows_event.table, getContext());
|
||||
size_t bytes = onWriteOrDeleteData<1>(write_rows_event.rows, buffer->first, ++metadata.data_version);
|
||||
buffers.add(buffer->first.rows(), buffer->first.bytes(), write_rows_event.rows.size(), bytes);
|
||||
}
|
||||
else if (receive_event->type() == MYSQL_UPDATE_ROWS_EVENT)
|
||||
{
|
||||
UpdateRowsEvent & update_rows_event = static_cast<UpdateRowsEvent &>(*receive_event);
|
||||
Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(update_rows_event.table, global_context);
|
||||
Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(update_rows_event.table, getContext());
|
||||
size_t bytes = onUpdateData(update_rows_event.rows, buffer->first, ++metadata.data_version, buffer->second);
|
||||
buffers.add(buffer->first.rows(), buffer->first.bytes(), update_rows_event.rows.size(), bytes);
|
||||
}
|
||||
else if (receive_event->type() == MYSQL_DELETE_ROWS_EVENT)
|
||||
{
|
||||
DeleteRowsEvent & delete_rows_event = static_cast<DeleteRowsEvent &>(*receive_event);
|
||||
Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(delete_rows_event.table, global_context);
|
||||
Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(delete_rows_event.table, getContext());
|
||||
size_t bytes = onWriteOrDeleteData<-1>(delete_rows_event.rows, buffer->first, ++metadata.data_version);
|
||||
buffers.add(buffer->first.rows(), buffer->first.bytes(), delete_rows_event.rows.size(), bytes);
|
||||
}
|
||||
@ -687,7 +701,7 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr
|
||||
QueryEvent & query_event = static_cast<QueryEvent &>(*receive_event);
|
||||
Position position_before_ddl;
|
||||
position_before_ddl.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set);
|
||||
metadata.transaction(position_before_ddl, [&]() { buffers.commit(global_context); });
|
||||
metadata.transaction(position_before_ddl, [&]() { buffers.commit(getContext()); });
|
||||
metadata.transaction(client.getPosition(),[&](){ executeDDLAtomic(query_event); });
|
||||
}
|
||||
else
|
||||
@ -718,7 +732,7 @@ void MaterializeMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_event
|
||||
{
|
||||
try
|
||||
{
|
||||
Context query_context = createQueryContext(global_context);
|
||||
auto query_context = createQueryContext(getContext());
|
||||
String comment = "Materialize MySQL step 2: execute MySQL DDL for sync data";
|
||||
String event_database = query_event.schema == mysql_database_name ? database_name : "";
|
||||
tryToExecuteQuery(query_prefix + query_event.query, query_context, event_database, comment);
|
||||
@ -761,13 +775,13 @@ bool MaterializeMySQLSyncThread::Buffers::checkThresholds(size_t check_block_row
|
||||
|| total_blocks_bytes >= check_total_bytes;
|
||||
}
|
||||
|
||||
void MaterializeMySQLSyncThread::Buffers::commit(const Context & context)
|
||||
void MaterializeMySQLSyncThread::Buffers::commit(ContextPtr context)
|
||||
{
|
||||
try
|
||||
{
|
||||
for (auto & table_name_and_buffer : data)
|
||||
{
|
||||
Context query_context = createQueryContext(context);
|
||||
auto query_context = createQueryContext(context);
|
||||
OneBlockInputStream input(table_name_and_buffer.second->first);
|
||||
BlockOutputStreamPtr out = getTableOutput(database, table_name_and_buffer.first, query_context, true);
|
||||
copyData(input, *out);
|
||||
@ -787,7 +801,7 @@ void MaterializeMySQLSyncThread::Buffers::commit(const Context & context)
|
||||
}
|
||||
|
||||
MaterializeMySQLSyncThread::Buffers::BufferAndSortingColumnsPtr MaterializeMySQLSyncThread::Buffers::getTableDataBuffer(
|
||||
const String & table_name, const Context & context)
|
||||
const String & table_name, ContextPtr context)
|
||||
{
|
||||
const auto & iterator = data.find(table_name);
|
||||
if (iterator == data.end())
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user