mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Add current database to table names in JOIN section for distributed queries
This should fix JOIN w/o explicit database. v2: rewrite only JOIN section, since there is old behavior that relies on default_database for IN section, see [1]: - 01487_distributed_in_not_default_db - 01152_cross_replication [1]: https://s3.amazonaws.com/clickhouse-test-reports/33611/d0ea3c76fa51131171b1825939680867eb1c04da/fast_test__actions_.html Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
parent
4dcb332be9
commit
c341b3b237
@ -46,6 +46,7 @@
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/JoinedTables.h>
|
||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/createBlockSelector.h>
|
||||
@ -126,7 +127,12 @@ namespace
|
||||
|
||||
/// select query has database, table and table function names as AST pointers
|
||||
/// Creates a copy of query, changes database, table and table function names.
|
||||
ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, const std::string & table, ASTPtr table_function_ptr = nullptr)
|
||||
ASTPtr rewriteSelectQuery(
|
||||
ContextPtr context,
|
||||
const ASTPtr & query,
|
||||
const std::string & remote_database,
|
||||
const std::string & remote_table,
|
||||
ASTPtr table_function_ptr = nullptr)
|
||||
{
|
||||
auto modified_query_ast = query->clone();
|
||||
|
||||
@ -140,7 +146,7 @@ ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, co
|
||||
if (table_function_ptr)
|
||||
select_query.addTableFunction(table_function_ptr);
|
||||
else
|
||||
select_query.replaceDatabaseAndTable(database, table);
|
||||
select_query.replaceDatabaseAndTable(remote_database, remote_table);
|
||||
|
||||
/// Restore long column names (cause our short names are ambiguous).
|
||||
/// TODO: aliased table functions & CREATE TABLE AS table function cases
|
||||
@ -148,11 +154,20 @@ ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, co
|
||||
{
|
||||
RestoreQualifiedNamesVisitor::Data data;
|
||||
data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query->as<ASTSelectQuery &>(), 0));
|
||||
data.remote_table.database = database;
|
||||
data.remote_table.table = table;
|
||||
data.remote_table.database = remote_database;
|
||||
data.remote_table.table = remote_table;
|
||||
RestoreQualifiedNamesVisitor(data).visit(modified_query_ast);
|
||||
}
|
||||
|
||||
/// To make local JOIN works, default database should be added to table names.
|
||||
/// But only for JOIN section, since the following should work using default_database:
|
||||
/// - SELECT * FROM d WHERE value IN (SELECT l.value FROM l) ORDER BY value
|
||||
/// (see 01487_distributed_in_not_default_db)
|
||||
AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase(),
|
||||
/* only_replace_current_database_function_= */false,
|
||||
/* only_replace_in_join_= */true);
|
||||
visitor.visit(modified_query_ast);
|
||||
|
||||
return modified_query_ast;
|
||||
}
|
||||
|
||||
@ -600,7 +615,8 @@ void StorageDistributed::read(
|
||||
throw Exception(ErrorCodes::ILLEGAL_FINAL, "Final modifier is not allowed together with parallel reading from replicas feature");
|
||||
|
||||
const auto & modified_query_ast = rewriteSelectQuery(
|
||||
query_info.query, remote_database, remote_table, remote_table_function_ptr);
|
||||
local_context, query_info.query,
|
||||
remote_database, remote_table, remote_table_function_ptr);
|
||||
|
||||
Block header =
|
||||
InterpreterSelectQuery(query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
|
||||
|
@ -0,0 +1,14 @@
|
||||
-- { echoOn }
|
||||
select * from dist_02175 l join local_02175 r using dummy;
|
||||
0
|
||||
0
|
||||
select * from dist_02175 l global join local_02175 r using dummy;
|
||||
0
|
||||
0
|
||||
-- explicit database for distributed table
|
||||
select * from remote('127.1', currentDatabase(), dist_02175) l join local_02175 r using dummy;
|
||||
0
|
||||
0
|
||||
select * from remote('127.1', currentDatabase(), dist_02175) l global join local_02175 r using dummy;
|
||||
0
|
||||
0
|
@ -0,0 +1,19 @@
|
||||
-- Tags: shard
|
||||
|
||||
drop table if exists local_02175;
|
||||
drop table if exists dist_02175;
|
||||
|
||||
create table local_02175 engine=Memory() as select * from system.one;
|
||||
create table dist_02175 as local_02175 engine=Distributed(test_cluster_two_shards, currentDatabase(), local_02175);
|
||||
|
||||
-- { echoOn }
|
||||
select * from dist_02175 l join local_02175 r using dummy;
|
||||
select * from dist_02175 l global join local_02175 r using dummy;
|
||||
|
||||
-- explicit database for distributed table
|
||||
select * from remote('127.1', currentDatabase(), dist_02175) l join local_02175 r using dummy;
|
||||
select * from remote('127.1', currentDatabase(), dist_02175) l global join local_02175 r using dummy;
|
||||
|
||||
-- { echoOff }
|
||||
drop table local_02175;
|
||||
drop table dist_02175;
|
Loading…
Reference in New Issue
Block a user