ClickHouse/src/Interpreters/executeQuery.cpp

919 lines
34 KiB
C++
Raw Normal View History

#include <Common/formatReadable.h>
2019-10-07 18:56:03 +00:00
#include <Common/PODArray.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Common/ThreadProfileEvents.h>
#include <IO/ConcatReadBuffer.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/LimitReadBuffer.h>
#include <IO/copyData.h>
#include <DataStreams/BlockIO.h>
#include <DataStreams/copyData.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/InputStreamFromASTInsertQuery.h>
#include <DataStreams/CountingBlockOutputStream.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTShowProcesslistQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
2020-06-15 20:01:58 +00:00
#include <Parsers/ASTWatchQuery.h>
#include <Parsers/Lexer.h>
2019-05-28 18:30:10 +00:00
#include <Storages/StorageInput.h>
#include <Access/EnabledQuota.h>
#include <Interpreters/InterpreterFactory.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/InterpreterSetQuery.h>
2020-10-08 09:06:04 +00:00
#include <Interpreters/ApplyWithGlobalVisitor.h>
2019-05-18 21:07:23 +00:00
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/executeQuery.h>
#include <Interpreters/Context.h>
2019-07-19 07:44:18 +00:00
#include <Common/ProfileEvents.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Common/SensitiveDataMasker.h>
2011-10-30 11:30:52 +00:00
2019-03-26 18:28:37 +00:00
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
2019-03-26 18:28:37 +00:00
#include <Processors/Formats/IOutputFormat.h>
2011-10-30 11:30:52 +00:00
2020-03-28 03:02:26 +00:00
2019-06-20 07:17:21 +00:00
namespace ProfileEvents
{
extern const Event QueryMaskingRulesMatch;
extern const Event FailedQuery;
extern const Event FailedInsertQuery;
extern const Event FailedSelectQuery;
extern const Event QueryTimeMicroseconds;
extern const Event SelectQueryTimeMicroseconds;
extern const Event InsertQueryTimeMicroseconds;
2019-06-20 07:17:21 +00:00
}
2011-10-30 11:30:52 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int INTO_OUTFILE_NOT_ALLOWED;
extern const int QUERY_WAS_CANCELLED;
}
2011-10-30 11:30:52 +00:00
static void checkASTSizeLimits(const IAST & ast, const Settings & settings)
{
if (settings.max_ast_depth)
ast.checkDepth(settings.max_ast_depth);
if (settings.max_ast_elements)
ast.checkSize(settings.max_ast_elements);
}
2020-06-15 20:01:58 +00:00
static String joinLines(const String & query)
{
2020-06-15 20:01:58 +00:00
/// Care should be taken. We don't join lines inside non-whitespace tokens (e.g. multiline string literals)
2020-06-15 20:05:21 +00:00
/// and we don't join line after comment (because it can be single-line comment).
2020-06-15 20:01:58 +00:00
/// All other whitespaces replaced to a single whitespace.
String res;
const char * begin = query.data();
const char * end = begin + query.size();
Lexer lexer(begin, end);
Token token = lexer.nextToken();
for (; !token.isEnd(); token = lexer.nextToken())
{
if (token.type == TokenType::Whitespace)
{
res += ' ';
}
else if (token.type == TokenType::Comment)
{
res.append(token.begin, token.end);
if (token.end < end && *token.end == '\n')
res += '\n';
}
else
res.append(token.begin, token.end);
}
return res;
}
2019-06-20 07:17:21 +00:00
static String prepareQueryForLogging(const String & query, Context & context)
{
String res = query;
// wiping sensitive data before cropping query by log_queries_cut_to_length,
// otherwise something like credit card without last digit can go to log
2020-04-22 06:01:33 +00:00
if (auto * masker = SensitiveDataMasker::getInstance())
2019-06-20 07:17:21 +00:00
{
auto matches = masker->wipeSensitiveData(res);
if (matches > 0)
{
ProfileEvents::increment(ProfileEvents::QueryMaskingRulesMatch, matches);
}
}
res = res.substr(0, context.getSettingsRef().log_queries_cut_to_length);
2019-06-20 07:17:21 +00:00
return res;
}
/// Log query into text log (not into system table).
2019-03-04 18:28:42 +00:00
static void logQuery(const String & query, const Context & context, bool internal)
{
2019-03-04 18:28:42 +00:00
if (internal)
{
2020-05-30 21:57:37 +00:00
LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(internal) {}", joinLines(query));
2019-03-04 18:28:42 +00:00
}
else
{
const auto & current_query_id = context.getClientInfo().current_query_id;
const auto & initial_query_id = context.getClientInfo().initial_query_id;
const auto & current_user = context.getClientInfo().current_user;
2020-05-30 21:57:37 +00:00
LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(from {}{}{}) {}",
2020-05-23 21:41:35 +00:00
context.getClientInfo().current_address.toString(),
(current_user != "default" ? ", user: " + context.getClientInfo().current_user : ""),
(!initial_query_id.empty() && current_query_id != initial_query_id ? ", initial_query_id: " + initial_query_id : std::string()),
joinLines(query));
2019-03-04 18:28:42 +00:00
}
}
/// Call this inside catch block.
static void setExceptionStackTrace(QueryLogElement & elem)
{
/// Disable memory tracker for stack trace.
/// Because if exception is "Memory limit (for query) exceed", then we probably can't allocate another one string.
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
try
{
throw;
}
catch (const std::exception & e)
{
elem.stack_trace = getExceptionStackTraceString(e);
}
catch (...) {}
}
/// Log exception (with query info) into text log (not into system table).
static void logException(Context & context, QueryLogElement & elem)
{
2020-05-23 21:41:35 +00:00
if (elem.stack_trace.empty())
2020-05-30 21:57:37 +00:00
LOG_ERROR(&Poco::Logger::get("executeQuery"), "{} (from {}) (in query: {})",
2020-05-23 21:41:35 +00:00
elem.exception, context.getClientInfo().current_address.toString(), joinLines(elem.query));
else
2020-05-30 21:57:37 +00:00
LOG_ERROR(&Poco::Logger::get("executeQuery"), "{} (from {}) (in query: {})"
2020-05-23 21:41:35 +00:00
", Stack trace (when copying this message, always include the lines below):\n\n{}",
elem.exception, context.getClientInfo().current_address.toString(), joinLines(elem.query), elem.stack_trace);
}
inline UInt64 time_in_microseconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::microseconds>(timepoint.time_since_epoch()).count();
}
inline UInt64 time_in_seconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::seconds>(timepoint.time_since_epoch()).count();
}
static void onExceptionBeforeStart(const String & query_for_logging, Context & context, time_t current_time, UInt64 current_time_microseconds, ASTPtr ast)
{
/// Exception before the query execution.
if (auto quota = context.getQuota())
quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false);
2018-08-23 01:31:28 +00:00
const Settings & settings = context.getSettingsRef();
/// Log the start of query execution into the table if necessary.
QueryLogElement elem;
elem.type = QueryLogElementType::EXCEPTION_BEFORE_START;
// all callers to onExceptionBeforeStart method construct the timespec for event_time and
// event_time_microseconds from the same time point. So, it can be assumed that both of these
// times are equal upto the precision of a second.
elem.event_time = current_time;
elem.event_time_microseconds = current_time_microseconds;
elem.query_start_time = current_time;
elem.query_start_time_microseconds = current_time_microseconds;
elem.current_database = context.getCurrentDatabase();
2019-06-20 07:17:21 +00:00
elem.query = query_for_logging;
2020-01-22 12:29:30 +00:00
elem.exception_code = getCurrentExceptionCode();
elem.exception = getCurrentExceptionMessage(false);
elem.client_info = context.getClientInfo();
2018-08-23 01:31:28 +00:00
if (settings.calculate_text_stack_trace)
setExceptionStackTrace(elem);
logException(context, elem);
/// Update performance counters before logging to query_log
CurrentThread::finalizePerformanceCounters();
if (settings.log_queries && elem.type >= settings.log_queries_min_type)
if (auto query_log = context.getQueryLog())
query_log->add(elem);
ProfileEvents::increment(ProfileEvents::FailedQuery);
if (ast)
{
if (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>())
{
ProfileEvents::increment(ProfileEvents::FailedSelectQuery);
}
else if (ast->as<ASTInsertQuery>())
{
ProfileEvents::increment(ProfileEvents::FailedInsertQuery);
}
}
}
static void setQuerySpecificSettings(ASTPtr & ast, Context & context)
{
if (auto * ast_insert_into = dynamic_cast<ASTInsertQuery *>(ast.get()))
{
if (ast_insert_into->watch)
context.setSetting("output_format_enable_streaming", 1);
}
}
2015-06-18 02:11:05 +00:00
static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
const char * begin,
const char * end,
Context & context,
bool internal,
2019-02-08 13:24:24 +00:00
QueryProcessingStage::Enum stage,
bool has_query_tail,
ReadBuffer * istr)
{
// current_time and current_time_microseconds are both constructed from the same time point
// to ensure that both the times are equal upto the precision of a second.
const auto now = std::chrono::system_clock::now();
auto current_time = time_in_seconds(now);
auto current_time_microseconds = time_in_microseconds(now);
/// If we already executing query and it requires to execute internal query, than
/// don't replace thread context with given (it can be temporary). Otherwise, attach context to thread.
if (!internal)
{
context.makeQueryContext();
CurrentThread::attachQueryContext(context);
}
const Settings & settings = context.getSettingsRef();
ParserQuery parser(end, settings.enable_debug_queries);
ASTPtr ast;
2019-01-18 16:30:35 +00:00
const char * query_end;
/// Don't limit the size of internal queries.
size_t max_query_size = 0;
if (!internal)
max_query_size = settings.max_query_size;
try
{
/// TODO Parser should fail early when max_query_size limit is reached.
ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
2019-03-11 13:22:51 +00:00
auto * insert_query = ast->as<ASTInsertQuery>();
2019-03-29 14:50:48 +00:00
if (insert_query && insert_query->settings_ast)
InterpreterSetQuery(insert_query->settings_ast, context).executeForCurrentContext();
2019-01-18 16:30:35 +00:00
if (insert_query && insert_query->data)
2019-02-08 13:24:24 +00:00
{
2019-01-18 16:30:35 +00:00
query_end = insert_query->data;
2019-02-08 13:24:24 +00:00
insert_query->has_tail = has_query_tail;
}
2019-01-18 16:30:35 +00:00
else
2019-06-15 18:22:48 +00:00
{
2019-01-18 16:30:35 +00:00
query_end = end;
2019-06-15 18:22:48 +00:00
}
}
catch (...)
{
2019-03-04 18:28:42 +00:00
/// Anyway log the query.
String query = String(begin, begin + std::min(end - begin, static_cast<ptrdiff_t>(max_query_size)));
2019-06-20 07:17:21 +00:00
auto query_for_logging = prepareQueryForLogging(query, context);
logQuery(query_for_logging, context, internal);
2019-03-04 18:28:42 +00:00
if (!internal)
{
onExceptionBeforeStart(query_for_logging, context, current_time, current_time_microseconds, ast);
}
throw;
}
setQuerySpecificSettings(ast, context);
2019-01-18 16:30:35 +00:00
/// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion.
String query(begin, query_end);
BlockIO res;
2020-03-09 00:28:05 +00:00
String query_for_logging;
2019-06-20 07:17:21 +00:00
try
{
2020-10-08 09:06:04 +00:00
bool ast_modified = false;
2019-06-15 18:22:48 +00:00
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
2019-06-16 17:32:37 +00:00
if (context.hasQueryParameters())
{
ReplaceQueryParameterVisitor visitor(context.getQueryParameters());
visitor.visit(ast);
2020-10-08 09:06:04 +00:00
ast_modified = true;
}
2019-06-15 18:22:48 +00:00
2020-10-08 09:06:04 +00:00
/// Propagate WITH statement to children ASTSelect.
if (settings.enable_global_with_statement)
{
ApplyWithGlobalVisitor().visit(ast);
ast_modified = true;
2019-07-07 23:25:15 +00:00
}
2019-06-04 18:15:32 +00:00
2020-10-08 09:06:04 +00:00
if (ast_modified)
query = serializeAST(*ast);
2019-06-20 07:17:21 +00:00
query_for_logging = prepareQueryForLogging(query, context);
logQuery(query_for_logging, context, internal);
/// Check the limits.
checkASTSizeLimits(*ast, settings);
/// Put query to process list. But don't put SHOW PROCESSLIST query itself.
ProcessList::EntryPtr process_list_entry;
2019-03-11 13:22:51 +00:00
if (!internal && !ast->as<ASTShowProcesslistQuery>())
{
2019-06-20 07:17:21 +00:00
/// processlist also has query masked now, to avoid secrets leaks though SHOW PROCESSLIST by other users.
process_list_entry = context.getProcessList().insert(query_for_logging, ast.get(), context);
context.setProcessListElement(&process_list_entry->get());
}
/// Load external tables if they were provided
context.initializeExternalTablesIfSet();
2019-05-28 18:30:10 +00:00
auto * insert_query = ast->as<ASTInsertQuery>();
if (insert_query && insert_query->select)
2019-05-28 18:30:10 +00:00
{
/// Prepare Input storage before executing interpreter if we already got a buffer with data.
2019-05-28 18:30:10 +00:00
if (istr)
{
ASTPtr input_function;
2019-05-30 21:33:06 +00:00
insert_query->tryFindInputFunction(input_function);
if (input_function)
{
StoragePtr storage = context.executeTableFunction(input_function);
auto & input_storage = dynamic_cast<StorageInput &>(*storage);
auto input_metadata_snapshot = input_storage.getInMemoryMetadataPtr();
BlockInputStreamPtr input_stream = std::make_shared<InputStreamFromASTInsertQuery>(
ast, istr, input_metadata_snapshot->getSampleBlock(), context, input_function);
input_storage.setInputStream(input_stream);
}
2019-05-28 18:30:10 +00:00
}
}
else
/// reset Input callbacks if query is not INSERT SELECT
context.resetInputCallbacks();
auto interpreter = InterpreterFactory::get(ast, context, stage);
2019-03-26 18:28:37 +00:00
std::shared_ptr<const EnabledQuota> quota;
if (!interpreter->ignoreQuota())
{
quota = context.getQuota();
if (quota)
{
quota->used(Quota::QUERIES, 1);
quota->checkExceeded(Quota::ERRORS);
}
}
2020-09-15 10:40:39 +00:00
StreamLocalLimits limits;
if (!interpreter->ignoreLimits())
{
2020-09-14 14:13:58 +00:00
limits.mode = LimitsMode::LIMITS_CURRENT;
limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode);
}
2020-05-28 08:24:59 +00:00
res = interpreter->execute();
QueryPipeline & pipeline = res.pipeline;
bool use_processors = pipeline.initialized();
2019-03-26 18:28:37 +00:00
2020-05-27 18:20:26 +00:00
if (res.pipeline.initialized())
use_processors = true;
2020-04-22 06:01:33 +00:00
if (const auto * insert_interpreter = typeid_cast<const InterpreterInsertQuery *>(&*interpreter))
{
/// Save insertion table (not table function). TODO: support remote() table function.
2020-03-02 20:23:58 +00:00
auto table_id = insert_interpreter->getDatabaseTable();
if (!table_id.empty())
context.setInsertionTable(std::move(table_id));
}
if (process_list_entry)
{
/// Query was killed before execution
if ((*process_list_entry)->isKilled())
throw Exception("Query '" + (*process_list_entry)->getInfo().client_info.current_query_id + "' is killed in pending state",
ErrorCodes::QUERY_WAS_CANCELLED);
2019-03-26 18:28:37 +00:00
else if (!use_processors)
(*process_list_entry)->setQueryStreams(res);
}
/// Hold element of process list till end of query execution.
res.process_list_entry = process_list_entry;
2019-03-26 18:28:37 +00:00
if (use_processors)
{
/// Limits on the result, the quota on the result, and also callback for progress.
/// Limits apply only to the final result.
pipeline.setProgressCallback(context.getProgressCallback());
pipeline.setProcessListElement(context.getProcessListElement());
2020-05-27 18:20:26 +00:00
if (stage == QueryProcessingStage::Complete && !pipeline.isCompleted())
{
2019-03-26 18:28:37 +00:00
pipeline.resize(1);
2019-04-05 11:34:11 +00:00
pipeline.addSimpleTransform([&](const Block & header)
{
2019-03-26 18:28:37 +00:00
auto transform = std::make_shared<LimitsCheckingTransform>(header, limits);
transform->setQuota(quota);
return transform;
});
}
}
2019-03-26 18:28:37 +00:00
else
{
/// Limits on the result, the quota on the result, and also callback for progress.
/// Limits apply only to the final result.
2019-03-26 18:28:37 +00:00
if (res.in)
{
2019-03-26 18:28:37 +00:00
res.in->setProgressCallback(context.getProgressCallback());
res.in->setProcessListElement(context.getProcessListElement());
if (stage == QueryProcessingStage::Complete)
{
if (!interpreter->ignoreQuota())
res.in->setQuota(quota);
if (!interpreter->ignoreLimits())
res.in->setLimits(limits);
2019-03-26 18:28:37 +00:00
}
}
if (res.out)
{
2020-04-22 06:01:33 +00:00
if (auto * stream = dynamic_cast<CountingBlockOutputStream *>(res.out.get()))
2019-03-26 18:28:37 +00:00
{
stream->setProcessListElement(context.getProcessListElement());
}
}
}
/// Everything related to query log.
{
QueryLogElement elem;
elem.type = QueryLogElementType::QUERY_START;
elem.event_time = current_time;
elem.event_time_microseconds = current_time_microseconds;
elem.query_start_time = current_time;
elem.query_start_time_microseconds = current_time_microseconds;
elem.current_database = context.getCurrentDatabase();
2019-06-20 07:17:21 +00:00
elem.query = query_for_logging;
elem.client_info = context.getClientInfo();
bool log_queries = settings.log_queries && !internal;
/// Log into system table start of query execution, if need.
if (log_queries)
{
if (settings.log_query_settings)
elem.query_settings = std::make_shared<Settings>(context.getSettingsRef());
if (elem.type >= settings.log_queries_min_type)
{
if (auto query_log = context.getQueryLog())
query_log->add(elem);
}
}
/// Common code for finish and exception callbacks
auto status_info_to_query_log = [](QueryLogElement &element, const QueryStatusInfo &info, const ASTPtr query_ast) mutable
{
DB::UInt64 query_time = info.elapsed_seconds * 1000000;
ProfileEvents::increment(ProfileEvents::QueryTimeMicroseconds, query_time);
if (query_ast->as<ASTSelectQuery>() || query_ast->as<ASTSelectWithUnionQuery>())
{
ProfileEvents::increment(ProfileEvents::SelectQueryTimeMicroseconds, query_time);
}
else if (query_ast->as<ASTInsertQuery>())
{
ProfileEvents::increment(ProfileEvents::InsertQueryTimeMicroseconds, query_time);
}
element.query_duration_ms = info.elapsed_seconds * 1000;
element.read_rows = info.read_rows;
element.read_bytes = info.read_bytes;
element.written_rows = info.written_rows;
element.written_bytes = info.written_bytes;
element.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0;
element.thread_ids = std::move(info.thread_ids);
element.profile_counters = std::move(info.profile_counters);
};
/// Also make possible for caller to log successful query finish and exception during execution.
auto finish_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type,
status_info_to_query_log]
(IBlockInputStream * stream_in, IBlockOutputStream * stream_out, QueryPipeline * query_pipeline) mutable
{
QueryStatus * process_list_elem = context.getProcessListElement();
if (!process_list_elem)
return;
/// Update performance counters before logging to query_log
CurrentThread::finalizePerformanceCounters();
2018-08-27 18:16:32 +00:00
QueryStatusInfo info = process_list_elem->getInfo(true, context.getSettingsRef().log_profile_events);
double elapsed_seconds = info.elapsed_seconds;
elem.type = QueryLogElementType::QUERY_FINISH;
// construct event_time and event_time_microseconds using the same time point
// so that the two times will always be equal up to a precision of a second.
const auto time_now = std::chrono::system_clock::now();
elem.event_time = time_in_seconds(time_now);
elem.event_time_microseconds = time_in_microseconds(time_now);
status_info_to_query_log(elem, info, ast);
auto progress_callback = context.getProgressCallback();
2019-05-06 06:57:48 +00:00
if (progress_callback)
progress_callback(Progress(WriteProgress(info.written_rows, info.written_bytes)));
if (stream_in)
{
const BlockStreamProfileInfo & stream_in_info = stream_in->getProfileInfo();
/// NOTE: INSERT SELECT query contains zero metrics
elem.result_rows = stream_in_info.rows;
elem.result_bytes = stream_in_info.bytes;
}
else if (stream_out) /// will be used only for ordinary INSERT queries
{
2020-04-22 06:01:33 +00:00
if (const auto * counting_stream = dynamic_cast<const CountingBlockOutputStream *>(stream_out))
{
2019-01-22 19:56:53 +00:00
/// NOTE: Redundancy. The same values could be extracted from process_list_elem->progress_out.query_settings = process_list_elem->progress_in
elem.result_rows = counting_stream->getProgress().read_rows;
elem.result_bytes = counting_stream->getProgress().read_bytes;
}
}
else if (query_pipeline)
{
if (const auto * output_format = query_pipeline->getOutputFormat())
{
elem.result_rows = output_format->getResultRows();
elem.result_bytes = output_format->getResultBytes();
}
}
if (elem.read_rows != 0)
{
2020-05-30 21:57:37 +00:00
LOG_INFO(&Poco::Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.",
2020-05-30 21:35:52 +00:00
elem.read_rows, ReadableSize(elem.read_bytes), elapsed_seconds,
2020-05-23 21:41:35 +00:00
static_cast<size_t>(elem.read_rows / elapsed_seconds),
2020-05-30 21:35:52 +00:00
ReadableSize(elem.read_bytes / elapsed_seconds));
}
2020-02-02 02:27:15 +00:00
elem.thread_ids = std::move(info.thread_ids);
elem.profile_counters = std::move(info.profile_counters);
if (log_queries && elem.type >= log_queries_min_type)
{
if (auto query_log = context.getQueryLog())
query_log->add(elem);
}
};
auto exception_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type, quota(quota),
status_info_to_query_log] () mutable
{
if (quota)
quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false);
elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING;
// event_time and event_time_microseconds are being constructed from the same time point
// to ensure that both the times will be equal upto the precision of a second.
const auto time_now = std::chrono::system_clock::now();
elem.event_time = time_in_seconds(time_now);
elem.event_time_microseconds = time_in_microseconds(time_now);
elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time);
2020-01-22 12:29:30 +00:00
elem.exception_code = getCurrentExceptionCode();
elem.exception = getCurrentExceptionMessage(false);
QueryStatus * process_list_elem = context.getProcessListElement();
2018-08-27 18:16:32 +00:00
const Settings & current_settings = context.getSettingsRef();
/// Update performance counters before logging to query_log
CurrentThread::finalizePerformanceCounters();
if (process_list_elem)
{
2018-08-27 18:16:32 +00:00
QueryStatusInfo info = process_list_elem->getInfo(true, current_settings.log_profile_events, false);
status_info_to_query_log(elem, info, ast);
}
2018-08-27 18:16:32 +00:00
if (current_settings.calculate_text_stack_trace)
2018-08-23 01:31:28 +00:00
setExceptionStackTrace(elem);
logException(context, elem);
/// In case of exception we log internal queries also
if (log_queries && elem.type >= log_queries_min_type)
{
if (auto query_log = context.getQueryLog())
query_log->add(elem);
}
ProfileEvents::increment(ProfileEvents::FailedQuery);
2020-05-22 23:37:14 +00:00
if (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>())
{
ProfileEvents::increment(ProfileEvents::FailedSelectQuery);
}
2020-05-22 23:37:14 +00:00
else if (ast->as<ASTInsertQuery>())
{
ProfileEvents::increment(ProfileEvents::FailedInsertQuery);
}
};
2019-04-05 10:52:07 +00:00
res.finish_callback = std::move(finish_callback);
res.exception_callback = std::move(exception_callback);
2019-03-26 18:28:37 +00:00
if (!internal && res.in)
{
std::stringstream log_str;
log_str << "Query pipeline:\n";
res.in->dumpTree(log_str);
2020-05-30 21:57:37 +00:00
LOG_DEBUG(&Poco::Logger::get("executeQuery"), log_str.str());
}
}
}
catch (...)
{
if (!internal)
2019-06-20 07:17:21 +00:00
{
if (query_for_logging.empty())
query_for_logging = prepareQueryForLogging(query, context);
onExceptionBeforeStart(query_for_logging, context, current_time, current_time_microseconds, ast);
2019-06-20 07:17:21 +00:00
}
throw;
}
2020-02-27 15:40:11 +00:00
return std::make_tuple(ast, std::move(res));
2015-06-18 02:11:05 +00:00
}
BlockIO executeQuery(
const String & query,
Context & context,
bool internal,
QueryProcessingStage::Enum stage,
bool may_have_embedded_data)
2015-06-18 02:11:05 +00:00
{
2019-11-04 03:53:26 +00:00
ASTPtr ast;
BlockIO streams;
2019-11-04 03:53:26 +00:00
std::tie(ast, streams) = executeQueryImpl(query.data(), query.data() + query.size(), context,
internal, stage, !may_have_embedded_data, nullptr);
if (const auto * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get()))
2019-11-04 03:53:26 +00:00
{
String format_name = ast_query_with_output->format
? getIdentifierName(ast_query_with_output->format)
: context.getDefaultFormat();
2019-11-04 03:53:26 +00:00
if (format_name == "Null")
streams.null_format = true;
}
return streams;
}
2020-05-19 14:06:33 +00:00
BlockIO executeQuery(
const String & query,
Context & context,
bool internal,
QueryProcessingStage::Enum stage,
bool may_have_embedded_data,
bool allow_processors)
{
2020-05-19 15:40:15 +00:00
BlockIO res = executeQuery(query, context, internal, stage, may_have_embedded_data);
2020-05-19 14:06:33 +00:00
if (!allow_processors && res.pipeline.initialized())
res.in = res.getInputStream();
return res;
}
2011-10-30 11:30:52 +00:00
void executeQuery(
ReadBuffer & istr,
WriteBuffer & ostr,
bool allow_into_outfile,
Context & context,
std::function<void(const String &, const String &, const String &, const String &)> set_result_details)
2011-10-30 11:30:52 +00:00
{
PODArray<char> parse_buf;
const char * begin;
const char * end;
/// If 'istr' is empty now, fetch next data into buffer.
2020-07-31 14:53:41 +00:00
if (!istr.hasPendingData())
istr.next();
size_t max_query_size = context.getSettingsRef().max_query_size;
bool may_have_tail;
2018-03-26 19:41:55 +00:00
if (istr.buffer().end() - istr.position() > static_cast<ssize_t>(max_query_size))
{
/// If remaining buffer space in 'istr' is enough to parse query up to 'max_query_size' bytes, then parse inplace.
begin = istr.position();
end = istr.buffer().end();
istr.position() += end - begin;
/// Actually we don't know will query has additional data or not.
/// But we can't check istr.eof(), because begin and end pointers will became invalid
may_have_tail = true;
}
else
{
/// If not - copy enough data into 'parse_buf'.
WriteBufferFromVector<PODArray<char>> out(parse_buf);
LimitReadBuffer limit(istr, max_query_size + 1, false);
copyData(limit, out);
2020-01-10 21:42:26 +00:00
out.finalize();
begin = parse_buf.data();
end = begin + parse_buf.size();
/// Can check stream for eof, because we have copied data
may_have_tail = !istr.eof();
}
ASTPtr ast;
BlockIO streams;
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, may_have_tail, &istr);
2019-03-26 18:28:37 +00:00
auto & pipeline = streams.pipeline;
try
{
if (streams.out)
{
InputStreamFromASTInsertQuery in(ast, &istr, streams.out->getHeader(), context, nullptr);
copyData(in, *streams.out);
}
2020-10-10 01:43:07 +00:00
else if (streams.in)
{
/// FIXME: try to prettify this cast using `as<>()`
const auto * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
WriteBuffer * out_buf = &ostr;
std::optional<WriteBufferFromFile> out_file_buf;
if (ast_query_with_output && ast_query_with_output->out_file)
{
if (!allow_into_outfile)
throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED);
const auto & out_file = ast_query_with_output->out_file->as<ASTLiteral &>().value.safeGet<std::string>();
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
out_buf = &*out_file_buf;
}
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
2019-08-08 20:02:30 +00:00
? getIdentifierName(ast_query_with_output->format)
: context.getDefaultFormat();
2019-01-24 19:22:26 +00:00
if (ast_query_with_output && ast_query_with_output->settings_ast)
InterpreterSetQuery(ast_query_with_output->settings_ast, context).executeForCurrentContext();
BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader());
/// Save previous progress callback if any. TODO Do it more conveniently.
auto previous_progress_callback = context.getProgressCallback();
/// NOTE Progress callback takes shared ownership of 'out'.
streams.in->setProgressCallback([out, previous_progress_callback] (const Progress & progress)
{
if (previous_progress_callback)
previous_progress_callback(progress);
out->onProgress(progress);
});
if (set_result_details)
set_result_details(context.getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone());
2019-02-02 12:24:26 +00:00
copyData(*streams.in, *out, [](){ return false; }, [&out](const Block &) { out->flush(); });
}
2020-10-10 01:43:07 +00:00
else if (pipeline.initialized())
2019-03-26 18:28:37 +00:00
{
const ASTQueryWithOutput * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
WriteBuffer * out_buf = &ostr;
std::optional<WriteBufferFromFile> out_file_buf;
if (ast_query_with_output && ast_query_with_output->out_file)
{
if (!allow_into_outfile)
throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED);
const auto & out_file = typeid_cast<const ASTLiteral &>(*ast_query_with_output->out_file).value.safeGet<std::string>();
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
out_buf = &*out_file_buf;
}
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
2019-08-08 20:02:30 +00:00
? getIdentifierName(ast_query_with_output->format)
2019-03-26 18:28:37 +00:00
: context.getDefaultFormat();
if (ast_query_with_output && ast_query_with_output->settings_ast)
InterpreterSetQuery(ast_query_with_output->settings_ast, context).executeForCurrentContext();
2020-05-27 18:20:26 +00:00
if (!pipeline.isCompleted())
2019-04-12 15:32:53 +00:00
{
2020-05-27 18:20:26 +00:00
pipeline.addSimpleTransform([](const Block & header)
{
return std::make_shared<MaterializingTransform>(header);
});
2019-04-12 15:32:53 +00:00
2020-05-27 18:20:26 +00:00
auto out = context.getOutputFormatProcessor(format_name, *out_buf, pipeline.getHeader());
out->setAutoFlush();
2019-03-26 18:28:37 +00:00
2020-05-27 18:20:26 +00:00
/// Save previous progress callback if any. TODO Do it more conveniently.
auto previous_progress_callback = context.getProgressCallback();
2019-03-26 18:28:37 +00:00
2020-05-27 18:20:26 +00:00
/// NOTE Progress callback takes shared ownership of 'out'.
pipeline.setProgressCallback([out, previous_progress_callback] (const Progress & progress)
{
if (previous_progress_callback)
previous_progress_callback(progress);
out->onProgress(progress);
});
2019-03-26 18:28:37 +00:00
2020-05-27 18:20:26 +00:00
if (set_result_details)
set_result_details(context.getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone());
2019-03-26 18:28:37 +00:00
2020-05-27 18:20:26 +00:00
pipeline.setOutputFormat(std::move(out));
}
else
{
pipeline.setProgressCallback(context.getProgressCallback());
}
{
2019-05-16 14:57:27 +00:00
auto executor = pipeline.execute();
executor->execute(pipeline.getNumThreads());
}
2019-03-26 18:28:37 +00:00
}
}
catch (...)
{
streams.onException();
throw;
}
streams.onFinish();
2015-06-18 02:11:05 +00:00
}
2012-03-11 08:52:56 +00:00
2011-10-30 11:30:52 +00:00
}