ClickHouse/src/Interpreters/TableJoin.h

252 lines
10 KiB
C++
Raw Normal View History

2018-11-02 18:53:23 +00:00
#pragma once
#include <Core/Names.h>
#include <Core/NamesAndTypes.h>
#include <Core/SettingsEnums.h>
#include <Parsers/ASTTablesInSelectQuery.h>
2019-09-09 19:43:37 +00:00
#include <Interpreters/IJoin.h>
#include <Interpreters/join_common.h>
2019-10-11 17:56:26 +00:00
#include <Interpreters/asof.h>
#include <DataStreams/IBlockStream_fwd.h>
2019-09-09 19:43:37 +00:00
#include <DataStreams/SizeLimits.h>
#include <DataTypes/getLeastSupertype.h>
2020-04-08 18:59:52 +00:00
#include <Storages/IStorage_fwd.h>
2018-11-02 18:53:23 +00:00
#include <utility>
#include <memory>
2021-06-11 14:47:30 +00:00
2018-11-02 18:53:23 +00:00
namespace DB
{
class Context;
class ASTSelectQuery;
2019-02-13 19:00:52 +00:00
struct DatabaseAndTableWithAlias;
class Block;
2020-04-08 18:59:52 +00:00
class DictionaryReader;
2021-06-29 09:22:53 +00:00
class StorageJoin;
class StorageDictionary;
2020-09-08 11:06:36 +00:00
2020-09-08 11:07:26 +00:00
struct ColumnWithTypeAndName;
2020-09-08 11:06:36 +00:00
using ColumnsWithTypeAndName = std::vector<ColumnWithTypeAndName>;
2019-09-09 19:43:37 +00:00
struct Settings;
2018-11-02 18:53:23 +00:00
2020-07-08 14:25:23 +00:00
class IVolume;
using VolumePtr = std::shared_ptr<IVolume>;
enum class JoinTableSide
{
Left,
Right
};
class TableJoin
2018-11-02 18:53:23 +00:00
{
public:
using NameToTypeMap = std::unordered_map<String, DataTypePtr>;
private:
2018-11-02 18:53:23 +00:00
/** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k`
* The join is made by column k.
* During the JOIN,
* - in the "right" table, it will be available by alias `k`, since `Project` action for the subquery was executed.
* - in the "left" table, it will be accessible by the name `expr(x)`, since `Project` action has not been executed yet.
* You must remember both of these options.
*
* Query of the form `SELECT ... from t1 ANY LEFT JOIN (SELECT ... from t2) ON expr(t1 columns) = expr(t2 columns)`
* to the subquery will be added expression `expr(t2 columns)`.
* It's possible to use name `expr(t2 columns)`.
*/
2019-07-30 18:39:37 +00:00
friend class TreeRewriter;
2019-07-30 18:39:37 +00:00
const SizeLimits size_limits;
const size_t default_max_bytes = 0;
const bool join_use_nulls = false;
const size_t max_joined_block_rows = 0;
JoinAlgorithm join_algorithm = JoinAlgorithm::AUTO;
const bool partial_merge_join_optimizations = false;
const size_t partial_merge_join_rows_in_right_blocks = 0;
const size_t partial_merge_join_left_table_buffer_bytes = 0;
const size_t max_files_to_merge = 0;
const String temporary_files_codec = "LZ4";
2019-09-09 19:43:37 +00:00
2018-11-02 18:53:23 +00:00
Names key_names_left;
Names key_names_right; /// Duplicating names are qualified.
ASTs on_filter_condition_asts_left;
ASTs on_filter_condition_asts_right;
2018-11-02 18:53:23 +00:00
ASTs key_asts_left;
ASTs key_asts_right;
ASTTableJoin table_join;
ASOF::Inequality asof_inequality = ASOF::Inequality::GreaterOrEquals;
2018-11-02 18:53:23 +00:00
/// All columns which can be read from joined table. Duplicating names are qualified.
2019-07-30 18:39:37 +00:00
NamesAndTypesList columns_from_joined_table;
/// Columns will be added to block by JOIN.
/// It's a subset of columns_from_joined_table
/// Note: without corrected Nullability or type, see correctedColumnsAddedByJoin
NamesAndTypesList columns_added_by_join;
/// Target type to convert key columns before join
NameToTypeMap left_type_map;
NameToTypeMap right_type_map;
ActionsDAGPtr left_converting_actions;
ActionsDAGPtr right_converting_actions;
2019-07-30 18:39:37 +00:00
/// Name -> original name. Names are the same as in columns_from_joined_table list.
std::unordered_map<String, String> original_names;
/// Original name -> name. Only renamed columns.
2019-07-30 18:39:37 +00:00
std::unordered_map<String, String> renames;
2020-07-08 14:25:23 +00:00
VolumePtr tmp_volume;
2021-06-29 09:22:53 +00:00
std::shared_ptr<StorageJoin> right_storage_join;
std::shared_ptr<StorageDictionary> right_storage_dictionary;
std::shared_ptr<DictionaryReader> dictionary_reader;
Names requiredJoinedNames() const;
/// Create converting actions and change key column names if required
ActionsDAGPtr applyKeyConvertToTable(
const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, Names & names_to_rename) const;
/// Calculates common supertypes for corresponding join key columns.
template <typename LeftNamesAndTypes, typename RightNamesAndTypes>
bool inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool to_supertype);
NamesAndTypesList correctedColumnsAddedByJoin() const;
public:
2020-04-08 18:59:52 +00:00
TableJoin() = default;
TableJoin(const Settings &, VolumePtr tmp_volume);
/// for StorageJoin
TableJoin(SizeLimits limits, bool use_nulls, ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness,
2021-06-11 11:03:53 +00:00
const Names & key_names_right_)
: size_limits(limits)
, default_max_bytes(0)
, join_use_nulls(use_nulls)
, join_algorithm(JoinAlgorithm::HASH)
, key_names_right(key_names_right_)
{
table_join.kind = kind;
table_join.strictness = strictness;
}
2019-09-09 19:43:37 +00:00
ASTTableJoin::Kind kind() const { return table_join.kind; }
ASTTableJoin::Strictness strictness() const { return table_join.strictness; }
2020-02-02 16:12:48 +00:00
bool sameStrictnessAndKind(ASTTableJoin::Strictness, ASTTableJoin::Kind) const;
const SizeLimits & sizeLimits() const { return size_limits; }
VolumePtr getTemporaryVolume() { return tmp_volume; }
bool allowMergeJoin() const;
bool preferMergeJoin() const { return join_algorithm == JoinAlgorithm::PREFER_PARTIAL_MERGE; }
bool forceMergeJoin() const { return join_algorithm == JoinAlgorithm::PARTIAL_MERGE; }
bool forceHashJoin() const
{
/// HashJoin always used for DictJoin
return dictionary_reader || join_algorithm == JoinAlgorithm::HASH;
}
bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(table_join.kind); }
bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(table_join.kind); }
size_t defaultMaxBytes() const { return default_max_bytes; }
size_t maxJoinedBlockRows() const { return max_joined_block_rows; }
size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; }
size_t maxBytesInLeftBuffer() const { return partial_merge_join_left_table_buffer_bytes; }
size_t maxFilesToMerge() const { return max_files_to_merge; }
const String & temporaryFilesCodec() const { return temporary_files_codec; }
bool enablePartialMergeJoinOptimizations() const { return partial_merge_join_optimizations; }
bool needStreamWithNonJoinedRows() const;
2019-09-09 19:43:37 +00:00
void resetCollected();
void addUsingKey(const ASTPtr & ast);
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
2019-02-06 16:44:47 +00:00
/* Conditions for left/right table from JOIN ON section.
*
* Conditions for left and right tables stored separately and united with 'and' function into one column.
* For example for query:
* SELECT ... JOIN ... ON t1.id == t2.id AND expr11(t1) AND expr21(t2) AND expr12(t1) AND expr22(t2)
*
* We will build two new ASTs: `expr11(t1) AND expr12(t1)`, `expr21(t2) AND expr22(t2)`
* Such columns will be added and calculated for left and right tables respectively.
* Only rows where conditions are met (where new columns have non-zero value) will be joined.
*
* NOTE: non-equi condition containing columns from different tables (like `... ON t1.id = t2.id AND t1.val > t2.val)
* doesn't supported yet, it can be added later.
*/
void addJoinCondition(const ASTPtr & ast, bool is_left);
ASTPtr joinConditionColumn(JoinTableSide side) const;
std::pair<String, String> joinConditionColumnNames() const;
bool hasUsing() const { return table_join.using_expression_list != nullptr; }
bool hasOn() const { return table_join.on_expression != nullptr; }
NamesWithAliases getNamesWithAliases(const NameSet & required_columns) const;
2020-03-08 23:48:08 +00:00
NamesWithAliases getRequiredColumns(const Block & sample, const Names & action_required_columns) const;
2018-11-02 18:53:23 +00:00
2019-07-30 18:39:37 +00:00
void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix);
size_t rightKeyInclusion(const String & name) const;
2019-09-11 15:57:09 +00:00
NameSet requiredRightKeys() const;
bool leftBecomeNullable(const DataTypePtr & column_type) const;
bool rightBecomeNullable(const DataTypePtr & column_type) const;
void addJoinedColumn(const NameAndTypePair & joined_column);
void addJoinedColumnsAndCorrectTypes(NamesAndTypesList & left_columns, bool correct_nullability);
/// Calculate converting actions, rename key columns in required
/// For `USING` join we will convert key columns inplace and affect into types in the result table
/// For `JOIN ON` we will create new columns with converted keys to join by.
bool createConvertingActions(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns);
/// Key columns should be converted before join.
ActionsDAGPtr leftConvertingActions() const { return left_converting_actions; }
ActionsDAGPtr rightConvertingActions() const { return right_converting_actions; }
void setAsofInequality(ASOF::Inequality inequality) { asof_inequality = inequality; }
ASOF::Inequality getAsofInequality() { return asof_inequality; }
2019-10-11 17:56:26 +00:00
ASTPtr leftKeysList() const;
ASTPtr rightKeysList() const; /// For ON syntax only
const Names & keyNamesLeft() const { return key_names_left; }
const Names & keyNamesRight() const { return key_names_right; }
const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; }
Names columnsAddedByJoin() const
{
Names res;
for (const auto & col : columns_added_by_join)
res.push_back(col.name);
return res;
}
/// StorageJoin overrides key names (cause of different names qualification)
void setRightKeys(const Names & keys) { key_names_right = keys; }
Block getRequiredRightKeys(const Block & right_table_keys, std::vector<String> & keys_sources) const;
String renamedRightColumnName(const String & name) const;
2021-08-06 14:15:11 +00:00
std::unordered_map<String, String> leftToRightKeyRemap() const;
2021-06-29 09:22:53 +00:00
void setStorageJoin(std::shared_ptr<StorageJoin> storage);
void setStorageJoin(std::shared_ptr<StorageDictionary> storage);
std::shared_ptr<StorageJoin> getStorageJoin();
bool tryInitDictJoin(const Block & sample_block, ContextPtr context);
bool isSpecialStorage() const;
const DictionaryReader * getDictionaryReader() const;
2018-11-02 18:53:23 +00:00
};
}