This commit is contained in:
vdimir 2022-12-07 16:27:01 +00:00
parent 127373ab06
commit 4b50e5c15e
No known key found for this signature in database
GPG Key ID: 6EE4CE2BEDC51862
5 changed files with 21 additions and 11 deletions

View File

@ -225,7 +225,8 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
, right_sample_block(right_sample_block_)
, log(&Poco::Logger::get("HashJoin"))
{
LOG_DEBUG(log, "HashJoin. Datatype: {}, kind: {}, strictness: {}", data->type, kind, strictness);
LOG_DEBUG(log, "Datatype: {}, kind: {}, strictness: {}", data->type, kind, strictness);
LOG_DEBUG(log, "Keys: {}", TableJoin::formatClauses(table_join->getClauses(), true));
if (isCrossOrComma(kind))
{
@ -1492,7 +1493,7 @@ void HashJoin::joinBlockImpl(
{
const auto & right_key = required_right_keys.getByPosition(i);
auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name);
if (!block.findByName(right_col_name /*right_key.name*/))
if (!block.findByName(right_col_name))
{
const auto & left_name = required_right_keys_sources[i];
@ -1512,7 +1513,7 @@ void HashJoin::joinBlockImpl(
block.insert(std::move(right_col));
if constexpr (jf.need_replication)
right_keys_to_replicate.push_back(block.getPositionByName(right_key.name));
right_keys_to_replicate.push_back(block.getPositionByName(right_col_name));
}
}
}

View File

@ -653,9 +653,6 @@ void TableJoin::setStorageJoin(std::shared_ptr<const IKeyValueEntity> storage)
void TableJoin::setStorageJoin(std::shared_ptr<StorageJoin> storage)
{
if (clauses.empty())
throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN,
"StorageJoin keys should match JOIN keys, expected JOIN ON [{}]", fmt::join(storage->getKeyNames(), ", "));
right_storage_join = storage;
}

View File

@ -48,6 +48,7 @@
#include <IO/WriteHelpers.h>
#include <Storages/IStorage.h>
#include <Storages/StorageJoin.h>
#include <Common/checkStackSize.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
@ -60,6 +61,7 @@ namespace ErrorCodes
extern const int EMPTY_LIST_OF_COLUMNS_QUERIED;
extern const int EMPTY_NESTED_TABLE;
extern const int EXPECTED_ALL_OR_ANY;
extern const int INCOMPATIBLE_TYPE_OF_JOIN;
extern const int INVALID_JOIN_ON_EXPRESSION;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
@ -757,6 +759,10 @@ void collectJoinedColumns(TableJoin & analyzed_join, ASTTableJoin & table_join,
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"Cannot get JOIN keys from JOIN ON section: {}", queryToString(table_join.on_expression));
if (const auto storage_join = analyzed_join.getStorageJoin())
throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN,
"StorageJoin keys should match JOIN keys, expected JOIN ON [{}]", fmt::join(storage_join->getKeyNames(), ", "));
bool join_on_const_ok = tryJoinOnConst(analyzed_join, table_join.on_expression, context);
if (!join_on_const_ok)
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,

View File

@ -174,8 +174,12 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr<TableJoin> analyzed_join,
"Table {} needs the same join_use_nulls setting as present in LEFT or FULL JOIN",
getStorageID().getNameForLogs());
const auto & key_names_right = analyzed_join->getOnlyClause().key_names_right;
const auto & key_names_left = analyzed_join->getOnlyClause().key_names_left;
const auto & join_on = analyzed_join->getOnlyClause();
if (join_on.on_filter_condition_left || join_on.on_filter_condition_right)
throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, "ON section of JOIN with filter conditions is not implemented");
const auto & key_names_right = join_on.key_names_right;
const auto & key_names_left = join_on.key_names_left;
if (key_names.size() != key_names_right.size() || key_names.size() != key_names_left.size())
throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN,
"Number of keys in JOIN ON section ({}) doesn't match number of keys in Join engine ({})",

View File

@ -22,9 +22,11 @@ SELECT * FROM (SELECT key3 AS c, key1 AS a, key2 AS b FROM t1) AS t1 ALL INNER J
SELECT * FROM (SELECT key3 AS c, key1 AS a, key2 AS b FROM t1) AS t1 ALL INNER JOIN tj ON t1.a = tj.key1 AND t1.b = tj.key2 AND t1.c = tj.key3 ORDER BY t1.a;
SELECT * FROM (SELECT key3 AS c, key1 AS a, key2 AS b FROM t1) AS t1 ALL INNER JOIN tj ON t1.c = tj.key3 AND t1.a = tj.key1 AND t1.b = tj.key2 ORDER BY t1.a;
SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND 1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }
SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND 0; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }
SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND NULL; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }
SELECT * FROM t1 ALL INNER JOIN tj ON 1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }
SELECT * FROM t1 ALL INNER JOIN tj ON 0; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }
SELECT * FROM t1 ALL INNER JOIN tj ON NULL; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }
SELECT * FROM t1 ALL INNER JOIN tj ON 1 == 1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }
SELECT * FROM t1 ALL INNER JOIN tj ON 1 != 1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }
SELECT * FROM t1 ALL INNER JOIN tj USING (key2, key3); -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }
SELECT * FROM t1 ALL INNER JOIN tj USING (key1, key2, attr); -- { serverError INCOMPATIBLE_TYPE_OF_JOIN }