ClickHouse/src/Interpreters/TreeRewriter.h
Amos Bird 264cff6415
Projections
TODO (suggested by Nikolai)

1. Build query plan fro current query (inside storage::read) up to WithMergableState
2. Check, that plan is simple enough: Aggregating - Expression - Filter - ReadFromStorage (or simplier)
3. Check, that filter is the same as filter in projection, and also expression calculates the same aggregation keys as in projection
4. Return WithMergableState if projection applies

3 will be easier to do with ActionsDAG, cause it sees all functions, and dependencies are direct (but it is possible with ExpressionActions also)

Also need to figure out how prewhere works for projections, and
row_filter_policies.

wip
2021-05-11 18:12:23 +08:00

124 lines
4.6 KiB
C++

#pragma once
#include <Core/Block.h>
#include <Core/NamesAndTypes.h>
#include <Interpreters/Aliases.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Storages/IStorage_fwd.h>
namespace DB
{
class ASTFunction;
struct ASTTablesInSelectQueryElement;
class TableJoin;
struct Settings;
struct SelectQueryOptions;
using Scalars = std::map<String, Block>;
struct StorageInMemoryMetadata;
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
struct TreeRewriterResult
{
ConstStoragePtr storage;
StorageMetadataPtr metadata_snapshot;
std::shared_ptr<TableJoin> analyzed_join;
const ASTTablesInSelectQueryElement * ast_join = nullptr;
NamesAndTypesList source_columns;
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;
Aliases aliases;
std::vector<const ASTFunction *> aggregates;
std::vector<const ASTFunction *> window_function_asts;
/// 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.
NameToNameMap array_join_alias_to_name;
/// The backward mapping for array_join_alias_to_name.
/// Note: not used further.
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.
bool optimize_trivial_count = false;
bool can_use_projection = false;
/// Cache isRemote() call for storage, because it may be too heavy.
bool is_remote_storage = false;
/// Results of scalar sub queries
Scalars scalars;
TreeRewriterResult(
const NamesAndTypesList & source_columns_,
ConstStoragePtr storage_ = {},
const StorageMetadataPtr & metadata_snapshot_ = {},
bool add_special = true);
void collectSourceColumns(bool add_special);
void collectUsedColumns(const ASTPtr & query, bool is_select);
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
NameSet getArrayJoinSourceNameSet() const;
const Scalars & getScalars() const { return scalars; }
};
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.
/// 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 : WithContext
{
public:
explicit TreeRewriter(ContextPtr context_) : WithContext(context_) {}
/// 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 = {},
const std::vector<TableWithColumnNamesAndTypes> & tables_with_columns = {},
const Names & required_result_columns = {},
std::shared_ptr<TableJoin> table_join = {}) const;
private:
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings);
};
}