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:
Stig Bakken 2021-12-16 09:26:37 +01:00
parent c5fe3a8f8a
commit 6fbb9f5e87
10 changed files with 413 additions and 112 deletions

View File

@ -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) \

View File

@ -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,15 +252,19 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
WriteBufferFromOwnString buf;
bool single_line = false;
bool insert_buf = true;
if (ast.getKind() == ASTExplainQuery::ParsedAST)
switch (ast.getKind())
{
case ASTExplainQuery::ParsedAST:
{
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING);
dumpAST(*ast.getExplainedQuery(), buf);
break;
}
else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax)
case ASTExplainQuery::AnalyzedSyntax:
{
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING);
@ -267,8 +273,9 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
ExplainAnalyzedSyntaxVisitor(data).visit(query);
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false));
break;
}
else if (ast.getKind() == ASTExplainQuery::QueryPlan)
case ASTExplainQuery::QueryPlan:
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY);
@ -302,8 +309,9 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
}
else
plan.explainPlan(buf, settings.query_plan_options);
break;
}
else if (ast.getKind() == ASTExplainQuery::QueryPipeline)
case ASTExplainQuery::QueryPipeline:
{
if (dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
{
@ -340,8 +348,9 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
}
else
throw Exception("Only SELECT and INSERT is supported for EXPLAIN PIPELINE query", ErrorCodes::INCORRECT_QUERY);
break;
}
else if (ast.getKind() == ASTExplainQuery::QueryEstimates)
case ASTExplainQuery::QueryEstimates:
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
throw Exception("Only SELECT is supported for EXPLAIN ESTIMATE query", ErrorCodes::INCORRECT_QUERY);
@ -359,8 +368,25 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
if (settings.optimize)
plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext()));
plan.explainEstimate(res_columns);
insert_buf = false;
break;
}
if (ast.getKind() != ASTExplainQuery::QueryEstimates)
case ASTExplainQuery::TableOverride:
{
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;
}
}
if (insert_buf)
{
if (single_line)
res_columns[0]->insertData(buf.str().data(), buf.str().size());

View 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";
}
}
}

View 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;
};
}

View File

@ -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,9 +74,22 @@ protected:
ast_settings->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:
ExplainKind kind;
@ -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();

View File

@ -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 : "";
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;
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,6 +74,7 @@ void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState
format_storage_elem(storage->ttl_table, "TTL");
}
if (is_standalone)
settings.ostr << nl_or_nothing << ')';
}

View File

@ -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;

View File

@ -969,14 +969,15 @@ bool ParserTableOverrideDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte
ASTPtr sample_by;
ASTPtr ttl_table;
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>();
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);

View File

@ -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;

View File

@ -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))