ClickHouse/src/Interpreters/TreeRewriter.h

122 lines
4.5 KiB
C++
Raw Normal View History

2018-11-02 18:53:23 +00:00
#pragma once
2019-10-19 20:36:35 +00:00
#include <Core/Block.h>
2019-09-03 16:56:32 +00:00
#include <Core/NamesAndTypes.h>
2019-01-25 11:43:19 +00:00
#include <Interpreters/Aliases.h>
#include <Interpreters/Context_fwd.h>
2020-03-03 14:25:45 +00:00
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Storages/IStorage_fwd.h>
2018-11-02 18:53:23 +00:00
namespace DB
{
class ASTFunction;
2020-06-25 20:59:10 +00:00
struct ASTTablesInSelectQueryElement;
class TableJoin;
struct Settings;
2019-09-03 16:56:32 +00:00
struct SelectQueryOptions;
2019-10-19 20:36:35 +00:00
using Scalars = std::map<String, Block>;
struct StorageInMemoryMetadata;
2020-06-18 11:02:31 +00:00
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
struct TreeRewriterResult
2018-11-02 18:53:23 +00:00
{
ConstStoragePtr storage;
StorageMetadataPtr metadata_snapshot;
std::shared_ptr<TableJoin> analyzed_join;
2020-06-25 20:59:10 +00:00
const ASTTablesInSelectQueryElement * ast_join = nullptr;
2018-11-02 18:53:23 +00:00
NamesAndTypesList source_columns;
2020-03-03 14:25:45 +00:00
NameSet source_columns_set; /// Set of names of source_columns.
/// Set of columns that are enough to read from the table to evaluate the expression. It does not include joined columns.
NamesAndTypesList required_source_columns;
2018-11-02 18:53:23 +00:00
Aliases aliases;
std::vector<const ASTFunction *> aggregates;
2018-11-02 18:53:23 +00:00
2020-12-22 01:37:45 +00:00
std::vector<const ASTFunction *> window_function_asts;
2020-12-09 11:14:40 +00:00
2018-11-02 18:53:23 +00:00
/// Which column is needed to be ARRAY-JOIN'ed to get the specified.
/// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v".
NameToNameMap array_join_result_to_source;
/// For the ARRAY JOIN section, mapping from the alias to the full column name.
/// For example, for `ARRAY JOIN [1,2] AS b` "b" -> "array(1,2)" will enter here.
/// Note: not used further.
2018-11-02 18:53:23 +00:00
NameToNameMap array_join_alias_to_name;
/// The backward mapping for array_join_alias_to_name.
/// Note: not used further.
2018-11-02 18:53:23 +00:00
NameToNameMap array_join_name_to_alias;
/// Predicate optimizer overrides the sub queries
bool rewrite_subqueries = false;
/// Whether the query contains explicit columns like "SELECT column1 + column2 FROM table1".
/// Queries like "SELECT count() FROM table1", "SELECT 1" don't contain explicit columns.
bool has_explicit_columns = false;
/// Whether it's possible to use the trivial count optimization,
/// i.e. use a fast call of IStorage::totalRows() (or IStorage::totalRowsByPartitionPredicate())
/// instead of actual retrieving columns and counting rows.
2020-06-04 22:01:40 +00:00
bool optimize_trivial_count = false;
/// Cache isRemote() call for storage, because it may be too heavy.
bool is_remote_storage = false;
2019-10-19 20:36:35 +00:00
/// Results of scalar sub queries
Scalars scalars;
TreeRewriterResult(
const NamesAndTypesList & source_columns_,
ConstStoragePtr storage_ = {},
const StorageMetadataPtr & metadata_snapshot_ = {},
bool add_special = true);
2020-03-03 14:25:45 +00:00
2020-04-29 16:26:01 +00:00
void collectSourceColumns(bool add_special);
2020-06-04 22:01:40 +00:00
void collectUsedColumns(const ASTPtr & query, bool is_select);
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
2020-12-12 16:42:15 +00:00
NameSet getArrayJoinSourceNameSet() const;
2019-10-19 20:36:35 +00:00
const Scalars & getScalars() const { return scalars; }
2018-11-02 18:53:23 +00:00
};
using TreeRewriterResultPtr = std::shared_ptr<const TreeRewriterResult>;
/// Tree Rewriter in terms of CMU slides @sa https://15721.courses.cs.cmu.edu/spring2020/slides/19-optimizer1.pdf
///
/// Optimises AST tree and collect information for further expression analysis in ExpressionAnalyzer.
2018-11-08 11:17:31 +00:00
/// Result AST has the following invariants:
/// * all aliases are substituted
/// * qualified names are translated
/// * scalar subqueries are executed replaced with constants
/// * unneeded columns are removed from SELECT clause
/// * duplicated columns are removed from ORDER BY, LIMIT BY, USING(...).
class TreeRewriter : WithConstContext
2018-11-02 18:53:23 +00:00
{
public:
explicit TreeRewriter(ContextConstPtr context_) : WithConstContext(context_) {}
2018-11-08 11:17:31 +00:00
/// Analyze and rewrite not select query
TreeRewriterResultPtr analyze(
ASTPtr & query,
const NamesAndTypesList & source_columns_,
ConstStoragePtr storage = {},
const StorageMetadataPtr & metadata_snapshot = {},
bool allow_aggregations = false) const;
/// Analyze and rewrite select query
TreeRewriterResultPtr analyzeSelect(
ASTPtr & query,
TreeRewriterResult && result,
const SelectQueryOptions & select_options = {},
2020-03-03 14:25:45 +00:00
const std::vector<TableWithColumnNamesAndTypes> & tables_with_columns = {},
2020-04-08 18:59:52 +00:00
const Names & required_result_columns = {},
std::shared_ptr<TableJoin> table_join = {}) const;
2018-11-08 11:17:31 +00:00
2019-01-09 16:16:59 +00:00
private:
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings);
2018-11-02 18:53:23 +00:00
};
}