ClickHouse/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.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

84 lines
2.2 KiB
C++

#pragma once
#include <Parsers/IAST.h>
#include <Storages/MergeTree/MergeTreeDataFormatVersion.h>
#include <common/types.h>
#include <Storages/StorageInMemoryMetadata.h>
namespace DB
{
class MergeTreeData;
class WriteBuffer;
class ReadBuffer;
/** The basic parameters of ReplicatedMergeTree table engine for saving in ZooKeeper.
* Lets you verify that they match local ones.
*/
struct ReplicatedMergeTreeTableMetadata
{
String date_column;
String sampling_expression;
UInt64 index_granularity;
int merging_params_mode;
String sign_column;
String primary_key;
MergeTreeDataFormatVersion data_format_version;
String partition_key;
String sorting_key;
String skip_indices;
String projections;
String constraints;
String ttl_table;
UInt64 index_granularity_bytes;
ReplicatedMergeTreeTableMetadata() = default;
explicit ReplicatedMergeTreeTableMetadata(const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot);
void read(ReadBuffer & in);
static ReplicatedMergeTreeTableMetadata parse(const String & s);
void write(WriteBuffer & out) const;
String toString() const;
struct Diff
{
bool sorting_key_changed = false;
String new_sorting_key;
bool sampling_expression_changed = false;
String new_sampling_expression;
bool skip_indices_changed = false;
String new_skip_indices;
bool constraints_changed = false;
String new_constraints;
bool projections_changed = false;
String new_projections;
bool ttl_table_changed = false;
String new_ttl_table;
bool empty() const
{
return !sorting_key_changed && !sampling_expression_changed && !skip_indices_changed && !projections_changed
&& !ttl_table_changed && !constraints_changed;
}
};
void checkEquals(const ReplicatedMergeTreeTableMetadata & from_zk, const ColumnsDescription & columns, ContextPtr context) const;
Diff checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk) const;
private:
void checkImmutableFieldsEquals(const ReplicatedMergeTreeTableMetadata & from_zk) const;
bool index_granularity_bytes_found_in_zk = false;
};
}