2018-11-02 18:53:23 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Core/Names.h>
|
|
|
|
#include <Core/NamesAndTypes.h>
|
2020-02-19 19:11:23 +00:00
|
|
|
#include <Core/SettingsCollection.h>
|
2019-09-02 19:58:45 +00:00
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2019-09-09 19:43:37 +00:00
|
|
|
#include <Interpreters/IJoin.h>
|
2019-10-11 17:56:26 +00:00
|
|
|
#include <Interpreters/asof.h>
|
2019-09-03 14:36:02 +00:00
|
|
|
#include <DataStreams/IBlockStream_fwd.h>
|
2019-09-09 19:43:37 +00:00
|
|
|
#include <DataStreams/SizeLimits.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>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
class Context;
|
|
|
|
class ASTSelectQuery;
|
2019-02-13 19:00:52 +00:00
|
|
|
struct DatabaseAndTableWithAlias;
|
2019-09-02 19:58:45 +00:00
|
|
|
class Block;
|
2020-04-08 18:59:52 +00:00
|
|
|
class DictionaryReader;
|
2019-09-02 19:58:45 +00:00
|
|
|
|
2019-09-09 19:43:37 +00:00
|
|
|
struct Settings;
|
2018-11-02 18:53:23 +00:00
|
|
|
|
2020-05-04 20:15:38 +00:00
|
|
|
class VolumeJBOD;
|
|
|
|
using VolumeJBODPtr = std::shared_ptr<VolumeJBOD>;
|
2020-01-19 14:26:28 +00:00
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
class TableJoin
|
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 SyntaxAnalyzer;
|
|
|
|
|
2019-09-09 19:43:37 +00:00
|
|
|
const SizeLimits size_limits;
|
2020-04-08 18:59:52 +00:00
|
|
|
const size_t default_max_bytes = 0;
|
|
|
|
const bool join_use_nulls = false;
|
2020-01-17 11:45:43 +00:00
|
|
|
const size_t max_joined_block_rows = 0;
|
2020-04-08 18:59:52 +00:00
|
|
|
JoinAlgorithm join_algorithm = JoinAlgorithm::AUTO;
|
2019-09-26 18:51:17 +00:00
|
|
|
const bool partial_merge_join_optimizations = false;
|
|
|
|
const size_t partial_merge_join_rows_in_right_blocks = 0;
|
2020-04-28 13:55:50 +00:00
|
|
|
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 key_asts_left;
|
|
|
|
ASTs key_asts_right;
|
2019-09-02 19:58:45 +00:00
|
|
|
ASTTableJoin table_join;
|
2019-10-11 17:56:26 +00:00
|
|
|
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;
|
2019-09-02 19:58:45 +00:00
|
|
|
/// Columns will be added to block by JOIN. It's a subset of columns_from_joined_table with corrected Nullability
|
|
|
|
NamesAndTypesList columns_added_by_join;
|
|
|
|
|
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 ranamed columns.
|
|
|
|
std::unordered_map<String, String> renames;
|
|
|
|
|
2020-05-04 20:15:38 +00:00
|
|
|
VolumeJBODPtr tmp_volume;
|
2019-10-15 16:31:49 +00:00
|
|
|
|
2019-07-30 18:39:37 +00:00
|
|
|
public:
|
2020-04-08 18:59:52 +00:00
|
|
|
TableJoin() = default;
|
2020-05-04 20:15:38 +00:00
|
|
|
TableJoin(const Settings &, VolumeJBODPtr tmp_volume);
|
2019-09-09 19:43:37 +00:00
|
|
|
|
|
|
|
/// for StorageJoin
|
2020-04-07 09:48:47 +00:00
|
|
|
TableJoin(SizeLimits limits, bool use_nulls, ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness,
|
2019-09-09 19:43:37 +00:00
|
|
|
const Names & key_names_right_)
|
|
|
|
: size_limits(limits)
|
2019-10-15 16:31:49 +00:00
|
|
|
, default_max_bytes(0)
|
2019-09-09 19:43:37 +00:00
|
|
|
, join_use_nulls(use_nulls)
|
2020-02-19 19:11:23 +00:00
|
|
|
, join_algorithm(JoinAlgorithm::HASH)
|
2019-09-09 19:43:37 +00:00
|
|
|
, key_names_right(key_names_right_)
|
|
|
|
{
|
|
|
|
table_join.kind = kind;
|
|
|
|
table_join.strictness = strictness;
|
|
|
|
}
|
|
|
|
|
2020-04-08 18:59:52 +00:00
|
|
|
StoragePtr joined_storage;
|
|
|
|
std::shared_ptr<DictionaryReader> dictionary_reader;
|
|
|
|
|
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;
|
2019-09-09 19:43:37 +00:00
|
|
|
const SizeLimits & sizeLimits() const { return size_limits; }
|
2020-05-04 20:15:38 +00:00
|
|
|
VolumeJBODPtr getTemporaryVolume() { return tmp_volume; }
|
2020-02-11 18:27:52 +00:00
|
|
|
bool allowMergeJoin() const;
|
2020-04-09 20:00:57 +00:00
|
|
|
bool allowDictJoin(const String & dict_key, const Block & sample_block, Names &, NamesAndTypesList &) const;
|
2020-02-20 11:26:00 +00:00
|
|
|
bool preferMergeJoin() const { return join_algorithm == JoinAlgorithm::PREFER_PARTIAL_MERGE; }
|
2020-02-19 19:11:23 +00:00
|
|
|
bool forceMergeJoin() const { return join_algorithm == JoinAlgorithm::PARTIAL_MERGE; }
|
|
|
|
bool forceHashJoin() const { return join_algorithm == JoinAlgorithm::HASH; }
|
2019-09-12 14:09:05 +00:00
|
|
|
|
2019-09-18 12:24:35 +00:00
|
|
|
bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(table_join.kind); }
|
|
|
|
bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(table_join.kind); }
|
2019-10-15 16:31:49 +00:00
|
|
|
size_t defaultMaxBytes() const { return default_max_bytes; }
|
2020-01-17 11:45:43 +00:00
|
|
|
size_t maxJoinedBlockRows() const { return max_joined_block_rows; }
|
2019-09-23 19:36:47 +00:00
|
|
|
size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; }
|
2020-04-28 13:55:50 +00:00
|
|
|
size_t maxFilesToMerge() const { return max_files_to_merge; }
|
|
|
|
const String & temporaryFilesCodec() const { return temporary_files_codec; }
|
2019-09-26 18:51:17 +00:00
|
|
|
bool enablePartialMergeJoinOptimizations() const { return partial_merge_join_optimizations; }
|
2019-09-09 19:43:37 +00:00
|
|
|
|
2020-05-20 09:29:23 +00:00
|
|
|
void resetCollected();
|
2019-02-13 15:18:02 +00:00
|
|
|
void addUsingKey(const ASTPtr & ast);
|
|
|
|
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
|
2019-02-06 16:44:47 +00:00
|
|
|
|
2019-09-02 19:58:45 +00:00
|
|
|
bool hasUsing() const { return table_join.using_expression_list != nullptr; }
|
2019-10-29 13:52:05 +00:00
|
|
|
bool hasOn() const { return table_join.on_expression != nullptr; }
|
2019-01-30 15:51:39 +00:00
|
|
|
|
2019-07-30 18:39:37 +00:00
|
|
|
NameSet getQualifiedColumnsSet() const;
|
2019-09-04 16:20:02 +00:00
|
|
|
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);
|
2019-05-13 18:58:15 +00:00
|
|
|
size_t rightKeyInclusion(const String & name) const;
|
2019-09-11 15:57:09 +00:00
|
|
|
NameSet requiredRightKeys() const;
|
2019-09-02 19:58:45 +00:00
|
|
|
|
2020-04-13 17:03:11 +00:00
|
|
|
bool leftBecomeNullable(const DataTypePtr & column_type) const;
|
|
|
|
bool rightBecomeNullable(const DataTypePtr & column_type) const;
|
2019-09-02 19:58:45 +00:00
|
|
|
void addJoinedColumn(const NameAndTypePair & joined_column);
|
|
|
|
void addJoinedColumnsAndCorrectNullability(Block & sample_block) const;
|
|
|
|
|
2019-10-11 17:56:26 +00:00
|
|
|
void setAsofInequality(ASOF::Inequality inequality) { asof_inequality = inequality; }
|
|
|
|
ASOF::Inequality getAsofInequality() { return asof_inequality; }
|
|
|
|
|
2019-09-02 19:58:45 +00:00
|
|
|
ASTPtr leftKeysList() const;
|
|
|
|
ASTPtr rightKeysList() const; /// For ON syntax only
|
|
|
|
|
|
|
|
Names requiredJoinedNames() const;
|
|
|
|
const Names & keyNamesLeft() const { return key_names_left; }
|
2019-09-09 19:43:37 +00:00
|
|
|
const Names & keyNamesRight() const { return key_names_right; }
|
2019-09-02 19:58:45 +00:00
|
|
|
const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; }
|
|
|
|
const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; }
|
|
|
|
|
2019-12-19 20:37:10 +00:00
|
|
|
/// StorageJoin overrides key names (cause of different names qualification)
|
|
|
|
void setRightKeys(const Names & keys) { key_names_right = keys; }
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
static bool sameJoin(const TableJoin * x, const TableJoin * y);
|
2018-11-02 18:53:23 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|