mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Better transformQueryForExternalDatabase for analyzer
This commit is contained in:
parent
6919db3d46
commit
426f4ec233
@ -75,7 +75,7 @@ ASTPtr ArrayJoinNode::toASTImpl(const ConvertToASTOptions & options) const
|
||||
array_join_ast->expression_list = array_join_ast->children.back();
|
||||
|
||||
ASTPtr tables_in_select_query_ast = std::make_shared<ASTTablesInSelectQuery>();
|
||||
addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[table_expression_child_index]);
|
||||
addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[table_expression_child_index], options);
|
||||
|
||||
auto array_join_query_element_ast = std::make_shared<ASTTablesInSelectQueryElement>();
|
||||
array_join_query_element_ast->children.push_back(std::move(array_join_ast));
|
||||
|
@ -103,11 +103,6 @@ public:
|
||||
*/
|
||||
QueryTreeNodePtr getColumnSource() const;
|
||||
|
||||
void dropColumnSource()
|
||||
{
|
||||
getSourceWeakPointer().reset();
|
||||
}
|
||||
|
||||
/** Get column source.
|
||||
* If column source is not valid null is returned.
|
||||
*/
|
||||
|
@ -99,17 +99,17 @@ QueryTreeNodePtr JoinNode::cloneImpl() const
|
||||
return std::make_shared<JoinNode>(getLeftTableExpression(), getRightTableExpression(), getJoinExpression(), locality, strictness, kind);
|
||||
}
|
||||
|
||||
ASTPtr JoinNode::toASTImpl(const ConvertToASTOptions & /* options */) const
|
||||
ASTPtr JoinNode::toASTImpl(const ConvertToASTOptions & options) const
|
||||
{
|
||||
ASTPtr tables_in_select_query_ast = std::make_shared<ASTTablesInSelectQuery>();
|
||||
|
||||
addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[left_table_expression_child_index]);
|
||||
addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[left_table_expression_child_index], options);
|
||||
|
||||
size_t join_table_index = tables_in_select_query_ast->children.size();
|
||||
|
||||
auto join_ast = toASTTableJoin();
|
||||
|
||||
addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index]);
|
||||
addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index], options);
|
||||
|
||||
auto & table_element = tables_in_select_query_ast->children.at(join_table_index)->as<ASTTablesInSelectQueryElement &>();
|
||||
table_element.children.push_back(std::move(join_ast));
|
||||
|
@ -293,7 +293,7 @@ ASTPtr QueryNode::toASTImpl(const ConvertToASTOptions & options) const
|
||||
select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(projection_ast));
|
||||
|
||||
ASTPtr tables_in_select_query_ast = std::make_shared<ASTTablesInSelectQuery>();
|
||||
addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, getJoinTree());
|
||||
addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, getJoinTree(), options);
|
||||
select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast));
|
||||
|
||||
if (getPrewhere())
|
||||
|
@ -75,7 +75,7 @@ void TableFunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_
|
||||
|
||||
if (!settings_changes.empty())
|
||||
{
|
||||
buffer << '\n' << std::string(indent + 6, ' ') << "SETTINGS";
|
||||
buffer << '\n' << std::string(indent + 2, ' ') << "SETTINGS";
|
||||
for (const auto & change : settings_changes)
|
||||
buffer << fmt::format(" {}={}", change.name, toString(change.value));
|
||||
}
|
||||
@ -90,6 +90,9 @@ bool TableFunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const
|
||||
if (storage && rhs_typed.storage)
|
||||
return storage_id == rhs_typed.storage_id;
|
||||
|
||||
if (settings_changes != rhs_typed.settings_changes)
|
||||
return false;
|
||||
|
||||
return table_expression_modifiers == rhs_typed.table_expression_modifiers;
|
||||
}
|
||||
|
||||
@ -108,6 +111,7 @@ void TableFunctionNode::updateTreeHashImpl(HashState & state) const
|
||||
if (table_expression_modifiers)
|
||||
table_expression_modifiers->updateTreeHash(state);
|
||||
|
||||
state.update(settings_changes.size());
|
||||
for (const auto & change : settings_changes)
|
||||
{
|
||||
state.update(change.name.size());
|
||||
|
@ -268,7 +268,7 @@ static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expre
|
||||
return result_table_expression;
|
||||
}
|
||||
|
||||
void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression)
|
||||
void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options)
|
||||
{
|
||||
auto table_expression_node_type = table_expression->getNodeType();
|
||||
|
||||
@ -297,7 +297,7 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q
|
||||
[[fallthrough]];
|
||||
case QueryTreeNodeType::JOIN:
|
||||
{
|
||||
auto table_expression_tables_in_select_query_ast = table_expression->toAST();
|
||||
auto table_expression_tables_in_select_query_ast = table_expression->toAST(convert_to_ast_options);
|
||||
tables_in_select_query_ast->children.reserve(table_expression_tables_in_select_query_ast->children.size());
|
||||
for (auto && table_element_ast : table_expression_tables_in_select_query_ast->children)
|
||||
tables_in_select_query_ast->children.push_back(std::move(table_element_ast));
|
||||
|
@ -40,7 +40,7 @@ std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr &
|
||||
/** Add table expression in tables in select query children.
|
||||
* If table expression node is not of identifier node, table node, query node, table function node, join node or array join node type throws logical error exception.
|
||||
*/
|
||||
void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression);
|
||||
void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options);
|
||||
|
||||
/// Extract table, table function, query, union from join tree
|
||||
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node);
|
||||
|
@ -9,6 +9,8 @@
|
||||
#include <Planner/Planner.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
struct TestExternalDatabaseQueryState;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -71,14 +73,11 @@ public:
|
||||
/// Set number_of_current_replica and count_participating_replicas in client_info
|
||||
void setProperClientInfo(size_t replica_number, size_t count_participating_replicas);
|
||||
|
||||
const QueryTreeNodePtr & getQueryTree() const { return query_tree; }
|
||||
|
||||
SelectQueryInfo getSelectQueryInfo()
|
||||
{
|
||||
planner.buildQueryPlanIfNeeded(); return planner.buildSelectQueryInfo();
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
/// Access select query info from unit tests (see gtest_transform_query_for_external_database)
|
||||
friend struct ::TestExternalDatabaseQueryState;
|
||||
|
||||
ASTPtr query;
|
||||
ContextMutablePtr context;
|
||||
SelectQueryOptions select_query_options;
|
||||
|
@ -23,18 +23,24 @@ using namespace DB;
|
||||
|
||||
|
||||
/// TODO: use gtest fixture
|
||||
struct State
|
||||
struct TestExternalDatabaseQueryState
|
||||
{
|
||||
State(const State&) = delete;
|
||||
TestExternalDatabaseQueryState(const TestExternalDatabaseQueryState&) = delete;
|
||||
|
||||
ContextMutablePtr context;
|
||||
|
||||
static const State & instance()
|
||||
static const TestExternalDatabaseQueryState & instance()
|
||||
{
|
||||
static State state;
|
||||
static TestExternalDatabaseQueryState state;
|
||||
return state;
|
||||
}
|
||||
|
||||
static SelectQueryInfo getSelectQueryInfo(InterpreterSelectQueryAnalyzer & interpreter)
|
||||
{
|
||||
interpreter.planner.buildQueryPlanIfNeeded();
|
||||
return interpreter.planner.buildSelectQueryInfo();
|
||||
}
|
||||
|
||||
const NamesAndTypesList & getColumns(size_t idx = 0) const
|
||||
{
|
||||
return tables[idx].columns;
|
||||
@ -86,7 +92,7 @@ private:
|
||||
}),
|
||||
};
|
||||
|
||||
explicit State()
|
||||
explicit TestExternalDatabaseQueryState()
|
||||
: context(Context::createCopy(getContext().context))
|
||||
{
|
||||
tryRegisterFunctions();
|
||||
@ -123,7 +129,7 @@ private:
|
||||
};
|
||||
|
||||
static void checkOld(
|
||||
const State & state,
|
||||
const TestExternalDatabaseQueryState & state,
|
||||
size_t table_num,
|
||||
const std::string & query,
|
||||
const std::string & expected)
|
||||
@ -165,7 +171,7 @@ static QueryTreeNodePtr findTableExpression(const QueryTreeNodePtr & node, const
|
||||
|
||||
/// `column_names` - Normally it's passed to query plan step. But in test we do it manually.
|
||||
static void checkNewAnalyzer(
|
||||
const State & state,
|
||||
const TestExternalDatabaseQueryState & state,
|
||||
const Names & column_names,
|
||||
const std::string & query,
|
||||
const std::string & expected)
|
||||
@ -175,7 +181,7 @@ static void checkNewAnalyzer(
|
||||
|
||||
SelectQueryOptions select_options;
|
||||
InterpreterSelectQueryAnalyzer interpreter(ast, state.context, select_options);
|
||||
SelectQueryInfo query_info = interpreter.getSelectQueryInfo();
|
||||
SelectQueryInfo query_info = state.getSelectQueryInfo(interpreter);
|
||||
const auto * query_node = query_info.query_tree->as<QueryNode>();
|
||||
if (!query_node)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryNode expected");
|
||||
@ -189,7 +195,7 @@ static void checkNewAnalyzer(
|
||||
}
|
||||
|
||||
static void check(
|
||||
const State & state,
|
||||
const TestExternalDatabaseQueryState & state,
|
||||
size_t table_num,
|
||||
const Names & column_names,
|
||||
const std::string & query,
|
||||
@ -208,7 +214,7 @@ static void check(
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, InWithSingleElement)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"column"},
|
||||
"SELECT column FROM test.table WHERE 1 IN (1)",
|
||||
@ -226,7 +232,7 @@ TEST(TransformQueryForExternalDatabase, InWithSingleElement)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, InWithMultipleColumns)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"column"},
|
||||
"SELECT column FROM test.table WHERE (1,1) IN ((1,1))",
|
||||
@ -239,7 +245,7 @@ TEST(TransformQueryForExternalDatabase, InWithMultipleColumns)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, InWithTable)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"column"},
|
||||
"SELECT column FROM test.table WHERE 1 IN external_table",
|
||||
@ -257,7 +263,7 @@ TEST(TransformQueryForExternalDatabase, InWithTable)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Like)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"field"},
|
||||
"SELECT field FROM test.table WHERE field LIKE '%hello%'",
|
||||
@ -269,7 +275,7 @@ TEST(TransformQueryForExternalDatabase, Like)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Substring)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"field"},
|
||||
"SELECT field FROM test.table WHERE left(field, 10) = RIGHT(field, 10) AND SUBSTRING(field FROM 1 FOR 2) = 'Hello'",
|
||||
@ -278,7 +284,7 @@ TEST(TransformQueryForExternalDatabase, Substring)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"column"},
|
||||
"SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(toString(column), 10) = RIGHT(toString(column), 10) AND column IN (1, 42) AND SUBSTRING(toString(column) FROM 1 FOR 2) = 'Hello' AND column != 4",
|
||||
@ -290,7 +296,7 @@ TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Issue7245)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"apply_id", "apply_type", "apply_status", "create_time"},
|
||||
"SELECT apply_id FROM test.table WHERE apply_type = 2 AND create_time > addDays(toDateTime('2019-01-01 01:02:03'),-7) AND apply_status IN (3,4)",
|
||||
@ -299,7 +305,7 @@ TEST(TransformQueryForExternalDatabase, Issue7245)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Aliases)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"field"},
|
||||
"SELECT field AS value, field AS display FROM table WHERE field NOT IN ('') AND display LIKE '%test%'",
|
||||
@ -308,7 +314,7 @@ TEST(TransformQueryForExternalDatabase, Aliases)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 2, {"column", "apply_id"},
|
||||
"SELECT column FROM test.table "
|
||||
@ -319,7 +325,7 @@ TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, NoStrict)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"field"},
|
||||
"SELECT field FROM table WHERE field IN (SELECT attr FROM table2)",
|
||||
@ -328,7 +334,7 @@ TEST(TransformQueryForExternalDatabase, NoStrict)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Strict)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
state.context->setSetting("external_table_strict_query", true);
|
||||
|
||||
check(state, 1, {"field"},
|
||||
@ -349,7 +355,7 @@ TEST(TransformQueryForExternalDatabase, Strict)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Null)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"field"},
|
||||
"SELECT field FROM table WHERE field IS NULL",
|
||||
@ -368,7 +374,7 @@ TEST(TransformQueryForExternalDatabase, Null)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, ToDate)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"a", "b", "foo"},
|
||||
"SELECT foo FROM table WHERE a=10 AND b=toDate('2019-10-05')",
|
||||
@ -377,7 +383,7 @@ TEST(TransformQueryForExternalDatabase, ToDate)
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, Analyzer)
|
||||
{
|
||||
const State & state = State::instance();
|
||||
const TestExternalDatabaseQueryState & state = TestExternalDatabaseQueryState::instance();
|
||||
|
||||
check(state, 1, {"field"},
|
||||
"SELECT count() FROM table WHERE field LIKE '%name_%'",
|
||||
|
@ -365,7 +365,7 @@ String transformQueryForExternalDatabase(
|
||||
|
||||
if (column_names.empty())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "No column names for query '{}' to external table '{}.{}'",
|
||||
database, table, query_info.query_tree->formatASTForErrorMessage());
|
||||
query_info.query_tree->formatASTForErrorMessage(), database, table);
|
||||
|
||||
auto clone_query = getASTForExternalDatabaseFromQueryTree(query_info.query_tree);
|
||||
|
||||
|
@ -1,18 +1,13 @@
|
||||
#include <Storages/transformQueryForExternalDatabaseAnalyzer.h>
|
||||
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
#include <Analyzer/ConstantValue.h>
|
||||
#include <Analyzer/TableNode.h>
|
||||
#include <Analyzer/JoinNode.h>
|
||||
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -60,7 +60,7 @@ QUERY id: 0
|
||||
IDENTIFIER id: 11, identifier: foo
|
||||
CONSTANT id: 12, constant_value: \'default\', constant_value_type: String
|
||||
CONSTANT id: 13, constant_value: \'\', constant_value_type: String
|
||||
SETTINGS connection_wait_timeout=123 connect_timeout=40123002 read_write_timeout=40123001 connection_pool_size=3
|
||||
SETTINGS connection_wait_timeout=123 connect_timeout=40123002 read_write_timeout=40123001 connection_pool_size=3
|
||||
|
||||
SELECT
|
||||
key AS key,
|
||||
|
Loading…
Reference in New Issue
Block a user