avoiding only for paste join now

This commit is contained in:
yariks5s 2024-01-10 18:18:50 +00:00
parent f656a6d799
commit 92c6eba922
4 changed files with 48 additions and 14 deletions

View File

@ -1207,7 +1207,7 @@ private:
static void validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope);
static void checkDuplicateTableNamesOrAlias(QueryTreeNodePtr & join_node, QueryTreeNodePtr & left_table_expr, QueryTreeNodePtr & right_table_expr);
static void checkDuplicateTableNamesOrAlias(QueryTreeNodePtr & join_node, QueryTreeNodePtr & left_table_expr, QueryTreeNodePtr & right_table_expr, IdentifierResolveScope & scope);
static std::pair<bool, UInt64> recursivelyCollectMaxOrdinaryExpressions(QueryTreeNodePtr & node, QueryTreeNodes & into);
@ -6779,8 +6779,14 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif
}
}
void QueryAnalyzer::checkDuplicateTableNamesOrAlias(QueryTreeNodePtr & join_node, QueryTreeNodePtr & left_table_expr, QueryTreeNodePtr & right_table_expr)
void QueryAnalyzer::checkDuplicateTableNamesOrAlias(QueryTreeNodePtr & join_node, QueryTreeNodePtr & left_table_expr, QueryTreeNodePtr & right_table_expr, IdentifierResolveScope & scope)
{
if (scope.context->getSettingsRef().joined_subquery_requires_alias)
return;
if (join_node->as<JoinNode &>().getKind() != JoinKind::Paste)
return;
Names column_names;
auto * left_node = left_table_expr->as<QueryNode>();
auto * right_node = right_table_expr->as<QueryNode>();
@ -6813,7 +6819,7 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS
validateJoinTableExpressionWithoutAlias(join_node, join_node_typed.getRightTableExpression(), scope);
if (!join_node_typed.getLeftTableExpression()->hasAlias() && !join_node_typed.getRightTableExpression()->hasAlias())
checkDuplicateTableNamesOrAlias(join_node, join_node_typed.getLeftTableExpression(), join_node_typed.getRightTableExpression());
checkDuplicateTableNamesOrAlias(join_node, join_node_typed.getLeftTableExpression(), join_node_typed.getRightTableExpression(), scope);
if (join_node_typed.isOnJoinExpression())
{

View File

@ -452,7 +452,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
storage_snapshot = storage->getStorageSnapshotForQuery(metadata_snapshot, query_ptr, context);
}
if (has_input || !joined_tables.resolveTables())
if (has_input || !joined_tables.resolveTables(analysis_result))
joined_tables.makeFakeTable(storage, metadata_snapshot, source_header);
if (context->getCurrentTransaction() && context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
@ -513,7 +513,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
rewriteMultipleJoins(query_ptr, joined_tables.tablesWithColumns(), context->getCurrentDatabase(), context->getSettingsRef());
joined_tables.reset(getSelectQuery());
joined_tables.resolveTables();
joined_tables.resolveTables(analysis_result);
if (auto view_source = context->getViewSource())
{
// If we are using a virtual block view to replace a table and that table is used

View File

@ -25,6 +25,7 @@
#include <Storages/StorageJoin.h>
#include <Storages/StorageValues.h>
#include "Common/Exception.h"
#include "Interpreters/ExpressionAnalyzer.h"
namespace DB
{
@ -238,7 +239,24 @@ StoragePtr JoinedTables::getLeftTableStorage()
return DatabaseCatalog::instance().getTable(table_id, context);
}
bool JoinedTables::resolveTables()
void JoinedTables::checkDuplicateNames()
{
Names column_names = {};
for (const auto & t : tables_with_columns)
for (auto & name : t.columns.getNames())
column_names.push_back(name);
if (column_names.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Names of joining columns cannot be empty");
std::sort(column_names.begin(), column_names.end());
for (size_t i = 0; i < column_names.size() - 1; i++) // Check if there is not any duplicates because it will lead to broken result
if (column_names[i] == column_names[i+1])
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Name of columns and aliases should be unique for this query (you can add/change aliases so they will not be duplicated)"
"While processing '{}'", table_expressions[i]->formatForErrorMessage());
}
bool JoinedTables::resolveTables(ExpressionAnalysisResult result)
{
const auto & settings = context->getSettingsRef();
bool include_alias_cols = include_all_columns || settings.asterisk_include_alias_columns;
@ -249,15 +267,23 @@ bool JoinedTables::resolveTables()
if (settings.joined_subquery_requires_alias && tables_with_columns.size() > 1)
{
for (size_t i = 0; i < tables_with_columns.size(); ++i)
if (result.hasJoin())
{
const auto & t = tables_with_columns[i];
if (t.table.table.empty() && t.table.alias.empty())
if (result.join->getTableJoin().kind() == JoinKind::Paste)
checkDuplicateNames();
else
{
throw Exception(ErrorCodes::ALIAS_REQUIRED,
"No alias for subquery or table function "
"in JOIN (set joined_subquery_requires_alias=0 to disable restriction). "
"While processing '{}'", table_expressions[i]->formatForErrorMessage());
for (size_t i = 0; i < tables_with_columns.size(); ++i)
{
const auto & t = tables_with_columns[i];
if (t.table.table.empty() && t.table.alias.empty())
{
throw Exception(ErrorCodes::ALIAS_REQUIRED,
"No alias for subquery or table function "
"in JOIN (set joined_subquery_requires_alias=0 to disable restriction). "
"While processing '{}'", table_expressions[i]->formatForErrorMessage());
}
}
}
}
}

View File

@ -6,6 +6,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/StorageID.h>
#include <Storages/IStorage_fwd.h>
#include "Interpreters/ExpressionAnalyzer.h"
namespace DB
{
@ -27,7 +28,8 @@ public:
void reset(const ASTSelectQuery & select_query);
StoragePtr getLeftTableStorage();
bool resolveTables();
bool resolveTables(ExpressionAnalysisResult result);
void checkDuplicateNames();
/// Make fake tables_with_columns[0] in case we have predefined input in InterpreterSelectQuery
void makeFakeTable(StoragePtr storage, const StorageMetadataPtr & metadata_snapshot, const Block & source_header);