ClickHouse/src/Storages/ProjectionsDescription.h

113 lines
3.8 KiB
C++
Raw Normal View History

#pragma once
#include <Core/Types.h>
#include <memory>
#include <vector>
#include <Interpreters/ExpressionActions.h>
#include <Parsers/IAST_fwd.h>
#include <Storages/ColumnsDescription.h>
#include <boost/multi_index/member.hpp>
#include <boost/multi_index/ordered_index.hpp>
#include <boost/multi_index/sequenced_index.hpp>
#include <boost/multi_index_container.hpp>
namespace DB
{
struct StorageInMemoryMetadata;
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
/// Description of projections for Storage
struct ProjectionDescription
{
/// Definition AST of projection
ASTPtr definition_ast;
/// Subquery AST for projection calculation
ASTPtr query_ast;
/// Projection name
String name;
/// Projection type (normal, aggregate, etc.)
String type = "normal";
Names required_columns;
Names getRequiredColumns() const { return required_columns; }
/// Names of projection columns (not to be confused with required columns)
Names column_names;
/// Data types of projection columns
DataTypes data_types;
/// Sample block with projection columns. (NOTE: columns in block are empty, but not nullptr)
Block sample_block;
StorageMetadataPtr metadata;
size_t key_size = 0;
/// Parse projection from definition AST
static ProjectionDescription
getProjectionFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr query_context);
ProjectionDescription() = default;
/// We need custom copy constructors because we don't want
/// unintentionaly share AST variables and modify them.
ProjectionDescription(const ProjectionDescription & other);
ProjectionDescription & operator=(const ProjectionDescription & other);
bool operator==(const ProjectionDescription & other) const;
bool operator!=(const ProjectionDescription & other) const { return !(*this == other); }
/// Recalculate projection with new columns because projection expression may change
/// if something change in columns.
void recalculateWithNewColumns(const ColumnsDescription & new_columns, ContextPtr query_context);
bool isPrimaryKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const;
};
/// All projections in storage
struct ProjectionsDescription
{
/// Convert description to string
String toString() const;
/// Parse description from string
static ProjectionsDescription parse(const String & str, const ColumnsDescription & columns, ContextPtr query_context);
/// Return common expression for all stored projections
ExpressionActionsPtr getSingleExpressionForProjections(const ColumnsDescription & columns, ContextPtr query_context) const;
bool operator==(const ProjectionsDescription & other) const { return projections == other.projections; }
bool operator!=(const ProjectionsDescription & other) const { return !(*this == other); }
auto begin() const { return projections.begin(); }
auto end() const { return projections.end(); }
size_t size() const { return projections.size(); }
bool empty() const { return projections.empty(); }
bool has(const String & projection_name) const;
const ProjectionDescription & get(const String & projection_name) const;
void
add(ProjectionDescription && projection, const String & after_projection = String(), bool first = false, bool if_not_exists = false);
void remove(const String & projection_name);
private:
/// Keep the sequence of columns and allow to lookup by name.
using Container = boost::multi_index_container<
ProjectionDescription,
boost::multi_index::indexed_by<
boost::multi_index::sequenced<>,
boost::multi_index::ordered_unique<boost::multi_index::member<ProjectionDescription, String, &ProjectionDescription::name>>>>;
Container projections;
};
}