fixed bugs for all right join

add test cases

fixed typos

add document

fixed stype error and rename some variables

fixed test error

add test cases

try to recude hash join code size

debug

fixed

fixed coredump

fixed. condition result type is not bool

enable when allow_experimental_analyzer=0

fixed tests

fixed

update tests

update tests

roll backup when use the old analyzer

fixed typos

fixed

test

large obj check

test

test
This commit is contained in:
lgbo-ustc 2024-03-07 19:51:54 +08:00
parent f7dd4deca0
commit cd9c87811f
13 changed files with 695 additions and 140 deletions

View File

@ -164,6 +164,51 @@ Result:
│ 4 │ -4 │ 4 │
└───┴────┴─────┘
```
## Join with inequality conditions
Clickhouse currently supports inner, left, right and full join with inequality conditions, including with `OR` operator. You need to set `allow_experimental_analyzer = 1` and select `hash` or `grace_hash` join algorithm.
**Example**
Table `t1`:
```
┌─key──┬─attr─┬─a─┬─b─┬─c─┐
│ key1 │ a │ 1 │ 1 │ 2 │
│ key1 │ b │ 2 │ 3 │ 2 │
│ key1 │ c │ 3 │ 2 │ 1 │
│ key1 │ d │ 4 │ 7 │ 2 │
│ key1 │ e │ 5 │ 5 │ 5 │
│ key2 │ a2 │ 1 │ 1 │ 1 │
│ key4 │ f │ 2 │ 3 │ 4 │
└──────┴──────┴───┴───┴───┘
```
Table `t2`
```
┌─key──┬─attr─┬─a─┬─b─┬─c─┐
│ key1 │ A │ 1 │ 2 │ 1 │
│ key1 │ B │ 2 │ 1 │ 2 │
│ key1 │ C │ 3 │ 4 │ 5 │
│ key1 │ D │ 4 │ 1 │ 6 │
│ key3 │ a3 │ 1 │ 1 │ 1 │
│ key4 │ F │ 1 │ 1 │ 1 │
└──────┴──────┴───┴───┴───┘
```
```sql
SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
```
```
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 0 0 \N
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 0 0 \N
```
## NULL values in JOIN keys

View File

@ -17,7 +17,6 @@
#include <boost/algorithm/string.hpp>
namespace DB
{
@ -866,4 +865,5 @@ Block concatenateBlocks(const std::vector<Block> & blocks)
out.setColumns(std::move(columns));
return out;
}
}

View File

@ -321,6 +321,7 @@ class IColumn;
M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \
\
M(Bool, join_use_nulls, false, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \
M(Bool, enable_mixed_join_condition, false, "Support conditions involve columns from both tables in on join expression.", IMPORTANT) \
\
M(JoinStrictness, join_default_strictness, JoinStrictness::All, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \
M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \

View File

@ -8,7 +8,6 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/queryToString.h>
namespace DB
{

View File

@ -55,6 +55,7 @@ namespace ErrorCodes
extern const int SET_SIZE_LIMIT_EXCEEDED;
extern const int TYPE_MISMATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int INVALID_JOIN_ON_EXPRESSION;
}
namespace
@ -124,14 +125,14 @@ namespace JoinStuff
}
}
template <bool use_flags, bool multiple_disjuncts, typename FindResult>
template <bool use_flags, bool flag_per_row, typename FindResult>
void JoinUsedFlags::setUsed(const FindResult & f)
{
if constexpr (!use_flags)
return;
/// Could be set simultaneously from different threads.
if constexpr (multiple_disjuncts)
if constexpr (flag_per_row)
{
auto & mapped = f.getMapped();
flags[mapped.block][mapped.row_num].store(true, std::memory_order_relaxed);
@ -142,14 +143,14 @@ namespace JoinStuff
}
}
template <bool use_flags, bool multiple_disjuncts>
template <bool use_flags, bool flag_per_row>
void JoinUsedFlags::setUsed(const Block * block, size_t row_num, size_t offset)
{
if constexpr (!use_flags)
return;
/// Could be set simultaneously from different threads.
if constexpr (multiple_disjuncts)
if constexpr (flag_per_row)
{
flags[block][row_num].store(true, std::memory_order_relaxed);
}
@ -159,13 +160,13 @@ namespace JoinStuff
}
}
template <bool use_flags, bool multiple_disjuncts, typename FindResult>
template <bool use_flags, bool flag_per_row, typename FindResult>
bool JoinUsedFlags::getUsed(const FindResult & f)
{
if constexpr (!use_flags)
return true;
if constexpr (multiple_disjuncts)
if constexpr (flag_per_row)
{
auto & mapped = f.getMapped();
return flags[mapped.block][mapped.row_num].load();
@ -176,13 +177,13 @@ namespace JoinStuff
}
}
template <bool use_flags, bool multiple_disjuncts, typename FindResult>
template <bool use_flags, bool flag_per_row, typename FindResult>
bool JoinUsedFlags::setUsedOnce(const FindResult & f)
{
if constexpr (!use_flags)
return true;
if constexpr (multiple_disjuncts)
if constexpr (flag_per_row)
{
auto & mapped = f.getMapped();
@ -711,7 +712,8 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample)
bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) ||
table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) ||
isRightOrFull(kind) ||
multiple_disjuncts;
multiple_disjuncts ||
table_join->getFullJoinExpression();
if (save_key_columns)
{
saved_block_sample = right_table_keys.cloneEmpty();
@ -841,7 +843,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
if (rows)
data->empty = false;
bool multiple_disjuncts = !table_join->oneDisjunct();
bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join);
const auto & onexprs = table_join->getClauses();
for (size_t onexpr_idx = 0; onexpr_idx < onexprs.size(); ++onexpr_idx)
{
@ -865,7 +867,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
auto join_mask_col = JoinCommon::getColumnAsMask(source_block, onexprs[onexpr_idx].condColumnNames().second);
/// Save blocks that do not hold conditions in ON section
ColumnUInt8::MutablePtr not_joined_map = nullptr;
if (!multiple_disjuncts && isRightOrFull(kind) && join_mask_col.hasData())
if (!flag_per_row && isRightOrFull(kind) && join_mask_col.hasData())
{
const auto & join_mask = join_mask_col.getData();
/// Save rows that do not hold conditions
@ -895,7 +897,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
join_mask_col.getData(),
data->pool, is_inserted);
if (multiple_disjuncts)
if (flag_per_row)
used_flags.reinit<kind_, strictness_>(stored_block);
else if (is_inserted)
/// Number of buckets + 1 value from zero storage
@ -903,19 +905,19 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
});
}
if (!multiple_disjuncts && save_nullmap && is_inserted)
if (!flag_per_row && save_nullmap && is_inserted)
{
data->blocks_nullmaps_allocated_size += null_map_holder->allocatedBytes();
data->blocks_nullmaps.emplace_back(stored_block, null_map_holder);
}
if (!multiple_disjuncts && not_joined_map && is_inserted)
if (!flag_per_row && not_joined_map && is_inserted)
{
data->blocks_nullmaps_allocated_size += not_joined_map->allocatedBytes();
data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map));
}
if (!multiple_disjuncts && !is_inserted)
if (!flag_per_row && !is_inserted)
{
LOG_TRACE(log, "Skipping inserting block with {} rows", rows);
data->blocks_allocated_size -= stored_block->allocatedBytes();
@ -1351,7 +1353,7 @@ struct JoinFeatures
static constexpr bool need_flags = MapGetter<KIND, STRICTNESS>::flagged;
};
template <bool multiple_disjuncts>
template <bool flag_per_row>
class KnownRowsHolder;
/// Keep already joined rows to prevent duplication if many disjuncts
@ -1426,18 +1428,18 @@ public:
}
};
template <typename Map, bool add_missing, bool multiple_disjuncts, typename AddedColumns>
template <typename Map, bool add_missing, bool flag_per_row, typename AddedColumns>
void addFoundRowAll(
const typename Map::mapped_type & mapped,
AddedColumns & added,
IColumn::Offset & current_offset,
KnownRowsHolder<multiple_disjuncts> & known_rows [[maybe_unused]],
KnownRowsHolder<flag_per_row> & known_rows [[maybe_unused]],
JoinStuff::JoinUsedFlags * used_flags [[maybe_unused]])
{
if constexpr (add_missing)
added.applyLazyDefaults();
if constexpr (multiple_disjuncts)
if constexpr (flag_per_row)
{
std::unique_ptr<std::vector<KnownRowsHolder<true>::Type>> new_known_rows_ptr;
@ -1454,7 +1456,7 @@ void addFoundRowAll(
new_known_rows_ptr->push_back(std::make_pair(it->block, it->row_num));
if (used_flags)
{
used_flags->JoinStuff::JoinUsedFlags::setUsedOnce<true, multiple_disjuncts>(
used_flags->JoinStuff::JoinUsedFlags::setUsedOnce<true, flag_per_row>(
FindResultImpl<const RowRef, false>(*it, true, 0));
}
}
@ -1494,24 +1496,31 @@ void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unuse
}
template<typename AddedColumns>
ColumnPtr buildAdditionFilter(
ColumnPtr buildAdditionalFilter(
size_t left_start_row,
const std::vector<std::pair<const Block *, size_t>> & selected_rows,
const std::vector<RowRef> & selected_rows,
const std::vector<size_t> & row_replicate_offset,
AddedColumns & added_columns)
{
if (selected_rows.empty())
return ColumnUInt8::create();
const Block & sample_right_block = *selected_rows.begin()->first;
const Block & sample_right_block = *selected_rows.begin()->block;
if (!sample_right_block)
return ColumnUInt8::create();
auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes();
if (required_cols.empty())
{
Block block;
added_columns.additional_filter_expression->execute(block);
return block.getByPosition(0).column->cloneResized(selected_rows.size());
}
NameSet required_column_names;
for (auto & col : required_cols)
{
required_column_names.insert(col.name);
}
Block executed_block;
size_t right_col_pos = 0;
for (const auto & col : sample_right_block.getColumnsWithTypeAndName())
@ -1521,8 +1530,8 @@ ColumnPtr buildAdditionFilter(
auto new_col = col.column->cloneEmpty();
for (const auto & selected_row : selected_rows)
{
const auto & src_col = selected_row.first->getByPosition(right_col_pos);
new_col->insertFrom(*src_col.column, selected_row.second);
const auto & src_col = selected_row.block->getByPosition(right_col_pos);
new_col->insertFrom(*src_col.column, selected_row.row_num);
}
executed_block.insert({std::move(new_col), col.type, col.name});
}
@ -1530,7 +1539,7 @@ ColumnPtr buildAdditionFilter(
}
if (!executed_block)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected at least one column from right table");
return ColumnUInt8::create();
}
for (const auto & col_name : required_column_names)
@ -1552,21 +1561,20 @@ ColumnPtr buildAdditionFilter(
}
executed_block.insert({std::move(new_col), src_col->type, col_name});
}
// LOG_TRACE(getLogger("HashJoin"), "Additional filter execute block:\n{}", executed_block.dumpContent());
LOG_TRACE(getLogger("HashJoin"), "Additional filter execute block:\n{}", executed_block.dumpContent());
added_columns.additional_filter_expression->execute(executed_block);
// LOG_TRACE(getLogger("HashJoin"), "Addition filter execute result block:\n{}", executed_block.dumpContent());
LOG_TRACE(getLogger("HashJoin"), "Addition filter execute result block:\n{}", executed_block.dumpContent());
return executed_block.getByPosition(0).column;
}
template <bool multiple_disjuncts, bool need_flags>
void appendFoundRowAll(
template <bool flag_per_row>
void addFoundRowRefAll(
const RowRefList & row_list,
std::vector<std::pair<const Block *, size_t>> & selected_rows,
std::vector<RowRef> & selected_rows,
IColumn::Offset & current_offset,
KnownRowsHolder<multiple_disjuncts> & known_rows [[maybe_unused]],
JoinStuff::JoinUsedFlags * used_flags [[maybe_unused]])
KnownRowsHolder<flag_per_row> & known_rows [[maybe_unused]])
{
if constexpr (multiple_disjuncts)
if constexpr (flag_per_row)
{
std::unique_ptr<std::vector<KnownRowsHolder<true>::Type>> new_known_rows_ptr;
for (auto it = row_list.begin(); it.ok(); ++it)
@ -1574,19 +1582,13 @@ void appendFoundRowAll(
auto row_ref = std::make_pair(it->block, it->row_num);
if (!known_rows.isKnown(row_ref))
{
selected_rows.push_back(row_ref);
selected_rows.emplace_back(row_ref.first, row_ref.second);
++current_offset;
if (!new_known_rows_ptr)
{
new_known_rows_ptr = std::make_unique<std::vector<KnownRowsHolder<true>::Type>>();
}
new_known_rows_ptr->push_back(row_ref);
if constexpr (need_flags)
{
used_flags->JoinStuff::JoinUsedFlags::setUsedOnce<true, multiple_disjuncts>(
FindResultImpl<const RowRef, false>(*it, true, 0));
}
}
}
@ -1597,38 +1599,44 @@ void appendFoundRowAll(
{
for (auto it = row_list.begin(); it.ok(); ++it)
{
selected_rows.emplace_back(std::pair(it->block, it->row_num));
selected_rows.emplace_back(it->block, it->row_num);
++current_offset;
}
}
}
/// First to collect all matched rows by join keys, then filter out rows which is not true in additional filter expression.
template <JoinKind KIND, JoinStrictness STRICTNESS, typename KeyGetter, typename Map, bool need_filter, bool multiple_disjuncts, typename AddedColumns>
/// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression.
template <
typename KeyGetter,
typename Map,
bool need_filter,
bool need_replication,
bool need_flags,
bool add_missing,
bool flag_per_row,
typename AddedColumns>
NO_INLINE size_t joinRightColumnsWithAddtitionalFilter(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]])
{
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
size_t left_block_rows = added_columns.rows_to_add;
if constexpr (need_filter)
added_columns.filter = IColumn::Filter(left_block_rows, 0);
std::unique_ptr<Arena> pool;
if constexpr (join_features.need_replication)
if constexpr (need_replication)
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(left_block_rows);
std::vector<size_t> row_replicate_offset;
row_replicate_offset.reserve(left_block_rows);
using FindResult = typename KeyGetter::FindResult;
size_t max_joined_block_rows = added_columns.max_joined_block_rows;
size_t left_row_iter = 0;
std::vector<std::pair<const Block*, size_t>> selected_rows;
std::vector<RowRef> selected_rows;
selected_rows.reserve(left_block_rows);
std::vector<FindResult> find_results;
find_results.reserve(left_block_rows);
@ -1646,14 +1654,14 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter(
selected_rows.clear();
for (; left_row_iter < left_block_rows; ++left_row_iter)
{
if constexpr (join_features.need_replication)
if constexpr (need_replication)
{
if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows))
{
break;
}
}
KnownRowsHolder<multiple_disjuncts> known_rows;
KnownRowsHolder<flag_per_row> known_rows;
for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx)
{
const auto & join_keys = added_columns.join_on_keys[join_clause_idx];
@ -1669,15 +1677,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter(
{
auto & mapped = find_result.getMapped();
find_results.push_back(find_result);
if constexpr (join_features.is_all_join)
{
appendFoundRowAll<multiple_disjuncts, join_features.need_flags>(
mapped, selected_rows, current_added_rows, known_rows, &used_flags);
}
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsupported join type. kind:{}, strictness:{}", KIND, STRICTNESS);
}
addFoundRowRefAll<flag_per_row>(mapped, selected_rows, current_added_rows, known_rows);
}
}
row_replicate_offset.push_back(current_added_rows);
@ -1687,6 +1687,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter(
auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col)
{
const PaddedPODArray<UInt8> * filter_flags = nullptr;
filter_col = filter_col->convertToFullIfNeeded();
if (filter_col->isNullable())
{
auto nested_col = typeid_cast<const ColumnNullable &>(*filter_col).getNestedColumnPtr();
@ -1699,41 +1700,58 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter(
size_t prev_replicated_row = 0;
auto selected_right_row_it = selected_rows.begin();
size_t find_result_index = 0;
for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i)
{
bool any_matched = false;
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
/// For all right join, flag_per_row is true, we need mark used flags for each row.
if constexpr (flag_per_row)
{
if ((*filter_flags)[replicated_row])
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
{
any_matched = true;
added_columns.appendFromBlock(*selected_right_row_it->first, selected_right_row_it->second, join_features.add_missing);
total_added_rows += 1;
if ((*filter_flags)[replicated_row])
{
any_matched = true;
added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing);
total_added_rows += 1;
used_flags.template setUsed<need_flags, flag_per_row>(selected_right_row_it->block, selected_right_row_it->row_num, 0);
}
++selected_right_row_it;
}
++selected_right_row_it;
}
if (!any_matched)
{
if constexpr (join_features.is_anti_join && join_features.left)
setUsed<need_filter>(added_columns.filter, i - 1);
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, total_added_rows);
}
else
{
if constexpr (join_features.is_all_join)
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
{
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(find_results[i - 1]);
setUsed<need_filter>(added_columns.filter, left_start_row + i - 1);
if ((*filter_flags)[replicated_row])
{
any_matched = true;
added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing);
total_added_rows += 1;
}
++selected_right_row_it;
}
}
if (!any_matched)
{
addNotFoundRow<add_missing, need_replication>(added_columns, total_added_rows);
}
else
{
if constexpr (!flag_per_row)
used_flags.template setUsed<need_flags, false>(find_results[find_result_index]);
setUsed<need_filter>(added_columns.filter, left_start_row + i - 1);
if constexpr (add_missing)
added_columns.applyLazyDefaults();
}
find_result_index += (prev_replicated_row != row_replicate_offset[i]);
if constexpr (join_features.need_replication)
if constexpr (need_replication)
{
(*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows;
}
prev_replicated_row = row_replicate_offset[i];
}
};
while (left_row_iter < left_block_rows && !exceeded_max_block_rows)
@ -1752,10 +1770,10 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter(
left_row_iter,
left_start_row);
}
auto filter_col = buildAdditionFilter(left_start_row, selected_rows, row_replicate_offset, added_columns);
auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns);
copy_final_matched_rows(left_start_row, filter_col);
if constexpr (join_features.need_replication)
if constexpr (need_replication)
{
// Add a check for current_added_rows to avoid run the filter expression on too small size batch.
if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024)
@ -1765,7 +1783,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter(
}
}
if constexpr (join_features.need_replication)
if constexpr (need_replication)
{
added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter);
added_columns.filter.resize_assume_reserved(left_row_iter);
@ -1776,7 +1794,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter(
/// Joins right table columns which indexes are present in right_indexes using specified map.
/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS).
template <JoinKind KIND, JoinStrictness STRICTNESS, typename KeyGetter, typename Map, bool need_filter, bool multiple_disjuncts, typename AddedColumns>
template <JoinKind KIND, JoinStrictness STRICTNESS, typename KeyGetter, typename Map, bool need_filter, bool flag_per_row, typename AddedColumns>
NO_INLINE size_t joinRightColumns(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
@ -1811,7 +1829,7 @@ NO_INLINE size_t joinRightColumns(
bool right_row_found = false;
KnownRowsHolder<multiple_disjuncts> known_rows;
KnownRowsHolder<flag_per_row> known_rows;
for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx)
{
const auto & join_keys = added_columns.join_on_keys[onexpr_idx];
@ -1834,10 +1852,10 @@ NO_INLINE size_t joinRightColumns(
if (row_ref.block)
{
setUsed<need_filter>(added_columns.filter, i);
if constexpr (multiple_disjuncts)
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(row_ref.block, row_ref.row_num, 0);
if constexpr (flag_per_row)
used_flags.template setUsed<join_features.need_flags, flag_per_row>(row_ref.block, row_ref.row_num, 0);
else
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(find_result);
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing);
}
@ -1847,14 +1865,14 @@ NO_INLINE size_t joinRightColumns(
else if constexpr (join_features.is_all_join)
{
setUsed<need_filter>(added_columns.filter, i);
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(find_result);
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
addFoundRowAll<Map, join_features.add_missing>(mapped, added_columns, current_offset, known_rows, used_flags_opt);
}
else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right)
{
/// Use first appeared left key + it needs left columns replication
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, multiple_disjuncts>(find_result);
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
if (used_once)
{
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
@ -1864,7 +1882,7 @@ NO_INLINE size_t joinRightColumns(
}
else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner)
{
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, multiple_disjuncts>(find_result);
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
/// Use first appeared left key only
if (used_once)
@ -1882,12 +1900,12 @@ NO_INLINE size_t joinRightColumns(
else if constexpr (join_features.is_anti_join)
{
if constexpr (join_features.right && join_features.need_flags)
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(find_result);
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
}
else /// ANY LEFT, SEMI LEFT, old ANY (RightAny)
{
setUsed<need_filter>(added_columns.filter, i);
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(find_result);
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing);
if (join_features.is_any_or_semi_join)
@ -1922,19 +1940,46 @@ size_t joinRightColumnsSwitchMultipleDisjuncts(
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]])
{
if (added_columns.additional_filter_expression)
{
return mapv.size() > 1 ? joinRightColumnsWithAddtitionalFilter<KIND, STRICTNESS, KeyGetter, Map, need_filter, true>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags)
: joinRightColumnsWithAddtitionalFilter<KIND, STRICTNESS, KeyGetter, Map, need_filter, false>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
else
auto join_without_additional_filter = [&]()
{
return mapv.size() > 1 ? joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, need_filter, true>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags)
: joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, need_filter, false>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
};
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
if constexpr (join_features.is_all_join)
{
if (added_columns.additional_filter_expression)
{
constexpr bool mark_per_row_used = join_features.right || join_features.full;
return mapv.size() > 1
? joinRightColumnsWithAddtitionalFilter<
KeyGetter,
Map,
need_filter,
join_features.need_replication,
join_features.need_flags,
join_features.add_missing,
true>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags)
: joinRightColumnsWithAddtitionalFilter<
KeyGetter,
Map,
need_filter,
join_features.need_replication,
join_features.need_flags,
join_features.add_missing,
mark_per_row_used>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
else
{
return join_without_additional_filter();
}
}
else
{
return join_without_additional_filter();
}
}
@ -2421,10 +2466,10 @@ struct AdderNonJoined
class NotJoinedHash final : public NotJoinedBlocks::RightColumnsFiller
{
public:
NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_, bool multiple_disjuncts_)
NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_, bool flag_per_row_)
: parent(parent_)
, max_block_size(max_block_size_)
, multiple_disjuncts(multiple_disjuncts_)
, flag_per_row(flag_per_row_)
, current_block_start(0)
{
if (parent.data == nullptr)
@ -2451,7 +2496,7 @@ public:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness);
}
if (!multiple_disjuncts)
if (!flag_per_row)
{
fillNullsFromBlocks(columns_right, rows_added);
}
@ -2462,7 +2507,7 @@ public:
private:
const HashJoin & parent;
UInt64 max_block_size;
bool multiple_disjuncts;
bool flag_per_row;
size_t current_block_start;
@ -2528,7 +2573,7 @@ private:
{
size_t rows_added = 0;
if (multiple_disjuncts)
if (flag_per_row)
{
if (!used_position.has_value())
used_position = parent.data->blocks.begin();
@ -2620,8 +2665,8 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block,
return {};
size_t left_columns_count = left_sample_block.columns();
bool multiple_disjuncts = !table_join->oneDisjunct();
if (!multiple_disjuncts)
bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join);
if (!flag_per_row)
{
/// With multiple disjuncts, all keys are in sample_block_with_columns_to_add, so invariant is not held
size_t expected_columns_count = left_columns_count + required_right_keys.columns() + sample_block_with_columns_to_add.columns();
@ -2633,7 +2678,7 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block,
}
}
auto non_joined = std::make_unique<NotJoinedHash>(*this, max_block_size, multiple_disjuncts);
auto non_joined = std::make_unique<NotJoinedHash>(*this, max_block_size, flag_per_row);
return std::make_unique<NotJoinedBlocks>(std::move(non_joined), result_sample_block, left_columns_count, *table_join);
}
@ -2642,8 +2687,8 @@ void HashJoin::reuseJoinedData(const HashJoin & join)
data = join.data;
from_storage_join = true;
bool multiple_disjuncts = !table_join->oneDisjunct();
if (multiple_disjuncts)
bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join);
if (flag_per_row)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageJoin with ORs is not supported");
for (auto & map : data->maps)
@ -2724,17 +2769,28 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona
if (!type->equals(*std::make_shared<DataTypeUInt8>()))
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unexpected expression in JOIN ON section. Expected boolean (UInt8), got '{}'",
expression_sample_block.getByPosition(0).type->getName());
"Unexpected expression in JOIN ON section. Expected boolean (UInt8), got '{}'. expression:\n{}",
expression_sample_block.getByPosition(0).type->getName(),
additional_filter_expression->dumpActions());
}
bool is_supported = (strictness == JoinStrictness::All) && (kind == JoinKind::Inner || kind == JoinKind::Left || kind == JoinKind::Right);
bool is_supported = (strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind));
if (!is_supported)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Non equi condition '{}' from JOIN ON section is supported only for ALL INNER/LEFT/RIGHT JOINs",
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"Non equi condition '{}' from JOIN ON section is supported only for ALL INNER/LEFT/FULL/RIGHT JOINs.",
expression_sample_block.getByPosition(0).name);
}
}
bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const
{
if (!table_join_->oneDisjunct())
return true;
/// If it'a a all right join with inequal conditions, we need to mark each row
if (table_join_->getFullJoinExpression() && isRightOrFull(table_join_->kind()))
return true;
return false;
}
}

View File

@ -9,7 +9,6 @@
#include <Interpreters/IJoin.h>
#include <Interpreters/AggregationCommon.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/RowRefs.h>
#include <Common/Arena.h>
@ -32,6 +31,7 @@ namespace DB
{
class TableJoin;
class ExpressionActions;
namespace JoinStuff
{
@ -61,16 +61,16 @@ public:
bool getUsedSafe(size_t i) const;
bool getUsedSafe(const Block * block_ptr, size_t row_idx) const;
template <bool use_flags, bool multiple_disjuncts, typename T>
template <bool use_flags, bool flag_per_row, typename T>
void setUsed(const T & f);
template <bool use_flags, bool multiple_disjunct>
template <bool use_flags, bool flag_per_row>
void setUsed(const Block * block, size_t row_num, size_t offset);
template <bool use_flags, bool multiple_disjuncts, typename T>
template <bool use_flags, bool flag_per_row, typename T>
bool getUsed(const T & f);
template <bool use_flags, bool multiple_disjuncts, typename T>
template <bool use_flags, bool flag_per_row, typename T>
bool setUsedOnce(const T & f);
};
@ -472,7 +472,8 @@ private:
bool empty() const;
void validateAdditionalFilterExpression(ExpressionActionsPtr additional_filter_expression);
void validateAdditionalFilterExpression(std::shared_ptr<ExpressionActions> additional_filter_expression);
bool needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const;
};
}

View File

@ -4,8 +4,6 @@
#include <Core/NamesAndTypes.h>
#include <Core/SettingsEnums.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/IJoin.h>
#include <Interpreters/JoinUtils.h>
#include <QueryPipeline/SizeLimits.h>
@ -30,6 +28,7 @@ class ASTSelectQuery;
struct DatabaseAndTableWithAlias;
class Block;
class DictionaryJoinAdapter;
class ExpressionActions;
class StorageJoin;
class StorageDictionary;
class IKeyValueEntity;
@ -156,7 +155,7 @@ private:
Clauses clauses;
/// Originally used for inequal join. If there is no any inequal join condition, it will be nullptr.
ExpressionActionsPtr full_join_expression = nullptr;
std::shared_ptr<ExpressionActions> full_join_expression = nullptr;
ASTTableJoin table_join;
@ -302,8 +301,8 @@ public:
std::vector<JoinOnClause> & getClauses() { return clauses; }
const std::vector<JoinOnClause> & getClauses() const { return clauses; }
const ExpressionActionsPtr & getFullJoinExpression() const { return full_join_expression; }
ExpressionActionsPtr & getFullJoinExpression() { return full_join_expression; }
const std::shared_ptr<ExpressionActions> & getFullJoinExpression() const { return full_join_expression; }
std::shared_ptr<ExpressionActions> & getFullJoinExpression() { return full_join_expression; }
Names getAllNames(JoinTableSide side) const;

View File

@ -1323,7 +1323,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
ExpressionActionsPtr & full_join_expression = table_join->getFullJoinExpression();
full_join_expression = std::make_shared<ExpressionActions>(
join_clauses_and_actions.full_join_expressions_actions,
ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::no));
ExpressionActionsSettings::fromContext(planner_context->getQueryContext()));
}
}
else if (join_node.isUsingJoinExpression())

View File

@ -10,6 +10,7 @@
#include <DataTypes/getLeastSupertype.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Storages/IStorage.h>
#include <Storages/StorageJoin.h>
@ -313,10 +314,21 @@ void buildJoinClause(
}
else
{
/// expression involves both tables.
/// `expr1(left.col1, right.col2) == expr2(left.col3, right.col4)`
const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node);
join_clause.addMixedCondition(node);
auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().enable_mixed_join_condition;
if (support_mixed_join_condition)
{
/// expression involves both tables.
/// `expr1(left.col1, right.col2) == expr2(left.col3, right.col4)`
const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node);
join_clause.addMixedCondition(node);
}
else
{
throw Exception(
ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"JOIN {} join expression contains column from left and right table",
join_node.formatASTForErrorMessage());
}
}
}
@ -326,7 +338,7 @@ void buildJoinClause(
left_table_expressions,
right_table_expressions,
join_node);
// expression_sides.empty() = true, the expression is constant
if (expression_sides.empty() || expression_sides.size() == 1)
{
auto expression_side = expression_sides.empty() ? JoinTableSide::Right : *expression_sides.begin();
@ -336,9 +348,20 @@ void buildJoinClause(
}
else
{
/// expression involves both tables.
const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node);
join_clause.addMixedCondition(node);
auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().enable_mixed_join_condition;
if (support_mixed_join_condition)
{
/// expression involves both tables.
const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node);
join_clause.addMixedCondition(node);
}
else
{
throw Exception(
ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"JOIN {} join expression contains column from left and right table",
join_node.formatASTForErrorMessage());
}
}
}
}
@ -592,6 +615,20 @@ JoinClausesAndActions buildJoinClausesAndActions(
auto full_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true);
result.full_join_expressions_actions = full_join_expressions_actions;
}
auto outputs = result.full_join_expressions_actions->getOutputs();
if (outputs.size() != 1)
{
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Only one output is expected. but got:\n{}", result.full_join_expressions_actions->dumpDAG());
}
auto output_type = removeNullable(outputs[0]->result_type);
WhichDataType which_type(output_type);
if (!which_type.isUInt8())
{
DataTypePtr uint8_ty = std::make_shared<DataTypeUInt8>();
auto true_col = ColumnWithTypeAndName(uint8_ty->createColumnConst(1, 1), uint8_ty, "true");
const auto * true_node = &result.full_join_expressions_actions->addColumn(true_col);
result.full_join_expressions_actions = ActionsDAG::buildFilterActionsDAG({outputs[0], true_node});
}
}
return result;
@ -805,10 +842,11 @@ std::shared_ptr<IJoin> chooseJoinAlgorithm(std::shared_ptr<TableJoin> & table_jo
const Block & right_table_expression_header,
const PlannerContextPtr & planner_context)
{
if (table_join->getFullJoinExpression() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH))
if (table_join->getFullJoinExpression() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH)
&& !(table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) && table_join->oneDisjunct()))
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"JOIN with mixed conditions supports only hash join algorithm");
"JOIN with mixed conditions supports only hash join or grace hash join with one disjunct.");
}
trySetStorageInTableJoin(right_table_expression, table_join);

View File

@ -149,7 +149,7 @@ public:
{
mixed_filter_condition_nodes.push_back(condition_node);
}
const ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes() const
{
return mixed_filter_condition_nodes;
@ -157,7 +157,6 @@ public:
bool hasMixedFilterCondition() const
{
// return has_mixed_filter_condition;
return !mixed_filter_condition_nodes.empty();
}
/// Dump clause into buffer

View File

@ -0,0 +1,366 @@
-- { echoOn }
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 key4 F 1 1 1
SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 c 3 2 1 key1 B 2 1 2
key1 c 3 2 1 key1 C 3 4 5
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 0 0 \N
SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 0 0 \N
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 0 0 \N
SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
1 1 1 1 1 1
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key4 f 2 3 4 key4 F 1 1 1
SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 c 3 2 1 key1 B 2 1 2
key1 c 3 2 1 key1 C 3 4 5
key1 d 4 7 2 key1 D 4 1 6
SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 D 4 1 6
SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
1 1 1 1 1 1
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key3 a3 1 1 1
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key4 f 2 3 4 key4 F 1 1 1
SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key3 a3 1 1 1
0 0 \N key4 F 1 1 1
key1 a 1 1 2 key1 A 1 2 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 c 3 2 1 key1 B 2 1 2
key1 c 3 2 1 key1 C 3 4 5
key1 d 4 7 2 key1 D 4 1 6
SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key1 A 1 2 1
0 0 \N key3 a3 1 1 1
0 0 \N key4 F 1 1 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 D 4 1 6
SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
1 1 1 1 1 1
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key3 a3 1 1 1
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 key4 F 1 1 1
SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key3 a3 1 1 1
0 0 \N key4 F 1 1 1
key1 a 1 1 2 key1 A 1 2 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 c 3 2 1 key1 B 2 1 2
key1 c 3 2 1 key1 C 3 4 5
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 0 0 \N
SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key1 A 1 2 1
0 0 \N key3 a3 1 1 1
0 0 \N key4 F 1 1 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 0 0 \N
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 0 0 \N
SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 FULL JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
1 1 1 1 1 1
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 key4 F 1 1 1
SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 c 3 2 1 key1 B 2 1 2
key1 c 3 2 1 key1 C 3 4 5
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 0 0 \N
SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 0 0 \N
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 0 0 \N
SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
1 1 1 1 1 1
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key4 f 2 3 4 key4 F 1 1 1
SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 c 3 2 1 key1 B 2 1 2
key1 c 3 2 1 key1 C 3 4 5
key1 d 4 7 2 key1 D 4 1 6
SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 D 4 1 6
SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
1 1 1 1 1 1
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key3 a3 1 1 1
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key4 f 2 3 4 key4 F 1 1 1
SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key3 a3 1 1 1
0 0 \N key4 F 1 1 1
key1 a 1 1 2 key1 A 1 2 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 c 3 2 1 key1 B 2 1 2
key1 c 3 2 1 key1 C 3 4 5
key1 d 4 7 2 key1 D 4 1 6
SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key1 A 1 2 1
0 0 \N key3 a3 1 1 1
0 0 \N key4 F 1 1 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 D 4 1 6
SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
1 1 1 1 1 1
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key3 a3 1 1 1
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 key4 F 1 1 1
SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key3 a3 1 1 1
0 0 \N key4 F 1 1 1
key1 a 1 1 2 key1 A 1 2 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 c 3 2 1 key1 B 2 1 2
key1 c 3 2 1 key1 C 3 4 5
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 0 0 \N
SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
0 0 \N key1 A 1 2 1
0 0 \N key3 a3 1 1 1
0 0 \N key4 F 1 1 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 0 0 \N
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 0 0 \N
key4 f 2 3 4 0 0 \N
SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 FULL JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
1 1 1 1 1 1
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 a 1 1 2 key3 a3 1 1 1
key1 a 1 1 2 key4 F 1 1 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 key1 A 1 2 1
key2 a2 1 1 1 key3 a3 1 1 1
key2 a2 1 1 1 key4 F 1 1 1
key4 f 2 3 4 key1 B 2 1 2
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 a 1 1 2 key3 a3 1 1 1
key1 a 1 1 2 key4 F 1 1 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key2 a2 1 1 1 key1 A 1 2 1
key2 a2 1 1 1 key3 a3 1 1 1
key2 a2 1 1 1 key4 F 1 1 1
key4 f 2 3 4 key1 B 2 1 2
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 a 1 1 2 key3 a3 1 1 1
key1 a 1 1 2 key4 F 1 1 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key2 a2 1 1 1 key1 A 1 2 1
key2 a2 1 1 1 key3 a3 1 1 1
key2 a2 1 1 1 key4 F 1 1 1
key4 f 2 3 4 key1 B 2 1 2
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
key1 a 1 1 2 key1 A 1 2 1
key1 a 1 1 2 key1 B 2 1 2
key1 a 1 1 2 key1 C 3 4 5
key1 a 1 1 2 key1 D 4 1 6
key1 a 1 1 2 key3 a3 1 1 1
key1 a 1 1 2 key4 F 1 1 1
key1 b 2 3 2 key1 B 2 1 2
key1 b 2 3 2 key1 C 3 4 5
key1 b 2 3 2 key1 D 4 1 6
key1 c 3 2 1 key1 C 3 4 5
key1 c 3 2 1 key1 D 4 1 6
key1 d 4 7 2 key1 D 4 1 6
key1 e 5 5 5 0 0 \N
key2 a2 1 1 1 key1 A 1 2 1
key2 a2 1 1 1 key3 a3 1 1 1
key2 a2 1 1 1 key4 F 1 1 1
key4 f 2 3 4 key1 B 2 1 2

View File

@ -0,0 +1,51 @@
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
CREATE TABLE t1 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64)) ENGINE = MergeTree ORDER BY key;
INSERT INTO t1 VALUES ('key1', 'a', 1, 1, 2), ('key1', 'b', 2, 3, 2), ('key1', 'c', 3, 2, 1), ('key1', 'd', 4, 7, 2), ('key1', 'e', 5, 5, 5), ('key2', 'a2', 1, 1, 1), ('key4', 'f', 2, 3, 4);
CREATE TABLE t2 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64)) ENGINE = MergeTree ORDER BY key;
INSERT INTO t2 VALUES ('key1', 'A', 1, 2, 1), ('key1', 'B', 2, 1, 2), ('key1', 'C', 3, 4, 5), ('key1', 'D', 4, 1, 6), ('key3', 'a3', 1, 1, 1), ('key4', 'F', 1,1,1);
SET allow_experimental_analyzer=1;
SET enable_mixed_join_condition=1;
-- { echoOn }
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
{% endfor -%}
{% endfor -%}
{% for algorithm in ['hash'] -%}
SET join_algorithm='{{ algorithm }}';
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
{% endfor -%}
{% endfor -%}
-- { echoOff }
-- test error messages
{% for algorithm in ['partial_merge', 'full_sorting_merge', 'parallel_hash', 'auto', 'direct'] -%}
SET join_algorithm='{{ algorithm }}';
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED }
SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED }
SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED }
{% endfor -%}
{% endfor -%}
{% for algorithm in ['grace_hash', 'partial_merge', 'full_sorting_merge', 'parallel_hash', 'auto', 'direct'] -%}
SET join_algorithm='{{ algorithm }}';
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED }
{% endfor -%}
{% endfor -%}
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;

View File

@ -9,7 +9,7 @@ TU_EXCLUDES=(
Aggregator
)
if find $1 -name '*.o' | xargs wc -c | grep --regexp='\.o$' | sort -rn | awk '{ if ($1 > 50000000) print }' \
if find $1 -name '*.o' | xargs wc -c | grep --regexp='\.o$' | sort -rn | awk '{ if ($1 > 100000000) print }' \
| grep -v -f <(printf "%s\n" "${TU_EXCLUDES[@]}")
then
echo "^ It's not allowed to have so large translation units."