ClickHouse/src/Interpreters/InterpreterExplainQuery.cpp
Azat Khuzhin d04cda0367 Fix QueryPlan lifetime (for EXPLAIN PIPELINE graph=1) for queries with nested interpreter
Example of such queries are distributed queries, which creates local
InterpreterSelectQuery, which will have it's own QueryPlan but returns
Pipes that has that IQueryPlanStep attached.

After EXPLAIN PIPELINE graph=1 tries to use them, and will get SIGSEGV.

- TSAN:

<details>

```
==2782113==ERROR: AddressSanitizer: heap-use-after-free on address 0x6120000223c0 at pc 0x00002b8f3f3e bp 0x7fff18cfbff0 sp 0x7fff18cfbfe8
READ of size 8 at 0x6120000223c0 thread T22 (TCPHandler)
    #0 0x2b8f3f3d in DB::printPipelineCompact(std::__1::vector<std::__1::shared_ptr<DB::IProcessor>, std::__1::allocator<std::__1::shared_ptr<DB::IProcessor> > > const&, DB::WriteBuffer&, bool) /build/obj-x86_64-linux-gnu/../src/Processors/printPipeline.cpp:116:53
    #1 0x29ee698c in DB::InterpreterExplainQuery::executeImpl() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterExplainQuery.cpp:275:17
    #2 0x29ee2e40 in DB::InterpreterExplainQuery::execute() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterExplainQuery.cpp:73:14
    #3 0x2a7b44a2 in DB::executeQueryImpl(char const*, char const*, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool, DB::ReadBuffer*) /build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:389:28
    #4 0x2a7b1cb3 in DB::executeQuery(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool) /build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:675:30
    #5 0x2b7993b2 in DB::TCPHandler::runImpl() /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:253:24
    #6 0x2b7b649a in DB::TCPHandler::run() /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:1217:9
    #7 0x31d9c57e in Poco::Net::TCPServerConnection::start() /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:43:3
    #8 0x31d9d281 in Poco::Net::TCPServerDispatcher::run() /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:114:20
    #9 0x3206b5d5 in Poco::PooledThread::run() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:199:14
    #10 0x320657ad in Poco::ThreadImpl::runnableEntry(void*) /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:345:27
    #11 0x7ffff7f853e8 in start_thread (/usr/lib/libpthread.so.0+0x93e8)
    #12 0x7ffff7ea2292 in clone (/usr/lib/libc.so.6+0x100292)

0x6120000223c0 is located 0 bytes inside of 272-byte region [0x6120000223c0,0x6120000224d0)
freed by thread T22 (TCPHandler) here:
    #0 0x122f3b62 in operator delete(void*, unsigned long) (/src/ch/tmp/master-20200831/clickhouse+0x122f3b62)
    #1 0x2bd9e9fa in std::__1::default_delete<DB::IQueryPlanStep>::operator()(DB::IQueryPlanStep*) const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2363:5
    #2 0x2bd9e9fa in std::__1::unique_ptr<DB::IQueryPlanStep, std::__1::default_delete<DB::IQueryPlanStep> >::reset(DB::IQueryPlanStep*) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2618:7
    #3 0x2bd9e9fa in std::__1::unique_ptr<DB::IQueryPlanStep, std::__1::default_delete<DB::IQueryPlanStep> >::~unique_ptr() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2572:19
    #4 0x2bd9e9fa in DB::QueryPlan::Node::~Node() /build/obj-x86_64-linux-gnu/../src/Processors/QueryPlan/QueryPlan.h:66:12
    #5 0x2bd9e9fa in void std::__1::allocator_traits<std::__1::allocator<std::__1::__list_node<DB::QueryPlan::Node, void*> > >::__destroy<DB::QueryPlan::Node>(std::__1::integral_constant<bool, false>, std::__1::allocator<std::__1::__list_node<DB::QueryPlan::Node, void*> >&,
 DB::QueryPlan::Node*) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1798:23
    #6 0x2bd9e9fa in void std::__1::allocator_traits<std::__1::allocator<std::__1::__list_node<DB::QueryPlan::Node, void*> > >::destroy<DB::QueryPlan::Node>(std::__1::allocator<std::__1::__list_node<DB::QueryPlan::Node, void*> >&, DB::QueryPlan::Node*) /build/obj-x86_64-lin
ux-gnu/../contrib/libcxx/include/memory:1630:14
    #7 0x2bd9e9fa in std::__1::__list_imp<DB::QueryPlan::Node, std::__1::allocator<DB::QueryPlan::Node> >::clear() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/list:762:13
    #8 0x29fece08 in DB::InterpreterSelectQuery::execute() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:492:1
    #9 0x2abf7484 in DB::ClusterProxy::(anonymous namespace)::createLocalStream(std::__1::shared_ptr<DB::IAST> const&, DB::Block const&, DB::Context const&, DB::QueryProcessingStage::Enum) /build/obj-x86_64-linux-gnu/../src/Interpreters/ClusterProxy/SelectStreamFactory.cpp:
78:33
    #10 0x2abea85d in DB::ClusterProxy::SelectStreamFactory::createForShard(DB::Cluster::ShardInfo const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::shared_ptr<DB::IAST> const&, DB::Context const&, std::__1::shar
ed_ptr<DB::Throttler> const&, DB::SelectQueryInfo const&, std::__1::vector<DB::Pipe, std::__1::allocator<DB::Pipe> >&)::$_0::operator()() const /build/obj-x86_64-linux-gnu/../src/Interpreters/ClusterProxy/SelectStreamFactory.cpp:133:51
    #11 0x2abea85d in DB::ClusterProxy::SelectStreamFactory::createForShard(DB::Cluster::ShardInfo const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::shared_ptr<DB::IAST> const&, DB::Context const&, std::__1::shar
ed_ptr<DB::Throttler> const&, DB::SelectQueryInfo const&, std::__1::vector<DB::Pipe, std::__1::allocator<DB::Pipe> >&) /build/obj-x86_64-linux-gnu/../src/Interpreters/ClusterProxy/SelectStreamFactory.cpp:189:13
    #12 0x2abe6d99 in DB::ClusterProxy::executeQuery(DB::ClusterProxy::IStreamFactory&, std::__1::shared_ptr<DB::Cluster> const&, Poco::Logger*, std::__1::shared_ptr<DB::IAST> const&, DB::Context const&, DB::Settings const&, DB::SelectQueryInfo const&) /build/obj-x86_64-lin
ux-gnu/../src/Interpreters/ClusterProxy/executeQuery.cpp:107:24
    #13 0x2abc4b74 in DB::StorageDistributed::read(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&, s
td::__1::shared_ptr<DB::StorageInMemoryMetadata const> const&, DB::SelectQueryInfo const&, DB::Context const&, DB::QueryProcessingStage::Enum, unsigned long, unsigned int) /build/obj-x86_64-linux-gnu/../src/Storages/StorageDistributed.cpp:514:12
    #14 0x2bda1c5a in DB::ReadFromStorageStep::ReadFromStorageStep(std::__1::shared_ptr<DB::RWLockImpl::LockHolderImpl>, std::__1::shared_ptr<DB::StorageInMemoryMetadata const>&, DB::SelectQueryOptions, std::__1::shared_ptr<DB::IStorage>, std::__1::vector<std::__1::basic_st
ring<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&, DB::SelectQueryInfo const&, std::__1::shared_ptr<DB::Context>, DB::QueryProcessingStage
::Enum, unsigned long, unsigned long) /build/obj-x86_64-linux-gnu/../src/Processors/QueryPlan/ReadFromStorageStep.cpp:39:26
    #15 0x2a01ca70 in std::__1::__unique_if<DB::ReadFromStorageStep>::__unique_single std::__1::make_unique<DB::ReadFromStorageStep, std::__1::shared_ptr<DB::RWLockImpl::LockHolderImpl>&, std::__1::shared_ptr<DB::StorageInMemoryMetadata const>&, DB::SelectQueryOptions&, std
::__1::shared_ptr<DB::IStorage>&, std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > >&, DB::SelectQueryInfo&, st
d::__1::shared_ptr<DB::Context>&, DB::QueryProcessingStage::Enum&, unsigned long&, unsigned long&>(std::__1::shared_ptr<DB::RWLockImpl::LockHolderImpl>&, std::__1::shared_ptr<DB::StorageInMemoryMetadata const>&, DB::SelectQueryOptions&, std::__1::shared_ptr<DB::IStorage>&,
std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > >&, DB::SelectQueryInfo&, std::__1::shared_ptr<DB::Context>&,
DB::QueryProcessingStage::Enum&, unsigned long&, unsigned long&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3028:32
    #16 0x29ff556a in DB::InterpreterSelectQuery::executeFetchColumns(DB::QueryProcessingStage::Enum, DB::QueryPlan&, std::__1::shared_ptr<DB::PrewhereInfo> const&, std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::
__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:1383:26
    #17 0x29fe6b83 in DB::InterpreterSelectQuery::executeImpl(DB::QueryPlan&, std::__1::shared_ptr<DB::IBlockInputStream> const&, std::__1::optional<DB::Pipe>) /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:795:9
    #18 0x29fe5771 in DB::InterpreterSelectQuery::buildQueryPlan(DB::QueryPlan&) /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:473:5
    #19 0x2a47d370 in DB::InterpreterSelectWithUnionQuery::buildQueryPlan(DB::QueryPlan&) /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectWithUnionQuery.cpp:182:38
    #20 0x29ee5bff in DB::InterpreterExplainQuery::executeImpl() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterExplainQuery.cpp:265:21
    #21 0x29ee2e40 in DB::InterpreterExplainQuery::execute() /build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterExplainQuery.cpp:73:14
    #22 0x2a7b44a2 in DB::executeQueryImpl(char const*, char const*, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool, DB::ReadBuffer*) /build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:389:28
    #23 0x2a7b1cb3 in DB::executeQuery(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool) /build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:675:30
    #24 0x2b7993b2 in DB::TCPHandler::runImpl() /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:253:24
    #25 0x2b7b649a in DB::TCPHandler::run() /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:1217:9
    #26 0x31d9c57e in Poco::Net::TCPServerConnection::start() /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:43:3
    #27 0x31d9d281 in Poco::Net::TCPServerDispatcher::run() /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:114:20
    #28 0x3206b5d5 in Poco::PooledThread::run() /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:199:14
    #29 0x320657ad in Poco::ThreadImpl::runnableEntry(void*) /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:345:27
    #30 0x7ffff7f853e8 in start_thread (/usr/lib/libpthread.so.0+0x93e8)
```

