Merge pull request #59357 from hanfei1991/hanfei/stats_uniq

Introduce statistics of type "number of distinct values"
This commit is contained in:
Han Fei 2024-06-05 12:56:52 +00:00 committed by GitHub
commit ac430bb01d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
90 changed files with 1511 additions and 897 deletions

View File

@ -39,8 +39,8 @@ If you need to update rows frequently, we recommend using the [`ReplacingMergeTr
``` sql ``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
( (
name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY] [SETTINGS (name = value, ...)], name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTICS(stat1)] [TTL expr1] [PRIMARY KEY] [SETTINGS (name = value, ...)],
name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY] [SETTINGS (name = value, ...)], name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTICS(stat2)] [TTL expr2] [PRIMARY KEY] [SETTINGS (name = value, ...)],
... ...
INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1], INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1],
INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2], INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2],
@ -1043,12 +1043,12 @@ ClickHouse versions 22.3 through 22.7 use a different cache configuration, see [
## Column Statistics (Experimental) {#column-statistics} ## Column Statistics (Experimental) {#column-statistics}
The statistic declaration is in the columns section of the `CREATE` query for tables from the `*MergeTree*` Family when we enable `set allow_experimental_statistic = 1`. The statistics declaration is in the columns section of the `CREATE` query for tables from the `*MergeTree*` Family when we enable `set allow_experimental_statistics = 1`.
``` sql ``` sql
CREATE TABLE tab CREATE TABLE tab
( (
a Int64 STATISTIC(tdigest), a Int64 STATISTICS(TDigest, Uniq),
b Float64 b Float64
) )
ENGINE = MergeTree ENGINE = MergeTree
@ -1058,19 +1058,23 @@ ORDER BY a
We can also manipulate statistics with `ALTER` statements. We can also manipulate statistics with `ALTER` statements.
```sql ```sql
ALTER TABLE tab ADD STATISTIC b TYPE tdigest; ALTER TABLE tab ADD STATISTICS b TYPE TDigest, Uniq;
ALTER TABLE tab DROP STATISTIC a TYPE tdigest; ALTER TABLE tab DROP STATISTICS a;
``` ```
These lightweight statistics aggregate information about distribution of values in columns. These lightweight statistics aggregate information about distribution of values in columns. Statistics are stored in every part and updated when every insert comes.
They can be used for query optimization when we enable `set allow_statistic_optimize = 1`. They can be used for prewhere optimization only if we enable `set allow_statistics_optimize = 1`.
#### Available Types of Column Statistics {#available-types-of-column-statistics} #### Available Types of Column Statistics {#available-types-of-column-statistics}
- `tdigest` - `TDigest`
Stores distribution of values from numeric columns in [TDigest](https://github.com/tdunning/t-digest) sketch. Stores distribution of values from numeric columns in [TDigest](https://github.com/tdunning/t-digest) sketch.
- `Uniq`
Estimate the number of distinct values of a column by HyperLogLog.
## Column-level Settings {#column-level-settings} ## Column-level Settings {#column-level-settings}
Certain MergeTree settings can be override at column level: Certain MergeTree settings can be override at column level:

View File

@ -5108,7 +5108,7 @@ a Tuple(
) )
``` ```
## allow_experimental_statistic {#allow_experimental_statistic} ## allow_experimental_statistics {#allow_experimental_statistics}
Allows defining columns with [statistics](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) and [manipulate statistics](../../engines/table-engines/mergetree-family/mergetree.md#column-statistics). Allows defining columns with [statistics](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) and [manipulate statistics](../../engines/table-engines/mergetree-family/mergetree.md#column-statistics).

View File

@ -16,7 +16,7 @@ Most `ALTER TABLE` queries modify table settings or data:
- [INDEX](/docs/en/sql-reference/statements/alter/skipping-index.md) - [INDEX](/docs/en/sql-reference/statements/alter/skipping-index.md)
- [CONSTRAINT](/docs/en/sql-reference/statements/alter/constraint.md) - [CONSTRAINT](/docs/en/sql-reference/statements/alter/constraint.md)
- [TTL](/docs/en/sql-reference/statements/alter/ttl.md) - [TTL](/docs/en/sql-reference/statements/alter/ttl.md)
- [STATISTIC](/docs/en/sql-reference/statements/alter/statistic.md) - [STATISTICS](/docs/en/sql-reference/statements/alter/statistics.md)
- [APPLY DELETED MASK](/docs/en/sql-reference/statements/alter/apply-deleted-mask.md) - [APPLY DELETED MASK](/docs/en/sql-reference/statements/alter/apply-deleted-mask.md)
:::note :::note

View File

@ -1,25 +0,0 @@
---
slug: /en/sql-reference/statements/alter/statistic
sidebar_position: 45
sidebar_label: STATISTIC
---
# Manipulating Column Statistics
The following operations are available:
- `ALTER TABLE [db].table ADD STATISTIC (columns list) TYPE type` - Adds statistic description to tables metadata.
- `ALTER TABLE [db].table DROP STATISTIC (columns list) TYPE type` - Removes statistic description from tables metadata and deletes statistic files from disk.
- `ALTER TABLE [db].table CLEAR STATISTIC (columns list) TYPE type` - Deletes statistic files from disk.
- `ALTER TABLE [db.]table MATERIALIZE STATISTIC (columns list) TYPE type` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
The first two commands are lightweight in a sense that they only change metadata or remove files.
Also, they are replicated, syncing statistics metadata via ZooKeeper.
:::note
Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants).
:::

View File

@ -0,0 +1,33 @@
---
slug: /en/sql-reference/statements/alter/statistics
sidebar_position: 45
sidebar_label: STATISTICS
---
# Manipulating Column Statistics
The following operations are available:
- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata.
- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata.
- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns.
- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`.
- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
The first two commands are lightweight in a sense that they only change metadata or remove files.
Also, they are replicated, syncing statistics metadata via ZooKeeper.
There is an example adding two statistics types to two columns:
```
ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq;
```
:::note
Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants).
:::

View File

@ -51,10 +51,11 @@ enum class AccessType : uint8_t
M(ALTER_CLEAR_INDEX, "CLEAR INDEX", TABLE, ALTER_INDEX) \ M(ALTER_CLEAR_INDEX, "CLEAR INDEX", TABLE, ALTER_INDEX) \
M(ALTER_INDEX, "INDEX", GROUP, ALTER_TABLE) /* allows to execute ALTER ORDER BY or ALTER {ADD|DROP...} INDEX */\ M(ALTER_INDEX, "INDEX", GROUP, ALTER_TABLE) /* allows to execute ALTER ORDER BY or ALTER {ADD|DROP...} INDEX */\
\ \
M(ALTER_ADD_STATISTIC, "ALTER ADD STATISTIC", TABLE, ALTER_STATISTIC) \ M(ALTER_ADD_STATISTICS, "ALTER ADD STATISTIC", TABLE, ALTER_STATISTICS) \
M(ALTER_DROP_STATISTIC, "ALTER DROP STATISTIC", TABLE, ALTER_STATISTIC) \ M(ALTER_DROP_STATISTICS, "ALTER DROP STATISTIC", TABLE, ALTER_STATISTICS) \
M(ALTER_MATERIALIZE_STATISTIC, "ALTER MATERIALIZE STATISTIC", TABLE, ALTER_STATISTIC) \ M(ALTER_MODIFY_STATISTICS, "ALTER MODIFY STATISTIC", TABLE, ALTER_STATISTICS) \
M(ALTER_STATISTIC, "STATISTIC", GROUP, ALTER_TABLE) /* allows to execute ALTER STATISTIC */\ M(ALTER_MATERIALIZE_STATISTICS, "ALTER MATERIALIZE STATISTIC", TABLE, ALTER_STATISTICS) \
M(ALTER_STATISTICS, "STATISTIC", GROUP, ALTER_TABLE) /* allows to execute ALTER STATISTIC */\
\ \
M(ALTER_ADD_PROJECTION, "ADD PROJECTION", TABLE, ALTER_PROJECTION) \ M(ALTER_ADD_PROJECTION, "ADD PROJECTION", TABLE, ALTER_PROJECTION) \
M(ALTER_DROP_PROJECTION, "DROP PROJECTION", TABLE, ALTER_PROJECTION) \ M(ALTER_DROP_PROJECTION, "DROP PROJECTION", TABLE, ALTER_PROJECTION) \

View File

@ -334,6 +334,18 @@ public:
compress(); // Allows reading/writing TDigests with different epsilon/max_centroids params compress(); // Allows reading/writing TDigests with different epsilon/max_centroids params
} }
Float64 getCountEqual(Float64 value) const
{
Float64 result = 0;
for (const auto & c : centroids)
{
/// std::cerr << "c "<< c.mean << " "<< c.count << std::endl;
if (value == c.mean)
result += c.count;
}
return result;
}
Float64 getCountLessThan(Float64 value) const Float64 getCountLessThan(Float64 value) const
{ {
bool first = true; bool first = true;

View File

@ -586,7 +586,7 @@
M(705, TABLE_NOT_EMPTY) \ M(705, TABLE_NOT_EMPTY) \
M(706, LIBSSH_ERROR) \ M(706, LIBSSH_ERROR) \
M(707, GCP_ERROR) \ M(707, GCP_ERROR) \
M(708, ILLEGAL_STATISTIC) \ M(708, ILLEGAL_STATISTICS) \
M(709, CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT) \ M(709, CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT) \
M(710, FAULT_INJECTED) \ M(710, FAULT_INJECTED) \
M(711, FILECACHE_ACCESS_DENIED) \ M(711, FILECACHE_ACCESS_DENIED) \

View File

@ -160,8 +160,8 @@ class IColumn;
M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \
M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \ M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \
\ \
M(Bool, allow_statistic_optimize, false, "Allows using statistic to optimize queries", 0) \ M(Bool, allow_statistics_optimize, false, "Allows using statistics to optimize queries", 0) \
M(Bool, allow_experimental_statistic, false, "Allows using statistic", 0) \ M(Bool, allow_experimental_statistics, false, "Allows using statistics", 0) \
\ \
M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \
M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \

View File

@ -175,11 +175,11 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
else else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong parameter type in ALTER query"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong parameter type in ALTER query");
if (!getContext()->getSettings().allow_experimental_statistic && ( if (!getContext()->getSettings().allow_experimental_statistics && (
command_ast->type == ASTAlterCommand::ADD_STATISTIC || command_ast->type == ASTAlterCommand::ADD_STATISTICS ||
command_ast->type == ASTAlterCommand::DROP_STATISTIC || command_ast->type == ASTAlterCommand::DROP_STATISTICS ||
command_ast->type == ASTAlterCommand::MATERIALIZE_STATISTIC)) command_ast->type == ASTAlterCommand::MATERIALIZE_STATISTICS))
throw Exception(ErrorCodes::INCORRECT_QUERY, "Alter table with statistic is now disabled. Turn on allow_experimental_statistic"); throw Exception(ErrorCodes::INCORRECT_QUERY, "Alter table with statistics is now disabled. Turn on allow_experimental_statistics");
} }
if (typeid_cast<DatabaseReplicated *>(database.get())) if (typeid_cast<DatabaseReplicated *>(database.get()))
@ -343,19 +343,24 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS
required_access.emplace_back(AccessType::ALTER_SAMPLE_BY, database, table); required_access.emplace_back(AccessType::ALTER_SAMPLE_BY, database, table);
break; break;
} }
case ASTAlterCommand::ADD_STATISTIC: case ASTAlterCommand::ADD_STATISTICS:
{ {
required_access.emplace_back(AccessType::ALTER_ADD_STATISTIC, database, table); required_access.emplace_back(AccessType::ALTER_ADD_STATISTICS, database, table);
break; break;
} }
case ASTAlterCommand::DROP_STATISTIC: case ASTAlterCommand::MODIFY_STATISTICS:
{ {
required_access.emplace_back(AccessType::ALTER_DROP_STATISTIC, database, table); required_access.emplace_back(AccessType::ALTER_MODIFY_STATISTICS, database, table);
break; break;
} }
case ASTAlterCommand::MATERIALIZE_STATISTIC: case ASTAlterCommand::DROP_STATISTICS:
{ {
required_access.emplace_back(AccessType::ALTER_MATERIALIZE_STATISTIC, database, table); required_access.emplace_back(AccessType::ALTER_DROP_STATISTICS, database, table);
break;
}
case ASTAlterCommand::MATERIALIZE_STATISTICS:
{
required_access.emplace_back(AccessType::ALTER_MATERIALIZE_STATISTICS, database, table);
break; break;
} }
case ASTAlterCommand::ADD_INDEX: case ASTAlterCommand::ADD_INDEX:

View File

@ -448,9 +448,9 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
column_declaration->children.push_back(column_declaration->codec); column_declaration->children.push_back(column_declaration->codec);
} }
if (column.stat) if (!column.statistics.empty())
{ {
column_declaration->stat_type = column.stat->ast; column_declaration->stat_type = column.statistics.getAST();
column_declaration->children.push_back(column_declaration->stat_type); column_declaration->children.push_back(column_declaration->stat_type);
} }
@ -675,11 +675,13 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec, enable_zstd_qat_codec); col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec, enable_zstd_qat_codec);
} }
column.statistics.column_name = column.name; /// We assign column name here for better exception error message.
if (col_decl.stat_type) if (col_decl.stat_type)
{ {
if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistic) if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistic is now disabled. Turn on allow_experimental_statistic"); throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistics is now disabled. Turn on allow_experimental_statistics");
column.stat = StatisticDescription::getStatisticFromColumnDeclaration(col_decl); column.statistics = ColumnStatisticsDescription::fromColumnDeclaration(col_decl);
column.statistics.data_type = column.type;
} }
if (col_decl.ttl) if (col_decl.ttl)

View File

@ -67,8 +67,8 @@ namespace
static void visit(ASTSelectQuery & select, ASTPtr & node, Data & data) static void visit(ASTSelectQuery & select, ASTPtr & node, Data & data)
{ {
/// we need to read statistic when `allow_statistic_optimize` is enabled. /// we need to read statistic when `allow_statistics_optimize` is enabled.
bool only_analyze = !data.getContext()->getSettings().allow_statistic_optimize; bool only_analyze = !data.getContext()->getSettings().allow_statistics_optimize;
InterpreterSelectQuery interpreter( InterpreterSelectQuery interpreter(
node, data.getContext(), SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze(only_analyze).modify()); node, data.getContext(), SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze(only_analyze).modify());

View File

@ -657,7 +657,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
MergeTreeWhereOptimizer where_optimizer{ MergeTreeWhereOptimizer where_optimizer{
std::move(column_compressed_sizes), std::move(column_compressed_sizes),
metadata_snapshot, metadata_snapshot,
storage->getConditionEstimatorByPredicate(storage_snapshot, nullptr, context), storage->getConditionSelectivityEstimatorByPredicate(storage_snapshot, nullptr, context),
queried_columns, queried_columns,
supported_prewhere_columns, supported_prewhere_columns,
log}; log};

View File

@ -55,7 +55,7 @@ namespace ErrorCodes
extern const int CANNOT_UPDATE_COLUMN; extern const int CANNOT_UPDATE_COLUMN;
extern const int UNEXPECTED_EXPRESSION; extern const int UNEXPECTED_EXPRESSION;
extern const int THERE_IS_NO_COLUMN; extern const int THERE_IS_NO_COLUMN;
extern const int ILLEGAL_STATISTIC; extern const int ILLEGAL_STATISTICS;
} }
@ -781,7 +781,7 @@ void MutationsInterpreter::prepare(bool dry_run)
} }
else if (command.type == MutationCommand::MATERIALIZE_INDEX) else if (command.type == MutationCommand::MATERIALIZE_INDEX)
{ {
mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTICS_PROJECTION);
auto it = std::find_if( auto it = std::find_if(
std::cbegin(indices_desc), std::end(indices_desc), std::cbegin(indices_desc), std::end(indices_desc),
[&](const IndexDescription & index) [&](const IndexDescription & index)
@ -801,20 +801,20 @@ void MutationsInterpreter::prepare(bool dry_run)
materialized_indices.emplace(command.index_name); materialized_indices.emplace(command.index_name);
} }
} }
else if (command.type == MutationCommand::MATERIALIZE_STATISTIC) else if (command.type == MutationCommand::MATERIALIZE_STATISTICS)
{ {
mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTICS_PROJECTION);
for (const auto & stat_column_name: command.statistic_columns) for (const auto & stat_column_name: command.statistics_columns)
{ {
if (!columns_desc.has(stat_column_name) || !columns_desc.get(stat_column_name).stat) if (!columns_desc.has(stat_column_name) || columns_desc.get(stat_column_name).statistics.empty())
throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Unknown statistic column: {}", stat_column_name); throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Unknown statistics column: {}", stat_column_name);
dependencies.emplace(stat_column_name, ColumnDependency::STATISTIC); dependencies.emplace(stat_column_name, ColumnDependency::STATISTICS);
materialized_statistics.emplace(stat_column_name); materialized_statistics.emplace(stat_column_name);
} }
} }
else if (command.type == MutationCommand::MATERIALIZE_PROJECTION) else if (command.type == MutationCommand::MATERIALIZE_PROJECTION)
{ {
mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTICS_PROJECTION);
const auto & projection = projections_desc.get(command.projection_name); const auto & projection = projections_desc.get(command.projection_name);
if (!source.hasProjection(projection.name) || source.hasBrokenProjection(projection.name)) if (!source.hasProjection(projection.name) || source.hasBrokenProjection(projection.name))
{ {
@ -825,18 +825,18 @@ void MutationsInterpreter::prepare(bool dry_run)
} }
else if (command.type == MutationCommand::DROP_INDEX) else if (command.type == MutationCommand::DROP_INDEX)
{ {
mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTICS_PROJECTION);
materialized_indices.erase(command.index_name); materialized_indices.erase(command.index_name);
} }
else if (command.type == MutationCommand::DROP_STATISTIC) else if (command.type == MutationCommand::DROP_STATISTICS)
{ {
mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTICS_PROJECTION);
for (const auto & stat_column_name: command.statistic_columns) for (const auto & stat_column_name: command.statistics_columns)
materialized_statistics.erase(stat_column_name); materialized_statistics.erase(stat_column_name);
} }
else if (command.type == MutationCommand::DROP_PROJECTION) else if (command.type == MutationCommand::DROP_PROJECTION)
{ {
mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTICS_PROJECTION);
materialized_projections.erase(command.projection_name); materialized_projections.erase(command.projection_name);
} }
else if (command.type == MutationCommand::MATERIALIZE_TTL) else if (command.type == MutationCommand::MATERIALIZE_TTL)
@ -888,7 +888,7 @@ void MutationsInterpreter::prepare(bool dry_run)
{ {
if (dependency.kind == ColumnDependency::SKIP_INDEX if (dependency.kind == ColumnDependency::SKIP_INDEX
|| dependency.kind == ColumnDependency::PROJECTION || dependency.kind == ColumnDependency::PROJECTION
|| dependency.kind == ColumnDependency::STATISTIC) || dependency.kind == ColumnDependency::STATISTICS)
dependencies.insert(dependency); dependencies.insert(dependency);
} }
} }
@ -1360,7 +1360,7 @@ QueryPipelineBuilder MutationsInterpreter::execute()
Block MutationsInterpreter::getUpdatedHeader() const Block MutationsInterpreter::getUpdatedHeader() const
{ {
// If it's an index/projection materialization, we don't write any data columns, thus empty header is used // If it's an index/projection materialization, we don't write any data columns, thus empty header is used
return mutation_kind.mutation_kind == MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION ? Block{} : *updated_header; return mutation_kind.mutation_kind == MutationKind::MUTATE_INDEX_STATISTICS_PROJECTION ? Block{} : *updated_header;
} }
const ColumnDependencies & MutationsInterpreter::getColumnDependencies() const const ColumnDependencies & MutationsInterpreter::getColumnDependencies() const

View File

@ -102,7 +102,7 @@ public:
enum MutationKindEnum enum MutationKindEnum
{ {
MUTATE_UNKNOWN, MUTATE_UNKNOWN,
MUTATE_INDEX_STATISTIC_PROJECTION, MUTATE_INDEX_STATISTICS_PROJECTION,
MUTATE_OTHER, MUTATE_OTHER,
} mutation_kind = MUTATE_UNKNOWN; } mutation_kind = MUTATE_UNKNOWN;

View File

@ -42,8 +42,8 @@ ASTPtr ASTAlterCommand::clone() const
res->projection_decl = res->children.emplace_back(projection_decl->clone()).get(); res->projection_decl = res->children.emplace_back(projection_decl->clone()).get();
if (projection) if (projection)
res->projection = res->children.emplace_back(projection->clone()).get(); res->projection = res->children.emplace_back(projection->clone()).get();
if (statistic_decl) if (statistics_decl)
res->statistic_decl = res->children.emplace_back(statistic_decl->clone()).get(); res->statistics_decl = res->children.emplace_back(statistics_decl->clone()).get();
if (partition) if (partition)
res->partition = res->children.emplace_back(partition->clone()).get(); res->partition = res->children.emplace_back(partition->clone()).get();
if (predicate) if (predicate)
@ -200,27 +200,33 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState &
partition->formatImpl(settings, state, frame); partition->formatImpl(settings, state, frame);
} }
} }
else if (type == ASTAlterCommand::ADD_STATISTIC) else if (type == ASTAlterCommand::ADD_STATISTICS)
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD STATISTIC " << (if_not_exists ? "IF NOT EXISTS " : "") settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD STATISTICS " << (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : ""); << (settings.hilite ? hilite_none : "");
statistic_decl->formatImpl(settings, state, frame); statistics_decl->formatImpl(settings, state, frame);
} }
else if (type == ASTAlterCommand::DROP_STATISTIC) else if (type == ASTAlterCommand::MODIFY_STATISTICS)
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_statistic ? "CLEAR " : "DROP ") << "STATISTIC " settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY STATISTICS "
<< (settings.hilite ? hilite_none : "");
statistics_decl->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::DROP_STATISTICS)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_statistics ? "CLEAR " : "DROP ") << "STATISTICS "
<< (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
statistic_decl->formatImpl(settings, state, frame); statistics_decl->formatImpl(settings, state, frame);
if (partition) if (partition)
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
partition->formatImpl(settings, state, frame); partition->formatImpl(settings, state, frame);
} }
} }
else if (type == ASTAlterCommand::MATERIALIZE_STATISTIC) else if (type == ASTAlterCommand::MATERIALIZE_STATISTICS)
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE STATISTIC " << (settings.hilite ? hilite_none : ""); settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE STATISTICS " << (settings.hilite ? hilite_none : "");
statistic_decl->formatImpl(settings, state, frame); statistics_decl->formatImpl(settings, state, frame);
if (partition) if (partition)
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : "");
@ -507,7 +513,7 @@ void ASTAlterCommand::forEachPointerToChild(std::function<void(void**)> f)
f(reinterpret_cast<void **>(&constraint)); f(reinterpret_cast<void **>(&constraint));
f(reinterpret_cast<void **>(&projection_decl)); f(reinterpret_cast<void **>(&projection_decl));
f(reinterpret_cast<void **>(&projection)); f(reinterpret_cast<void **>(&projection));
f(reinterpret_cast<void **>(&statistic_decl)); f(reinterpret_cast<void **>(&statistics_decl));
f(reinterpret_cast<void **>(&partition)); f(reinterpret_cast<void **>(&partition));
f(reinterpret_cast<void **>(&predicate)); f(reinterpret_cast<void **>(&predicate));
f(reinterpret_cast<void **>(&update_assignments)); f(reinterpret_cast<void **>(&update_assignments));

View File

@ -55,9 +55,10 @@ public:
DROP_PROJECTION, DROP_PROJECTION,
MATERIALIZE_PROJECTION, MATERIALIZE_PROJECTION,
ADD_STATISTIC, ADD_STATISTICS,
DROP_STATISTIC, DROP_STATISTICS,
MATERIALIZE_STATISTIC, MODIFY_STATISTICS,
MATERIALIZE_STATISTICS,
DROP_PARTITION, DROP_PARTITION,
DROP_DETACHED_PARTITION, DROP_DETACHED_PARTITION,
@ -135,7 +136,7 @@ public:
*/ */
IAST * projection = nullptr; IAST * projection = nullptr;
IAST * statistic_decl = nullptr; IAST * statistics_decl = nullptr;
/** Used in DROP PARTITION, ATTACH PARTITION FROM, FORGET PARTITION, UPDATE, DELETE queries. /** Used in DROP PARTITION, ATTACH PARTITION FROM, FORGET PARTITION, UPDATE, DELETE queries.
* The value or ID of the partition is stored here. * The value or ID of the partition is stored here.
@ -180,7 +181,7 @@ public:
bool clear_index = false; /// for CLEAR INDEX (do not drop index from metadata) bool clear_index = false; /// for CLEAR INDEX (do not drop index from metadata)
bool clear_statistic = false; /// for CLEAR STATISTIC (do not drop statistic from metadata) bool clear_statistics = false; /// for CLEAR STATISTICS (do not drop statistics from metadata)
bool clear_projection = false; /// for CLEAR PROJECTION (do not drop projection from metadata) bool clear_projection = false; /// for CLEAR PROJECTION (do not drop projection from metadata)

View File

@ -1,42 +0,0 @@
#include <Parsers/ASTStatisticDeclaration.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
#include <Parsers/ASTFunction.h>
namespace DB
{
ASTPtr ASTStatisticDeclaration::clone() const
{
auto res = std::make_shared<ASTStatisticDeclaration>();
res->set(res->columns, columns->clone());
res->type = type;
return res;
}
std::vector<String> ASTStatisticDeclaration::getColumnNames() const
{
std::vector<String> result;
result.reserve(columns->children.size());
for (const ASTPtr & column_ast : columns->children)
{
result.push_back(column_ast->as<ASTIdentifier &>().name());
}
return result;
}
void ASTStatisticDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
columns->formatImpl(s, state, frame);
s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : "");
s.ostr << backQuoteIfNeed(type);
}
}

View File

@ -0,0 +1,60 @@
#include <Parsers/ASTStatisticsDeclaration.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
#include <Parsers/ASTFunction.h>
namespace DB
{
ASTPtr ASTStatisticsDeclaration::clone() const
{
auto res = std::make_shared<ASTStatisticsDeclaration>();
res->set(res->columns, columns->clone());
if (types)
res->set(res->types, types->clone());
return res;
}
std::vector<String> ASTStatisticsDeclaration::getColumnNames() const
{
std::vector<String> result;
result.reserve(columns->children.size());
for (const ASTPtr & column_ast : columns->children)
{
result.push_back(column_ast->as<ASTIdentifier &>().name());
}
return result;
}
std::vector<String> ASTStatisticsDeclaration::getTypeNames() const
{
chassert(types != nullptr);
std::vector<String> result;
result.reserve(types->children.size());
for (const ASTPtr & column_ast : types->children)
{
result.push_back(column_ast->as<ASTFunction &>().name);
}
return result;
}
void ASTStatisticsDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
columns->formatImpl(s, state, frame);
s.ostr << (s.hilite ? hilite_keyword : "");
if (types)
{
s.ostr << " TYPE " << (s.hilite ? hilite_none : "");
types->formatImpl(s, state, frame);
}
}
}

View File

@ -9,17 +9,17 @@ class ASTFunction;
/** name BY columns TYPE typename(args) in create query /** name BY columns TYPE typename(args) in create query
*/ */
class ASTStatisticDeclaration : public IAST class ASTStatisticsDeclaration : public IAST
{ {
public: public:
IAST * columns; IAST * columns;
/// TODO type should be a list of ASTFunction, for example, 'tdigest(256), hyperloglog(128)', etc. IAST * types;
String type;
/** Get the text that identifies this element. */ /** Get the text that identifies this element. */
String getID(char) const override { return "Stat"; } String getID(char) const override { return "Stat"; }
std::vector<String> getColumnNames() const; std::vector<String> getColumnNames() const;
std::vector<String> getTypeNames() const;
ASTPtr clone() const override; ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;

View File

@ -13,7 +13,7 @@ namespace DB
MR_MACROS(ADD_CONSTRAINT, "ADD CONSTRAINT") \ MR_MACROS(ADD_CONSTRAINT, "ADD CONSTRAINT") \
MR_MACROS(ADD_INDEX, "ADD INDEX") \ MR_MACROS(ADD_INDEX, "ADD INDEX") \
MR_MACROS(ADD_PROJECTION, "ADD PROJECTION") \ MR_MACROS(ADD_PROJECTION, "ADD PROJECTION") \
MR_MACROS(ADD_STATISTIC, "ADD STATISTIC") \ MR_MACROS(ADD_STATISTICS, "ADD STATISTICS") \
MR_MACROS(ADD, "ADD") \ MR_MACROS(ADD, "ADD") \
MR_MACROS(ADMIN_OPTION_FOR, "ADMIN OPTION FOR") \ MR_MACROS(ADMIN_OPTION_FOR, "ADMIN OPTION FOR") \
MR_MACROS(AFTER, "AFTER") \ MR_MACROS(AFTER, "AFTER") \
@ -83,7 +83,7 @@ namespace DB
MR_MACROS(CLEAR_COLUMN, "CLEAR COLUMN") \ MR_MACROS(CLEAR_COLUMN, "CLEAR COLUMN") \
MR_MACROS(CLEAR_INDEX, "CLEAR INDEX") \ MR_MACROS(CLEAR_INDEX, "CLEAR INDEX") \
MR_MACROS(CLEAR_PROJECTION, "CLEAR PROJECTION") \ MR_MACROS(CLEAR_PROJECTION, "CLEAR PROJECTION") \
MR_MACROS(CLEAR_STATISTIC, "CLEAR STATISTIC") \ MR_MACROS(CLEAR_STATISTICS, "CLEAR STATISTICS") \
MR_MACROS(CLUSTER, "CLUSTER") \ MR_MACROS(CLUSTER, "CLUSTER") \
MR_MACROS(CLUSTERS, "CLUSTERS") \ MR_MACROS(CLUSTERS, "CLUSTERS") \
MR_MACROS(CN, "CN") \ MR_MACROS(CN, "CN") \
@ -150,7 +150,7 @@ namespace DB
MR_MACROS(DROP_PART, "DROP PART") \ MR_MACROS(DROP_PART, "DROP PART") \
MR_MACROS(DROP_PARTITION, "DROP PARTITION") \ MR_MACROS(DROP_PARTITION, "DROP PARTITION") \
MR_MACROS(DROP_PROJECTION, "DROP PROJECTION") \ MR_MACROS(DROP_PROJECTION, "DROP PROJECTION") \
MR_MACROS(DROP_STATISTIC, "DROP STATISTIC") \ MR_MACROS(DROP_STATISTICS, "DROP STATISTICS") \
MR_MACROS(DROP_TABLE, "DROP TABLE") \ MR_MACROS(DROP_TABLE, "DROP TABLE") \
MR_MACROS(DROP_TEMPORARY_TABLE, "DROP TEMPORARY TABLE") \ MR_MACROS(DROP_TEMPORARY_TABLE, "DROP TEMPORARY TABLE") \
MR_MACROS(DROP, "DROP") \ MR_MACROS(DROP, "DROP") \
@ -279,7 +279,7 @@ namespace DB
MR_MACROS(MATERIALIZE_COLUMN, "MATERIALIZE COLUMN") \ MR_MACROS(MATERIALIZE_COLUMN, "MATERIALIZE COLUMN") \
MR_MACROS(MATERIALIZE_INDEX, "MATERIALIZE INDEX") \ MR_MACROS(MATERIALIZE_INDEX, "MATERIALIZE INDEX") \
MR_MACROS(MATERIALIZE_PROJECTION, "MATERIALIZE PROJECTION") \ MR_MACROS(MATERIALIZE_PROJECTION, "MATERIALIZE PROJECTION") \
MR_MACROS(MATERIALIZE_STATISTIC, "MATERIALIZE STATISTIC") \ MR_MACROS(MATERIALIZE_STATISTICS, "MATERIALIZE STATISTICS") \
MR_MACROS(MATERIALIZE_TTL, "MATERIALIZE TTL") \ MR_MACROS(MATERIALIZE_TTL, "MATERIALIZE TTL") \
MR_MACROS(MATERIALIZE, "MATERIALIZE") \ MR_MACROS(MATERIALIZE, "MATERIALIZE") \
MR_MACROS(MATERIALIZED, "MATERIALIZED") \ MR_MACROS(MATERIALIZED, "MATERIALIZED") \
@ -304,6 +304,7 @@ namespace DB
MR_MACROS(MODIFY_QUERY, "MODIFY QUERY") \ MR_MACROS(MODIFY_QUERY, "MODIFY QUERY") \
MR_MACROS(MODIFY_REFRESH, "MODIFY REFRESH") \ MR_MACROS(MODIFY_REFRESH, "MODIFY REFRESH") \
MR_MACROS(MODIFY_SAMPLE_BY, "MODIFY SAMPLE BY") \ MR_MACROS(MODIFY_SAMPLE_BY, "MODIFY SAMPLE BY") \
MR_MACROS(MODIFY_STATISTICS, "MODIFY STATISTICS") \
MR_MACROS(MODIFY_SETTING, "MODIFY SETTING") \ MR_MACROS(MODIFY_SETTING, "MODIFY SETTING") \
MR_MACROS(MODIFY_SQL_SECURITY, "MODIFY SQL SECURITY") \ MR_MACROS(MODIFY_SQL_SECURITY, "MODIFY SQL SECURITY") \
MR_MACROS(MODIFY_TTL, "MODIFY TTL") \ MR_MACROS(MODIFY_TTL, "MODIFY TTL") \
@ -447,7 +448,7 @@ namespace DB
MR_MACROS(SQL_SECURITY, "SQL SECURITY") \ MR_MACROS(SQL_SECURITY, "SQL SECURITY") \
MR_MACROS(SS, "SS") \ MR_MACROS(SS, "SS") \
MR_MACROS(START_TRANSACTION, "START TRANSACTION") \ MR_MACROS(START_TRANSACTION, "START TRANSACTION") \
MR_MACROS(STATISTIC, "STATISTIC") \ MR_MACROS(STATISTICS, "STATISTICS") \
MR_MACROS(STEP, "STEP") \ MR_MACROS(STEP, "STEP") \
MR_MACROS(STORAGE, "STORAGE") \ MR_MACROS(STORAGE, "STORAGE") \
MR_MACROS(STRICT, "STRICT") \ MR_MACROS(STRICT, "STRICT") \

View File

@ -703,7 +703,7 @@ bool ParserCodec::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return true; return true;
} }
bool ParserStatisticType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserStatisticsType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{ {
ParserList stat_type_parser(std::make_unique<ParserIdentifierWithOptionalParameters>(), ParserList stat_type_parser(std::make_unique<ParserIdentifierWithOptionalParameters>(),
std::make_unique<ParserToken>(TokenType::Comma), false); std::make_unique<ParserToken>(TokenType::Comma), false);
@ -722,7 +722,7 @@ bool ParserStatisticType::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
++pos; ++pos;
auto function_node = std::make_shared<ASTFunction>(); auto function_node = std::make_shared<ASTFunction>();
function_node->name = "STATISTIC"; function_node->name = "STATISTICS";
function_node->arguments = stat_type; function_node->arguments = stat_type;
function_node->children.push_back(function_node->arguments); function_node->children.push_back(function_node->arguments);

View File

@ -202,11 +202,11 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
}; };
/// STATISTIC(tdigest(200)) /// STATISTICS(tdigest(200))
class ParserStatisticType : public IParserBase class ParserStatisticsType : public IParserBase
{ {
protected: protected:
const char * getName() const override { return "statistic"; } const char * getName() const override { return "statistics"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
}; };

View File

@ -49,10 +49,11 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_clear_index(Keyword::CLEAR_INDEX); ParserKeyword s_clear_index(Keyword::CLEAR_INDEX);
ParserKeyword s_materialize_index(Keyword::MATERIALIZE_INDEX); ParserKeyword s_materialize_index(Keyword::MATERIALIZE_INDEX);
ParserKeyword s_add_statistic(Keyword::ADD_STATISTIC); ParserKeyword s_add_statistics(Keyword::ADD_STATISTICS);
ParserKeyword s_drop_statistic(Keyword::DROP_STATISTIC); ParserKeyword s_drop_statistics(Keyword::DROP_STATISTICS);
ParserKeyword s_clear_statistic(Keyword::CLEAR_STATISTIC); ParserKeyword s_modify_statistics(Keyword::MODIFY_STATISTICS);
ParserKeyword s_materialize_statistic(Keyword::MATERIALIZE_STATISTIC); ParserKeyword s_clear_statistics(Keyword::CLEAR_STATISTICS);
ParserKeyword s_materialize_statistics(Keyword::MATERIALIZE_STATISTICS);
ParserKeyword s_add_constraint(Keyword::ADD_CONSTRAINT); ParserKeyword s_add_constraint(Keyword::ADD_CONSTRAINT);
ParserKeyword s_drop_constraint(Keyword::DROP_CONSTRAINT); ParserKeyword s_drop_constraint(Keyword::DROP_CONSTRAINT);
@ -126,7 +127,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserIdentifier parser_remove_property; ParserIdentifier parser_remove_property;
ParserCompoundColumnDeclaration parser_col_decl; ParserCompoundColumnDeclaration parser_col_decl;
ParserIndexDeclaration parser_idx_decl; ParserIndexDeclaration parser_idx_decl;
ParserStatisticDeclaration parser_stat_decl; ParserStatisticsDeclaration parser_stat_decl;
ParserStatisticsDeclarationWithoutTypes parser_stat_decl_without_types;
ParserConstraintDeclaration parser_constraint_decl; ParserConstraintDeclaration parser_constraint_decl;
ParserProjectionDeclaration parser_projection_decl; ParserProjectionDeclaration parser_projection_decl;
ParserCompoundColumnDeclaration parser_modify_col_decl(false, false, true); ParserCompoundColumnDeclaration parser_modify_col_decl(false, false, true);
@ -154,7 +156,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ASTPtr command_constraint; ASTPtr command_constraint;
ASTPtr command_projection_decl; ASTPtr command_projection_decl;
ASTPtr command_projection; ASTPtr command_projection;
ASTPtr command_statistic_decl; ASTPtr command_statistics_decl;
ASTPtr command_partition; ASTPtr command_partition;
ASTPtr command_predicate; ASTPtr command_predicate;
ASTPtr command_update_assignments; ASTPtr command_update_assignments;
@ -368,36 +370,43 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
return false; return false;
} }
} }
else if (s_add_statistic.ignore(pos, expected)) else if (s_add_statistics.ignore(pos, expected))
{ {
if (s_if_not_exists.ignore(pos, expected)) if (s_if_not_exists.ignore(pos, expected))
command->if_not_exists = true; command->if_not_exists = true;
if (!parser_stat_decl.parse(pos, command_statistic_decl, expected)) if (!parser_stat_decl.parse(pos, command_statistics_decl, expected))
return false; return false;
command->type = ASTAlterCommand::ADD_STATISTIC; command->type = ASTAlterCommand::ADD_STATISTICS;
} }
else if (s_drop_statistic.ignore(pos, expected)) else if (s_modify_statistics.ignore(pos, expected))
{
if (!parser_stat_decl.parse(pos, command_statistics_decl, expected))
return false;
command->type = ASTAlterCommand::MODIFY_STATISTICS;
}
else if (s_drop_statistics.ignore(pos, expected))
{ {
if (s_if_exists.ignore(pos, expected)) if (s_if_exists.ignore(pos, expected))
command->if_exists = true; command->if_exists = true;
if (!parser_stat_decl.parse(pos, command_statistic_decl, expected)) if (!parser_stat_decl_without_types.parse(pos, command_statistics_decl, expected))
return false; return false;
command->type = ASTAlterCommand::DROP_STATISTIC; command->type = ASTAlterCommand::DROP_STATISTICS;
} }
else if (s_clear_statistic.ignore(pos, expected)) else if (s_clear_statistics.ignore(pos, expected))
{ {
if (s_if_exists.ignore(pos, expected)) if (s_if_exists.ignore(pos, expected))
command->if_exists = true; command->if_exists = true;
if (!parser_stat_decl.parse(pos, command_statistic_decl, expected)) if (!parser_stat_decl_without_types.parse(pos, command_statistics_decl, expected))
return false; return false;
command->type = ASTAlterCommand::DROP_STATISTIC; command->type = ASTAlterCommand::DROP_STATISTICS;
command->clear_statistic = true; command->clear_statistics = true;
command->detach = false; command->detach = false;
if (s_in_partition.ignore(pos, expected)) if (s_in_partition.ignore(pos, expected))
@ -406,15 +415,15 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
return false; return false;
} }
} }
else if (s_materialize_statistic.ignore(pos, expected)) else if (s_materialize_statistics.ignore(pos, expected))
{ {
if (s_if_exists.ignore(pos, expected)) if (s_if_exists.ignore(pos, expected))
command->if_exists = true; command->if_exists = true;
if (!parser_stat_decl.parse(pos, command_statistic_decl, expected)) if (!parser_stat_decl_without_types.parse(pos, command_statistics_decl, expected))
return false; return false;
command->type = ASTAlterCommand::MATERIALIZE_STATISTIC; command->type = ASTAlterCommand::MATERIALIZE_STATISTICS;
command->detach = false; command->detach = false;
if (s_in_partition.ignore(pos, expected)) if (s_in_partition.ignore(pos, expected))
@ -931,8 +940,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->projection_decl = command->children.emplace_back(std::move(command_projection_decl)).get(); command->projection_decl = command->children.emplace_back(std::move(command_projection_decl)).get();
if (command_projection) if (command_projection)
command->projection = command->children.emplace_back(std::move(command_projection)).get(); command->projection = command->children.emplace_back(std::move(command_projection)).get();
if (command_statistic_decl) if (command_statistics_decl)
command->statistic_decl = command->children.emplace_back(std::move(command_statistic_decl)).get(); command->statistics_decl = command->children.emplace_back(std::move(command_statistics_decl)).get();
if (command_partition) if (command_partition)
command->partition = command->children.emplace_back(std::move(command_partition)).get(); command->partition = command->children.emplace_back(std::move(command_partition)).get();
if (command_predicate) if (command_predicate)

View File

@ -7,7 +7,7 @@
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIndexDeclaration.h> #include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTStatisticDeclaration.h> #include <Parsers/ASTStatisticsDeclaration.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTProjectionDeclaration.h> #include <Parsers/ASTProjectionDeclaration.h>
#include <Parsers/ASTSelectWithUnionQuery.h> #include <Parsers/ASTSelectWithUnionQuery.h>
@ -225,15 +225,15 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
return true; return true;
} }
bool ParserStatisticDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserStatisticsDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{ {
ParserKeyword s_type(Keyword::TYPE); ParserKeyword s_type(Keyword::TYPE);
ParserList columns_p(std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false); ParserList columns_p(std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserIdentifier type_p; ParserList types_p(std::make_unique<ParserDataType>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ASTPtr columns; ASTPtr columns;
ASTPtr type; ASTPtr types;
if (!columns_p.parse(pos, columns, expected)) if (!columns_p.parse(pos, columns, expected))
return false; return false;
@ -241,12 +241,29 @@ bool ParserStatisticDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
if (!s_type.ignore(pos, expected)) if (!s_type.ignore(pos, expected))
return false; return false;
if (!type_p.parse(pos, type, expected)) if (!types_p.parse(pos, types, expected))
return false; return false;
auto stat = std::make_shared<ASTStatisticDeclaration>(); auto stat = std::make_shared<ASTStatisticsDeclaration>();
stat->set(stat->columns, columns);
stat->set(stat->types, types);
node = stat;
return true;
}
bool ParserStatisticsDeclarationWithoutTypes::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserList columns_p(std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ASTPtr columns;
if (!columns_p.parse(pos, columns, expected))
return false;
auto stat = std::make_shared<ASTStatisticsDeclaration>();
stat->set(stat->columns, columns); stat->set(stat->columns, columns);
stat->type = type->as<ASTIdentifier &>().name();
node = stat; node = stat;
return true; return true;

View File

@ -138,7 +138,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ParserKeyword s_auto_increment{Keyword::AUTO_INCREMENT}; ParserKeyword s_auto_increment{Keyword::AUTO_INCREMENT};
ParserKeyword s_comment{Keyword::COMMENT}; ParserKeyword s_comment{Keyword::COMMENT};
ParserKeyword s_codec{Keyword::CODEC}; ParserKeyword s_codec{Keyword::CODEC};
ParserKeyword s_stat{Keyword::STATISTIC}; ParserKeyword s_stat{Keyword::STATISTICS};
ParserKeyword s_ttl{Keyword::TTL}; ParserKeyword s_ttl{Keyword::TTL};
ParserKeyword s_remove{Keyword::REMOVE}; ParserKeyword s_remove{Keyword::REMOVE};
ParserKeyword s_modify_setting(Keyword::MODIFY_SETTING); ParserKeyword s_modify_setting(Keyword::MODIFY_SETTING);
@ -155,7 +155,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ParserLiteral literal_parser; ParserLiteral literal_parser;
ParserCodec codec_parser; ParserCodec codec_parser;
ParserCollation collation_parser; ParserCollation collation_parser;
ParserStatisticType stat_type_parser; ParserStatisticsType stat_type_parser;
ParserExpression expression_parser; ParserExpression expression_parser;
ParserSetQuery settings_parser(true); ParserSetQuery settings_parser(true);
@ -452,16 +452,27 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
}; };
class ParserStatisticDeclaration : public IParserBase class ParserStatisticsDeclaration : public IParserBase
{ {
public: public:
ParserStatisticDeclaration() = default; ParserStatisticsDeclaration() = default;
protected: protected:
const char * getName() const override { return "statistics declaration"; } const char * getName() const override { return "statistics declaration"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
}; };
class ParserStatisticsDeclarationWithoutTypes : public IParserBase
{
public:
ParserStatisticsDeclarationWithoutTypes() = default;
protected:
const char * getName() const override { return "statistics declaration"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserConstraintDeclaration : public IParserBase class ParserConstraintDeclaration : public IParserBase
{ {
protected: protected:

View File

@ -83,7 +83,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
MergeTreeWhereOptimizer where_optimizer{ MergeTreeWhereOptimizer where_optimizer{
std::move(column_compressed_sizes), std::move(column_compressed_sizes),
storage_metadata, storage_metadata,
storage.getConditionEstimatorByPredicate(storage_snapshot, source_step_with_filter->getFilterActionsDAG(), context), storage.getConditionSelectivityEstimatorByPredicate(storage_snapshot, source_step_with_filter->getFilterActionsDAG(), context),
queried_columns, queried_columns,
storage.supportedPrewhereColumns(), storage.supportedPrewhereColumns(),
getLogger("QueryPlanOptimizePrewhere")}; getLogger("QueryPlanOptimizePrewhere")};

View File

@ -118,7 +118,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s
optimizePrimaryKeyCondition(stack); optimizePrimaryKeyCondition(stack);
/// NOTE: optimizePrewhere can modify the stack. /// NOTE: optimizePrewhere can modify the stack.
/// Prewhere optimization relies on PK optimization (getConditionEstimatorByPredicate) /// Prewhere optimization relies on PK optimization (getConditionSelectivityEstimatorByPredicate)
if (optimization_settings.optimize_prewhere) if (optimization_settings.optimize_prewhere)
optimizePrewhere(stack, nodes); optimizePrewhere(stack, nodes);

View File

@ -25,7 +25,7 @@
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIndexDeclaration.h> #include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTProjectionDeclaration.h> #include <Parsers/ASTProjectionDeclaration.h>
#include <Parsers/ASTStatisticDeclaration.h> #include <Parsers/ASTStatisticsDeclaration.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSetQuery.h> #include <Parsers/ASTSetQuery.h>
#include <Parsers/queryToString.h> #include <Parsers/queryToString.h>
@ -44,7 +44,7 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_STATISTIC; extern const int ILLEGAL_STATISTICS;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int NOT_FOUND_COLUMN_IN_BLOCK;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
@ -263,17 +263,32 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
return command; return command;
} }
else if (command_ast->type == ASTAlterCommand::ADD_STATISTIC) else if (command_ast->type == ASTAlterCommand::ADD_STATISTICS)
{ {
AlterCommand command; AlterCommand command;
command.ast = command_ast->clone(); command.ast = command_ast->clone();
command.statistic_decl = command_ast->statistic_decl->clone(); command.statistics_decl = command_ast->statistics_decl->clone();
command.type = AlterCommand::ADD_STATISTIC; command.type = AlterCommand::ADD_STATISTICS;
const auto & ast_stat_decl = command_ast->statistic_decl->as<ASTStatisticDeclaration &>(); const auto & ast_stat_decl = command_ast->statistics_decl->as<ASTStatisticsDeclaration &>();
command.statistic_columns = ast_stat_decl.getColumnNames(); command.statistics_columns = ast_stat_decl.getColumnNames();
command.statistic_type = ast_stat_decl.type; command.statistics_types = ast_stat_decl.getTypeNames();
command.if_not_exists = command_ast->if_not_exists;
return command;
}
else if (command_ast->type == ASTAlterCommand::MODIFY_STATISTICS)
{
AlterCommand command;
command.ast = command_ast->clone();
command.statistics_decl = command_ast->statistics_decl->clone();
command.type = AlterCommand::MODIFY_STATISTICS;
const auto & ast_stat_decl = command_ast->statistics_decl->as<ASTStatisticsDeclaration &>();
command.statistics_columns = ast_stat_decl.getColumnNames();
command.statistics_types = ast_stat_decl.getTypeNames();
command.if_not_exists = command_ast->if_not_exists; command.if_not_exists = command_ast->if_not_exists;
return command; return command;
@ -337,17 +352,17 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
return command; return command;
} }
else if (command_ast->type == ASTAlterCommand::DROP_STATISTIC) else if (command_ast->type == ASTAlterCommand::DROP_STATISTICS)
{ {
AlterCommand command; AlterCommand command;
command.ast = command_ast->clone(); command.ast = command_ast->clone();
command.type = AlterCommand::DROP_STATISTIC; command.statistics_decl = command_ast->statistics_decl->clone();
const auto & ast_stat_decl = command_ast->statistic_decl->as<ASTStatisticDeclaration &>(); command.type = AlterCommand::DROP_STATISTICS;
const auto & ast_stat_decl = command_ast->statistics_decl->as<ASTStatisticsDeclaration &>();
command.statistic_columns = ast_stat_decl.getColumnNames(); command.statistics_columns = ast_stat_decl.getColumnNames();
command.statistic_type = ast_stat_decl.type;
command.if_exists = command_ast->if_exists; command.if_exists = command_ast->if_exists;
command.clear = command_ast->clear_statistic; command.clear = command_ast->clear_statistics;
if (command_ast->partition) if (command_ast->partition)
command.partition = command_ast->partition->clone(); command.partition = command_ast->partition->clone();
@ -676,41 +691,56 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
metadata.secondary_indices.erase(erase_it); metadata.secondary_indices.erase(erase_it);
} }
} }
else if (type == ADD_STATISTIC) else if (type == ADD_STATISTICS)
{ {
for (const auto & statistic_column_name : statistic_columns) for (const auto & statistics_column_name : statistics_columns)
{ {
if (!metadata.columns.has(statistic_column_name)) if (!metadata.columns.has(statistics_column_name))
{ {
throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Cannot add statistic {} with type {}: this column is not found", statistic_column_name, statistic_type); throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot add statistics for column {}: this column is not found", statistics_column_name);
} }
if (!if_exists && metadata.columns.get(statistic_column_name).stat)
throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Cannot add statistic {} with type {}: statistic on this column with this type already exists", statistic_column_name, statistic_type);
} }
auto stats = StatisticDescription::getStatisticsFromAST(statistic_decl, metadata.columns); auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns);
for (auto && stat : stats) for (const auto & stats : stats_vec)
{ {
metadata.columns.modify(stat.column_name, metadata.columns.modify(stats.column_name,
[&](ColumnDescription & column) { column.stat = std::move(stat); }); [&](ColumnDescription & column) { column.statistics.merge(stats, column.name, column.type, if_not_exists); });
} }
} }
else if (type == DROP_STATISTIC) else if (type == DROP_STATISTICS)
{ {
for (const auto & stat_column_name : statistic_columns) for (const auto & statistics_column_name : statistics_columns)
{ {
if (!metadata.columns.has(stat_column_name) || !metadata.columns.get(stat_column_name).stat) if (!metadata.columns.has(statistics_column_name)
|| metadata.columns.get(statistics_column_name).statistics.empty())
{ {
if (if_exists) if (if_exists)
return; return;
throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "Wrong statistic name. Cannot find statistic {} with type {} to drop", backQuote(stat_column_name), statistic_type); throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Wrong statistics name. Cannot find statistics {} to drop", backQuote(statistics_column_name));
} }
if (!partition && !clear)
if (!clear && !partition)
metadata.columns.modify(statistics_column_name,
[&](ColumnDescription & column) { column.statistics.clear(); });
}
}
else if (type == MODIFY_STATISTICS)
{
for (const auto & statistics_column_name : statistics_columns)
{
if (!metadata.columns.has(statistics_column_name))
{ {
metadata.columns.modify(stat_column_name, throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot add statistics for column {}: this column is not found", statistics_column_name);
[&](ColumnDescription & column) { column.stat = std::nullopt; });
} }
} }
auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns);
for (const auto & stats : stats_vec)
{
metadata.columns.modify(stats.column_name,
[&](ColumnDescription & column) { column.statistics.assign(stats); });
}
} }
else if (type == ADD_CONSTRAINT) else if (type == ADD_CONSTRAINT)
{ {
@ -833,8 +863,8 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
rename_visitor.visit(column_to_modify.default_desc.expression); rename_visitor.visit(column_to_modify.default_desc.expression);
if (column_to_modify.ttl) if (column_to_modify.ttl)
rename_visitor.visit(column_to_modify.ttl); rename_visitor.visit(column_to_modify.ttl);
if (column_to_modify.name == column_name && column_to_modify.stat) if (column_to_modify.name == column_name && !column_to_modify.statistics.empty())
column_to_modify.stat->column_name = rename_to; column_to_modify.statistics.column_name = rename_to;
}); });
} }
if (metadata.table_ttl.definition_ast) if (metadata.table_ttl.definition_ast)
@ -958,7 +988,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada
if (isRemovingProperty() || type == REMOVE_TTL || type == REMOVE_SAMPLE_BY) if (isRemovingProperty() || type == REMOVE_TTL || type == REMOVE_SAMPLE_BY)
return false; return false;
if (type == DROP_INDEX || type == DROP_PROJECTION || type == RENAME_COLUMN || type == DROP_STATISTIC) if (type == DROP_INDEX || type == DROP_PROJECTION || type == RENAME_COLUMN || type == DROP_STATISTICS)
return true; return true;
/// Drop alias is metadata alter, in other case mutation is required. /// Drop alias is metadata alter, in other case mutation is required.
@ -1065,10 +1095,10 @@ std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(Storage
result.predicate = nullptr; result.predicate = nullptr;
} }
else if (type == DROP_STATISTIC) else if (type == DROP_STATISTICS)
{ {
result.type = MutationCommand::Type::DROP_STATISTIC; result.type = MutationCommand::Type::DROP_STATISTICS;
result.statistic_columns = statistic_columns; result.statistics_columns = statistics_columns;
if (clear) if (clear)
result.clear = true; result.clear = true;

View File

@ -38,8 +38,9 @@ struct AlterCommand
DROP_CONSTRAINT, DROP_CONSTRAINT,
ADD_PROJECTION, ADD_PROJECTION,
DROP_PROJECTION, DROP_PROJECTION,
ADD_STATISTIC, ADD_STATISTICS,
DROP_STATISTIC, DROP_STATISTICS,
MODIFY_STATISTICS,
MODIFY_TTL, MODIFY_TTL,
MODIFY_SETTING, MODIFY_SETTING,
RESET_SETTING, RESET_SETTING,
@ -123,9 +124,9 @@ struct AlterCommand
/// For ADD/DROP PROJECTION /// For ADD/DROP PROJECTION
String projection_name; String projection_name;
ASTPtr statistic_decl = nullptr; ASTPtr statistics_decl = nullptr;
std::vector<String> statistic_columns; std::vector<String> statistics_columns;
String statistic_type; std::vector<String> statistics_types;
/// For MODIFY TTL /// For MODIFY TTL
ASTPtr ttl = nullptr; ASTPtr ttl = nullptr;

View File

@ -26,8 +26,8 @@ struct ColumnDependency
/// TTL is set for @column_name. /// TTL is set for @column_name.
TTL_TARGET, TTL_TARGET,
/// Exists any statistic, that requires @column_name /// Exists any statistics, that requires @column_name
STATISTIC, STATISTICS,
}; };
ColumnDependency(const String & column_name_, Kind kind_) ColumnDependency(const String & column_name_, Kind kind_)

View File

@ -72,7 +72,7 @@ ColumnDescription & ColumnDescription::operator=(const ColumnDescription & other
codec = other.codec ? other.codec->clone() : nullptr; codec = other.codec ? other.codec->clone() : nullptr;
settings = other.settings; settings = other.settings;
ttl = other.ttl ? other.ttl->clone() : nullptr; ttl = other.ttl ? other.ttl->clone() : nullptr;
stat = other.stat; statistics = other.statistics;
return *this; return *this;
} }
@ -95,7 +95,7 @@ ColumnDescription & ColumnDescription::operator=(ColumnDescription && other) noe
ttl = other.ttl ? other.ttl->clone() : nullptr; ttl = other.ttl ? other.ttl->clone() : nullptr;
other.ttl.reset(); other.ttl.reset();
stat = std::move(other.stat); statistics = std::move(other.statistics);
return *this; return *this;
} }
@ -107,7 +107,7 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const
return name == other.name return name == other.name
&& type->equals(*other.type) && type->equals(*other.type)
&& default_desc == other.default_desc && default_desc == other.default_desc
&& stat == other.stat && statistics == other.statistics
&& ast_to_str(codec) == ast_to_str(other.codec) && ast_to_str(codec) == ast_to_str(other.codec)
&& settings == other.settings && settings == other.settings
&& ast_to_str(ttl) == ast_to_str(other.ttl); && ast_to_str(ttl) == ast_to_str(other.ttl);
@ -154,10 +154,10 @@ void ColumnDescription::writeText(WriteBuffer & buf) const
DB::writeText(")", buf); DB::writeText(")", buf);
} }
if (stat) if (!statistics.empty())
{ {
writeChar('\t', buf); writeChar('\t', buf);
writeEscapedString(queryToString(stat->ast), buf); writeEscapedString(queryToString(statistics.getAST()), buf);
} }
if (ttl) if (ttl)

View File

@ -89,7 +89,7 @@ struct ColumnDescription
ASTPtr codec; ASTPtr codec;
SettingsChanges settings; SettingsChanges settings;
ASTPtr ttl; ASTPtr ttl;
std::optional<StatisticDescription> stat; ColumnStatisticsDescription statistics;
ColumnDescription() = default; ColumnDescription() = default;
ColumnDescription(const ColumnDescription & other) { *this = other; } ColumnDescription(const ColumnDescription & other) { *this = other; }

View File

@ -12,7 +12,7 @@
#include <Processors/QueryPlan/ReadFromPreparedSource.h> #include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/QueryPlan/QueryPlan.h> #include <Processors/QueryPlan/QueryPlan.h>
#include <Storages/AlterCommands.h> #include <Storages/AlterCommands.h>
#include <Storages/Statistics/Estimator.h> #include <Storages/Statistics/ConditionSelectivityEstimator.h>
#include <Backups/RestorerFromBackup.h> #include <Backups/RestorerFromBackup.h>
#include <Backups/IBackup.h> #include <Backups/IBackup.h>
@ -236,7 +236,7 @@ StorageID IStorage::getStorageID() const
return storage_id; return storage_id;
} }
ConditionEstimator IStorage::getConditionEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const ConditionSelectivityEstimator IStorage::getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const
{ {
return {}; return {};
} }

View File

@ -68,7 +68,7 @@ using DatabaseAndTableName = std::pair<String, String>;
class BackupEntriesCollector; class BackupEntriesCollector;
class RestorerFromBackup; class RestorerFromBackup;
class ConditionEstimator; class ConditionSelectivityEstimator;
struct ColumnSize struct ColumnSize
{ {
@ -135,7 +135,7 @@ public:
/// Returns true if the storage supports queries with the PREWHERE section. /// Returns true if the storage supports queries with the PREWHERE section.
virtual bool supportsPrewhere() const { return false; } virtual bool supportsPrewhere() const { return false; }
virtual ConditionEstimator getConditionEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const; virtual ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const;
/// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported. /// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported.
/// This is needed for engines whose aggregates data from multiple tables, like Merge. /// This is needed for engines whose aggregates data from multiple tables, like Merge.

View File

@ -673,16 +673,16 @@ String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(bool with_subc
return *minimum_size_column; return *minimum_size_column;
} }
Statistics IMergeTreeDataPart::loadStatistics() const ColumnsStatistics IMergeTreeDataPart::loadStatistics() const
{ {
const auto & metadata_snaphost = storage.getInMemoryMetadata(); const auto & metadata_snaphost = storage.getInMemoryMetadata();
auto total_statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snaphost.getColumns()); auto total_statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snaphost.getColumns());
Statistics result; ColumnsStatistics result;
for (auto & stat : total_statistics) for (auto & stat : total_statistics)
{ {
String file_name = stat->getFileName() + STAT_FILE_SUFFIX; String file_name = stat->getFileName() + STATS_FILE_SUFFIX;
String file_path = fs::path(getDataPartStorage().getRelativePath()) / file_name; String file_path = fs::path(getDataPartStorage().getRelativePath()) / file_name;
if (!metadata_manager->exists(file_name)) if (!metadata_manager->exists(file_name))

View File

@ -171,7 +171,7 @@ public:
void remove(); void remove();
Statistics loadStatistics() const; ColumnsStatistics loadStatistics() const;
/// Initialize columns (from columns.txt if exists, or create from column files if not). /// Initialize columns (from columns.txt if exists, or create from column files if not).
/// Load various metadata into memory: checksums from checksums.txt, index if required, etc. /// Load various metadata into memory: checksums from checksums.txt, index if required, etc.

View File

@ -119,7 +119,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter(
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,
const VirtualsDescriptionPtr & virtual_columns, const VirtualsDescriptionPtr & virtual_columns,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc, const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,
@ -136,7 +136,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter(
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,
const VirtualsDescriptionPtr & virtual_columns, const VirtualsDescriptionPtr & virtual_columns,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc, const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,
@ -156,7 +156,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter(
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,
const VirtualsDescriptionPtr & virtual_columns, const VirtualsDescriptionPtr & virtual_columns,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc, const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,

View File

@ -84,7 +84,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter(
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,
const VirtualsDescriptionPtr & virtual_columns_, const VirtualsDescriptionPtr & virtual_columns_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc, const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension, const String & marks_file_extension,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,

View File

@ -658,7 +658,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const
/// because all of them were already recalculated and written /// because all of them were already recalculated and written
/// as key part of vertical merge /// as key part of vertical merge
std::vector<MergeTreeIndexPtr>{}, std::vector<MergeTreeIndexPtr>{},
std::vector<StatisticPtr>{}, /// TODO: think about it ColumnsStatistics{}, /// TODO(hanfei)
&global_ctx->written_offset_columns, &global_ctx->written_offset_columns,
global_ctx->to->getIndexGranularity()); global_ctx->to->getIndexGranularity());

View File

@ -73,7 +73,7 @@
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h> #include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
#include <Storages/MergeTree/MergeTreeDataPartBuilder.h> #include <Storages/MergeTree/MergeTreeDataPartBuilder.h>
#include <Storages/MergeTree/MergeTreeDataPartCompact.h> #include <Storages/MergeTree/MergeTreeDataPartCompact.h>
#include <Storages/Statistics/Estimator.h> #include <Storages/Statistics/ConditionSelectivityEstimator.h>
#include <Storages/MergeTree/MergeTreeSelectProcessor.h> #include <Storages/MergeTree/MergeTreeSelectProcessor.h>
#include <Storages/MergeTree/checkDataPart.h> #include <Storages/MergeTree/checkDataPart.h>
#include <Storages/MutationCommands.h> #include <Storages/MutationCommands.h>
@ -471,10 +471,10 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const
return storage_policy; return storage_policy;
} }
ConditionEstimator MergeTreeData::getConditionEstimatorByPredicate( ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByPredicate(
const StorageSnapshotPtr & storage_snapshot, const ActionsDAGPtr & filter_dag, ContextPtr local_context) const const StorageSnapshotPtr & storage_snapshot, const ActionsDAGPtr & filter_dag, ContextPtr local_context) const
{ {
if (!local_context->getSettings().allow_statistic_optimize) if (!local_context->getSettings().allow_statistics_optimize)
return {}; return {};
const auto & parts = assert_cast<const MergeTreeData::SnapshotData &>(*storage_snapshot->data).parts; const auto & parts = assert_cast<const MergeTreeData::SnapshotData &>(*storage_snapshot->data).parts;
@ -486,23 +486,29 @@ ConditionEstimator MergeTreeData::getConditionEstimatorByPredicate(
ASTPtr expression_ast; ASTPtr expression_ast;
ConditionEstimator result; ConditionSelectivityEstimator result;
PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag, local_context); PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag, local_context);
if (partition_pruner.isUseless()) if (partition_pruner.isUseless())
{ {
/// Read all partitions. /// Read all partitions.
for (const auto & part : parts) for (const auto & part : parts)
try
{ {
auto stats = part->loadStatistics(); auto stats = part->loadStatistics();
/// TODO: We only have one stats file for every part. /// TODO: We only have one stats file for every part.
for (const auto & stat : stats) for (const auto & stat : stats)
result.merge(part->info.getPartNameV1(), part->rows_count, stat); result.merge(part->info.getPartNameV1(), part->rows_count, stat);
} }
catch (...)
{
tryLogCurrentException(log, fmt::format("while loading statistics on part {}", part->info.getPartNameV1()));
}
} }
else else
{ {
for (const auto & part : parts) for (const auto & part : parts)
try
{ {
if (!partition_pruner.canBePruned(*part)) if (!partition_pruner.canBePruned(*part))
{ {
@ -511,6 +517,10 @@ ConditionEstimator MergeTreeData::getConditionEstimatorByPredicate(
result.merge(part->info.getPartNameV1(), part->rows_count, stat); result.merge(part->info.getPartNameV1(), part->rows_count, stat);
} }
} }
catch (...)
{
tryLogCurrentException(log, fmt::format("while loading statistics on part {}", part->info.getPartNameV1()));
}
} }
return result; return result;
@ -691,8 +701,8 @@ void MergeTreeData::checkProperties(
for (const auto & col : new_metadata.columns) for (const auto & col : new_metadata.columns)
{ {
if (col.stat) if (!col.statistics.empty())
MergeTreeStatisticsFactory::instance().validate(*col.stat, col.type); MergeTreeStatisticsFactory::instance().validate(col.statistics, col.type);
} }
checkKeyExpression(*new_sorting_key.expression, new_sorting_key.sample_block, "Sorting", allow_nullable_key_); checkKeyExpression(*new_sorting_key.expression, new_sorting_key.sample_block, "Sorting", allow_nullable_key_);
@ -3469,13 +3479,13 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
new_metadata.getColumns().getPhysical(command.column_name)); new_metadata.getColumns().getPhysical(command.column_name));
const auto & old_column = old_metadata.getColumns().get(command.column_name); const auto & old_column = old_metadata.getColumns().get(command.column_name);
if (old_column.stat) if (!old_column.statistics.empty())
{ {
const auto & new_column = new_metadata.getColumns().get(command.column_name); const auto & new_column = new_metadata.getColumns().get(command.column_name);
if (!old_column.type->equals(*new_column.type)) if (!old_column.type->equals(*new_column.type))
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"ALTER types of column {} with statistic is not not safe " "ALTER types of column {} with statistics is not not safe "
"because it can change the representation of statistic", "because it can change the representation of statistics",
backQuoteIfNeed(command.column_name)); backQuoteIfNeed(command.column_name));
} }
} }
@ -8510,7 +8520,7 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::createE
const auto & index_factory = MergeTreeIndexFactory::instance(); const auto & index_factory = MergeTreeIndexFactory::instance();
MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns,
index_factory.getMany(metadata_snapshot->getSecondaryIndices()), index_factory.getMany(metadata_snapshot->getSecondaryIndices()),
Statistics{}, ColumnsStatistics{},
compression_codec, txn ? txn->tid : Tx::PrehistoricTID); compression_codec, txn ? txn->tid : Tx::PrehistoricTID);
bool sync_on_insert = settings->fsync_after_insert; bool sync_on_insert = settings->fsync_after_insert;

View File

@ -426,7 +426,7 @@ public:
bool supportsPrewhere() const override { return true; } bool supportsPrewhere() const override { return true; }
ConditionEstimator getConditionEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const override; ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const override;
bool supportsFinal() const override; bool supportsFinal() const override;

View File

@ -59,7 +59,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter(
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,
const VirtualsDescriptionPtr & virtual_columns, const VirtualsDescriptionPtr & virtual_columns,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc, const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,

View File

@ -64,7 +64,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter(
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,
const VirtualsDescriptionPtr & virtual_columns, const VirtualsDescriptionPtr & virtual_columns,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc, const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,

View File

@ -20,7 +20,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact(
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const VirtualsDescriptionPtr & virtual_columns_, const VirtualsDescriptionPtr & virtual_columns_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc_, const std::vector<MergeTreeIndexPtr> & indices_to_recalc_,
const Statistics & stats_to_recalc, const ColumnsStatistics & stats_to_recalc,
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_, const MergeTreeWriterSettings & settings_,

View File

@ -21,7 +21,7 @@ public:
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const VirtualsDescriptionPtr & virtual_columns_, const VirtualsDescriptionPtr & virtual_columns_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc, const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const Statistics & stats_to_recalc, const ColumnsStatistics & stats_to_recalc,
const String & marks_file_extension, const String & marks_file_extension,
const CompressionCodecPtr & default_codec, const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings, const MergeTreeWriterSettings & settings,

View File

@ -150,7 +150,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk(
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const VirtualsDescriptionPtr & virtual_columns_, const VirtualsDescriptionPtr & virtual_columns_,
const MergeTreeIndices & indices_to_recalc_, const MergeTreeIndices & indices_to_recalc_,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_, const MergeTreeWriterSettings & settings_,
@ -265,7 +265,7 @@ void MergeTreeDataPartWriterOnDisk::initStatistics()
stats_streams.emplace_back(std::make_unique<MergeTreeDataPartWriterOnDisk::Stream<true>>( stats_streams.emplace_back(std::make_unique<MergeTreeDataPartWriterOnDisk::Stream<true>>(
stats_name, stats_name,
data_part_storage, data_part_storage,
stats_name, STAT_FILE_SUFFIX, stats_name, STATS_FILE_SUFFIX,
default_codec, settings.max_compress_block_size, default_codec, settings.max_compress_block_size,
settings.query_write_settings)); settings.query_write_settings));
} }

View File

@ -111,7 +111,7 @@ public:
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const VirtualsDescriptionPtr & virtual_columns_, const VirtualsDescriptionPtr & virtual_columns_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc, const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension, const String & marks_file_extension,
const CompressionCodecPtr & default_codec, const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings, const MergeTreeWriterSettings & settings,
@ -155,7 +155,7 @@ protected:
const MergeTreeIndices skip_indices; const MergeTreeIndices skip_indices;
const Statistics stats; const ColumnsStatistics stats;
std::vector<StatisticStreamPtr> stats_streams; std::vector<StatisticStreamPtr> stats_streams;
const String marks_file_extension; const String marks_file_extension;

View File

@ -86,7 +86,7 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide(
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const VirtualsDescriptionPtr & virtual_columns_, const VirtualsDescriptionPtr & virtual_columns_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc_, const std::vector<MergeTreeIndexPtr> & indices_to_recalc_,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_, const MergeTreeWriterSettings & settings_,

View File

@ -31,7 +31,7 @@ public:
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,
const VirtualsDescriptionPtr & virtual_columns_, const VirtualsDescriptionPtr & virtual_columns_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc, const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
const String & marks_file_extension, const String & marks_file_extension,
const CompressionCodecPtr & default_codec, const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings, const MergeTreeWriterSettings & settings,

View File

@ -469,7 +469,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
if (context->getSettingsRef().materialize_skip_indexes_on_insert) if (context->getSettingsRef().materialize_skip_indexes_on_insert)
indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices()); indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices());
Statistics statistics; ColumnsStatistics statistics;
if (context->getSettingsRef().materialize_statistics_on_insert) if (context->getSettingsRef().materialize_statistics_on_insert)
statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()); statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns());
@ -754,7 +754,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
metadata_snapshot, metadata_snapshot,
columns, columns,
MergeTreeIndices{}, MergeTreeIndices{},
Statistics{}, /// TODO(hanfei): It should be helpful to write statistics for projection result. /// TODO(hanfei): It should be helpful to write statistics for projection result.
ColumnsStatistics{},
compression_codec, compression_codec,
Tx::PrehistoricTID, Tx::PrehistoricTID,
false, false, data.getContext()->getWriteSettings()); false, false, data.getContext()->getWriteSettings());

View File

@ -53,7 +53,7 @@ static Int64 findMinPosition(const NameSet & condition_table_columns, const Name
MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
std::unordered_map<std::string, UInt64> column_sizes_, std::unordered_map<std::string, UInt64> column_sizes_,
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,
const ConditionEstimator & estimator_, const ConditionSelectivityEstimator & estimator_,
const Names & queried_columns_, const Names & queried_columns_,
const std::optional<NameSet> & supported_columns_, const std::optional<NameSet> & supported_columns_,
LoggerPtr log_) LoggerPtr log_)
@ -92,7 +92,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons
where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere;
where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere;
where_optimizer_context.is_final = select.final(); where_optimizer_context.is_final = select.final();
where_optimizer_context.use_statistic = context->getSettingsRef().allow_statistic_optimize; where_optimizer_context.use_statistics = context->getSettingsRef().allow_statistics_optimize;
RPNBuilderTreeContext tree_context(context, std::move(block_with_constants), {} /*prepared_sets*/); RPNBuilderTreeContext tree_context(context, std::move(block_with_constants), {} /*prepared_sets*/);
RPNBuilderTreeNode node(select.where().get(), tree_context); RPNBuilderTreeNode node(select.where().get(), tree_context);
@ -123,7 +123,7 @@ MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::op
where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere;
where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere;
where_optimizer_context.is_final = is_final; where_optimizer_context.is_final = is_final;
where_optimizer_context.use_statistic = context->getSettingsRef().allow_statistic_optimize; where_optimizer_context.use_statistics = context->getSettingsRef().allow_statistics_optimize;
RPNBuilderTreeContext tree_context(context); RPNBuilderTreeContext tree_context(context);
RPNBuilderTreeNode node(&filter_dag->findInOutputs(filter_column_name), tree_context); RPNBuilderTreeNode node(&filter_dag->findInOutputs(filter_column_name), tree_context);
@ -273,15 +273,17 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree
/// Do not move conditions involving all queried columns. /// Do not move conditions involving all queried columns.
&& cond.table_columns.size() < queried_columns.size(); && cond.table_columns.size() < queried_columns.size();
if (where_optimizer_context.use_statistic) if (cond.viable)
cond.good = isConditionGood(node, table_columns);
if (where_optimizer_context.use_statistics)
{ {
cond.good = cond.viable; cond.good = cond.viable;
cond.selectivity = estimator.estimateSelectivity(node);
LOG_TEST(log, "Condition {} has selectivity {}", node.getColumnName(), cond.selectivity); cond.estimated_row_count = estimator.estimateRowCount(node);
}
else if (cond.viable) if (node.getASTNode() != nullptr)
{ LOG_DEBUG(log, "Condition {} has estimated row count {}", node.getASTNode()->dumpTree(), cond.estimated_row_count);
cond.good = isConditionGood(node, table_columns);
} }
if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere)

View File

@ -4,7 +4,7 @@
#include <Interpreters/Context_fwd.h> #include <Interpreters/Context_fwd.h>
#include <Storages/SelectQueryInfo.h> #include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/RPNBuilder.h> #include <Storages/MergeTree/RPNBuilder.h>
#include <Storages/Statistics/Estimator.h> #include <Storages/Statistics/ConditionSelectivityEstimator.h>
#include <boost/noncopyable.hpp> #include <boost/noncopyable.hpp>
@ -38,7 +38,7 @@ public:
MergeTreeWhereOptimizer( MergeTreeWhereOptimizer(
std::unordered_map<std::string, UInt64> column_sizes_, std::unordered_map<std::string, UInt64> column_sizes_,
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,
const ConditionEstimator & estimator_, const ConditionSelectivityEstimator & estimator_,
const Names & queried_columns_, const Names & queried_columns_,
const std::optional<NameSet> & supported_columns_, const std::optional<NameSet> & supported_columns_,
LoggerPtr log_); LoggerPtr log_);
@ -76,7 +76,7 @@ private:
bool good = false; bool good = false;
/// the lower the better /// the lower the better
Float64 selectivity = 1.0; Float64 estimated_row_count = 0;
/// Does the condition contain primary key column? /// Does the condition contain primary key column?
/// If so, it is better to move it further to the end of PREWHERE chain depending on minimal position in PK of any /// If so, it is better to move it further to the end of PREWHERE chain depending on minimal position in PK of any
@ -85,7 +85,7 @@ private:
auto tuple() const auto tuple() const
{ {
return std::make_tuple(!viable, !good, -min_position_in_primary_key, selectivity, columns_size, table_columns.size()); return std::make_tuple(!viable, !good, -min_position_in_primary_key, estimated_row_count, columns_size, table_columns.size());
} }
/// Is condition a better candidate for moving to PREWHERE? /// Is condition a better candidate for moving to PREWHERE?
@ -104,7 +104,7 @@ private:
bool move_all_conditions_to_prewhere = false; bool move_all_conditions_to_prewhere = false;
bool move_primary_key_columns_to_end_of_prewhere = false; bool move_primary_key_columns_to_end_of_prewhere = false;
bool is_final = false; bool is_final = false;
bool use_statistic = false; bool use_statistics = false;
}; };
struct OptimizeResult struct OptimizeResult
@ -147,7 +147,7 @@ private:
static NameSet determineArrayJoinedNames(const ASTSelectQuery & select); static NameSet determineArrayJoinedNames(const ASTSelectQuery & select);
const ConditionEstimator estimator; const ConditionSelectivityEstimator estimator;
const NameSet table_columns; const NameSet table_columns;
const Names queried_columns; const Names queried_columns;

View File

@ -20,7 +20,7 @@ MergedBlockOutputStream::MergedBlockOutputStream(
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const NamesAndTypesList & columns_list_, const NamesAndTypesList & columns_list_,
const MergeTreeIndices & skip_indices, const MergeTreeIndices & skip_indices,
const Statistics & statistics, const ColumnsStatistics & statistics,
CompressionCodecPtr default_codec_, CompressionCodecPtr default_codec_,
TransactionID tid, TransactionID tid,
bool reset_columns_, bool reset_columns_,

View File

@ -20,7 +20,7 @@ public:
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const NamesAndTypesList & columns_list_, const NamesAndTypesList & columns_list_,
const MergeTreeIndices & skip_indices, const MergeTreeIndices & skip_indices,
const Statistics & statistics, const ColumnsStatistics & statistics,
CompressionCodecPtr default_codec_, CompressionCodecPtr default_codec_,
TransactionID tid, TransactionID tid,
bool reset_columns_ = false, bool reset_columns_ = false,

View File

@ -16,7 +16,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream(
const Block & header_, const Block & header_,
CompressionCodecPtr default_codec, CompressionCodecPtr default_codec,
const MergeTreeIndices & indices_to_recalc, const MergeTreeIndices & indices_to_recalc,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
WrittenOffsetColumns * offset_columns_, WrittenOffsetColumns * offset_columns_,
const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularity & index_granularity,
const MergeTreeIndexGranularityInfo * index_granularity_info) const MergeTreeIndexGranularityInfo * index_granularity_info)

View File

@ -20,7 +20,7 @@ public:
const Block & header_, const Block & header_,
CompressionCodecPtr default_codec_, CompressionCodecPtr default_codec_,
const MergeTreeIndices & indices_to_recalc_, const MergeTreeIndices & indices_to_recalc_,
const Statistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc_,
WrittenOffsetColumns * offset_columns_ = nullptr, WrittenOffsetColumns * offset_columns_ = nullptr,
const MergeTreeIndexGranularity & index_granularity = {}, const MergeTreeIndexGranularity & index_granularity = {},
const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr); const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr);

View File

@ -130,7 +130,7 @@ static void splitAndModifyMutationCommands(
} }
} }
if (command.type == MutationCommand::Type::MATERIALIZE_INDEX if (command.type == MutationCommand::Type::MATERIALIZE_INDEX
|| command.type == MutationCommand::Type::MATERIALIZE_STATISTIC || command.type == MutationCommand::Type::MATERIALIZE_STATISTICS
|| command.type == MutationCommand::Type::MATERIALIZE_PROJECTION || command.type == MutationCommand::Type::MATERIALIZE_PROJECTION
|| command.type == MutationCommand::Type::MATERIALIZE_TTL || command.type == MutationCommand::Type::MATERIALIZE_TTL
|| command.type == MutationCommand::Type::DELETE || command.type == MutationCommand::Type::DELETE
@ -143,7 +143,7 @@ static void splitAndModifyMutationCommands(
} }
else if (command.type == MutationCommand::Type::DROP_INDEX else if (command.type == MutationCommand::Type::DROP_INDEX
|| command.type == MutationCommand::Type::DROP_PROJECTION || command.type == MutationCommand::Type::DROP_PROJECTION
|| command.type == MutationCommand::Type::DROP_STATISTIC) || command.type == MutationCommand::Type::DROP_STATISTICS)
{ {
for_file_renames.push_back(command); for_file_renames.push_back(command);
} }
@ -258,7 +258,7 @@ static void splitAndModifyMutationCommands(
for_interpreter.push_back(command); for_interpreter.push_back(command);
} }
else if (command.type == MutationCommand::Type::MATERIALIZE_INDEX else if (command.type == MutationCommand::Type::MATERIALIZE_INDEX
|| command.type == MutationCommand::Type::MATERIALIZE_STATISTIC || command.type == MutationCommand::Type::MATERIALIZE_STATISTICS
|| command.type == MutationCommand::Type::MATERIALIZE_PROJECTION || command.type == MutationCommand::Type::MATERIALIZE_PROJECTION
|| command.type == MutationCommand::Type::MATERIALIZE_TTL || command.type == MutationCommand::Type::MATERIALIZE_TTL
|| command.type == MutationCommand::Type::DELETE || command.type == MutationCommand::Type::DELETE
@ -269,7 +269,7 @@ static void splitAndModifyMutationCommands(
} }
else if (command.type == MutationCommand::Type::DROP_INDEX else if (command.type == MutationCommand::Type::DROP_INDEX
|| command.type == MutationCommand::Type::DROP_PROJECTION || command.type == MutationCommand::Type::DROP_PROJECTION
|| command.type == MutationCommand::Type::DROP_STATISTIC) || command.type == MutationCommand::Type::DROP_STATISTICS)
{ {
for_file_renames.push_back(command); for_file_renames.push_back(command);
} }
@ -532,16 +532,16 @@ static ExecuteTTLType shouldExecuteTTL(const StorageMetadataPtr & metadata_snaps
return has_ttl_expression ? ExecuteTTLType::RECALCULATE : ExecuteTTLType::NONE; return has_ttl_expression ? ExecuteTTLType::RECALCULATE : ExecuteTTLType::NONE;
} }
static std::set<StatisticPtr> getStatisticsToRecalculate(const StorageMetadataPtr & metadata_snapshot, const NameSet & materialized_stats) static std::set<ColumnStatisticsPtr> getStatisticsToRecalculate(const StorageMetadataPtr & metadata_snapshot, const NameSet & materialized_stats)
{ {
const auto & stats_factory = MergeTreeStatisticsFactory::instance(); const auto & stats_factory = MergeTreeStatisticsFactory::instance();
std::set<StatisticPtr> stats_to_recalc; std::set<ColumnStatisticsPtr> stats_to_recalc;
const auto & columns = metadata_snapshot->getColumns(); const auto & columns = metadata_snapshot->getColumns();
for (const auto & col_desc : columns) for (const auto & col_desc : columns)
{ {
if (col_desc.stat && materialized_stats.contains(col_desc.name)) if (!col_desc.statistics.empty() && materialized_stats.contains(col_desc.name))
{ {
stats_to_recalc.insert(stats_factory.get(*col_desc.stat)); stats_to_recalc.insert(stats_factory.get(col_desc.statistics));
} }
} }
return stats_to_recalc; return stats_to_recalc;
@ -655,7 +655,7 @@ static NameSet collectFilesToSkip(
const std::set<MergeTreeIndexPtr> & indices_to_recalc, const std::set<MergeTreeIndexPtr> & indices_to_recalc,
const String & mrk_extension, const String & mrk_extension,
const std::set<ProjectionDescriptionRawPtr> & projections_to_recalc, const std::set<ProjectionDescriptionRawPtr> & projections_to_recalc,
const std::set<StatisticPtr> & stats_to_recalc) const std::set<ColumnStatisticsPtr> & stats_to_recalc)
{ {
NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); NameSet files_to_skip = source_part->getFileNamesWithoutChecksums();
@ -683,7 +683,7 @@ static NameSet collectFilesToSkip(
files_to_skip.insert(projection->getDirectoryName()); files_to_skip.insert(projection->getDirectoryName());
for (const auto & stat : stats_to_recalc) for (const auto & stat : stats_to_recalc)
files_to_skip.insert(stat->getFileName() + STAT_FILE_SUFFIX); files_to_skip.insert(stat->getFileName() + STATS_FILE_SUFFIX);
if (isWidePart(source_part)) if (isWidePart(source_part))
{ {
@ -772,11 +772,11 @@ static NameToNameVector collectFilesForRenames(
if (source_part->checksums.has(command.column_name + ".proj")) if (source_part->checksums.has(command.column_name + ".proj"))
add_rename(command.column_name + ".proj", ""); add_rename(command.column_name + ".proj", "");
} }
else if (command.type == MutationCommand::Type::DROP_STATISTIC) else if (command.type == MutationCommand::Type::DROP_STATISTICS)
{ {
for (const auto & statistic_column_name : command.statistic_columns) for (const auto & statistics_column_name : command.statistics_columns)
if (source_part->checksums.has(STAT_FILE_PREFIX + statistic_column_name + STAT_FILE_SUFFIX)) if (source_part->checksums.has(STATS_FILE_PREFIX + statistics_column_name + STATS_FILE_SUFFIX))
add_rename(STAT_FILE_PREFIX + statistic_column_name + STAT_FILE_SUFFIX, ""); add_rename(STATS_FILE_PREFIX + statistics_column_name + STATS_FILE_SUFFIX, "");
} }
else if (isWidePart(source_part)) else if (isWidePart(source_part))
{ {
@ -797,9 +797,9 @@ static NameToNameVector collectFilesForRenames(
if (auto serialization = source_part->tryGetSerialization(command.column_name)) if (auto serialization = source_part->tryGetSerialization(command.column_name))
serialization->enumerateStreams(callback); serialization->enumerateStreams(callback);
/// if we drop a column with statistic, we should also drop the stat file. /// if we drop a column with statistics, we should also drop the stat file.
if (source_part->checksums.has(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX)) if (source_part->checksums.has(STATS_FILE_PREFIX + command.column_name + STATS_FILE_SUFFIX))
add_rename(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX, ""); add_rename(STATS_FILE_PREFIX + command.column_name + STATS_FILE_SUFFIX, "");
} }
else if (command.type == MutationCommand::Type::RENAME_COLUMN) else if (command.type == MutationCommand::Type::RENAME_COLUMN)
{ {
@ -833,9 +833,9 @@ static NameToNameVector collectFilesForRenames(
if (auto serialization = source_part->tryGetSerialization(command.column_name)) if (auto serialization = source_part->tryGetSerialization(command.column_name))
serialization->enumerateStreams(callback); serialization->enumerateStreams(callback);
/// if we rename a column with statistic, we should also rename the stat file. /// if we rename a column with statistics, we should also rename the stat file.
if (source_part->checksums.has(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX)) if (source_part->checksums.has(STATS_FILE_PREFIX + command.column_name + STATS_FILE_SUFFIX))
add_rename(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX, STAT_FILE_PREFIX + command.rename_to + STAT_FILE_SUFFIX); add_rename(STATS_FILE_PREFIX + command.column_name + STATS_FILE_SUFFIX, STATS_FILE_PREFIX + command.rename_to + STATS_FILE_SUFFIX);
} }
else if (command.type == MutationCommand::Type::READ_COLUMN) else if (command.type == MutationCommand::Type::READ_COLUMN)
{ {
@ -1022,7 +1022,7 @@ struct MutationContext
IMergeTreeDataPart::MinMaxIndexPtr minmax_idx; IMergeTreeDataPart::MinMaxIndexPtr minmax_idx;
std::set<MergeTreeIndexPtr> indices_to_recalc; std::set<MergeTreeIndexPtr> indices_to_recalc;
std::set<StatisticPtr> stats_to_recalc; std::set<ColumnStatisticsPtr> stats_to_recalc;
std::set<ProjectionDescriptionRawPtr> projections_to_recalc; std::set<ProjectionDescriptionRawPtr> projections_to_recalc;
MergeTreeData::DataPart::Checksums existing_indices_stats_checksums; MergeTreeData::DataPart::Checksums existing_indices_stats_checksums;
NameSet files_to_skip; NameSet files_to_skip;
@ -1473,12 +1473,12 @@ private:
{ {
if (command.type == MutationCommand::DROP_INDEX) if (command.type == MutationCommand::DROP_INDEX)
removed_indices.insert(command.column_name); removed_indices.insert(command.column_name);
else if (command.type == MutationCommand::DROP_STATISTIC) else if (command.type == MutationCommand::DROP_STATISTICS)
for (const auto & column_name : command.statistic_columns) for (const auto & column_name : command.statistics_columns)
removed_stats.insert(column_name); removed_stats.insert(column_name);
else if (command.type == MutationCommand::RENAME_COLUMN else if (command.type == MutationCommand::RENAME_COLUMN
&& ctx->source_part->checksums.files.contains(STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX)) && ctx->source_part->checksums.files.contains(STATS_FILE_PREFIX + command.column_name + STATS_FILE_SUFFIX))
renamed_stats[STAT_FILE_PREFIX + command.column_name + STAT_FILE_SUFFIX] = STAT_FILE_PREFIX + command.rename_to + STAT_FILE_SUFFIX; renamed_stats[STATS_FILE_PREFIX + command.column_name + STATS_FILE_SUFFIX] = STATS_FILE_PREFIX + command.rename_to + STATS_FILE_SUFFIX;
} }
bool is_full_part_storage = isFullPartStorage(ctx->new_data_part->getDataPartStorage()); bool is_full_part_storage = isFullPartStorage(ctx->new_data_part->getDataPartStorage());
@ -1514,23 +1514,23 @@ private:
} }
} }
Statistics stats_to_rewrite; ColumnsStatistics stats_to_rewrite;
const auto & columns = ctx->metadata_snapshot->getColumns(); const auto & columns = ctx->metadata_snapshot->getColumns();
for (const auto & col : columns) for (const auto & col : columns)
{ {
if (!col.stat || removed_stats.contains(col.name)) if (col.statistics.empty() || removed_stats.contains(col.name))
continue; continue;
if (ctx->materialized_statistics.contains(col.name)) if (ctx->materialized_statistics.contains(col.name))
{ {
stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(*col.stat)); stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(col.statistics));
} }
else else
{ {
/// We do not hard-link statistics which /// We do not hard-link statistics which
/// 1. In `DROP STATISTIC` statement. It is filtered by `removed_stats` /// 1. In `DROP STATISTICS` statement. It is filtered by `removed_stats`
/// 2. Not in column list anymore, including `DROP COLUMN`. It is not touched by this loop. /// 2. Not in column list anymore, including `DROP COLUMN`. It is not touched by this loop.
String stat_file_name = STAT_FILE_PREFIX + col.name + STAT_FILE_SUFFIX; String stat_file_name = STATS_FILE_PREFIX + col.name + STATS_FILE_SUFFIX;
auto it = ctx->source_part->checksums.files.find(stat_file_name); auto it = ctx->source_part->checksums.files.find(stat_file_name);
if (it != ctx->source_part->checksums.files.end()) if (it != ctx->source_part->checksums.files.end())
{ {
@ -1904,7 +1904,7 @@ private:
ctx->updated_header, ctx->updated_header,
ctx->compression_codec, ctx->compression_codec,
std::vector<MergeTreeIndexPtr>(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()), std::vector<MergeTreeIndexPtr>(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()),
Statistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()), ColumnsStatistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()),
nullptr, nullptr,
ctx->source_part->index_granularity, ctx->source_part->index_granularity,
&ctx->source_part->index_granularity_info &ctx->source_part->index_granularity_info

View File

@ -6,7 +6,7 @@
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Parsers/ASTAssignment.h> #include <Parsers/ASTAssignment.h>
#include <Parsers/ASTColumnDeclaration.h> #include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTStatisticDeclaration.h> #include <Parsers/ASTStatisticsDeclaration.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
@ -83,15 +83,15 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
res.index_name = command->index->as<ASTIdentifier &>().name(); res.index_name = command->index->as<ASTIdentifier &>().name();
return res; return res;
} }
else if (command->type == ASTAlterCommand::MATERIALIZE_STATISTIC) else if (command->type == ASTAlterCommand::MATERIALIZE_STATISTICS)
{ {
MutationCommand res; MutationCommand res;
res.ast = command->ptr(); res.ast = command->ptr();
res.type = MATERIALIZE_STATISTIC; res.type = MATERIALIZE_STATISTICS;
if (command->partition) if (command->partition)
res.partition = command->partition->clone(); res.partition = command->partition->clone();
res.predicate = nullptr; res.predicate = nullptr;
res.statistic_columns = command->statistic_decl->as<ASTStatisticDeclaration &>().getColumnNames(); res.statistics_columns = command->statistics_decl->as<ASTStatisticsDeclaration &>().getColumnNames();
return res; return res;
} }
else if (command->type == ASTAlterCommand::MATERIALIZE_PROJECTION) else if (command->type == ASTAlterCommand::MATERIALIZE_PROJECTION)
@ -150,16 +150,16 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
res.clear = true; res.clear = true;
return res; return res;
} }
else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_STATISTIC) else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_STATISTICS)
{ {
MutationCommand res; MutationCommand res;
res.ast = command->ptr(); res.ast = command->ptr();
res.type = MutationCommand::Type::DROP_STATISTIC; res.type = MutationCommand::Type::DROP_STATISTICS;
if (command->partition) if (command->partition)
res.partition = command->partition->clone(); res.partition = command->partition->clone();
if (command->clear_index) if (command->clear_index)
res.clear = true; res.clear = true;
res.statistic_columns = command->statistic_decl->as<ASTStatisticDeclaration &>().getColumnNames(); res.statistics_columns = command->statistics_decl->as<ASTStatisticsDeclaration &>().getColumnNames();
return res; return res;
} }
else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_PROJECTION) else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_PROJECTION)

View File

@ -30,12 +30,12 @@ struct MutationCommand
UPDATE, UPDATE,
MATERIALIZE_INDEX, MATERIALIZE_INDEX,
MATERIALIZE_PROJECTION, MATERIALIZE_PROJECTION,
MATERIALIZE_STATISTIC, MATERIALIZE_STATISTICS,
READ_COLUMN, /// Read column and apply conversions (MODIFY COLUMN alter query). READ_COLUMN, /// Read column and apply conversions (MODIFY COLUMN alter query).
DROP_COLUMN, DROP_COLUMN,
DROP_INDEX, DROP_INDEX,
DROP_PROJECTION, DROP_PROJECTION,
DROP_STATISTIC, DROP_STATISTICS,
MATERIALIZE_TTL, MATERIALIZE_TTL,
RENAME_COLUMN, RENAME_COLUMN,
MATERIALIZE_COLUMN, MATERIALIZE_COLUMN,
@ -51,10 +51,11 @@ struct MutationCommand
/// Columns with corresponding actions /// Columns with corresponding actions
std::unordered_map<String, ASTPtr> column_to_update_expression = {}; std::unordered_map<String, ASTPtr> column_to_update_expression = {};
/// For MATERIALIZE INDEX and PROJECTION and STATISTIC /// For MATERIALIZE INDEX and PROJECTION and STATISTICS
String index_name = {}; String index_name = {};
String projection_name = {}; String projection_name = {};
std::vector<String> statistic_columns = {}; std::vector<String> statistics_columns = {};
std::vector<String> statistics_types = {};
/// For MATERIALIZE INDEX, UPDATE and DELETE. /// For MATERIALIZE INDEX, UPDATE and DELETE.
ASTPtr partition = {}; ASTPtr partition = {};

View File

@ -0,0 +1,201 @@
#include <Storages/Statistics/ConditionSelectivityEstimator.h>
#include <Storages/MergeTree/RPNBuilder.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
void ConditionSelectivityEstimator::ColumnSelectivityEstimator::merge(String part_name, ColumnStatisticsPtr stats)
{
if (part_statistics.contains(part_name))
throw Exception(ErrorCodes::LOGICAL_ERROR, "part {} has been added in column {}", part_name, stats->columnName());
part_statistics[part_name] = stats;
}
Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess(Float64 val, Float64 rows) const
{
if (part_statistics.empty())
return default_normal_cond_factor * rows;
Float64 result = 0;
Float64 part_rows = 0;
for (const auto & [key, estimator] : part_statistics)
{
result += estimator->estimateLess(val);
part_rows += estimator->rowCount();
}
return result * rows / part_rows;
}
Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreater(Float64 val, Float64 rows) const
{
return rows - estimateLess(val, rows);
}
Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(Float64 val, Float64 rows) const
{
if (part_statistics.empty())
{
if (val < - threshold || val > threshold)
return default_normal_cond_factor * rows;
else
return default_good_cond_factor * rows;
}
Float64 result = 0;
Float64 partial_cnt = 0;
for (const auto & [key, estimator] : part_statistics)
{
result += estimator->estimateEqual(val);
partial_cnt += estimator->rowCount();
}
return result * rows / partial_cnt;
}
/// second return value represents how many columns in the node.
static std::pair<String, Int32> tryToExtractSingleColumn(const RPNBuilderTreeNode & node)
{
if (node.isConstant())
{
return {};
}
if (!node.isFunction())
{
auto column_name = node.getColumnName();
return {column_name, 1};
}
auto function_node = node.toFunctionNode();
size_t arguments_size = function_node.getArgumentsSize();
std::pair<String, Int32> result;
for (size_t i = 0; i < arguments_size; ++i)
{
auto function_argument = function_node.getArgumentAt(i);
auto subresult = tryToExtractSingleColumn(function_argument);
if (subresult.second == 0) /// the subnode contains 0 column
continue;
else if (subresult.second > 1) /// the subnode contains more than 1 column
return subresult;
else if (result.second == 0 || result.first == subresult.first) /// subnodes contain same column.
result = subresult;
else
return {"", 2};
}
return result;
}
std::pair<String, Float64> ConditionSelectivityEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const
{
if (!node.isFunction())
return {};
auto function_node = node.toFunctionNode();
if (function_node.getArgumentsSize() != 2)
return {};
String function_name = function_node.getFunctionName();
auto lhs_argument = function_node.getArgumentAt(0);
auto rhs_argument = function_node.getArgumentAt(1);
auto lhs_argument_column_name = lhs_argument.getColumnName();
auto rhs_argument_column_name = rhs_argument.getColumnName();
bool lhs_argument_is_column = column_name == (lhs_argument_column_name);
bool rhs_argument_is_column = column_name == (rhs_argument_column_name);
bool lhs_argument_is_constant = lhs_argument.isConstant();
bool rhs_argument_is_constant = rhs_argument.isConstant();
RPNBuilderTreeNode * constant_node = nullptr;
if (lhs_argument_is_column && rhs_argument_is_constant)
constant_node = &rhs_argument;
else if (lhs_argument_is_constant && rhs_argument_is_column)
constant_node = &lhs_argument;
else
return {};
Field output_value;
DataTypePtr output_type;
if (!constant_node->tryGetConstant(output_value, output_type))
return {};
const auto type = output_value.getType();
Float64 value;
if (type == Field::Types::Int64)
value = output_value.get<Int64>();
else if (type == Field::Types::UInt64)
value = output_value.get<UInt64>();
else if (type == Field::Types::Float64)
value = output_value.get<Float64>();
else
return {};
return std::make_pair(function_name, value);
}
Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode & node) const
{
auto result = tryToExtractSingleColumn(node);
if (result.second != 1)
{
return default_unknown_cond_factor;
}
String col = result.first;
auto it = column_estimators.find(col);
/// If there the estimator of the column is not found or there are no data at all,
/// we use dummy estimation.
bool dummy = total_rows == 0;
ColumnSelectivityEstimator estimator;
if (it != column_estimators.end())
{
estimator = it->second;
}
else
{
dummy = true;
}
auto [op, val] = extractBinaryOp(node, col);
if (op == "equals")
{
if (dummy)
{
if (val < - threshold || val > threshold)
return default_normal_cond_factor * total_rows;
else
return default_good_cond_factor * total_rows;
}
return estimator.estimateEqual(val, total_rows);
}
else if (op == "less" || op == "lessOrEquals")
{
if (dummy)
return default_normal_cond_factor * total_rows;
return estimator.estimateLess(val, total_rows);
}
else if (op == "greater" || op == "greaterOrEquals")
{
if (dummy)
return default_normal_cond_factor * total_rows;
return estimator.estimateGreater(val, total_rows);
}
else
return default_unknown_cond_factor * total_rows;
}
void ConditionSelectivityEstimator::merge(String part_name, UInt64 part_rows, ColumnStatisticsPtr column_stat)
{
if (!part_names.contains(part_name))
{
total_rows += part_rows;
part_names.insert(part_name);
}
if (column_stat != nullptr)
column_estimators[column_stat->columnName()].merge(part_name, column_stat);
}
}

View File

@ -0,0 +1,50 @@
#pragma once
#include <Storages/Statistics/Statistics.h>
namespace DB
{
class RPNBuilderTreeNode;
/// It estimates the selectivity of a condition.
class ConditionSelectivityEstimator
{
private:
friend class ColumnStatistics;
struct ColumnSelectivityEstimator
{
/// We store the part_name and part_statistics.
/// then simply get selectivity for every part_statistics and combine them.
std::map<String, ColumnStatisticsPtr> part_statistics;
void merge(String part_name, ColumnStatisticsPtr stats);
Float64 estimateLess(Float64 val, Float64 rows) const;
Float64 estimateGreater(Float64 val, Float64 rows) const;
Float64 estimateEqual(Float64 val, Float64 rows) const;
};
static constexpr auto default_good_cond_factor = 0.1;
static constexpr auto default_normal_cond_factor = 0.5;
static constexpr auto default_unknown_cond_factor = 1.0;
/// Conditions like "x = N" are considered good if abs(N) > threshold.
/// This is used to assume that condition is likely to have good selectivity.
static constexpr auto threshold = 2;
UInt64 total_rows = 0;
std::set<String> part_names;
std::map<String, ColumnSelectivityEstimator> column_estimators;
std::pair<String, Float64> extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const;
public:
/// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ...
/// Right now we only support simple condition like col = val / col < val
Float64 estimateRowCount(const RPNBuilderTreeNode & node) const;
void merge(String part_name, UInt64 part_rows, ColumnStatisticsPtr column_stat);
};
}

View File

@ -1,137 +0,0 @@
#include <Storages/Statistics/Estimator.h>
#include <Storages/MergeTree/RPNBuilder.h>
namespace DB
{
/// second return value represents how many columns in the node.
static std::pair<std::string, Int32> tryToExtractSingleColumn(const RPNBuilderTreeNode & node)
{
if (node.isConstant())
{
return {};
}
if (!node.isFunction())
{
auto column_name = node.getColumnName();
return {column_name, 1};
}
auto function_node = node.toFunctionNode();
size_t arguments_size = function_node.getArgumentsSize();
std::pair<std::string, Int32> result;
for (size_t i = 0; i < arguments_size; ++i)
{
auto function_argument = function_node.getArgumentAt(i);
auto subresult = tryToExtractSingleColumn(function_argument);
if (subresult.second == 0) /// the subnode contains 0 column
continue;
else if (subresult.second > 1) /// the subnode contains more than 1 column
return subresult;
else if (result.second == 0 || result.first == subresult.first) /// subnodes contain same column.
result = subresult;
else
return {"", 2};
}
return result;
}
std::pair<std::string, Float64> ConditionEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const
{
if (!node.isFunction())
return {};
auto function_node = node.toFunctionNode();
if (function_node.getArgumentsSize() != 2)
return {};
std::string function_name = function_node.getFunctionName();
auto lhs_argument = function_node.getArgumentAt(0);
auto rhs_argument = function_node.getArgumentAt(1);
auto lhs_argument_column_name = lhs_argument.getColumnName();
auto rhs_argument_column_name = rhs_argument.getColumnName();
bool lhs_argument_is_column = column_name == (lhs_argument_column_name);
bool rhs_argument_is_column = column_name == (rhs_argument_column_name);
bool lhs_argument_is_constant = lhs_argument.isConstant();
bool rhs_argument_is_constant = rhs_argument.isConstant();
RPNBuilderTreeNode * constant_node = nullptr;
if (lhs_argument_is_column && rhs_argument_is_constant)
constant_node = &rhs_argument;
else if (lhs_argument_is_constant && rhs_argument_is_column)
constant_node = &lhs_argument;
else
return {};
Field output_value;
DataTypePtr output_type;
if (!constant_node->tryGetConstant(output_value, output_type))
return {};
const auto type = output_value.getType();
Float64 value;
if (type == Field::Types::Int64)
value = output_value.get<Int64>();
else if (type == Field::Types::UInt64)
value = output_value.get<UInt64>();
else if (type == Field::Types::Float64)
value = output_value.get<Float64>();
else
return {};
return std::make_pair(function_name, value);
}
Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) const
{
auto result = tryToExtractSingleColumn(node);
if (result.second != 1)
{
return default_unknown_cond_factor;
}
String col = result.first;
auto it = column_estimators.find(col);
/// If there the estimator of the column is not found or there are no data at all,
/// we use dummy estimation.
bool dummy = total_count == 0;
ColumnEstimator estimator;
if (it != column_estimators.end())
{
estimator = it->second;
}
else
{
dummy = true;
}
auto [op, val] = extractBinaryOp(node, col);
if (op == "equals")
{
if (val < -threshold || val > threshold)
return default_normal_cond_factor;
else
return default_good_cond_factor;
}
else if (op == "less" || op == "lessThan")
{
if (dummy)
return default_normal_cond_factor;
return estimator.estimateLess(val) / total_count;
}
else if (op == "greater" || op == "greaterThan")
{
if (dummy)
return default_normal_cond_factor;
return estimator.estimateGreater(val) / total_count;
}
else
return default_unknown_cond_factor;
}
}

View File

@ -1,111 +0,0 @@
#pragma once
#include <Storages/Statistics/TDigestStatistic.h>
namespace DB
{
class RPNBuilderTreeNode;
/// It estimates the selectivity of a condition.
class ConditionEstimator
{
private:
static constexpr auto default_good_cond_factor = 0.1;
static constexpr auto default_normal_cond_factor = 0.5;
static constexpr auto default_unknown_cond_factor = 1.0;
/// Conditions like "x = N" are considered good if abs(N) > threshold.
/// This is used to assume that condition is likely to have good selectivity.
static constexpr auto threshold = 2;
UInt64 total_count = 0;
/// Minimum estimator for values in a part. It can contains multiple types of statistics.
/// But right now we only have tdigest;
struct PartColumnEstimator
{
UInt64 part_count = 0;
std::shared_ptr<TDigestStatistic> tdigest;
void merge(StatisticPtr statistic)
{
UInt64 cur_part_count = statistic->count();
if (part_count == 0)
part_count = cur_part_count;
if (typeid_cast<TDigestStatistic *>(statistic.get()))
{
tdigest = std::static_pointer_cast<TDigestStatistic>(statistic);
}
}
Float64 estimateLess(Float64 val) const
{
if (tdigest != nullptr)
return tdigest -> estimateLess(val);
return part_count * default_normal_cond_factor;
}
Float64 estimateGreator(Float64 val) const
{
if (tdigest != nullptr)
return part_count - tdigest -> estimateLess(val);
return part_count * default_normal_cond_factor;
}
};
/// An estimator for a column consists of several PartColumnEstimator.
/// We simply get selectivity for every part estimator and combine the result.
struct ColumnEstimator
{
std::map<std::string, PartColumnEstimator> estimators;
void merge(std::string part_name, StatisticPtr statistic)
{
estimators[part_name].merge(statistic);
}
Float64 estimateLess(Float64 val) const
{
if (estimators.empty())
return default_normal_cond_factor;
Float64 result = 0;
for (const auto & [key, estimator] : estimators)
result += estimator.estimateLess(val);
return result;
}
Float64 estimateGreater(Float64 val) const
{
if (estimators.empty())
return default_normal_cond_factor;
Float64 result = 0;
for (const auto & [key, estimator] : estimators)
result += estimator.estimateGreator(val);
return result;
}
};
std::map<String, ColumnEstimator> column_estimators;
/// std::optional<std::string> extractSingleColumn(const RPNBuilderTreeNode & node) const;
std::pair<std::string, Float64> extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const;
public:
ConditionEstimator() = default;
/// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ...
/// Right now we only support simple condition like col = val / col < val
Float64 estimateSelectivity(const RPNBuilderTreeNode & node) const;
void merge(std::string part_name, UInt64 part_count, StatisticPtr statistic)
{
total_count += part_count;
if (statistic != nullptr)
column_estimators[statistic->columnName()].merge(part_name, statistic);
}
};
}

View File

@ -1,11 +1,14 @@
#include <optional> #include <optional>
#include <numeric> #include <numeric>
#include <DataTypes/DataTypeNullable.h>
#include <Storages/Statistics/Statistics.h> #include <Storages/Statistics/Statistics.h>
#include <Storages/Statistics/TDigestStatistic.h> #include <Storages/Statistics/ConditionSelectivityEstimator.h>
#include <Storages/Statistics/TDigestStatistics.h>
#include <Storages/Statistics/UniqStatistics.h>
#include <Storages/StatisticsDescription.h> #include <Storages/StatisticsDescription.h>
#include <Storages/ColumnsDescription.h> #include <Storages/ColumnsDescription.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/Exception.h> #include <Common/Exception.h>
namespace DB namespace DB
@ -15,39 +18,133 @@ namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int INCORRECT_QUERY; extern const int INCORRECT_QUERY;
extern const int ILLEGAL_STATISTIC;
} }
void MergeTreeStatisticsFactory::registerCreator(StatisticType stat_type, Creator creator) /// Version / bitmask of statistics / data of statistics /
enum StatisticsFileVersion : UInt16
{ {
if (!creators.emplace(stat_type, std::move(creator)).second) V0 = 0,
throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistic creator type {} is not unique", stat_type); };
}
void MergeTreeStatisticsFactory::registerValidator(StatisticType stat_type, Validator validator) IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) {}
ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_)
: stats_desc(stats_desc_), rows(0)
{ {
if (!validators.emplace(stat_type, std::move(validator)).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistic validator type {} is not unique", stat_type);
} }
StatisticPtr TDigestCreator(const StatisticDescription & stat) void ColumnStatistics::update(const ColumnPtr & column)
{ {
return StatisticPtr(new TDigestStatistic(stat)); rows += column->size();
for (const auto & iter : stats)
{
iter.second->update(column);
}
} }
void TDigestValidator(const StatisticDescription &, DataTypePtr data_type) Float64 ColumnStatistics::estimateLess(Float64 val) const
{ {
data_type = removeNullable(data_type); if (stats.contains(StatisticsType::TDigest))
if (!data_type->isValueRepresentedByNumber()) return std::static_pointer_cast<TDigestStatistics>(stats.at(StatisticsType::TDigest))->estimateLess(val);
throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "TDigest does not support type {}", data_type->getName()); return rows * ConditionSelectivityEstimator::default_normal_cond_factor;
} }
Float64 ColumnStatistics::estimateGreater(Float64 val) const
{
return rows - estimateLess(val);
}
Float64 ColumnStatistics::estimateEqual(Float64 val) const
{
if (stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest))
{
auto uniq_static = std::static_pointer_cast<UniqStatistics>(stats.at(StatisticsType::Uniq));
/// 2048 is the default number of buckets in TDigest. In this case, TDigest stores exactly one value (with many rows)
/// for every bucket.
if (uniq_static->getCardinality() < 2048)
{
auto tdigest_static = std::static_pointer_cast<TDigestStatistics>(stats.at(StatisticsType::TDigest));
return tdigest_static->estimateEqual(val);
}
}
if (val < - ConditionSelectivityEstimator::threshold || val > ConditionSelectivityEstimator::threshold)
return rows * ConditionSelectivityEstimator::default_normal_cond_factor;
else
return rows * ConditionSelectivityEstimator::default_good_cond_factor;
}
void ColumnStatistics::serialize(WriteBuffer & buf)
{
writeIntBinary(V0, buf);
UInt64 stat_types_mask = 0;
for (const auto & [type, _]: stats)
stat_types_mask |= 1 << UInt8(type);
writeIntBinary(stat_types_mask, buf);
/// We write some basic statistics
writeIntBinary(rows, buf);
/// We write complex statistics
for (const auto & [type, stat_ptr]: stats)
stat_ptr->serialize(buf);
}
void ColumnStatistics::deserialize(ReadBuffer &buf)
{
UInt16 version;
readIntBinary(version, buf);
if (version != V0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown file format version: {}", version);
UInt64 stat_types_mask = 0;
readIntBinary(stat_types_mask, buf);
readIntBinary(rows, buf);
for (auto it = stats.begin(); it != stats.end();)
{
if (!(stat_types_mask & 1 << UInt8(it->first)))
{
stats.erase(it++);
}
else
{
it->second->deserialize(buf);
++it;
}
}
}
String ColumnStatistics::getFileName() const
{
return STATS_FILE_PREFIX + columnName();
}
const String & ColumnStatistics::columnName() const
{
return stats_desc.column_name;
}
UInt64 ColumnStatistics::rowCount() const
{
return rows;
}
void MergeTreeStatisticsFactory::registerCreator(StatisticsType stats_type, Creator creator)
{
if (!creators.emplace(stats_type, std::move(creator)).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistics creator type {} is not unique", stats_type);
}
void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Validator validator)
{
if (!validators.emplace(stats_type, std::move(validator)).second)
throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistics validator type {} is not unique", stats_type);
}
MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() MergeTreeStatisticsFactory::MergeTreeStatisticsFactory()
{ {
registerCreator(TDigest, TDigestCreator); registerCreator(StatisticsType::TDigest, TDigestCreator);
registerValidator(TDigest, TDigestValidator); registerCreator(StatisticsType::Uniq, UniqCreator);
registerValidator(StatisticsType::TDigest, TDigestValidator);
registerValidator(StatisticsType::Uniq, UniqValidator);
} }
MergeTreeStatisticsFactory & MergeTreeStatisticsFactory::instance() MergeTreeStatisticsFactory & MergeTreeStatisticsFactory::instance()
@ -56,33 +153,42 @@ MergeTreeStatisticsFactory & MergeTreeStatisticsFactory::instance()
return instance; return instance;
} }
void MergeTreeStatisticsFactory::validate(const StatisticDescription & stat, DataTypePtr data_type) const void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & stats, DataTypePtr data_type) const
{ {
auto it = validators.find(stat.type); for (const auto & [type, desc] : stats.types_to_desc)
if (it == validators.end())
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown Statistic type '{}'", stat.type); auto it = validators.find(type);
if (it == validators.end())
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown Statistic type '{}'", type);
}
it->second(desc, data_type);
} }
it->second(stat, data_type);
} }
StatisticPtr MergeTreeStatisticsFactory::get(const StatisticDescription & stat) const ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescription & stats) const
{ {
auto it = creators.find(stat.type); ColumnStatisticsPtr column_stat = std::make_shared<ColumnStatistics>(stats);
if (it == creators.end()) for (const auto & [type, desc] : stats.types_to_desc)
{ {
throw Exception(ErrorCodes::INCORRECT_QUERY, auto it = creators.find(type);
"Unknown Statistic type '{}'. Available types: tdigest", stat.type); if (it == creators.end())
{
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Unknown Statistic type '{}'. Available types: tdigest, uniq", type);
}
auto stat_ptr = (it->second)(desc, stats.data_type);
column_stat->stats[type] = stat_ptr;
} }
return std::make_shared<TDigestStatistic>(stat); return column_stat;
} }
Statistics MergeTreeStatisticsFactory::getMany(const ColumnsDescription & columns) const ColumnsStatistics MergeTreeStatisticsFactory::getMany(const ColumnsDescription & columns) const
{ {
Statistics result; ColumnsStatistics result;
for (const auto & col : columns) for (const auto & col : columns)
if (col.stat) if (!col.statistics.empty())
result.push_back(get(*col.stat)); result.push_back(get(col.statistics));
return result; return result;
} }

View File

@ -1,12 +1,8 @@
#pragma once #pragma once
#include <cstddef>
#include <memory> #include <memory>
#include <optional>
#include <boost/core/noncopyable.hpp> #include <boost/core/noncopyable.hpp>
#include <AggregateFunctions/QuantileTDigest.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>
@ -14,38 +10,23 @@
#include <Storages/StatisticsDescription.h> #include <Storages/StatisticsDescription.h>
/// this is for user-defined statistic.
constexpr auto STAT_FILE_PREFIX = "statistic_";
constexpr auto STAT_FILE_SUFFIX = ".stat";
namespace DB namespace DB
{ {
class IStatistic; /// this is for user-defined statistic.
using StatisticPtr = std::shared_ptr<IStatistic>; constexpr auto STATS_FILE_PREFIX = "statistics_";
using Statistics = std::vector<StatisticPtr>; constexpr auto STATS_FILE_SUFFIX = ".stats";
/// Statistic contains the distribution of values in a column. /// Statistics describe properties of the values in the column,
/// right now we support /// e.g. how many unique values exist,
/// - tdigest /// what are the N most frequent values,
class IStatistic /// how frequent is a value V, etc.
class IStatistics
{ {
public: public:
explicit IStatistic(const StatisticDescription & stat_) explicit IStatistics(const SingleStatisticsDescription & stat_);
: stat(stat_)
{
}
virtual ~IStatistic() = default;
String getFileName() const virtual ~IStatistics() = default;
{
return STAT_FILE_PREFIX + columnName();
}
const String & columnName() const
{
return stat.column_name;
}
virtual void serialize(WriteBuffer & buf) = 0; virtual void serialize(WriteBuffer & buf) = 0;
@ -53,40 +34,68 @@ public:
virtual void update(const ColumnPtr & column) = 0; virtual void update(const ColumnPtr & column) = 0;
virtual UInt64 count() = 0;
protected: protected:
SingleStatisticsDescription stat;
};
StatisticDescription stat; using StatisticsPtr = std::shared_ptr<IStatistics>;
class ColumnStatistics
{
public:
explicit ColumnStatistics(const ColumnStatisticsDescription & stats_);
void serialize(WriteBuffer & buf);
void deserialize(ReadBuffer & buf);
String getFileName() const;
const String & columnName() const;
UInt64 rowCount() const;
void update(const ColumnPtr & column);
Float64 estimateLess(Float64 val) const;
Float64 estimateGreater(Float64 val) const;
Float64 estimateEqual(Float64 val) const;
private:
friend class MergeTreeStatisticsFactory;
ColumnStatisticsDescription stats_desc;
std::map<StatisticsType, StatisticsPtr> stats;
UInt64 rows; /// the number of rows of the column
}; };
class ColumnsDescription; class ColumnsDescription;
using ColumnStatisticsPtr = std::shared_ptr<ColumnStatistics>;
using ColumnsStatistics = std::vector<ColumnStatisticsPtr>;
class MergeTreeStatisticsFactory : private boost::noncopyable class MergeTreeStatisticsFactory : private boost::noncopyable
{ {
public: public:
static MergeTreeStatisticsFactory & instance(); static MergeTreeStatisticsFactory & instance();
void validate(const StatisticDescription & stat, DataTypePtr data_type) const; void validate(const ColumnStatisticsDescription & stats, DataTypePtr data_type) const;
using Creator = std::function<StatisticPtr(const StatisticDescription & stat)>; using Creator = std::function<StatisticsPtr(const SingleStatisticsDescription & stats, DataTypePtr data_type)>;
using Validator = std::function<void(const StatisticDescription & stat, DataTypePtr data_type)>; using Validator = std::function<void(const SingleStatisticsDescription & stats, DataTypePtr data_type)>;
StatisticPtr get(const StatisticDescription & stat) const; ColumnStatisticsPtr get(const ColumnStatisticsDescription & stats) const;
Statistics getMany(const ColumnsDescription & columns) const; ColumnsStatistics getMany(const ColumnsDescription & columns) const;
void registerCreator(StatisticType type, Creator creator); void registerCreator(StatisticsType type, Creator creator);
void registerValidator(StatisticType type, Validator validator); void registerValidator(StatisticsType type, Validator validator);
protected: protected:
MergeTreeStatisticsFactory(); MergeTreeStatisticsFactory();
private: private:
using Creators = std::unordered_map<StatisticType, Creator>; using Creators = std::unordered_map<StatisticsType, Creator>;
using Validators = std::unordered_map<StatisticType, Validator>; using Validators = std::unordered_map<StatisticsType, Validator>;
Creators creators; Creators creators;
Validators validators; Validators validators;
}; };

View File

@ -1,38 +0,0 @@
#include <Storages/Statistics/TDigestStatistic.h>
namespace DB
{
Float64 TDigestStatistic::estimateLess(Float64 val) const
{
return data.getCountLessThan(val);
}
void TDigestStatistic::serialize(WriteBuffer & buf)
{
data.serialize(buf);
}
void TDigestStatistic::deserialize(ReadBuffer & buf)
{
data.deserialize(buf);
}
void TDigestStatistic::update(const ColumnPtr & column)
{
size_t size = column->size();
for (size_t i = 0; i < size; ++i)
{
/// TODO: support more types.
Float64 value = column->getFloat64(i);
data.add(value, 1);
}
}
UInt64 TDigestStatistic::count()
{
return static_cast<UInt64>(data.count);
}
}

View File

@ -1,28 +0,0 @@
#pragma once
#include <Storages/Statistics/Statistics.h>
namespace DB
{
/// TDigestStatistic is a kind of histogram.
class TDigestStatistic : public IStatistic
{
QuantileTDigest<Float64> data;
public:
explicit TDigestStatistic(const StatisticDescription & stat_) : IStatistic(stat_)
{
}
Float64 estimateLess(Float64 val) const;
void serialize(WriteBuffer & buf) override;
void deserialize(ReadBuffer & buf) override;
void update(const ColumnPtr & column) override;
UInt64 count() override;
};
}

View File

@ -0,0 +1,60 @@
#include <Storages/Statistics/TDigestStatistics.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_STATISTICS;
}
TDigestStatistics::TDigestStatistics(const SingleStatisticsDescription & stat_):
IStatistics(stat_)
{
}
Float64 TDigestStatistics::estimateLess(Float64 val) const
{
return data.getCountLessThan(val);
}
Float64 TDigestStatistics::estimateEqual(Float64 val) const
{
return data.getCountEqual(val);
}
void TDigestStatistics::serialize(WriteBuffer & buf)
{
data.serialize(buf);
}
void TDigestStatistics::deserialize(ReadBuffer & buf)
{
data.deserialize(buf);
}
void TDigestStatistics::update(const ColumnPtr & column)
{
size_t size = column->size();
for (size_t i = 0; i < size; ++i)
{
/// TODO: support more types.
Float64 value = column->getFloat64(i);
data.add(value, 1);
}
}
StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr)
{
return std::make_shared<TDigestStatistics>(stat);
}
void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
{
data_type = removeNullable(data_type);
if (!data_type->isValueRepresentedByNumber())
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' does not support type {}", data_type->getName());
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Storages/Statistics/Statistics.h>
#include <AggregateFunctions/QuantileTDigest.h>
namespace DB
{
/// TDigestStatistic is a kind of histogram.
class TDigestStatistics : public IStatistics
{
public:
explicit TDigestStatistics(const SingleStatisticsDescription & stat_);
Float64 estimateLess(Float64 val) const;
Float64 estimateEqual(Float64 val) const;
void serialize(WriteBuffer & buf) override;
void deserialize(ReadBuffer & buf) override;
void update(const ColumnPtr & column) override;
private:
QuantileTDigest<Float64> data;
};
StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr);
void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type);
}

View File

@ -0,0 +1,66 @@
#include <Storages/Statistics/UniqStatistics.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_STATISTICS;
}
UniqStatistics::UniqStatistics(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type)
: IStatistics(stat_)
{
arena = std::make_unique<Arena>();
AggregateFunctionProperties properties;
collector = AggregateFunctionFactory::instance().get("uniq", NullsAction::IGNORE_NULLS, {data_type}, Array(), properties);
data = arena->alignedAlloc(collector->sizeOfData(), collector->alignOfData());
collector->create(data);
}
UniqStatistics::~UniqStatistics()
{
collector->destroy(data);
}
UInt64 UniqStatistics::getCardinality()
{
auto column = DataTypeUInt64().createColumn();
collector->insertResultInto(data, *column, nullptr);
return column->getUInt(0);
}
void UniqStatistics::serialize(WriteBuffer & buf)
{
collector->serialize(data, buf);
}
void UniqStatistics::deserialize(ReadBuffer & buf)
{
collector->deserialize(data, buf);
}
void UniqStatistics::update(const ColumnPtr & column)
{
/// TODO(hanfei): For low cardinality, it's very slow to convert to full column. We can read the dictionary directly.
/// Here we intend to avoid crash in CI.
auto col_ptr = column->convertToFullColumnIfLowCardinality();
const IColumn * raw_ptr = col_ptr.get();
collector->addBatchSinglePlace(0, column->size(), data, &(raw_ptr), nullptr);
}
void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
{
data_type = removeNullable(data_type);
if (!data_type->isValueRepresentedByNumber())
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' does not support type {}", data_type->getName());
}
StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type)
{
return std::make_shared<UniqStatistics>(stat, data_type);
}
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <Common/Arena.h>
#include <Storages/Statistics/Statistics.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
namespace DB
{
class UniqStatistics : public IStatistics
{
public:
UniqStatistics(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type);
~UniqStatistics() override;
UInt64 getCardinality();
void serialize(WriteBuffer & buf) override;
void deserialize(ReadBuffer & buf) override;
void update(const ColumnPtr & column) override;
private:
std::unique_ptr<Arena> arena;
AggregateFunctionPtr collector;
AggregateDataPtr data;
};
StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type);
void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type);
}

View File

@ -1,6 +1,6 @@
#include <gtest/gtest.h> #include <gtest/gtest.h>
#include <Storages/Statistics/Statistics.h> #include <Storages/Statistics/TDigestStatistics.h>
TEST(Statistics, TDigestLessThan) TEST(Statistics, TDigestLessThan)
{ {

View File

@ -1,14 +1,16 @@
#include <Storages/StatisticsDescription.h>
#include <base/defines.h> #include <base/defines.h>
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTStatisticDeclaration.h> #include <Parsers/ASTStatisticsDeclaration.h>
#include <Parsers/formatAST.h> #include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Parsers/ParserCreateQuery.h> #include <Parsers/ParserCreateQuery.h>
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <Storages/extractKeyExpressionList.h> #include <Storages/extractKeyExpressionList.h>
#include <Storages/StatisticsDescription.h>
#include <Storages/ColumnsDescription.h> #include <Storages/ColumnsDescription.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
@ -19,93 +21,187 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int INCORRECT_QUERY; extern const int INCORRECT_QUERY;
extern const int ILLEGAL_STATISTICS;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
}; };
StatisticDescription & StatisticDescription::operator=(const StatisticDescription & other) SingleStatisticsDescription & SingleStatisticsDescription::operator=(const SingleStatisticsDescription & other)
{ {
if (this == &other) if (this == &other)
return *this; return *this;
type = other.type; type = other.type;
column_name = other.column_name;
ast = other.ast ? other.ast->clone() : nullptr; ast = other.ast ? other.ast->clone() : nullptr;
return *this; return *this;
} }
StatisticDescription & StatisticDescription::operator=(StatisticDescription && other) noexcept SingleStatisticsDescription & SingleStatisticsDescription::operator=(SingleStatisticsDescription && other) noexcept
{ {
if (this == &other) if (this == &other)
return *this; return *this;
type = std::exchange(other.type, StatisticType{}); type = std::exchange(other.type, StatisticsType{});
column_name = std::move(other.column_name);
ast = other.ast ? other.ast->clone() : nullptr; ast = other.ast ? other.ast->clone() : nullptr;
other.ast.reset(); other.ast.reset();
return *this; return *this;
} }
StatisticType stringToType(String type) static StatisticsType stringToStatisticsType(String type)
{ {
if (type == "tdigest") if (type == "tdigest")
return TDigest; return StatisticsType::TDigest;
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}. We only support statistic type `tdigest` right now.", type); if (type == "uniq")
return StatisticsType::Uniq;
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type);
} }
String StatisticDescription::getTypeName() const String SingleStatisticsDescription::getTypeName() const
{ {
if (type == TDigest) switch (type)
return "tdigest";
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}. We only support statistic type `tdigest` right now.", type);
}
std::vector<StatisticDescription> StatisticDescription::getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns)
{
const auto * stat_definition = definition_ast->as<ASTStatisticDeclaration>();
if (!stat_definition)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create statistic from non ASTStatisticDeclaration AST");
std::vector<StatisticDescription> stats;
stats.reserve(stat_definition->columns->children.size());
for (const auto & column_ast : stat_definition->columns->children)
{ {
StatisticDescription stat; case StatisticsType::TDigest:
stat.type = stringToType(Poco::toLower(stat_definition->type)); return "TDigest";
String column_name = column_ast->as<ASTIdentifier &>().name(); case StatisticsType::Uniq:
return "Uniq";
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type);
}
}
if (!columns.hasPhysical(column_name)) SingleStatisticsDescription::SingleStatisticsDescription(StatisticsType type_, ASTPtr ast_)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", column_name); : type(type_), ast(ast_)
{}
const auto & column = columns.getPhysical(column_name); bool SingleStatisticsDescription::operator==(const SingleStatisticsDescription & other) const
stat.column_name = column.name; {
stat.ast = makeASTFunction("STATISTIC", std::make_shared<ASTIdentifier>(stat_definition->type)); return type == other.type;
stats.push_back(stat); }
bool ColumnStatisticsDescription::operator==(const ColumnStatisticsDescription & other) const
{
return types_to_desc == other.types_to_desc;
}
bool ColumnStatisticsDescription::empty() const
{
return types_to_desc.empty();
}
bool ColumnStatisticsDescription::contains(const String & stat_type) const
{
return types_to_desc.contains(stringToStatisticsType(stat_type));
}
void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & other, const String & merging_column_name, DataTypePtr merging_column_type, bool if_not_exists)
{
chassert(merging_column_type);
if (column_name.empty())
{
column_name = merging_column_name;
data_type = merging_column_type;
} }
if (stats.empty()) for (const auto & [stats_type, stats_desc]: other.types_to_desc)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistic column list"); {
if (!if_not_exists && types_to_desc.contains(stats_type))
{
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, column_name);
}
else if (!types_to_desc.contains(stats_type))
types_to_desc.emplace(stats_type, stats_desc);
}
}
void ColumnStatisticsDescription::assign(const ColumnStatisticsDescription & other)
{
if (other.column_name != column_name)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", column_name, other.column_name);
types_to_desc = other.types_to_desc;
}
void ColumnStatisticsDescription::clear()
{
types_to_desc.clear();
}
std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns)
{
const auto * stat_definition_ast = definition_ast->as<ASTStatisticsDeclaration>();
if (!stat_definition_ast)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot cast AST to ASTSingleStatisticsDeclaration");
StatisticsTypeDescMap statistics_types;
for (const auto & stat_ast : stat_definition_ast->types->children)
{
String stat_type_name = stat_ast->as<ASTFunction &>().name;
auto stat_type = stringToStatisticsType(Poco::toLower(stat_type_name));
if (statistics_types.contains(stat_type))
throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistics type {} was specified more than once", stat_type_name);
SingleStatisticsDescription stat(stat_type, stat_ast->clone());
statistics_types.emplace(stat.type, stat);
}
std::vector<ColumnStatisticsDescription> result;
result.reserve(stat_definition_ast->columns->children.size());
for (const auto & column_ast : stat_definition_ast->columns->children)
{
ColumnStatisticsDescription stats;
String physical_column_name = column_ast->as<ASTIdentifier &>().name();
if (!columns.hasPhysical(physical_column_name))
throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", physical_column_name);
const auto & column = columns.getPhysical(physical_column_name);
stats.column_name = column.name;
stats.types_to_desc = statistics_types;
result.push_back(stats);
}
if (result.empty())
throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistics column list is not allowed.");
return result;
}
ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column)
{
const auto & stat_type_list_ast = column.stat_type->as<ASTFunction &>().arguments;
if (stat_type_list_ast->children.empty())
throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistics type for column {}", queryToString(column));
ColumnStatisticsDescription stats;
stats.column_name = column.name;
for (const auto & ast : stat_type_list_ast->children)
{
const auto & stat_type = ast->as<const ASTFunction &>().name;
SingleStatisticsDescription stat(stringToStatisticsType(Poco::toLower(stat_type)), ast->clone());
if (stats.types_to_desc.contains(stat.type))
throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", stats.column_name, stat_type);
stats.types_to_desc.emplace(stat.type, std::move(stat));
}
return stats; return stats;
} }
String queryToString(const IAST & query); ASTPtr ColumnStatisticsDescription::getAST() const
StatisticDescription StatisticDescription::getStatisticFromColumnDeclaration(const ASTColumnDeclaration & column)
{ {
const auto & stat_type_list_ast = column.stat_type->as<ASTFunction &>().arguments; auto function_node = std::make_shared<ASTFunction>();
if (stat_type_list_ast->children.size() != 1) function_node->name = "STATISTICS";
throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect only one statistic type for column {}", queryToString(column)); function_node->arguments = std::make_shared<ASTExpressionList>();
for (const auto & [type, desc] : types_to_desc)
const auto & stat_type = stat_type_list_ast->children[0]->as<ASTFunction &>().name; {
if (desc.ast == nullptr)
StatisticDescription stat; throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown ast");
stat.type = stringToType(Poco::toLower(stat_type)); function_node->arguments->children.push_back(desc.ast);
stat.column_name = column.name; }
stat.ast = column.stat_type; function_node->children.push_back(function_node->arguments);
return function_node;
return stat;
} }
} }

View File

@ -1,45 +1,66 @@
#pragma once #pragma once
#include <DataTypes/IDataType.h>
#include <Parsers/IAST_fwd.h> #include <Parsers/IAST_fwd.h>
#include <Parsers/ASTColumnDeclaration.h> #include <Parsers/ASTColumnDeclaration.h>
#include <base/types.h> #include <base/types.h>
namespace DB namespace DB
{ {
enum StatisticType enum class StatisticsType : UInt8
{ {
TDigest = 0, TDigest = 0,
Uniq = 1,
Max = 63,
}; };
class ColumnsDescription; struct SingleStatisticsDescription
struct StatisticDescription
{ {
/// the type of statistic, right now it's only tdigest. StatisticsType type;
StatisticType type;
/// Names of statistic columns
String column_name;
ASTPtr ast; ASTPtr ast;
String getTypeName() const; String getTypeName() const;
StatisticDescription() = default; SingleStatisticsDescription() = delete;
StatisticDescription(const StatisticDescription & other) { *this = other; } SingleStatisticsDescription(StatisticsType type_, ASTPtr ast_);
StatisticDescription & operator=(const StatisticDescription & other);
StatisticDescription(StatisticDescription && other) noexcept { *this = std::move(other); }
StatisticDescription & operator=(StatisticDescription && other) noexcept;
bool operator==(const StatisticDescription & other) const SingleStatisticsDescription(const SingleStatisticsDescription & other) { *this = other; }
{ SingleStatisticsDescription & operator=(const SingleStatisticsDescription & other);
return type == other.type && column_name == other.column_name; SingleStatisticsDescription(SingleStatisticsDescription && other) noexcept { *this = std::move(other); }
} SingleStatisticsDescription & operator=(SingleStatisticsDescription && other) noexcept;
static StatisticDescription getStatisticFromColumnDeclaration(const ASTColumnDeclaration & column); bool operator==(const SingleStatisticsDescription & other) const;
};
static std::vector<StatisticDescription> getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); class ColumnsDescription;
struct ColumnStatisticsDescription
{
bool operator==(const ColumnStatisticsDescription & other) const;
bool empty() const;
bool contains(const String & stat_type) const;
void merge(const ColumnStatisticsDescription & other, const String & column_name, DataTypePtr column_type, bool if_not_exists);
void assign(const ColumnStatisticsDescription & other);
void clear();
ASTPtr getAST() const;
static std::vector<ColumnStatisticsDescription> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns);
static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column);
using StatisticsTypeDescMap = std::map<StatisticsType, SingleStatisticsDescription>;
StatisticsTypeDescMap types_to_desc;
String column_name;
DataTypePtr data_type;
}; };
} }

View File

@ -1,7 +1,7 @@
<clickhouse> <clickhouse>
<profiles> <profiles>
<default> <default>
<allow_experimental_statistic>1</allow_experimental_statistic> <allow_experimental_statistics>1</allow_experimental_statistics>
</default> </default>
</profiles> </profiles>
</clickhouse> </clickhouse>

View File

@ -34,14 +34,14 @@ def check_stat_file_on_disk(node, table, part_name, column_name, exist):
[ [
"bash", "bash",
"-c", "-c",
"find {p} -type f -name statistic_{col}.stat".format( "find {p} -type f -name statistics_{col}.stats".format(
p=part_path, col=column_name p=part_path, col=column_name
), ),
], ],
privileged=True, privileged=True,
) )
logging.debug( logging.debug(
f"Checking stat file in {part_path} for column {column_name}, got {output}" f"Checking stats file in {part_path} for column {column_name}, got {output}"
) )
if exist: if exist:
assert len(output) != 0 assert len(output) != 0
@ -56,26 +56,26 @@ def run_test_single_node(started_cluster):
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "b", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "b", True)
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "c", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "c", True)
node1.query("ALTER TABLE test_stat DROP STATISTIC a type tdigest") node1.query("ALTER TABLE test_stat DROP STATISTICS a")
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "a", False) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "a", False)
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "b", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "b", True)
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "c", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "c", True)
node1.query("ALTER TABLE test_stat CLEAR STATISTIC b, c type tdigest") node1.query("ALTER TABLE test_stat CLEAR STATISTICS b, c")
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "a", False) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "a", False)
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "b", False) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "b", False)
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "c", False) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "c", False)
node1.query("ALTER TABLE test_stat MATERIALIZE STATISTIC b, c type tdigest") node1.query("ALTER TABLE test_stat MATERIALIZE STATISTICS b, c")
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "a", False) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "a", False)
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "b", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "b", True)
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "c", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "c", True)
node1.query("ALTER TABLE test_stat ADD STATISTIC a type tdigest") node1.query("ALTER TABLE test_stat ADD STATISTICS a type tdigest")
node1.query("ALTER TABLE test_stat MATERIALIZE STATISTIC a type tdigest") node1.query("ALTER TABLE test_stat MATERIALIZE STATISTICS a")
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_5", "a", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_5", "a", True)
check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_5", "b", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_5", "b", True)
@ -104,7 +104,7 @@ def test_single_node_wide(started_cluster):
node1.query( node1.query(
""" """
CREATE TABLE test_stat(a Int64 STATISTIC(tdigest), b Int64 STATISTIC(tdigest), c Int64 STATISTIC(tdigest)) CREATE TABLE test_stat(a Int64 STATISTICS(tdigest), b Int64 STATISTICS(tdigest), c Int64 STATISTICS(tdigest))
ENGINE = MergeTree() ORDER BY a ENGINE = MergeTree() ORDER BY a
SETTINGS min_bytes_for_wide_part = 0; SETTINGS min_bytes_for_wide_part = 0;
""" """
@ -117,7 +117,7 @@ def test_single_node_normal(started_cluster):
node1.query( node1.query(
""" """
CREATE TABLE test_stat(a Int64 STATISTIC(tdigest), b Int64 STATISTIC(tdigest), c Int64 STATISTIC(tdigest)) CREATE TABLE test_stat(a Int64 STATISTICS(tdigest), b Int64 STATISTICS(tdigest), c Int64 STATISTICS(tdigest))
ENGINE = MergeTree() ORDER BY a; ENGINE = MergeTree() ORDER BY a;
""" """
) )

View File

@ -24,10 +24,11 @@ ALTER DROP INDEX ['DROP INDEX'] TABLE ALTER INDEX
ALTER MATERIALIZE INDEX ['MATERIALIZE INDEX'] TABLE ALTER INDEX ALTER MATERIALIZE INDEX ['MATERIALIZE INDEX'] TABLE ALTER INDEX
ALTER CLEAR INDEX ['CLEAR INDEX'] TABLE ALTER INDEX ALTER CLEAR INDEX ['CLEAR INDEX'] TABLE ALTER INDEX
ALTER INDEX ['INDEX'] \N ALTER TABLE ALTER INDEX ['INDEX'] \N ALTER TABLE
ALTER ADD STATISTIC ['ALTER ADD STATISTIC'] TABLE ALTER STATISTIC ALTER ADD STATISTICS ['ALTER ADD STATISTIC'] TABLE ALTER STATISTICS
ALTER DROP STATISTIC ['ALTER DROP STATISTIC'] TABLE ALTER STATISTIC ALTER DROP STATISTICS ['ALTER DROP STATISTIC'] TABLE ALTER STATISTICS
ALTER MATERIALIZE STATISTIC ['ALTER MATERIALIZE STATISTIC'] TABLE ALTER STATISTIC ALTER MODIFY STATISTICS ['ALTER MODIFY STATISTIC'] TABLE ALTER STATISTICS
ALTER STATISTIC ['STATISTIC'] \N ALTER TABLE ALTER MATERIALIZE STATISTICS ['ALTER MATERIALIZE STATISTIC'] TABLE ALTER STATISTICS
ALTER STATISTICS ['STATISTIC'] \N ALTER TABLE
ALTER ADD PROJECTION ['ADD PROJECTION'] TABLE ALTER PROJECTION ALTER ADD PROJECTION ['ADD PROJECTION'] TABLE ALTER PROJECTION
ALTER DROP PROJECTION ['DROP PROJECTION'] TABLE ALTER PROJECTION ALTER DROP PROJECTION ['DROP PROJECTION'] TABLE ALTER PROJECTION
ALTER MATERIALIZE PROJECTION ['MATERIALIZE PROJECTION'] TABLE ALTER PROJECTION ALTER MATERIALIZE PROJECTION ['MATERIALIZE PROJECTION'] TABLE ALTER PROJECTION

View File

@ -1,53 +0,0 @@
DROP TABLE IF EXISTS t1;
CREATE TABLE t1
(
a Float64 STATISTIC(tdigest),
b Int64 STATISTIC(tdigest),
pk String,
) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY }
SET allow_experimental_statistic = 1;
CREATE TABLE t1
(
a Float64 STATISTIC(tdigest),
b Int64,
pk String STATISTIC(tdigest),
) Engine = MergeTree() ORDER BY pk; -- { serverError ILLEGAL_STATISTIC }
CREATE TABLE t1
(
a Float64 STATISTIC(tdigest, tdigest(10)),
b Int64,
) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY }
CREATE TABLE t1
(
a Float64 STATISTIC(xyz),
b Int64,
) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY }
CREATE TABLE t1
(
a Float64,
b Int64,
pk String,
) Engine = MergeTree() ORDER BY pk;
ALTER TABLE t1 ADD STATISTIC a TYPE xyz; -- { serverError INCORRECT_QUERY }
ALTER TABLE t1 ADD STATISTIC a TYPE tdigest;
ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC }
ALTER TABLE t1 ADD STATISTIC pk TYPE tdigest; -- { serverError ILLEGAL_STATISTIC }
ALTER TABLE t1 DROP STATISTIC b TYPE tdigest; -- { serverError ILLEGAL_STATISTIC }
ALTER TABLE t1 DROP STATISTIC a TYPE tdigest;
ALTER TABLE t1 DROP STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC }
ALTER TABLE t1 CLEAR STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC }
ALTER TABLE t1 MATERIALIZE STATISTIC b TYPE tdigest; -- { serverError ILLEGAL_STATISTIC }
ALTER TABLE t1 ADD STATISTIC a TYPE tdigest;
ALTER TABLE t1 ADD STATISTIC b TYPE tdigest;
ALTER TABLE t1 MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH;
ALTER TABLE t1 MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN }
DROP TABLE t1;

View File

@ -0,0 +1,57 @@
DROP TABLE IF EXISTS t1;
CREATE TABLE t1
(
a Float64 STATISTICS(tdigest),
b Int64 STATISTICS(tdigest),
pk String,
) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY }
SET allow_experimental_statistics = 1;
CREATE TABLE t1
(
a Float64 STATISTICS(tdigest),
b Int64,
pk String STATISTICS(tdigest),
) Engine = MergeTree() ORDER BY pk; -- { serverError ILLEGAL_STATISTICS }
CREATE TABLE t1
(
a Float64 STATISTICS(tdigest, tdigest(10)),
b Int64,
) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY }
CREATE TABLE t1
(
a Float64 STATISTICS(xyz),
b Int64,
) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY }
CREATE TABLE t1
(
a Float64,
b Int64,
pk String,
) Engine = MergeTree() ORDER BY pk;
ALTER TABLE t1 ADD STATISTICS a TYPE xyz; -- { serverError INCORRECT_QUERY }
ALTER TABLE t1 ADD STATISTICS a TYPE tdigest;
ALTER TABLE t1 ADD STATISTICS IF NOT EXISTS a TYPE tdigest;
ALTER TABLE t1 ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
-- Statistics can be created only on integer columns
ALTER TABLE t1 MODIFY STATISTICS a TYPE tdigest;
ALTER TABLE t1 ADD STATISTICS pk TYPE tdigest; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE t1 DROP STATISTICS b; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE t1 DROP STATISTICS a;
ALTER TABLE t1 DROP STATISTICS IF EXISTS a;
ALTER TABLE t1 CLEAR STATISTICS a; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE t1 CLEAR STATISTICS IF EXISTS a;
ALTER TABLE t1 MATERIALIZE STATISTICS b; -- { serverError ILLEGAL_STATISTICS }
ALTER TABLE t1 ADD STATISTICS a TYPE tdigest;
ALTER TABLE t1 ADD STATISTICS b TYPE tdigest;
ALTER TABLE t1 MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH;
ALTER TABLE t1 MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN }
DROP TABLE t1;

View File

@ -1,4 +1,4 @@
CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `b` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After insert After insert
Prewhere info Prewhere info
Prewhere filter Prewhere filter
@ -12,7 +12,7 @@ After drop statistic
10 10
CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After add statistic After add statistic
CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `b` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After materialize statistic After materialize statistic
Prewhere info Prewhere info
Prewhere filter Prewhere filter
@ -23,7 +23,7 @@ After merge
Prewhere filter Prewhere filter
Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) Prewhere filter column: and(less(a, 10), less(b, 10)) (removed)
20 20
CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `c` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After rename After rename
Prewhere info Prewhere info
Prewhere filter Prewhere filter

View File

@ -1,12 +1,12 @@
DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t1;
SET allow_experimental_statistic = 1; SET allow_experimental_statistics = 1;
SET allow_statistic_optimize = 1; SET allow_statistics_optimize = 1;
CREATE TABLE t1 CREATE TABLE t1
( (
a Float64 STATISTIC(tdigest), a Float64 STATISTICS(tdigest),
b Int64 STATISTIC(tdigest), b Int64 STATISTICS(tdigest),
pk String, pk String,
) Engine = MergeTree() ORDER BY pk ) Engine = MergeTree() ORDER BY pk
SETTINGS min_bytes_for_wide_part = 0; SETTINGS min_bytes_for_wide_part = 0;
@ -20,7 +20,7 @@ SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=
SELECT count(*) FROM t1 WHERE b < 10 and a < 10; SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
SELECT count(*) FROM t1 WHERE b < NULL and a < '10'; SELECT count(*) FROM t1 WHERE b < NULL and a < '10';
ALTER TABLE t1 DROP STATISTIC a, b TYPE tdigest; ALTER TABLE t1 DROP STATISTICS a, b;
SELECT 'After drop statistic'; SELECT 'After drop statistic';
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
@ -28,13 +28,13 @@ SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
SHOW CREATE TABLE t1; SHOW CREATE TABLE t1;
ALTER TABLE t1 ADD STATISTIC a, b TYPE tdigest; ALTER TABLE t1 ADD STATISTICS a, b TYPE tdigest;
SELECT 'After add statistic'; SELECT 'After add statistic';
SHOW CREATE TABLE t1; SHOW CREATE TABLE t1;
ALTER TABLE t1 MATERIALIZE STATISTIC a, b TYPE tdigest; ALTER TABLE t1 MATERIALIZE STATISTICS a, b;
INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000;
SELECT 'After materialize statistic'; SELECT 'After materialize statistic';

View File

@ -0,0 +1,35 @@
CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After insert
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed)
After merge
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed)
After modify TDigest
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed)
After drop
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed)

View File

@ -0,0 +1,71 @@
DROP TABLE IF EXISTS t1;
SET allow_experimental_statistics = 1;
SET allow_statistics_optimize = 1;
CREATE TABLE t1
(
a Float64 STATISTICS(tdigest),
b Int64 STATISTICS(tdigest),
c Int64 STATISTICS(tdigest, uniq),
pk String,
) Engine = MergeTree() ORDER BY pk
SETTINGS min_bytes_for_wide_part = 0;
SHOW CREATE TABLE t1;
INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000;
INSERT INTO t1 select 0, 0, 11, generateUUIDv4();
SELECT 'After insert';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
OPTIMIZE TABLE t1 FINAL;
SELECT 'After merge';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT 'After modify TDigest';
ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest;
ALTER TABLE t1 MATERIALIZE STATISTICS c;
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
ALTER TABLE t1 DROP STATISTICS c;
SELECT 'After drop';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
SET allow_suspicious_low_cardinality_types=1;
CREATE TABLE t2
(
a Float64 STATISTICS(tdigest),
b Int64 STATISTICS(tdigest),
c LowCardinality(Int64) STATISTICS(tdigest, uniq),
pk String,
) Engine = MergeTree() ORDER BY pk
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000;
DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t3;
CREATE TABLE t3
(
a Float64 STATISTICS(tdigest),
b Int64 STATISTICS(tdigest),
c Nullable(Int64) STATISTICS(tdigest, uniq),
pk String,
) Engine = MergeTree() ORDER BY pk
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000;
DROP TABLE IF EXISTS t3;

View File

@ -41,7 +41,7 @@ allow_experimental_query_deduplication 0
allow_experimental_refreshable_materialized_view 0 allow_experimental_refreshable_materialized_view 0
allow_experimental_s3queue 1 allow_experimental_s3queue 1
allow_experimental_shared_merge_tree 0 allow_experimental_shared_merge_tree 0
allow_experimental_statistic 0 allow_experimental_statistics 0
allow_experimental_undrop_table_query 1 allow_experimental_undrop_table_query 1
allow_experimental_usearch_index 0 allow_experimental_usearch_index 0
allow_experimental_window_functions 1 allow_experimental_window_functions 1
@ -58,7 +58,7 @@ allow_prefetched_read_pool_for_remote_filesystem 1
allow_push_predicate_when_subquery_contains_with 1 allow_push_predicate_when_subquery_contains_with 1
allow_settings_after_format_in_insert 0 allow_settings_after_format_in_insert 0
allow_simdjson 1 allow_simdjson 1
allow_statistic_optimize 0 allow_statistics_optimize 0
allow_suspicious_codecs 0 allow_suspicious_codecs 0
allow_suspicious_fixed_string_types 0 allow_suspicious_fixed_string_types 0
allow_suspicious_indices 0 allow_suspicious_indices 0

1 add_http_cors_header 0
41 allow_experimental_refreshable_materialized_view 0
42 allow_experimental_s3queue 1
43 allow_experimental_shared_merge_tree 0
44 allow_experimental_statistic allow_experimental_statistics 0
45 allow_experimental_undrop_table_query 1
46 allow_experimental_usearch_index 0
47 allow_experimental_window_functions 1
58 allow_push_predicate_when_subquery_contains_with 1
59 allow_settings_after_format_in_insert 0
60 allow_simdjson 1
61 allow_statistic_optimize allow_statistics_optimize 0
62 allow_suspicious_codecs 0
63 allow_suspicious_fixed_string_types 0
64 allow_suspicious_indices 0

View File

@ -1,10 +1,10 @@
10 10
10 10
10 10
statistic not used Condition less(b, 10_UInt8) moved to PREWHERE statistics not used Condition less(b, 10_UInt8) moved to PREWHERE
statistic not used Condition less(a, 10_UInt8) moved to PREWHERE statistics not used Condition less(a, 10_UInt8) moved to PREWHERE
statistic used after merge Condition less(a, 10_UInt8) moved to PREWHERE statistics used after merge Condition less(a, 10_UInt8) moved to PREWHERE
statistic used after merge Condition less(b, 10_UInt8) moved to PREWHERE statistics used after merge Condition less(b, 10_UInt8) moved to PREWHERE
statistic used after materialize Condition less(a, 10_UInt8) moved to PREWHERE statistics used after materialize Condition less(a, 10_UInt8) moved to PREWHERE
statistic used after materialize Condition less(b, 10_UInt8) moved to PREWHERE statistics used after materialize Condition less(b, 10_UInt8) moved to PREWHERE
2 0 2 0

View File

@ -1,34 +1,34 @@
DROP TABLE IF EXISTS t_statistic_materialize; DROP TABLE IF EXISTS t_statistics_materialize;
SET allow_experimental_analyzer = 1; SET allow_experimental_analyzer = 1;
SET allow_experimental_statistic = 1; SET allow_experimental_statistics = 1;
SET allow_statistic_optimize = 1; SET allow_statistics_optimize = 1;
SET materialize_statistics_on_insert = 0; SET materialize_statistics_on_insert = 0;
CREATE TABLE t_statistic_materialize CREATE TABLE t_statistics_materialize
( (
a Int64 STATISTIC(tdigest), a Int64 STATISTICS(tdigest),
b Int16 STATISTIC(tdigest), b Int16 STATISTICS(tdigest),
) ENGINE = MergeTree() ORDER BY tuple() ) ENGINE = MergeTree() ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics.
INSERT INTO t_statistic_materialize SELECT number, -number FROM system.numbers LIMIT 10000; INSERT INTO t_statistics_materialize SELECT number, -number FROM system.numbers LIMIT 10000;
SELECT count(*) FROM t_statistic_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistic not used'; SELECT count(*) FROM t_statistics_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics not used';
OPTIMIZE TABLE t_statistic_materialize FINAL; OPTIMIZE TABLE t_statistics_materialize FINAL;
SELECT count(*) FROM t_statistic_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistic used after merge'; SELECT count(*) FROM t_statistics_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after merge';
TRUNCATE TABLE t_statistic_materialize; TRUNCATE TABLE t_statistics_materialize;
SET mutations_sync = 2; SET mutations_sync = 2;
INSERT INTO t_statistic_materialize SELECT number, -number FROM system.numbers LIMIT 10000; INSERT INTO t_statistics_materialize SELECT number, -number FROM system.numbers LIMIT 10000;
ALTER TABLE t_statistic_materialize MATERIALIZE STATISTIC a, b TYPE tdigest; ALTER TABLE t_statistics_materialize MATERIALIZE STATISTICS a, b;
SELECT count(*) FROM t_statistic_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistic used after materialize'; SELECT count(*) FROM t_statistics_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after materialize';
DROP TABLE t_statistic_materialize; DROP TABLE t_statistics_materialize;
SYSTEM FLUSH LOGS; SYSTEM FLUSH LOGS;
@ -36,7 +36,7 @@ SELECT log_comment, message FROM system.text_log JOIN
( (
SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log
WHERE current_database = currentDatabase() WHERE current_database = currentDatabase()
AND query LIKE 'SELECT count(*) FROM t_statistic_materialize%' AND query LIKE 'SELECT count(*) FROM t_statistics_materialize%'
AND type = 'QueryFinish' AND type = 'QueryFinish'
) AS query_log USING (query_id) ) AS query_log USING (query_id)
WHERE message LIKE '%moved to PREWHERE%' WHERE message LIKE '%moved to PREWHERE%'
@ -45,5 +45,5 @@ ORDER BY event_time_microseconds;
SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds']) SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds'])
FROM system.query_log FROM system.query_log
WHERE current_database = currentDatabase() WHERE current_database = currentDatabase()
AND query LIKE 'INSERT INTO t_statistic_materialize SELECT%' AND query LIKE 'INSERT INTO t_statistics_materialize SELECT%'
AND type = 'QueryFinish'; AND type = 'QueryFinish';

View File

@ -526,7 +526,7 @@ class QueryResult:
for row in rows: for row in rows:
res_row = [] res_row = []
for c, t in zip(row, types): for c, t in zip(row, types):
logger.debug("Builging row. c:%s t:%s", c, t) logger.debug("Building row. c:%s t:%s", c, t)
if c is None: if c is None:
res_row.append("NULL") res_row.append("NULL")
continue continue