mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Implement EXPLAIN TABLE OVERRIDE
for pre-validating overrides.
Example: ```sql EXPLAIN TABLE OVERRIDE mysql('127.0.0.1:3306', 'db', 'table', 'user', 'pw') PARTITION BY tuple(toYYYYMM(created), id % 8) ``` Validations done: * check that storage overrides do not reference unknown or nullable columns * check that default specifier is not modified for columns
This commit is contained in:
parent
c5fe3a8f8a
commit
6fbb9f5e87
@ -605,6 +605,7 @@
|
||||
M(634, MONGODB_ERROR) \
|
||||
M(635, CANNOT_POLL) \
|
||||
M(636, CANNOT_EXTRACT_TABLE_STRUCTURE) \
|
||||
M(637, INVALID_TABLE_OVERRIDE) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
@ -7,10 +7,12 @@
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/TableOverrideUtils.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTExplainQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
|
||||
@ -250,117 +252,141 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
|
||||
|
||||
WriteBufferFromOwnString buf;
|
||||
bool single_line = false;
|
||||
bool insert_buf = true;
|
||||
|
||||
if (ast.getKind() == ASTExplainQuery::ParsedAST)
|
||||
switch (ast.getKind())
|
||||
{
|
||||
if (ast.getSettings())
|
||||
throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING);
|
||||
|
||||
dumpAST(*ast.getExplainedQuery(), buf);
|
||||
}
|
||||
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(getContext());
|
||||
ExplainAnalyzedSyntaxVisitor(data).visit(query);
|
||||
|
||||
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, 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(), getContext(), SelectQueryOptions());
|
||||
interpreter.buildQueryPlan(plan);
|
||||
|
||||
if (settings.optimize)
|
||||
plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext()));
|
||||
|
||||
if (settings.json)
|
||||
case ASTExplainQuery::ParsedAST:
|
||||
{
|
||||
/// Add extra layers to make plan look more like from postgres.
|
||||
auto plan_map = std::make_unique<JSONBuilder::JSONMap>();
|
||||
plan_map->add("Plan", plan.explainPlan(settings.query_plan_options));
|
||||
auto plan_array = std::make_unique<JSONBuilder::JSONArray>();
|
||||
plan_array->add(std::move(plan_map));
|
||||
if (ast.getSettings())
|
||||
throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING);
|
||||
|
||||
auto format_settings = getFormatSettings(getContext());
|
||||
format_settings.json.quote_64bit_integers = false;
|
||||
|
||||
JSONBuilder::FormatSettings json_format_settings{.settings = format_settings};
|
||||
JSONBuilder::FormatContext format_context{.out = buf};
|
||||
|
||||
plan_array->format(json_format_settings, format_context);
|
||||
|
||||
single_line = true;
|
||||
dumpAST(*ast.getExplainedQuery(), buf);
|
||||
break;
|
||||
}
|
||||
else
|
||||
plan.explainPlan(buf, settings.query_plan_options);
|
||||
}
|
||||
else if (ast.getKind() == ASTExplainQuery::QueryPipeline)
|
||||
{
|
||||
if (dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
|
||||
case ASTExplainQuery::AnalyzedSyntax:
|
||||
{
|
||||
auto settings = checkAndGetSettings<QueryPipelineSettings>(ast.getSettings());
|
||||
if (ast.getSettings())
|
||||
throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING);
|
||||
|
||||
ExplainAnalyzedSyntaxVisitor::Data data(getContext());
|
||||
ExplainAnalyzedSyntaxVisitor(data).visit(query);
|
||||
|
||||
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false));
|
||||
break;
|
||||
}
|
||||
case 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(), getContext(), SelectQueryOptions());
|
||||
interpreter.buildQueryPlan(plan);
|
||||
auto pipeline = plan.buildQueryPipeline(
|
||||
|
||||
if (settings.optimize)
|
||||
plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext()));
|
||||
|
||||
if (settings.json)
|
||||
{
|
||||
/// Add extra layers to make plan look more like from postgres.
|
||||
auto plan_map = std::make_unique<JSONBuilder::JSONMap>();
|
||||
plan_map->add("Plan", plan.explainPlan(settings.query_plan_options));
|
||||
auto plan_array = std::make_unique<JSONBuilder::JSONArray>();
|
||||
plan_array->add(std::move(plan_map));
|
||||
|
||||
auto format_settings = getFormatSettings(getContext());
|
||||
format_settings.json.quote_64bit_integers = false;
|
||||
|
||||
JSONBuilder::FormatSettings json_format_settings{.settings = format_settings};
|
||||
JSONBuilder::FormatContext format_context{.out = buf};
|
||||
|
||||
plan_array->format(json_format_settings, format_context);
|
||||
|
||||
single_line = true;
|
||||
}
|
||||
else
|
||||
plan.explainPlan(buf, settings.query_plan_options);
|
||||
break;
|
||||
}
|
||||
case ASTExplainQuery::QueryPipeline:
|
||||
{
|
||||
if (dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
|
||||
{
|
||||
auto settings = checkAndGetSettings<QueryPipelineSettings>(ast.getSettings());
|
||||
QueryPlan plan;
|
||||
|
||||
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions());
|
||||
interpreter.buildQueryPlan(plan);
|
||||
auto pipeline = plan.buildQueryPipeline(
|
||||
QueryPlanOptimizationSettings::fromContext(getContext()),
|
||||
BuildQueryPipelineSettings::fromContext(getContext()));
|
||||
|
||||
if (settings.graph)
|
||||
{
|
||||
/// Pipe holds QueryPlan, should not go out-of-scope
|
||||
auto pipe = QueryPipelineBuilder::getPipe(std::move(*pipeline));
|
||||
const auto & processors = pipe.getProcessors();
|
||||
|
||||
if (settings.compact)
|
||||
printPipelineCompact(processors, buf, settings.query_pipeline_options.header);
|
||||
else
|
||||
printPipeline(processors, buf);
|
||||
}
|
||||
else
|
||||
{
|
||||
plan.explainPipeline(buf, settings.query_pipeline_options);
|
||||
}
|
||||
}
|
||||
else if (dynamic_cast<const ASTInsertQuery *>(ast.getExplainedQuery().get()))
|
||||
{
|
||||
InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext());
|
||||
auto io = insert.execute();
|
||||
printPipeline(io.pipeline.getProcessors(), buf);
|
||||
}
|
||||
else
|
||||
throw Exception("Only SELECT and INSERT is supported for EXPLAIN PIPELINE query", ErrorCodes::INCORRECT_QUERY);
|
||||
break;
|
||||
}
|
||||
case ASTExplainQuery::QueryEstimates:
|
||||
{
|
||||
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
|
||||
throw Exception("Only SELECT is supported for EXPLAIN ESTIMATE query", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
|
||||
QueryPlan plan;
|
||||
|
||||
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions());
|
||||
interpreter.buildQueryPlan(plan);
|
||||
// collect the selected marks, rows, parts during build query pipeline.
|
||||
plan.buildQueryPipeline(
|
||||
QueryPlanOptimizationSettings::fromContext(getContext()),
|
||||
BuildQueryPipelineSettings::fromContext(getContext()));
|
||||
|
||||
if (settings.graph)
|
||||
{
|
||||
/// Pipe holds QueryPlan, should not go out-of-scope
|
||||
auto pipe = QueryPipelineBuilder::getPipe(std::move(*pipeline));
|
||||
const auto & processors = pipe.getProcessors();
|
||||
|
||||
if (settings.compact)
|
||||
printPipelineCompact(processors, buf, settings.query_pipeline_options.header);
|
||||
else
|
||||
printPipeline(processors, buf);
|
||||
}
|
||||
else
|
||||
{
|
||||
plan.explainPipeline(buf, settings.query_pipeline_options);
|
||||
}
|
||||
if (settings.optimize)
|
||||
plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext()));
|
||||
plan.explainEstimate(res_columns);
|
||||
insert_buf = false;
|
||||
break;
|
||||
}
|
||||
else if (dynamic_cast<const ASTInsertQuery *>(ast.getExplainedQuery().get()))
|
||||
case ASTExplainQuery::TableOverride:
|
||||
{
|
||||
InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext());
|
||||
auto io = insert.execute();
|
||||
printPipeline(io.pipeline.getProcessors(), buf);
|
||||
if (auto * table_function = ast.getTableFunction()->as<ASTFunction>(); !table_function || table_function->name != "mysql")
|
||||
{
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "EXPLAIN TABLE OVERRIDE is not supported for the {}() table function", table_function->name);
|
||||
}
|
||||
auto storage = getContext()->getQueryContext()->executeTableFunction(ast.getTableFunction());
|
||||
auto metadata_snapshot = storage->getInMemoryMetadata();
|
||||
TableOverrideAnalyzer::Result override_info;
|
||||
TableOverrideAnalyzer override_analyzer(ast.getTableOverride());
|
||||
override_analyzer.analyze(metadata_snapshot, override_info);
|
||||
override_info.appendTo(buf);
|
||||
break;
|
||||
}
|
||||
else
|
||||
throw Exception("Only SELECT and INSERT is supported for EXPLAIN PIPELINE query", ErrorCodes::INCORRECT_QUERY);
|
||||
}
|
||||
else if (ast.getKind() == ASTExplainQuery::QueryEstimates)
|
||||
{
|
||||
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
|
||||
throw Exception("Only SELECT is supported for EXPLAIN ESTIMATE query", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
|
||||
QueryPlan plan;
|
||||
|
||||
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions());
|
||||
interpreter.buildQueryPlan(plan);
|
||||
// collect the selected marks, rows, parts during build query pipeline.
|
||||
plan.buildQueryPipeline(
|
||||
QueryPlanOptimizationSettings::fromContext(getContext()),
|
||||
BuildQueryPipelineSettings::fromContext(getContext()));
|
||||
|
||||
if (settings.optimize)
|
||||
plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext()));
|
||||
plan.explainEstimate(res_columns);
|
||||
}
|
||||
if (ast.getKind() != ASTExplainQuery::QueryEstimates)
|
||||
if (insert_buf)
|
||||
{
|
||||
if (single_line)
|
||||
res_columns[0]->insertData(buf.str().data(), buf.str().size());
|
||||
|
174
src/Interpreters/TableOverrideUtils.cpp
Normal file
174
src/Interpreters/TableOverrideUtils.cpp
Normal file
@ -0,0 +1,174 @@
|
||||
#include <Interpreters/TableOverrideUtils.h>
|
||||
|
||||
#include <Common/quoteString.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/RequiredSourceColumnsVisitor.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTTableOverrides.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Storages/ColumnDefault.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_TABLE_OVERRIDE;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class MaybeNullableColumnsMatcher
|
||||
{
|
||||
public:
|
||||
using Visitor = ConstInDepthNodeVisitor<MaybeNullableColumnsMatcher, false>;
|
||||
using Data = RequiredSourceColumnsData;
|
||||
|
||||
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child)
|
||||
{
|
||||
if (const auto * f = node->as<ASTFunction>(); f && f->name == "assumeNotNull")
|
||||
return false;
|
||||
return RequiredSourceColumnsMatcher::needChildVisit(node, child);
|
||||
}
|
||||
|
||||
static void visit(const ASTPtr & ast, Data & data)
|
||||
{
|
||||
RequiredSourceColumnsMatcher::visit(ast, data);
|
||||
}
|
||||
};
|
||||
|
||||
using MaybeNullableColumnsVisitor = MaybeNullableColumnsMatcher::Visitor;
|
||||
|
||||
}
|
||||
|
||||
static void checkRequiredColumns(const IAST * ast, const NameToTypeMap & existing_types, NamesAndTypes & used_columns, const String & what, bool allow_nulls = false)
|
||||
{
|
||||
if (!ast)
|
||||
return;
|
||||
RequiredSourceColumnsData columns_data;
|
||||
RequiredSourceColumnsVisitor(columns_data).visit(ast->clone());
|
||||
auto required_columns = columns_data.requiredColumns();
|
||||
for (const auto & column : required_columns)
|
||||
{
|
||||
auto type = existing_types.find(column);
|
||||
if (type == existing_types.end())
|
||||
throw Exception(ErrorCodes::INVALID_TABLE_OVERRIDE, "{} override refers to unknown column {}", what, backQuote(column));
|
||||
}
|
||||
if (!allow_nulls)
|
||||
{
|
||||
RequiredSourceColumnsData nullable_data;
|
||||
MaybeNullableColumnsVisitor(nullable_data).visit(ast->clone());
|
||||
for (const auto & column : nullable_data.requiredColumns())
|
||||
{
|
||||
if (existing_types.find(column)->second->isNullable())
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_TABLE_OVERRIDE,
|
||||
"{} override refers to nullable column {} (use assumeNotNull() if the column does not in fact contain NULL values)",
|
||||
what,
|
||||
backQuote(column));
|
||||
}
|
||||
}
|
||||
for (const auto & col : required_columns)
|
||||
{
|
||||
used_columns.push_back({col, existing_types.find(col)->second});
|
||||
}
|
||||
}
|
||||
|
||||
void TableOverrideAnalyzer::analyze(const StorageInMemoryMetadata & metadata, Result & result) const
|
||||
{
|
||||
for (const auto & column : metadata.columns)
|
||||
result.existing_types[column.name] = column.type;
|
||||
checkRequiredColumns(override->storage->order_by, result.existing_types, result.order_by_columns, "ORDER BY");
|
||||
checkRequiredColumns(override->storage->primary_key, result.existing_types, result.primary_key_columns, "PRIMARY KEY");
|
||||
checkRequiredColumns(override->storage->partition_by, result.existing_types, result.partition_by_columns, "PARTITION BY");
|
||||
checkRequiredColumns(override->storage->sample_by, result.existing_types, result.sample_by_columns, "SAMPLE BY");
|
||||
checkRequiredColumns(override->storage->ttl_table, result.existing_types, result.ttl_columns, "TTL");
|
||||
if (override->columns && override->columns->columns)
|
||||
{
|
||||
for (const auto & column_ast : override->columns->columns->children)
|
||||
{
|
||||
auto * override_column = column_ast->as<ASTColumnDeclaration>();
|
||||
auto override_type = DataTypeFactory::instance().get(override_column->type);
|
||||
auto found = metadata.columns.tryGetColumnOrSubcolumn(ColumnsDescription::GetFlags::All, override_column->name);
|
||||
std::optional<ColumnDefaultKind> override_default_kind;
|
||||
if (!override_column->default_specifier.empty())
|
||||
override_default_kind = columnDefaultKindFromString(override_column->default_specifier);
|
||||
if (found)
|
||||
{
|
||||
std::optional<ColumnDefaultKind> existing_default_kind;
|
||||
if (auto col_default = metadata.columns.getDefault(found->name))
|
||||
existing_default_kind = col_default->kind;
|
||||
if (existing_default_kind != override_default_kind)
|
||||
throw Exception(ErrorCodes::INVALID_TABLE_OVERRIDE, "column {}: modifying default specifier is not allowed", backQuote(override_column->name));
|
||||
result.modified_columns.push_back({found->name, override_type});
|
||||
/// TODO: validate that the original type can be converted to the overridden type
|
||||
}
|
||||
else
|
||||
{
|
||||
if (override_default_kind && *override_default_kind == ColumnDefaultKind::Alias)
|
||||
result.added_columns.push_back({override_column->name, override_type});
|
||||
else
|
||||
throw Exception(ErrorCodes::INVALID_TABLE_OVERRIDE, "column {}: can only add ALIAS columns", backQuote(override_column->name));
|
||||
}
|
||||
/// TODO: validate default and materialized expressions (use checkRequiredColumns, allowing nulls)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void TableOverrideAnalyzer::Result::appendTo(WriteBuffer & ostr)
|
||||
{
|
||||
const auto & format_names = [&](const NamesAndTypes & names) -> String
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
bool first = true;
|
||||
for (const auto & name : names)
|
||||
{
|
||||
if (!first)
|
||||
buf << ", ";
|
||||
first = false;
|
||||
buf << backQuote(name.name) << " ";
|
||||
auto old_type = existing_types.find(name.name);
|
||||
if (old_type != existing_types.end() && old_type->second != name.type)
|
||||
buf << old_type->second->getName() << " -> ";
|
||||
buf << name.type->getName();
|
||||
}
|
||||
return buf.str();
|
||||
};
|
||||
if (!modified_columns.empty())
|
||||
{
|
||||
ostr << "Modified columns: " << format_names(modified_columns) << "\n";
|
||||
}
|
||||
if (!added_columns.empty())
|
||||
{
|
||||
ostr << "Added columns: " << format_names(added_columns) << "\n";
|
||||
}
|
||||
if (!order_by_columns.empty())
|
||||
{
|
||||
ostr << "ORDER BY uses columns: " << format_names(order_by_columns) << "\n";
|
||||
}
|
||||
if (!primary_key_columns.empty())
|
||||
{
|
||||
ostr << "PRIMARY KEY uses columns: " << format_names(primary_key_columns) << "\n";
|
||||
}
|
||||
if (!partition_by_columns.empty())
|
||||
{
|
||||
ostr << "PARTITION BY uses columns: " << format_names(partition_by_columns) << "\n";
|
||||
}
|
||||
if (!sample_by_columns.empty())
|
||||
{
|
||||
ostr << "SAMPLE BY uses columns: " << format_names(sample_by_columns) << "\n";
|
||||
}
|
||||
if (!ttl_columns.empty())
|
||||
{
|
||||
ostr << "TTL uses columns: " << format_names(ttl_columns) << "\n";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
38
src/Interpreters/TableOverrideUtils.h
Normal file
38
src/Interpreters/TableOverrideUtils.h
Normal file
@ -0,0 +1,38 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Parsers/ASTTableOverrides.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct StorageInMemoryMetadata;
|
||||
|
||||
using NameToTypeMap = std::map<String, DataTypePtr>;
|
||||
|
||||
struct TableOverrideAnalyzer
|
||||
{
|
||||
struct Result
|
||||
{
|
||||
NameToTypeMap existing_types;
|
||||
NamesAndTypes order_by_columns;
|
||||
NamesAndTypes primary_key_columns;
|
||||
NamesAndTypes partition_by_columns;
|
||||
NamesAndTypes sample_by_columns;
|
||||
NamesAndTypes ttl_columns;
|
||||
NamesAndTypes added_columns;
|
||||
NamesAndTypes modified_columns;
|
||||
|
||||
void appendTo(WriteBuffer &);
|
||||
};
|
||||
|
||||
ASTTableOverride * override;
|
||||
|
||||
explicit TableOverrideAnalyzer(ASTPtr ast) : override(assert_cast<ASTTableOverride *>(ast.get())) { }
|
||||
|
||||
void analyze(const StorageInMemoryMetadata & metadata, Result & result) const;
|
||||
};
|
||||
|
||||
}
|
@ -18,6 +18,7 @@ public:
|
||||
QueryPlan, /// 'EXPLAIN SELECT ...'
|
||||
QueryPipeline, /// 'EXPLAIN PIPELINE ...'
|
||||
QueryEstimates, /// 'EXPLAIN ESTIMATE ...'
|
||||
TableOverride, /// 'EXPLAIN TABLE OVERRIDE ...'
|
||||
};
|
||||
|
||||
explicit ASTExplainQuery(ExplainKind kind_) : kind(kind_) {}
|
||||
@ -45,8 +46,22 @@ public:
|
||||
ast_settings = std::move(settings_);
|
||||
}
|
||||
|
||||
void setTableFunction(ASTPtr table_function_)
|
||||
{
|
||||
children.emplace_back(table_function_);
|
||||
table_function = std::move(table_function_);
|
||||
}
|
||||
|
||||
void setTableOverride(ASTPtr table_override_)
|
||||
{
|
||||
children.emplace_back(table_override_);
|
||||
table_override = std::move(table_override_);
|
||||
}
|
||||
|
||||
const ASTPtr & getExplainedQuery() const { return query; }
|
||||
const ASTPtr & getSettings() const { return ast_settings; }
|
||||
const ASTPtr & getTableFunction() const { return table_function; }
|
||||
const ASTPtr & getTableOverride() const { return table_override; }
|
||||
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
|
||||
@ -59,8 +74,21 @@ protected:
|
||||
ast_settings->formatImpl(settings, state, frame);
|
||||
}
|
||||
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
query->formatImpl(settings, state, frame);
|
||||
if (query)
|
||||
{
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
query->formatImpl(settings, state, frame);
|
||||
}
|
||||
if (table_function)
|
||||
{
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
table_function->formatImpl(settings, state, frame);
|
||||
}
|
||||
if (table_override)
|
||||
{
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
table_override->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
@ -69,6 +97,10 @@ private:
|
||||
ASTPtr query;
|
||||
ASTPtr ast_settings;
|
||||
|
||||
/// Used by EXPLAIN TABLE OVERRIDE
|
||||
ASTPtr table_function;
|
||||
ASTPtr table_override;
|
||||
|
||||
static String toString(ExplainKind kind)
|
||||
{
|
||||
switch (kind)
|
||||
@ -78,6 +110,7 @@ private:
|
||||
case QueryPlan: return "EXPLAIN";
|
||||
case QueryPipeline: return "EXPLAIN PIPELINE";
|
||||
case QueryEstimates: return "EXPLAIN ESTIMATE";
|
||||
case TableOverride: return "EXPLAIN TABLE OVERRIDE";
|
||||
}
|
||||
|
||||
__builtin_unreachable();
|
||||
|
@ -31,13 +31,19 @@ void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState
|
||||
String hl_keyword = settings.hilite ? hilite_keyword : "";
|
||||
String hl_none = settings.hilite ? hilite_none : "";
|
||||
|
||||
settings.ostr << hl_keyword << "TABLE OVERRIDE " << hl_none;
|
||||
ASTIdentifier(table_name).formatImpl(settings, state, frame);
|
||||
if (is_standalone)
|
||||
{
|
||||
settings.ostr << hl_keyword << "TABLE OVERRIDE " << hl_none;
|
||||
ASTIdentifier(table_name).formatImpl(settings, state, frame);
|
||||
}
|
||||
if (!columns && (!storage || storage->children.empty()))
|
||||
return;
|
||||
auto override_frame = frame;
|
||||
++override_frame.indent;
|
||||
settings.ostr << nl_or_ws << '(' << nl_or_nothing;
|
||||
if (is_standalone)
|
||||
{
|
||||
++override_frame.indent;
|
||||
settings.ostr << nl_or_ws << '(' << nl_or_nothing;
|
||||
}
|
||||
String indent_str = settings.one_line ? "" : String(4 * override_frame.indent, ' ');
|
||||
size_t override_elems = 0;
|
||||
if (columns)
|
||||
@ -68,7 +74,8 @@ void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState
|
||||
format_storage_elem(storage->ttl_table, "TTL");
|
||||
}
|
||||
|
||||
settings.ostr << nl_or_nothing << ')';
|
||||
if (is_standalone)
|
||||
settings.ostr << nl_or_nothing << ')';
|
||||
}
|
||||
|
||||
ASTPtr ASTTableOverrideList::clone() const
|
||||
|
@ -15,7 +15,7 @@ class ASTStorage;
|
||||
|
||||
/// Storage and column overrides for a single table, for example:
|
||||
///
|
||||
/// TABLE OVERRIDE `foo` PARTITION BY toYYYYMM(`createtime`)
|
||||
/// TABLE OVERRIDE `foo` (PARTITION BY toYYYYMM(`createtime`))
|
||||
///
|
||||
class ASTTableOverride : public IAST
|
||||
{
|
||||
@ -23,6 +23,7 @@ public:
|
||||
String table_name;
|
||||
ASTColumns * columns = nullptr;
|
||||
ASTStorage * storage = nullptr;
|
||||
bool is_standalone = true;
|
||||
String getID(char) const override { return "TableOverride " + table_name; }
|
||||
ASTPtr clone() const override;
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
|
@ -969,14 +969,15 @@ bool ParserTableOverrideDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte
|
||||
ASTPtr sample_by;
|
||||
ASTPtr ttl_table;
|
||||
|
||||
if (!s_table_override.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!table_name_p.parse(pos, table_name, expected))
|
||||
return false;
|
||||
|
||||
if (!lparen_p.ignore(pos, expected))
|
||||
return false;
|
||||
if (is_standalone)
|
||||
{
|
||||
if (!s_table_override.ignore(pos, expected))
|
||||
return false;
|
||||
if (!table_name_p.parse(pos, table_name, expected))
|
||||
return false;
|
||||
if (!lparen_p.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
while (true)
|
||||
{
|
||||
@ -1034,7 +1035,7 @@ bool ParserTableOverrideDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte
|
||||
break;
|
||||
}
|
||||
|
||||
if (!rparen_p.ignore(pos, expected))
|
||||
if (is_standalone && !rparen_p.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
auto storage = std::make_shared<ASTStorage>();
|
||||
@ -1045,7 +1046,9 @@ bool ParserTableOverrideDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte
|
||||
storage->set(storage->ttl_table, ttl_table);
|
||||
|
||||
auto res = std::make_shared<ASTTableOverride>();
|
||||
res->table_name = table_name->as<ASTIdentifier>()->name();
|
||||
if (table_name)
|
||||
res->table_name = table_name->as<ASTIdentifier>()->name();
|
||||
res->is_standalone = is_standalone;
|
||||
res->set(res->storage, storage);
|
||||
if (columns)
|
||||
res->set(res->columns, columns);
|
||||
|
@ -389,6 +389,10 @@ protected:
|
||||
|
||||
class ParserTableOverrideDeclaration : public IParserBase
|
||||
{
|
||||
public:
|
||||
const bool is_standalone;
|
||||
ParserTableOverrideDeclaration(bool is_standalone_ = true) : is_standalone(is_standalone_) { }
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "table override declaration"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
@ -21,6 +21,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_pipeline("PIPELINE");
|
||||
ParserKeyword s_plan("PLAN");
|
||||
ParserKeyword s_estimates("ESTIMATE");
|
||||
ParserKeyword s_table_override("TABLE OVERRIDE");
|
||||
|
||||
if (s_explain.ignore(pos, expected))
|
||||
{
|
||||
@ -36,6 +37,8 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
kind = ASTExplainQuery::ExplainKind::QueryPlan; //-V1048
|
||||
else if (s_estimates.ignore(pos, expected))
|
||||
kind = ASTExplainQuery::ExplainKind::QueryEstimates; //-V1048
|
||||
else if (s_table_override.ignore(pos, expected))
|
||||
kind = ASTExplainQuery::ExplainKind::TableOverride;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
@ -65,6 +68,17 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else if (kind == ASTExplainQuery::ExplainKind::TableOverride)
|
||||
{
|
||||
ASTPtr table_function;
|
||||
if (!ParserFunction(true, true).parse(pos, table_function, expected))
|
||||
return false;
|
||||
ASTPtr table_override;
|
||||
if (!ParserTableOverrideDeclaration(false).parse(pos, table_override, expected))
|
||||
return false;
|
||||
explain_query->setTableFunction(table_function);
|
||||
explain_query->setTableOverride(table_override);
|
||||
}
|
||||
else if (select_p.parse(pos, query, expected) ||
|
||||
create_p.parse(pos, query, expected) ||
|
||||
insert_p.parse(pos, query, expected))
|
||||
|
Loading…
Reference in New Issue
Block a user