</details>
2020-09-01 10:04:08 +03:00

293 lines
8.4 KiB
C++

#include <Interpreters/InterpreterExplainQuery.h>
#include <DataStreams/BlockIO.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataTypes/DataTypeString.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/Context.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <IO/WriteBufferFromOStream.h>
#include <Storages/StorageView.h>
#include <sstream>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/printPipeline.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_QUERY;
extern const int INVALID_SETTING_VALUE;
extern const int UNKNOWN_SETTING;
extern const int LOGICAL_ERROR;
}
namespace
{
struct ExplainAnalyzedSyntaxMatcher
{
struct Data
{
const Context & context;
};
static bool needChildVisit(ASTPtr & node, ASTPtr &)
{
return !node->as<ASTSelectQuery>();
}
static void visit(ASTPtr & ast, Data & data)
{
if (auto * select = ast->as<ASTSelectQuery>())
visit(*select, ast, data);
}
static void visit(ASTSelectQuery & select, ASTPtr & node, Data & data)
{
InterpreterSelectQuery interpreter(
node, data.context, SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze().modify());
const SelectQueryInfo & query_info = interpreter.getQueryInfo();
if (query_info.view_query)
{
ASTPtr tmp;
StorageView::replaceWithSubquery(select, query_info.view_query->clone(), tmp);
}
}
};
using ExplainAnalyzedSyntaxVisitor = InDepthNodeVisitor<ExplainAnalyzedSyntaxMatcher, true>;
}
BlockIO InterpreterExplainQuery::execute()
{
BlockIO res;
res.in = executeImpl();
return res;
}
Block InterpreterExplainQuery::getSampleBlock()
{
Block block;
ColumnWithTypeAndName col;
col.name = "explain";
col.type = std::make_shared<DataTypeString>();
col.column = col.type->createColumn();
block.insert(col);
return block;
}
/// Split str by line feed and write as separate row to ColumnString.
static void fillColumn(IColumn & column, const std::string & str)
{
size_t start = 0;
size_t end = 0;
size_t size = str.size();
while (end < size)
{
if (str[end] == '\n')
{
column.insertData(str.data() + start, end - start);
start = end + 1;
}
++end;
}
if (start < end)
column.insertData(str.data() + start, end - start);
}
namespace
{
/// Settings. Different for each explain type.
struct QueryPlanSettings
{
QueryPlan::ExplainPlanOptions query_plan_options;
constexpr static char name[] = "PLAN";
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
{
{"header", query_plan_options.header},
{"description", query_plan_options.description},
{"actions", query_plan_options.actions}
};
};
struct QueryPipelineSettings
{
QueryPlan::ExplainPipelineOptions query_pipeline_options;
bool graph = false;
bool compact = true;
constexpr static char name[] = "PIPELINE";
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
{
{"header", query_pipeline_options.header},
{"graph", graph},
{"compact", compact},
};
};
template <typename Settings>
struct ExplainSettings : public Settings
{
using Settings::boolean_settings;
bool has(const std::string & name_) const
{
return boolean_settings.count(name_) > 0;
}
void setBooleanSetting(const std::string & name_, bool value)
{
auto it = boolean_settings.find(name_);
if (it == boolean_settings.end())
throw Exception("Unknown setting for ExplainSettings: " + name_, ErrorCodes::LOGICAL_ERROR);
it->second.get() = value;
}
std::string getSettingsList() const
{
std::string res;
for (const auto & setting : boolean_settings)
{
if (!res.empty())
res += ", ";
res += setting.first;
}
return res;
}
};
template <typename Settings>
ExplainSettings<Settings> checkAndGetSettings(const ASTPtr & ast_settings)
{
if (!ast_settings)
return {};
ExplainSettings<Settings> settings;
const auto & set_query = ast_settings->as<ASTSetQuery &>();
for (const auto & change : set_query.changes)
{
if (!settings.has(change.name))
throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN " + Settings::name + " query. "
"Supported settings: " + settings.getSettingsList(), ErrorCodes::UNKNOWN_SETTING);
if (change.value.getType() != Field::Types::UInt64)
throw Exception("Invalid type " + std::string(change.value.getTypeName()) + " for setting \"" + change.name +
"\" only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE);
auto value = change.value.get<UInt64>();
if (value > 1)
throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name +
"\". Only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE);
settings.setBooleanSetting(change.name, value);
}
return settings;
}
}
BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
{
const auto & ast = query->as<ASTExplainQuery &>();
Block sample_block = getSampleBlock();
MutableColumns res_columns = sample_block.cloneEmptyColumns();
std::stringstream ss;
if (ast.getKind() == ASTExplainQuery::ParsedAST)
{
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING);
dumpAST(*ast.getExplainedQuery(), ss);
}
else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax)
{
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING);
ExplainAnalyzedSyntaxVisitor::Data data{.context = context};
ExplainAnalyzedSyntaxVisitor(data).visit(query);
ast.getExplainedQuery()->format(IAST::FormatSettings(ss, false));
}
else if (ast.getKind() == ASTExplainQuery::QueryPlan)
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY);
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
QueryPlan plan;
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions());
interpreter.buildQueryPlan(plan);
plan.optimize();
WriteBufferFromOStream buffer(ss);
plan.explainPlan(buffer, settings.query_plan_options);
}
else if (ast.getKind() == ASTExplainQuery::QueryPipeline)
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY);
auto settings = checkAndGetSettings<QueryPipelineSettings>(ast.getSettings());
QueryPlan plan;
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions());
interpreter.buildQueryPlan(plan);
auto pipeline = plan.buildQueryPipeline();
WriteBufferFromOStream buffer(ss);
if (settings.graph)
{
/// Pipe holds QueryPlan, should not go out-of-scope
auto pipe = QueryPipeline::getPipe(std::move(*pipeline));
const auto & processors = pipe.getProcessors();
if (settings.compact)
printPipelineCompact(processors, buffer, settings.query_pipeline_options.header);
else
printPipeline(processors, buffer);
}
else
{
plan.explainPipeline(buffer, settings.query_pipeline_options);
}
}
fillColumn(*res_columns[0], ss.str());
return std::make_shared<OneBlockInputStream>(sample_block.cloneWithColumns(std::move(res_columns)));
}
}