Merge remote-tracking branch 'upstream/master' into materialized-postgresql-better-startup

This commit is contained in:
kssenii 2021-12-26 21:05:45 +03:00
commit 2a58048868
41 changed files with 668 additions and 1008 deletions

View File

@ -5,15 +5,15 @@ toc_title: MaterializedPostgreSQL
# [experimental] MaterializedPostgreSQL {#materialize-postgresql}
Creates ClickHouse database with an initial data dump of PostgreSQL database tables and starts replication process, i.e. executes background job to apply new changes as they happen on PostgreSQL database tables in the remote PostgreSQL database.
Creates a ClickHouse database with tables from PostgreSQL database. Firstly, database with engine `MaterializedPostgreSQL` creates a snapshot of PostgreSQL database and loads required tables. Required tables can include any subset of tables from any subset of schemas from specified database. Along with the snapshot database engine acquires LSN and once initial dump of tables is performed - it starts pulling updates from WAL. After database is created, newly added tables to PostgreSQL database are not automatically added to replication. They have to be added manually with `ATTACH TABLE db.table` query.
ClickHouse server works as PostgreSQL replica. It reads WAL and performs DML queries. DDL is not replicated, but can be handled (described below).
Replication is implemented with PostgreSQL Logical Replication Protocol, which does not allow to replicate DDL, but allows to know whether replication breaking changes happened (column type changes, adding/removing columns). Such changes are detected and according tables stop receiving updates. Such tables can be automatically reloaded in the background in case required setting is turned on. Safest way for now is to use `ATTACH`/ `DETACH` queries to reload table completely. If DDL does not break replication (for example, renaming a column) table will still receive updates (insertion is done by position).
## Creating a Database {#creating-a-database}
``` sql
CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster]
ENGINE = MaterializedPostgreSQL('host:port', ['database' | database], 'user', 'password') [SETTINGS ...]
ENGINE = MaterializedPostgreSQL('host:port', 'database', 'user', 'password') [SETTINGS ...]
```
**Engine Parameters**
@ -23,51 +23,39 @@ ENGINE = MaterializedPostgreSQL('host:port', ['database' | database], 'user', 'p
- `user` — PostgreSQL user.
- `password` — User password.
## Example of Use {#example-of-use}
``` sql
CREATE DATABASE postgresql;
ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password');
SHOW TABLES FROM postgres_db;
┌─name───┐
│ table1 │
└────────┘
SELECT * FROM postgresql_db.postgres_table;
```
## Dynamically adding new tables to replication {#dynamically-adding-table-to-replication}
After `MaterializedPostgreSQL` database is created, it does not automatically detect new tables in according PostgreSQL database. Such tables can be added manually:
``` sql
ATTACH TABLE postgres_database.new_table;
```
When specifying a specific list of tables in the database using the setting [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list), it will be updated to the current state, taking into account the tables which were added by the `ATTACH TABLE` query.
Warning: before version 21.13 adding table to replication left unremoved temprorary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in clickhouse version before 21.13, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. Issue is fixed in 21.13.
## Dynamically removing tables from replication {#dynamically-removing-table-from-replication}
It is possible to remove specific tables from replication:
``` sql
DETACH TABLE postgres_database.table_to_remove;
```
## Settings {#settings}
- [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list)
- [materialized_postgresql_schema](../../operations/settings/settings.md#materialized-postgresql-schema)
- [materialized_postgresql_schema_list](../../operations/settings/settings.md#materialized-postgresql-schema-list)
- [materialized_postgresql_allow_automatic_update](../../operations/settings/settings.md#materialized-postgresql-allow-automatic-update)
- [materialized_postgresql_max_block_size](../../operations/settings/settings.md#materialized-postgresql-max-block-size)
- [materialized_postgresql_replication_slot](../../operations/settings/settings.md#materialized-postgresql-replication-slot)
- [materialized_postgresql_snapshot](../../operations/settings/settings.md#materialized-postgresql-snapshot)
``` sql
CREATE DATABASE database1
ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password')
SETTINGS materialized_postgresql_tables_list = 'table1,table2,table3';
SELECT * FROM database1.table1;
```
The settings can be changed, if necessary, using a DDL query. But it is impossible to change the setting `materialized_postgresql_tables_list`. To update the list of tables in this setting use the `ATTACH TABLE` query.
``` sql
ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_block_size = <new_size>;
```
## PostgreSQL schema {#schema}
PostgreSQL [schema](https://www.postgresql.org/docs/9.1/ddl-schemas.html) can be configured in 3 ways (starting from version 21.12).
@ -150,13 +138,63 @@ WHERE oid = 'postgres_table'::regclass;
!!! warning "Warning"
Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used.
## Example of Use {#example-of-use}
## Settings {#settings}
1. materialized_postgresql_tables_list {#materialized-postgresql-tables-list}
Sets a comma-separated list of PostgreSQL database tables, which will be replicated via [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) database engine.
Default value: empty list — means whole PostgreSQL database will be replicated.
2. materialized_postgresql_schema {#materialized-postgresql-schema}
Default value: empty string. (Default schema is used)
3. materialized_postgresql_schema_list {#materialized-postgresql-schema-list}
Default value: empty list. (Default schema is used)
4. materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update}
Allows reloading table in the background, when schema changes are detected. DDL queries on the PostgreSQL side are not replicated via ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case, the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping the replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them.
Possible values:
- 0 — The table is not automatically updated in the background, when schema changes are detected.
- 1 — The table is automatically updated in the background, when schema changes are detected.
Default value: `0`.
5. materialized_postgresql_max_block_size {#materialized-postgresql-max-block-size}
Sets the number of rows collected in memory before flushing data into PostgreSQL database table.
Possible values:
- Positive integer.
Default value: `65536`.
6. materialized_postgresql_replication_slot {#materialized-postgresql-replication-slot}
A user-created replication slot. Must be used together with `materialized_postgresql_snapshot`.
7. materialized_postgresql_snapshot {#materialized-postgresql-snapshot}
A text string identifying a snapshot, from which [initial dump of PostgreSQL tables](../../engines/database-engines/materialized-postgresql.md) will be performed. Must be used together with `materialized_postgresql_replication_slot`.
``` sql
CREATE DATABASE postgresql_db
ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password');
CREATE DATABASE database1
ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password')
SETTINGS materialized_postgresql_tables_list = 'table1,table2,table3';
SELECT * FROM postgresql_db.postgres_table;
SELECT * FROM database1.table1;
```
The settings can be changed, if necessary, using a DDL query. But it is impossible to change the setting `materialized_postgresql_tables_list`. To update the list of tables in this setting use the `ATTACH TABLE` query.
``` sql
ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_block_size = <new_size>;
```
## Notes {#notes}
@ -165,7 +203,7 @@ SELECT * FROM postgresql_db.postgres_table;
Logical Replication Slots which exist on the primary are not available on standby replicas.
So if there is a failover, new primary (the old physical standby) wont be aware of any slots which were existing with old primary. This will lead to a broken replication from PostgreSQL.
A solution to this is to manage replication slots yourself and define a permanent replication slot (some information can be found [here](https://patroni.readthedocs.io/en/latest/SETTINGS.html)). You'll need to pass slot name via [materialized_postgresql_replication_slot](../../operations/settings/settings.md#materialized-postgresql-replication-slot) setting, and it has to be exported with `EXPORT SNAPSHOT` option. The snapshot identifier needs to be passed via [materialized_postgresql_snapshot](../../operations/settings/settings.md#materialized-postgresql-snapshot) setting.
A solution to this is to manage replication slots yourself and define a permanent replication slot (some information can be found [here](https://patroni.readthedocs.io/en/latest/SETTINGS.html)). You'll need to pass slot name via `materialized_postgresql_replication_slot` setting, and it has to be exported with `EXPORT SNAPSHOT` option. The snapshot identifier needs to be passed via `materialized_postgresql_snapshot` setting.
Please note that this should be used only if it is actually needed. If there is no real need for that or full understanding why, then it is better to allow the table engine to create and manage its own replication slot.
@ -214,3 +252,23 @@ SETTINGS
```bash
kubectl exec acid-demo-cluster-0 -c postgres -- su postgres -c 'patronictl failover --candidate acid-demo-cluster-1 --force'
```
### Required permissions
1. [CREATE PUBLICATION](https://postgrespro.ru/docs/postgresql/14/sql-createpublication) -- create query privilege.
2. [CREATE_REPLICATION_SLOT](https://postgrespro.ru/docs/postgrespro/10/protocol-replication#PROTOCOL-REPLICATION-CREATE-SLOT) -- replication privelege.
3. [pg_drop_replication_slot](https://postgrespro.ru/docs/postgrespro/9.5/functions-admin#functions-replication) -- replication privilege or superuser.
4. [DROP PUBLICATION](https://postgrespro.ru/docs/postgresql/10/sql-droppublication) -- owner of publication (`username` in MaterializedPostgreSQL engine itself).
It is possible to avoid executing `2` and `3` commands and having those permissions. Use settings `materialized_postgresql_replication_slot` and `materialized_postgresql_snapshot`. But with much care.
Access to tables:
1. pg_publication
2. pg_replication_slots
3. pg_publication_tables

View File

@ -7,7 +7,7 @@ toc_title: MaterializedPostgreSQL
Creates ClickHouse table with an initial data dump of PostgreSQL table and starts replication process, i.e. executes background job to apply new changes as they happen on PostgreSQL table in the remote PostgreSQL database.
If more than one table is required, it is highly recommended to use the [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md) database engine instead of the table engine and use the [materialized_postgresql_tables_list](../../../operations/settings/settings.md#materialized-postgresql-tables-list) setting, which specifies the tables to be replicated. It will be much better in terms of CPU, fewer connections and fewer replication slots inside the remote PostgreSQL database.
If more than one table is required, it is highly recommended to use the [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md) database engine instead of the table engine and use the `materialized_postgresql_tables_list` setting, which specifies the tables to be replicated (will also be possible to add database `schema`). It will be much better in terms of CPU, fewer connections and fewer replication slots inside the remote PostgreSQL database.
## Creating a Table {#creating-a-table}

View File

@ -36,6 +36,31 @@ The table structure can differ from the original PostgreSQL table structure:
- `schema` — Non-default table schema. Optional.
- `on conflict ...` — example: `ON CONFLICT DO NOTHING`. Optional. Note: adding this option will make insertion less efficient.
or via config (since version 21.11):
```
<named_collections>
<postgres1>
<host></host>
<port></port>
<username></username>
<password></password>
<table></table>
</postgres1>
<postgres2>
<host></host>
<port></port>
<username></username>
<password></password>
</postgres2>
</named_collections>
```
Some parameters can be overriden by key value arguments:
``` sql
SELECT * FROM postgresql(postgres1, schema='schema1', table='table1');
```
## Implementation Details {#implementation-details}
`SELECT` queries on PostgreSQL side run as `COPY (SELECT ...) TO STDOUT` inside read-only PostgreSQL transaction with commit after each `SELECT` query.

View File

@ -3682,49 +3682,6 @@ Possible values:
Default value: `0`.
## materialized_postgresql_max_block_size {#materialized-postgresql-max-block-size}
Sets the number of rows collected in memory before flushing data into PostgreSQL database table.
Possible values:
- Positive integer.
Default value: `65536`.
## materialized_postgresql_tables_list {#materialized-postgresql-tables-list}
Sets a comma-separated list of PostgreSQL database tables, which will be replicated via [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) database engine.
Default value: empty list — means whole PostgreSQL database will be replicated.
## materialized_postgresql_schema {#materialized-postgresql-schema}
Default value: empty string. (Default schema is used)
## materialized_postgresql_schema_list {#materialized-postgresql-schema-list}
Default value: empty list. (Default schema is used)
## materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update}
Allows reloading table in the background, when schema changes are detected. DDL queries on the PostgreSQL side are not replicated via ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case, the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping the replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them.
Possible values:
- 0 — The table is not automatically updated in the background, when schema changes are detected.
- 1 — The table is automatically updated in the background, when schema changes are detected.
Default value: `0`.
## materialized_postgresql_replication_slot {#materialized-postgresql-replication-slot}
A user-created replication slot. Must be used together with [materialized_postgresql_snapshot](#materialized-postgresql-snapshot).
## materialized_postgresql_snapshot {#materialized-postgresql-snapshot}
A text string identifying a snapshot, from which [initial dump of PostgreSQL tables](../../engines/database-engines/materialized-postgresql.md) will be performed. Must be used together with [materialized_postgresql_replication_slot](#materialized-postgresql-replication-slot).
## allow_experimental_projection_optimization {#allow-experimental-projection-optimization}
Enables or disables [projection](../../engines/table-engines/mergetree-family/mergetree.md#projections) optimization when processing `SELECT` queries.

View File

@ -792,9 +792,9 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp
int mainEntryClickHouseLocal(int argc, char ** argv)
{
DB::LocalServer app;
try
{
DB::LocalServer app;
app.init(argc, argv);
return app.run();
}

View File

@ -421,6 +421,9 @@ public:
*/
UInt8 rb_contains(UInt64 x) const
{
if (!std::is_same_v<T, UInt64> && x > rb_max())
return 0;
if (isSmall())
return small.find(x) != small.end();
else
@ -432,6 +435,9 @@ public:
*/
void rb_remove(UInt64 x)
{
if (!std::is_same_v<T, UInt64> && x > rb_max())
return;
if (isSmall())
toLarge();

View File

@ -8,6 +8,5 @@ namespace DB
{
using ColumnNumbers = std::vector<size_t>;
using ColumnNumbersList = std::vector<ColumnNumbers>;
}

View File

@ -109,8 +109,6 @@ public:
std::optional<NameAndTypePair> tryGetByName(const std::string & name) const;
};
using NamesAndTypesLists = std::vector<NamesAndTypesList>;
}
namespace std

View File

@ -19,7 +19,13 @@ namespace ErrorCodes
extern const int CANNOT_PARSE_BOOL;
}
static const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column)
namespace
{
constexpr char str_true[5] = "true";
constexpr char str_false[6] = "false";
const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column)
{
const auto * col = checkAndGetColumn<ColumnUInt8>(&column);
if (!checkAndGetColumn<ColumnUInt8>(&column))
@ -28,7 +34,7 @@ static const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column
return col;
}
static ColumnUInt8 * checkAndGetDeserializeColumnType(IColumn & column)
ColumnUInt8 * checkAndGetDeserializeColumnType(IColumn & column)
{
auto * col = typeid_cast<ColumnUInt8 *>(&column);
if (!checkAndGetColumn<ColumnUInt8>(&column))
@ -37,6 +43,166 @@ static ColumnUInt8 * checkAndGetDeserializeColumnType(IColumn & column)
return col;
}
void serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings)
{
const auto * col = checkAndGetSerializeColumnType(column);
if (col->getData()[row_num])
{
writeString(settings.bool_true_representation, ostr);
}
else
{
writeString(settings.bool_false_representation, ostr);
}
}
void serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &)
{
const auto * col = checkAndGetSerializeColumnType(column);
if (col->getData()[row_num])
ostr.write(str_true, sizeof(str_true) - 1);
else
ostr.write(str_false, sizeof(str_false) - 1);
}
bool tryDeserializeAllVariants(ColumnUInt8 * column, ReadBuffer & istr)
{
if (checkCharCaseInsensitive('1', istr))
{
column->insert(true);
}
else if (checkCharCaseInsensitive('0', istr))
{
column->insert(false);
}
/// 'True' and 'T'
else if (checkCharCaseInsensitive('t', istr))
{
/// Check if it's just short form `T` or full form `True`
if (checkCharCaseInsensitive('r', istr))
{
if (!checkStringCaseInsensitive("ue", istr))
return false;
}
column->insert(true);
}
/// 'False' and 'F'
else if (checkCharCaseInsensitive('f', istr))
{
/// Check if it's just short form `F` or full form `False`
if (checkCharCaseInsensitive('a', istr))
{
if (!checkStringCaseInsensitive("lse", istr))
return false;
}
column->insert(false);
}
/// 'Yes' and 'Y'
else if (checkCharCaseInsensitive('y', istr))
{
/// Check if it's just short form `Y` or full form `Yes`
if (checkCharCaseInsensitive('e', istr))
{
if (!checkCharCaseInsensitive('s', istr))
return false;
}
column->insert(true);
}
/// 'No' and 'N'
else if (checkCharCaseInsensitive('n', istr))
{
/// Check if it's just short form `N` or full form `No`
checkCharCaseInsensitive('o', istr);
column->insert(false);
}
/// 'On' and 'Off'
else if (checkCharCaseInsensitive('o', istr))
{
if (checkCharCaseInsensitive('n', istr))
column->insert(true);
else if (checkStringCaseInsensitive("ff", istr))
{
column->insert(false);
}
else
return false;
}
/// 'Enable' and 'Enabled'
else if (checkStringCaseInsensitive("enable", istr))
{
/// Check if it's 'enable' or 'enabled'
checkCharCaseInsensitive('d', istr);
column->insert(true);
}
/// 'Disable' and 'Disabled'
else if (checkStringCaseInsensitive("disable", istr))
{
/// Check if it's 'disable' or 'disabled'
checkCharCaseInsensitive('d', istr);
column->insert(false);
}
else
{
return false;
}
return true;
}
void deserializeImpl(
IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function<bool(ReadBuffer &)> check_end_of_value)
{
ColumnUInt8 * col = checkAndGetDeserializeColumnType(column);
PeekableReadBuffer buf(istr);
buf.setCheckpoint();
if (checkString(settings.bool_true_representation, buf) && check_end_of_value(buf))
{
col->insert(true);
return;
}
buf.rollbackToCheckpoint();
if (checkString(settings.bool_false_representation, buf) && check_end_of_value(buf))
{
col->insert(false);
buf.dropCheckpoint();
if (buf.hasUnreadData())
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if "
"bool_true_representation or bool_false_representation contains some delimiters of input format");
return;
}
buf.rollbackToCheckpoint();
if (tryDeserializeAllVariants(col, buf) && check_end_of_value(buf))
{
buf.dropCheckpoint();
if (buf.hasUnreadData())
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if "
"bool_true_representation or bool_false_representation contains some delimiters of input format");
return;
}
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Cannot parse boolean value here: '{}', should be '{}' or '{}' controlled by setting bool_true_representation and "
"bool_false_representation or one of "
"True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0",
String(buf.position(), std::min(10lu, buf.available())),
settings.bool_true_representation, settings.bool_false_representation);
}
}
SerializationBool::SerializationBool(const SerializationPtr &nested_)
: SerializationWrapper(nested_)
{
@ -160,166 +326,9 @@ void SerializationBool::deserializeWholeText(IColumn & column, ReadBuffer & istr
deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof(); });
}
void SerializationBool::serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
const auto * col = checkAndGetSerializeColumnType(column);
if (col->getData()[row_num])
{
writeString(settings.bool_true_representation, ostr);
}
else
{
writeString(settings.bool_false_representation, ostr);
}
}
void SerializationBool::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeSimple(column, row_num, ostr, settings);
}
void SerializationBool::serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
const auto * col = checkAndGetSerializeColumnType(column);
if (col->getData()[row_num])
ostr.write(str_true, sizeof(str_true) - 1);
else
ostr.write(str_false, sizeof(str_false) - 1);
}
void SerializationBool::deserializeImpl(
IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function<bool(ReadBuffer &)> check_end_of_value) const
{
ColumnUInt8 * col = checkAndGetDeserializeColumnType(column);
PeekableReadBuffer buf(istr);
buf.setCheckpoint();
if (checkString(settings.bool_true_representation, buf) && check_end_of_value(buf))
{
col->insert(true);
return;
}
buf.rollbackToCheckpoint();
if (checkString(settings.bool_false_representation, buf) && check_end_of_value(buf))
{
col->insert(false);
buf.dropCheckpoint();
if (buf.hasUnreadData())
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if "
"bool_true_representation or bool_false_representation contains some delimiters of input format");
return;
}
buf.rollbackToCheckpoint();
if (tryDeserializeAllVariants(col, buf) && check_end_of_value(buf))
{
buf.dropCheckpoint();
if (buf.hasUnreadData())
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if "
"bool_true_representation or bool_false_representation contains some delimiters of input format");
return;
}
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Cannot parse boolean value here: '{}', should be '{}' or '{}' controlled by setting bool_true_representation and "
"bool_false_representation or one of "
"True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0",
String(buf.position(), std::min(10lu, buf.available())),
settings.bool_true_representation, settings.bool_false_representation);
}
bool SerializationBool::tryDeserializeAllVariants(ColumnUInt8 * column, ReadBuffer & istr) const
{
if (checkCharCaseInsensitive('1', istr))
{
column->insert(true);
}
else if (checkCharCaseInsensitive('0', istr))
{
column->insert(false);
}
/// 'True' and 'T'
else if (checkCharCaseInsensitive('t', istr))
{
/// Check if it's just short form `T` or full form `True`
if (checkCharCaseInsensitive('r', istr))
{
if (!checkStringCaseInsensitive("ue", istr))
return false;
}
column->insert(true);
}
/// 'False' and 'F'
else if (checkCharCaseInsensitive('f', istr))
{
/// Check if it's just short form `F` or full form `False`
if (checkCharCaseInsensitive('a', istr))
{
if (!checkStringCaseInsensitive("lse", istr))
return false;
}
column->insert(false);
}
/// 'Yes' and 'Y'
else if (checkCharCaseInsensitive('y', istr))
{
/// Check if it's just short form `Y` or full form `Yes`
if (checkCharCaseInsensitive('e', istr))
{
if (!checkCharCaseInsensitive('s', istr))
return false;
}
column->insert(true);
}
/// 'No' and 'N'
else if (checkCharCaseInsensitive('n', istr))
{
/// Check if it's just short form `N` or full form `No`
checkCharCaseInsensitive('o', istr);
column->insert(false);
}
/// 'On' and 'Off'
else if (checkCharCaseInsensitive('o', istr))
{
if (checkCharCaseInsensitive('n', istr))
column->insert(true);
else if (checkStringCaseInsensitive("ff", istr))
{
column->insert(false);
}
else
return false;
}
/// 'Enable' and 'Enabled'
else if (checkStringCaseInsensitive("enable", istr))
{
/// Check if it's 'enable' or 'enabled'
checkCharCaseInsensitive('d', istr);
column->insert(true);
}
/// 'Disable' and 'Disabled'
else if (checkStringCaseInsensitive("disable", istr))
{
/// Check if it's 'disable' or 'disabled'
checkCharCaseInsensitive('d', istr);
column->insert(false);
}
else
{
return false;
}
return true;
}
}

View File

@ -9,10 +9,6 @@ namespace DB
class SerializationBool final : public SerializationWrapper
{
private:
static constexpr char str_true[5] = "true";
static constexpr char str_false[6] = "false";
public:
SerializationBool(const SerializationPtr & nested_);
@ -36,12 +32,6 @@ public:
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
protected:
void serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
void serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
void deserializeImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function<bool(ReadBuffer & buf)> check_end_of_value) const;
bool tryDeserializeAllVariants(ColumnUInt8 * column, ReadBuffer & istr) const;
};
}

View File

@ -183,7 +183,7 @@ public:
ActionsDAGPtr clone() const;
/// Execute actions for header. Input block must have empty columns.
/// Result should be equal to the execution of ExpressionActions built from this DAG.
/// Result should be equal to the execution of ExpressionActions build form this DAG.
/// Actions are not changed, no expressions are compiled.
///
/// In addition, check that result constants are constants according to DAG.

View File

@ -878,10 +878,9 @@ public:
Block intermediate_header;
/// What to count.
ColumnNumbers keys;
const ColumnNumbersList keys_vector;
const ColumnNumbers keys;
const AggregateDescriptions aggregates;
size_t keys_size;
const size_t keys_size;
const size_t aggregates_size;
/// The settings of approximate calculation of GROUP BY.
@ -939,46 +938,6 @@ public:
{
}
/// two dimensional vector of aggregating keys in params
Params(
const Block & src_header_,
const ColumnNumbers & keys_,
const ColumnNumbersList & keys_vector_,
const AggregateDescriptions & aggregates_,
bool overflow_row_,
size_t max_rows_to_group_by_,
OverflowMode group_by_overflow_mode_,
size_t group_by_two_level_threshold_,
size_t group_by_two_level_threshold_bytes_,
size_t max_bytes_before_external_group_by_,
bool empty_result_for_aggregation_by_empty_set_,
VolumePtr tmp_volume_,
size_t max_threads_,
size_t min_free_disk_space_,
bool compile_aggregate_expressions_,
size_t min_count_to_compile_aggregate_expression_,
const Block & intermediate_header_ = {})
: src_header(src_header_)
, intermediate_header(intermediate_header_)
, keys(keys_)
, keys_vector(keys_vector_)
, aggregates(aggregates_)
, keys_size(keys.size())
, aggregates_size(aggregates.size())
, overflow_row(overflow_row_)
, max_rows_to_group_by(max_rows_to_group_by_)
, group_by_overflow_mode(group_by_overflow_mode_)
, group_by_two_level_threshold(group_by_two_level_threshold_)
, group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_)
, max_bytes_before_external_group_by(max_bytes_before_external_group_by_)
, empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_)
, tmp_volume(tmp_volume_)
, max_threads(max_threads_)
, min_free_disk_space(min_free_disk_space_)
, compile_aggregate_expressions(compile_aggregate_expressions_)
, min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_)
{}
/// Only parameters that matter during merge.
Params(const Block & intermediate_header_,
const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_)

View File

@ -40,7 +40,6 @@
#include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.h>
#include "Core/NamesAndTypes.h"
#include <DataTypes/DataTypeFactory.h>
@ -342,64 +341,8 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false);
if (select_query->group_by_with_grouping_sets)
{
ASTs group_elements_ast;
const ASTExpressionList * group_ast_element = group_asts[i]->as<const ASTExpressionList>();
if (!group_ast_element)
throw Exception("Grouping Sets element " + group_asts[i]->getColumnName() + " should be an expression type", ErrorCodes::UNKNOWN_IDENTIFIER);
group_elements_ast = group_ast_element->children;
NamesAndTypesList grouping_set_list;
for (ssize_t j = 0; j < ssize_t(group_elements_ast.size()); ++j)
{
ssize_t group_size = group_elements_ast.size();
const auto & column_name = group_elements_ast[j]->getColumnName();
const auto * node = temp_actions->tryFindInIndex(column_name);
if (!node)
throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
/// Only removes constant keys if it's an initiator or distributed_group_by_no_merge is enabled.
if (getContext()->getClientInfo().distributed_depth == 0 || settings.distributed_group_by_no_merge > 0)
{
/// Constant expressions have non-null column pointer at this stage.
if (node->column && isColumnConst(*node->column))
{
/// But don't remove last key column if no aggregate functions, otherwise aggregation will not work.
if (!aggregate_descriptions.empty() || group_size > 1)
{
if (j + 1 < static_cast<ssize_t>(group_size))
group_elements_ast[j] = std::move(group_elements_ast.back());
group_elements_ast.pop_back();
--j;
continue;
}
}
}
NameAndTypePair key{column_name, node->result_type};
grouping_set_list.push_back(key);
/// Aggregation keys are unique.
if (!unique_keys.count(key.name))
{
unique_keys.insert(key.name);
aggregation_keys.push_back(key);
/// Key is no longer needed, therefore we can save a little by moving it.
aggregated_columns.push_back(std::move(key));
}
}
aggregation_keys_list.push_back(std::move(grouping_set_list));
}
else
{
const auto & column_name = group_asts[i]->getColumnName();
const auto * node = temp_actions->tryFindInIndex(column_name);
if (!node)
throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
@ -438,7 +381,6 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
aggregated_columns.push_back(std::move(key));
}
}
}
if (group_asts.empty())
{
@ -1197,25 +1139,11 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join);
ASTs asts = select_query->groupBy()->children;
if (select_query->group_by_with_grouping_sets)
{
for (const auto & ast : asts)
{
for (const auto & ast_element : ast->children)
{
step.addRequiredOutput(ast_element->getColumnName());
getRootActions(ast_element, only_types, step.actions());
}
}
}
else
{
for (const auto & ast : asts)
{
step.addRequiredOutput(ast->getColumnName());
getRootActions(ast, only_types, step.actions());
}
}
if (optimize_aggregation_in_order)
{

View File

@ -64,7 +64,6 @@ struct ExpressionAnalyzerData
bool has_aggregation = false;
NamesAndTypesList aggregation_keys;
NamesAndTypesLists aggregation_keys_list;
bool has_const_aggregation_keys = false;
AggregateDescriptions aggregate_descriptions;
@ -95,8 +94,6 @@ private:
explicit ExtractedSettings(const Settings & settings_);
};
Poco::Logger * poco_log = &Poco::Logger::get("ExpressionAnalyzer");
public:
/// Ctor for non-select queries. Generally its usage is:
/// auto actions = ExpressionAnalyzer(query, syntax, context).getActions();
@ -324,7 +321,6 @@ public:
const NamesAndTypesList & aggregationKeys() const { return aggregation_keys; }
bool hasConstAggregationKeys() const { return has_const_aggregation_keys; }
const NamesAndTypesLists & aggregationKeysList() const { return aggregation_keys_list; }
const AggregateDescriptions & aggregates() const { return aggregate_descriptions; }
const PreparedSets & getPreparedSets() const { return prepared_sets; }

View File

@ -44,7 +44,6 @@
#include <Processors/QueryPlan/ExtremesStep.h>
#include <Processors/QueryPlan/FillingStep.h>
#include <Processors/QueryPlan/FilterStep.h>
#include <Processors/QueryPlan/GroupingSetsStep.h>
#include <Processors/QueryPlan/JoinStep.h>
#include <Processors/QueryPlan/LimitByStep.h>
#include <Processors/QueryPlan/LimitStep.h>
@ -961,10 +960,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
bool aggregate_final =
expressions.need_aggregate &&
options.to_stage > QueryProcessingStage::WithMergeableState &&
!query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube && !query.group_by_with_grouping_sets;
// if (query.group_by_with_grouping_sets && query.group_by_with_totals)
// throw Exception("WITH TOTALS and GROUPING SETS are not supported together", ErrorCodes::NOT_IMPLEMENTED);
!query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube;
if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
{
@ -1183,7 +1179,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{
executeAggregation(
query_plan, expressions.before_aggregation, aggregate_overflow_row, aggregate_final, query_info.input_order_info);
/// We need to reset input order info, so that executeOrder can't use it
query_info.input_order_info.reset();
}
@ -1247,7 +1242,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{
if (query.group_by_with_totals)
{
bool final = !query.group_by_with_rollup && !query.group_by_with_cube && !query.group_by_with_grouping_sets;
bool final = !query.group_by_with_rollup && !query.group_by_with_cube;
executeTotalsAndHaving(
query_plan, expressions.hasHaving(), expressions.before_having, expressions.remove_having_filter, aggregate_overflow_row, final);
}
@ -1256,21 +1251,21 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
executeRollupOrCube(query_plan, Modificator::ROLLUP);
else if (query.group_by_with_cube)
executeRollupOrCube(query_plan, Modificator::CUBE);
else if (query.group_by_with_grouping_sets)
executeRollupOrCube(query_plan, Modificator::GROUPING_SETS);
if ((query.group_by_with_rollup || query.group_by_with_cube || query.group_by_with_grouping_sets) && expressions.hasHaving())
if ((query.group_by_with_rollup || query.group_by_with_cube) && expressions.hasHaving())
{
if (query.group_by_with_totals)
throw Exception("WITH TOTALS and WITH ROLLUP or CUBE or GROUPING SETS are not supported together in presence of HAVING", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(
"WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING",
ErrorCodes::NOT_IMPLEMENTED);
executeHaving(query_plan, expressions.before_having, expressions.remove_having_filter);
}
}
else if (expressions.hasHaving())
executeHaving(query_plan, expressions.before_having, expressions.remove_having_filter);
}
else if (query.group_by_with_totals || query.group_by_with_rollup || query.group_by_with_cube || query.group_by_with_grouping_sets)
throw Exception("WITH TOTALS, ROLLUP, CUBE or GROUPING SETS are not supported without aggregation", ErrorCodes::NOT_IMPLEMENTED);
else if (query.group_by_with_totals || query.group_by_with_rollup || query.group_by_with_cube)
throw Exception("WITH TOTALS, ROLLUP or CUBE are not supported without aggregation", ErrorCodes::NOT_IMPLEMENTED);
// Now we must execute:
// 1) expressions before window functions,
@ -2034,6 +2029,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
}
}
void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter)
{
auto where_step = std::make_unique<FilterStep>(
@ -2043,79 +2039,6 @@ void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsD
query_plan.addStep(std::move(where_step));
}
void InterpreterSelectQuery::initAggregatorParams(
const Block & current_data_stream_header,
AggregatorParamsPtr & params_ptr,
const AggregateDescriptions & aggregates,
bool overflow_row, const Settings & settings,
size_t group_by_two_level_threshold, size_t group_by_two_level_threshold_bytes)
{
auto & query = getSelectQuery();
if (query.group_by_with_grouping_sets)
{
ColumnNumbers keys;
ColumnNumbers all_keys;
ColumnNumbersList keys_vector;
std::unordered_set<size_t> keys_set;
for (const auto & aggregation_keys : query_analyzer->aggregationKeysList())
{
keys.clear();
for (const auto & key : aggregation_keys)
{
size_t key_name_pos = current_data_stream_header.getPositionByName(key.name);
keys_set.insert(key_name_pos);
keys.push_back(key_name_pos);
}
keys_vector.push_back(keys);
}
all_keys.assign(keys_set.begin(), keys_set.end());
params_ptr = std::make_unique<Aggregator::Params>(
current_data_stream_header,
all_keys,
keys_vector,
aggregates,
overflow_row,
settings.max_rows_to_group_by,
settings.group_by_overflow_mode,
group_by_two_level_threshold,
group_by_two_level_threshold_bytes,
settings.max_bytes_before_external_group_by,
settings.empty_result_for_aggregation_by_empty_set
|| (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty()
&& query_analyzer->hasConstAggregationKeys()),
context->getTemporaryVolume(),
settings.max_threads,
settings.min_free_disk_space_for_temporary_data,
settings.compile_aggregate_expressions,
settings.min_count_to_compile_aggregate_expression);
}
else
{
ColumnNumbers keys;
for (const auto & key : query_analyzer->aggregationKeys())
keys.push_back(current_data_stream_header.getPositionByName(key.name));
params_ptr = std::make_unique<Aggregator::Params>(
current_data_stream_header,
keys,
aggregates,
overflow_row,
settings.max_rows_to_group_by,
settings.group_by_overflow_mode,
group_by_two_level_threshold,
group_by_two_level_threshold_bytes,
settings.max_bytes_before_external_group_by,
settings.empty_result_for_aggregation_by_empty_set
|| (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty()
&& query_analyzer->hasConstAggregationKeys()),
context->getTemporaryVolume(),
settings.max_threads,
settings.min_free_disk_space_for_temporary_data,
settings.compile_aggregate_expressions,
settings.min_count_to_compile_aggregate_expression);
}
}
void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info)
{
@ -2127,6 +2050,9 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
return;
const auto & header_before_aggregation = query_plan.getCurrentDataStream().header;
ColumnNumbers keys;
for (const auto & key : query_analyzer->aggregationKeys())
keys.push_back(header_before_aggregation.getPositionByName(key.name));
AggregateDescriptions aggregates = query_analyzer->aggregates();
for (auto & descr : aggregates)
@ -2136,9 +2062,24 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
const Settings & settings = context->getSettingsRef();
AggregatorParamsPtr params_ptr;
initAggregatorParams(header_before_aggregation, params_ptr, aggregates, overflow_row, settings,
settings.group_by_two_level_threshold, settings.group_by_two_level_threshold_bytes);
Aggregator::Params params(
header_before_aggregation,
keys,
aggregates,
overflow_row,
settings.max_rows_to_group_by,
settings.group_by_overflow_mode,
settings.group_by_two_level_threshold,
settings.group_by_two_level_threshold_bytes,
settings.max_bytes_before_external_group_by,
settings.empty_result_for_aggregation_by_empty_set
|| (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty()
&& query_analyzer->hasConstAggregationKeys()),
context->getTemporaryVolume(),
settings.max_threads,
settings.min_free_disk_space_for_temporary_data,
settings.compile_aggregate_expressions,
settings.min_count_to_compile_aggregate_expression);
SortDescription group_by_sort_description;
@ -2156,7 +2097,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
auto aggregating_step = std::make_unique<AggregatingStep>(
query_plan.getCurrentDataStream(),
*params_ptr,
params,
final,
settings.max_block_size,
settings.aggregation_in_order_max_block_bytes,
@ -2165,6 +2106,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
storage_has_evenly_distributed_read,
std::move(group_by_info),
std::move(group_by_sort_description));
query_plan.addStep(std::move(aggregating_step));
}
@ -2216,27 +2158,47 @@ void InterpreterSelectQuery::executeTotalsAndHaving(
query_plan.addStep(std::move(totals_having_step));
}
void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modificator modificator)
{
const auto & header_before_transform = query_plan.getCurrentDataStream().header;
ColumnNumbers keys;
for (const auto & key : query_analyzer->aggregationKeys())
keys.push_back(header_before_transform.getPositionByName(key.name));
const Settings & settings = context->getSettingsRef();
AggregatorParamsPtr params_ptr;
initAggregatorParams(header_before_transform, params_ptr, query_analyzer->aggregates(), false, settings, 0, 0);
auto transform_params = std::make_shared<AggregatingTransformParams>(*params_ptr, true);
Aggregator::Params params(
header_before_transform,
keys,
query_analyzer->aggregates(),
false,
settings.max_rows_to_group_by,
settings.group_by_overflow_mode,
0,
0,
settings.max_bytes_before_external_group_by,
settings.empty_result_for_aggregation_by_empty_set,
context->getTemporaryVolume(),
settings.max_threads,
settings.min_free_disk_space_for_temporary_data,
settings.compile_aggregate_expressions,
settings.min_count_to_compile_aggregate_expression);
auto transform_params = std::make_shared<AggregatingTransformParams>(params, true);
QueryPlanStepPtr step;
if (modificator == Modificator::ROLLUP)
step = std::make_unique<RollupStep>(query_plan.getCurrentDataStream(), std::move(transform_params));
else if (modificator == Modificator::CUBE)
step = std::make_unique<CubeStep>(query_plan.getCurrentDataStream(), std::move(transform_params));
else
step = std::make_unique<GroupingSetsStep>(query_plan.getCurrentDataStream(), std::move(transform_params));
step = std::make_unique<CubeStep>(query_plan.getCurrentDataStream(), std::move(transform_params));
query_plan.addStep(std::move(step));
}
void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description)
{
if (!expression)

View File

@ -14,7 +14,6 @@
#include <Storages/TableLockHolder.h>
#include <Columns/FilterDescription.h>
#include "Interpreters/ActionsDAG.h"
namespace Poco
{
@ -30,7 +29,6 @@ class QueryPlan;
struct TreeRewriterResult;
using TreeRewriterResultPtr = std::shared_ptr<const TreeRewriterResult>;
using AggregatorParamsPtr = std::unique_ptr<Aggregator::Params>;
/** Interprets the SELECT query. Returns the stream of blocks with the results of the query before `to_stage` stage.
@ -128,12 +126,6 @@ private:
/// Different stages of query execution.
void initAggregatorParams(
const Block & current_data_stream_header,
AggregatorParamsPtr & params_ptr,
const AggregateDescriptions & aggregates,
bool overflow_row, const Settings & settings,
size_t group_by_two_level_threshold, size_t group_by_two_level_threshold_bytes);
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan);
void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter);
void executeAggregation(
@ -164,8 +156,7 @@ private:
enum class Modificator
{
ROLLUP = 0,
CUBE = 1,
GROUPING_SETS = 2
CUBE = 1
};
void executeRollupOrCube(QueryPlan & query_plan, Modificator modificator);

View File

@ -200,24 +200,12 @@ GroupByKeysInfo getGroupByKeysInfo(const ASTs & group_by_keys)
/// filling set with short names of keys
for (const auto & group_key : group_by_keys)
{
/// for grouping sets case
if (group_key->as<ASTExpressionList>())
{
const auto express_list_ast = group_key->as<const ASTExpressionList &>();
for (const auto & group_elem : express_list_ast.children)
{
data.key_names.insert(group_elem->getColumnName());
}
}
else
{
if (group_key->as<ASTFunction>())
data.has_function = true;
data.key_names.insert(group_key->getColumnName());
}
}
return data;
}

View File

@ -26,15 +26,7 @@ void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState
settings.ostr << ' ';
}
if (frame.surround_each_list_element_with_parens)
settings.ostr << "(";
FormatStateStacked frame_nested = frame;
frame_nested.surround_each_list_element_with_parens = false;
(*it)->formatImpl(settings, state, frame_nested);
if (frame.surround_each_list_element_with_parens)
settings.ostr << ")";
(*it)->formatImpl(settings, state, frame);
}
}
@ -49,7 +41,6 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For
}
++frame.indent;
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
@ -63,15 +54,7 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For
FormatStateStacked frame_nested = frame;
frame_nested.expression_list_always_start_on_new_line = false;
frame_nested.surround_each_list_element_with_parens = false;
if (frame.surround_each_list_element_with_parens)
settings.ostr << "(";
(*it)->formatImpl(settings, state, frame_nested);
if (frame.surround_each_list_element_with_parens)
settings.ostr << ")";
}
}

View File

@ -114,13 +114,10 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
if (groupBy())
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY" << (s.hilite ? hilite_none : "");
if (!group_by_with_grouping_sets)
{
s.one_line
? groupBy()->formatImpl(s, state, frame)
: groupBy()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
}
}
if (group_by_with_rollup)
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH ROLLUP" << (s.hilite ? hilite_none : "");
@ -128,18 +125,6 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
if (group_by_with_cube)
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH CUBE" << (s.hilite ? hilite_none : "");
if (group_by_with_grouping_sets)
{
frame.surround_each_list_element_with_parens = true;
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "GROUPING SETS" << (s.hilite ? hilite_none : "");
s.ostr << " (";
s.one_line
? groupBy()->formatImpl(s, state, frame)
: groupBy()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
s.ostr << ")";
frame.surround_each_list_element_with_parens = false;
}
if (group_by_with_totals)
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH TOTALS" << (s.hilite ? hilite_none : "");

View File

@ -83,7 +83,6 @@ public:
bool group_by_with_rollup = false;
bool group_by_with_cube = false;
bool group_by_with_constant_keys = false;
bool group_by_with_grouping_sets = false;
bool limit_with_ties = false;
ASTPtr & refSelect() { return getExpression(Expression::SELECT); }

View File

@ -749,61 +749,13 @@ bool ParserNotEmptyExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected
return nested_parser.parse(pos, node, expected) && !node->children.empty();
}
bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserOrderByElement>(), std::make_unique<ParserToken>(TokenType::Comma), false)
.parse(pos, node, expected);
}
bool ParserGroupingSetsExpressionListElements::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto command_list = std::make_shared<ASTExpressionList>();
node = command_list;
ParserToken s_comma(TokenType::Comma);
ParserToken s_open(TokenType::OpeningRoundBracket);
ParserToken s_close(TokenType::ClosingRoundBracket);
ParserExpressionWithOptionalAlias p_expression(false);
ParserList p_command(std::make_unique<ParserExpressionWithOptionalAlias>(false),
std::make_unique<ParserToken>(TokenType::Comma), true);
do
{
Pos begin = pos;
ASTPtr command;
if (!s_open.ignore(pos, expected))
{
pos = begin;
if (!p_expression.parse(pos, command, expected))
{
return false;
}
auto list = std::make_shared<ASTExpressionList>(',');
list->children.push_back(command);
command = std::move(list);
}
else
{
if (!p_command.parse(pos, command, expected))
return false;
if (!s_close.ignore(pos, expected))
break;
}
command_list->children.push_back(command);
}
while (s_comma.ignore(pos, expected));
return true;
}
bool ParserGroupingSetsExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserGroupingSetsExpressionListElements grouping_sets_elements;
return grouping_sets_elements.parse(pos, node, expected);
}
bool ParserTTLExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{

View File

@ -511,20 +511,6 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserGroupingSetsExpressionList : public IParserBase
{
protected:
const char * getName() const override { return "grouping sets expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserGroupingSetsExpressionListElements : public IParserBase
{
protected:
const char * getName() const override { return "grouping sets expression elements"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Parser for key-value pair, where value can be list of pairs.
class ParserKeyValuePair : public IParserBase

View File

@ -224,7 +224,6 @@ public:
bool need_parens = false;
bool expression_list_always_start_on_new_line = false; /// Line feed and indent before expression list even if it's of single element.
bool expression_list_prepend_whitespace = false; /// Prepend whitespace (if it is required)
bool surround_each_list_element_with_parens = false;
const IAST * current_select = nullptr;
};

View File

@ -50,7 +50,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_by("BY");
ParserKeyword s_rollup("ROLLUP");
ParserKeyword s_cube("CUBE");
ParserKeyword s_grouping_sets("GROUPING SETS");
ParserKeyword s_top("TOP");
ParserKeyword s_with_ties("WITH TIES");
ParserKeyword s_offset("OFFSET");
@ -66,7 +65,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword.
ParserExpressionWithOptionalAlias exp_elem(false);
ParserOrderByExpressionList order_list;
ParserGroupingSetsExpressionList grouping_sets_list;
ParserToken open_bracket(TokenType::OpeningRoundBracket);
ParserToken close_bracket(TokenType::ClosingRoundBracket);
@ -186,39 +184,24 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
select_query->group_by_with_rollup = true;
else if (s_cube.ignore(pos, expected))
select_query->group_by_with_cube = true;
else if (s_grouping_sets.ignore(pos, expected))
select_query->group_by_with_grouping_sets = true;
if ((select_query->group_by_with_rollup || select_query->group_by_with_cube || select_query->group_by_with_grouping_sets) &&
!open_bracket.ignore(pos, expected))
if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !open_bracket.ignore(pos, expected))
return false;
if (select_query->group_by_with_grouping_sets)
{
if (!grouping_sets_list.parse(pos, group_expression_list, expected))
return false;
}
else
{
if (!exp_list.parse(pos, group_expression_list, expected))
return false;
}
if ((select_query->group_by_with_rollup || select_query->group_by_with_cube || select_query->group_by_with_grouping_sets) &&
!close_bracket.ignore(pos, expected))
if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !close_bracket.ignore(pos, expected))
return false;
}
/// WITH ROLLUP, CUBE, GROUPING SETS or TOTALS
/// WITH ROLLUP, CUBE or TOTALS
if (s_with.ignore(pos, expected))
{
if (s_rollup.ignore(pos, expected))
select_query->group_by_with_rollup = true;
else if (s_cube.ignore(pos, expected))
select_query->group_by_with_cube = true;
else if (s_grouping_sets.ignore(pos, expected))
select_query->group_by_with_grouping_sets = true;
else if (s_totals.ignore(pos, expected))
select_query->group_by_with_totals = true;
else

View File

@ -56,8 +56,6 @@ private:
Processors aggregating;
Poco::Logger * log = &Poco::Logger::get("AggregatingStep");
};
}

View File

@ -1,46 +0,0 @@
#include <Processors/QueryPlan/GroupingSetsStep.h>
#include <Processors/Transforms/GroupingSetsTransform.h>
#include "QueryPipeline/QueryPipelineBuilder.h"
namespace DB
{
static ITransformingStep::Traits getTraits()
{
return ITransformingStep::Traits
{
{
.preserves_distinct_columns = false,
.returns_single_stream = true,
.preserves_number_of_streams = false,
.preserves_sorting = false,
},
{
.preserves_number_of_rows = false,
}
};
}
GroupingSetsStep::GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_)
: ITransformingStep(input_stream_, params_->getHeader(), getTraits())
, params(std::move(params_))
{
/// Aggregation keys are distinct
for (auto key : params->params.keys)
output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name);
}
void GroupingSetsStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
pipeline.resize(1);
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr
{
if (stream_type == QueryPipelineBuilder::StreamType::Totals)
return nullptr;
return std::make_shared<GroupingSetsTransform>(header, std::move(params));
});
}
}

View File

@ -1,25 +0,0 @@
#pragma once
#include <Processors/QueryPlan/ITransformingStep.h>
#include <QueryPipeline/SizeLimits.h>
#include "QueryPipeline/QueryPipelineBuilder.h"
namespace DB
{
struct AggregatingTransformParams;
using AggregatingTransformParamsPtr = std::shared_ptr<AggregatingTransformParams>;
class GroupingSetsStep : public ITransformingStep
{
public:
GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_);
String getName() const override { return "GroupingSets"; }
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
private:
AggregatingTransformParamsPtr params;
};
}

View File

@ -1,76 +0,0 @@
#include <Processors/Transforms/GroupingSetsTransform.h>
#include <Processors/Transforms/TotalsHavingTransform.h>
namespace DB
{
GroupingSetsTransform::GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params_)
: IAccumulatingTransform(std::move(header), params_->getHeader())
, params(std::move(params_))
, keys(params->params.keys)
, keys_vector(params->params.keys_vector)
, keys_vector_idx(0)
{
}
void GroupingSetsTransform::consume(Chunk chunk)
{
consumed_chunks.emplace_back(std::move(chunk));
}
Chunk GroupingSetsTransform::merge(Chunks && chunks, bool final)
{
BlocksList grouping_sets_blocks;
for (auto & chunk : chunks)
grouping_sets_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()));
auto grouping_sets_block = params->aggregator.mergeBlocks(grouping_sets_blocks, final);
auto num_rows = grouping_sets_block.rows();
return Chunk(grouping_sets_block.getColumns(), num_rows);
}
Chunk GroupingSetsTransform::generate()
{
if (!consumed_chunks.empty())
{
if (consumed_chunks.size() > 1)
grouping_sets_chunk = merge(std::move(consumed_chunks), false);
else
grouping_sets_chunk = std::move(consumed_chunks.front());
consumed_chunks.clear();
auto num_rows = grouping_sets_chunk.getNumRows();
current_columns = grouping_sets_chunk.getColumns();
current_zero_columns.clear();
for (auto key : keys)
current_zero_columns.emplace(key, current_columns[key]->cloneEmpty()->cloneResized(num_rows));
}
Chunk gen_chunk;
if (!current_columns.empty() && keys_vector_idx < keys_vector.size())
{
auto columns = current_columns;
std::set<size_t> key_vector(keys_vector[keys_vector_idx].begin(), keys_vector[keys_vector_idx].end());
for (auto key : keys)
{
if (!key_vector.contains(key))
columns[key] = current_zero_columns[key];
}
Chunks chunks;
chunks.emplace_back(std::move(columns), current_columns.front()->size());
gen_chunk = merge(std::move(chunks), false);
++keys_vector_idx;
}
finalizeChunk(gen_chunk);
return gen_chunk;
}
}

View File

@ -1,35 +0,0 @@
#pragma once
#include <Processors/IAccumulatingTransform.h>
#include <Processors/Transforms/AggregatingTransform.h>
namespace DB
{
class GroupingSetsTransform : public IAccumulatingTransform
{
public:
GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params);
String getName() const override { return "GroupingSetsTransform"; }
protected:
void consume(Chunk chunk) override;
Chunk generate() override;
private:
AggregatingTransformParamsPtr params;
const ColumnNumbers & keys;
const ColumnNumbersList & keys_vector;
Chunks consumed_chunks;
Chunk grouping_sets_chunk;
Columns current_columns;
std::unordered_map<size_t, ColumnPtr> current_zero_columns;
UInt64 keys_vector_idx = 0;
Poco::Logger * log = &Poco::Logger::get("GroupingSetsTransform");
Chunk merge(Chunks && chunks, bool final);
};
}

View File

@ -497,7 +497,7 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort
for (size_t i = 1; i < output_ports.size(); ++i)
assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes");
// Temporarily skip this check. TotalsHavingTransform may return finalized totals but not finalized data.
// Temporarily skip this check. TotaslHavingTransform may return finalized totals but not finalized data.
// if (totals_port)
// assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes");

View File

@ -4,7 +4,6 @@
#include <QueryPipeline/PipelineResourcesHolder.h>
#include <QueryPipeline/Chain.h>
#include <QueryPipeline/SizeLimits.h>
#include <base/logger_useful.h>
namespace DB
{
@ -137,8 +136,6 @@ private:
/// It is needed for debug. See QueryPipelineProcessorsCollector.
Processors * collected_processors = nullptr;
Poco::Logger * log = &Poco::Logger::get("Pipe");
/// This methods are for QueryPipeline. It is allowed to complete graph only there.
/// So, we may be sure that Pipe always has output port if not empty.
bool isCompleted() const { return !empty() && output_ports.empty(); }

View File

@ -1,37 +1,13 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# Tag no-fasttest: nc - command not found
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
PORT="$(($RANDOM%63000+2001))"
function thread1
{
while true; do
echo -e "HTTP/1.1 200 OK\n\n{\"a\": 1}" | nc -l -p $1 -q 1;
done
}
function thread2
{
while true; do
$CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 -q "SELECT * FROM url('http://127.0.0.1:$1/', JSONEachRow, 'a int, b int default 7, c default a + b') format Values" | grep -F '(1,7,8)' && break
done
}
# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout
export -f thread1;
export -f thread2;
TIMEOUT=60
timeout $TIMEOUT bash -c "thread1 $PORT" > /dev/null 2>&1 &
PID=$!
bash -c "thread2 $PORT" 2> /dev/null | grep -q -F '(1,7,8)' && echo "Ok" && kill -9 $PID
wait >/dev/null 2>&1
${CLICKHOUSE_CLIENT} --query "
SELECT * FROM url(
\$\$http://127.0.0.1:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+'{\"a\":1}'\$\$,
JSONEachRow,
'a int, b int default 7, c default a + b')
FORMAT Values"

View File

@ -1,89 +0,0 @@
1 0 1 4500
1 0 3 4700
1 0 5 4900
1 0 7 5100
1 0 9 5300
1 1 0 4500
1 2 0 5100
1 3 0 4700
1 4 0 5300
1 5 0 4900
2 0 2 4600
2 0 4 4800
2 0 6 5000
2 0 8 5200
2 0 10 5400
2 1 0 5000
2 2 0 4600
2 3 0 5200
2 4 0 4800
2 5 0 5400
0 0 1 1 4500
0 0 2 2 4600
0 0 3 3 4700
0 0 4 4 4800
0 0 5 5 4900
0 0 6 6 5000
0 0 7 7 5100
0 0 8 8 5200
0 0 9 9 5300
0 0 10 10 5400
1 1 0 0 4500
1 2 0 0 5100
1 3 0 0 4700
1 4 0 0 5300
1 5 0 0 4900
2 1 0 0 5000
2 2 0 0 4600
2 3 0 0 5200
2 4 0 0 4800
2 5 0 0 5400
0 0 0 49500
0 0 1 4500
0 0 2 4600
0 0 3 4700
0 0 4 4800
0 0 5 4900
0 0 6 5000
0 0 7 5100
0 0 8 5200
0 0 9 5300
0 0 10 5400
1 1 0 4500
1 2 0 5100
1 3 0 4700
1 4 0 5300
1 5 0 4900
2 1 0 5000
2 2 0 4600
2 3 0 5200
2 4 0 4800
2 5 0 5400
1 0 24500
1 1 4500
1 3 4700
1 5 4900
1 7 5100
1 9 5300
2 0 25000
2 2 4600
2 4 4800
2 6 5000
2 8 5200
2 10 5400
0 0 49500
1 0 24500
1 1 4500
1 3 4700
1 5 4900
1 7 5100
1 9 5300
2 0 25000
2 2 4600
2 4 4800
2 6 5000
2 8 5200
2 10 5400
0 0 49500

View File

@ -1,44 +0,0 @@
DROP TABLE IF EXISTS grouping_sets;
CREATE TABLE grouping_sets(fact_1_id Int32, fact_2_id Int32, fact_3_id Int32, fact_4_id Int32, sales_value Int32) ENGINE = Memory;
SELECT fact_1_id, fact_3_id, sum(sales_value), count() from grouping_sets GROUP BY GROUPING SETS(fact_1_id, fact_3_id) ORDER BY fact_1_id, fact_3_id;
INSERT INTO grouping_sets
SELECT
number % 2 + 1 AS fact_1_id,
number % 5 + 1 AS fact_2_id,
number % 10 + 1 AS fact_3_id,
number % 10 + 1 AS fact_4_id,
number % 100 AS sales_value
FROM system.numbers limit 1000;
SELECT fact_1_id, fact_2_id, fact_3_id, SUM(sales_value) AS sales_value from grouping_sets
GROUP BY GROUPING SETS ((fact_1_id, fact_2_id), (fact_1_id, fact_3_id))
ORDER BY fact_1_id, fact_2_id, fact_3_id;
SELECT fact_1_id, fact_2_id, fact_3_id, fact_4_id, SUM(sales_value) AS sales_value from grouping_sets
GROUP BY GROUPING SETS ((fact_1_id, fact_2_id), (fact_3_id, fact_4_id))
ORDER BY fact_1_id, fact_2_id, fact_3_id, fact_4_id;
SELECT fact_1_id, fact_2_id, fact_3_id, SUM(sales_value) AS sales_value from grouping_sets
GROUP BY GROUPING SETS ((fact_1_id, fact_2_id), (fact_3_id), ())
ORDER BY fact_1_id, fact_2_id, fact_3_id;
SELECT
fact_1_id,
fact_3_id,
SUM(sales_value) AS sales_value
FROM grouping_sets
GROUP BY grouping sets ((fact_1_id), (fact_1_id, fact_3_id)) WITH TOTALS
ORDER BY fact_1_id, fact_3_id;
SELECT
fact_1_id,
fact_3_id,
SUM(sales_value) AS sales_value
FROM grouping_sets
GROUP BY grouping sets (fact_1_id, (fact_1_id, fact_3_id)) WITH TOTALS
ORDER BY fact_1_id, fact_3_id;
DROP TABLE grouping_sets;

View File

@ -0,0 +1 @@
0

View File

@ -0,0 +1 @@
select bitmapContains(bitmapBuild([9]), 964291337)

View File

@ -0,0 +1,137 @@
<html id="html" style="color: white; background: black;">
<head>
<meta charset="UTF-8">
</head>
<body style="font-family: sans-serif">
<h1 id="loading" style="position: absolute; margin-left: 1em;">Loading (10 seconds, 20 MB)...</h1>
<div id="info" style="position: sticky; top: 1rem; z-index: 1; margin: 1rem; float: right; font-size: 16pt; padding: 0.5rem; border: 1px solid #111; display: none;"></div>
<canvas style="position: absolute" id="canvas"></canvas>
<script type="text/javascript">
let start_date = '2020-06-13';
let query = `
WITH '${start_date}'::Date AS start_date
SELECT groupArray([d, n, fail]) FROM
(
SELECT n, check_start_time::Date - start_date AS d, max(test_status LIKE 'F%' OR test_status LIKE 'E%') AS fail
FROM "gh-data".checks
INNER JOIN
(
SELECT test_name, toUInt16(rowNumberInAllBlocks()) AS n FROM
(
SELECT DISTINCT test_name
FROM "gh-data".checks
WHERE match(test_name, '^\\d+_') AND check_name ILIKE '%stateless%' AND check_start_time > now() - INTERVAL 1 DAY
ORDER BY test_name
)
) AS nums
USING (test_name)
WHERE
check_name ILIKE '%stateless%'
AND pull_request_number = 0
AND check_start_time >= start_date
GROUP BY d, n
ORDER BY d, n
)
FORMAT TSV`;
function load(query, callback) {
const xhr = new XMLHttpRequest;
xhr.open('POST', "https://play-ci.clickhouse.com/?user=play&add_http_cors_header=1", true);
xhr.onreadystatechange = function()
{
if (this.readyState === XMLHttpRequest.DONE && this.status == 200) {
callback(this.response);
}
}
xhr.send(query);
}
load(query, renderResponse);
let data;
let canvas = document.getElementById('canvas');
function renderResponse(response) {
data = JSON.parse(response);
const last_pixel = data[data.length - 1];
canvas.width = last_pixel[0] + 1;
canvas.height = last_pixel[1] + 1;
document.getElementById('html').style.height = canvas.height + 10 + 'px';
document.body.style.height = canvas.height + 10 + 'px';
let ctx = canvas.getContext('2d');
ctx.imageSmoothingEnabled = false;
ctx.mozImageSmoothingEnabled = false;
ctx.webkitImageSmoothingEnabled = false;
ctx.msImageSmoothingEnabled = false;
let image = ctx.createImageData(canvas.width, canvas.height);
let pixels = image.data;
data.map(elem => {
let x = elem[0];
let y = canvas.height - elem[1];
pixels[(x + y * canvas.width) * 4 + 0] = elem[2] ? 255 : 0; // r
pixels[(x + y * canvas.width) * 4 + 1] = elem[2] ? 0 : 100; // g
pixels[(x + y * canvas.width) * 4 + 2] = 0; // b
pixels[(x + y * canvas.width) * 4 + 3] = 255; // a
});
document.getElementById('loading').style.display = 'none';
ctx.putImageData(image, 0, 0);
}
let test_names_query = `
SELECT test_name, toUInt16(rowNumberInAllBlocks()) AS n FROM
(
SELECT DISTINCT test_name
FROM "gh-data".checks
WHERE match(test_name, '^\\d+_') AND check_name ILIKE '%stateless%' AND check_start_time > now() - INTERVAL 1 DAY
ORDER BY test_name
) FORMAT JSONCompact`
let test_names;
load(test_names_query, saveTestNames);
function saveTestNames(response)
{
test_names = JSON.parse(response).data;
}
canvas.addEventListener('mousemove', event => {
const x = event.layerX;
const y = event.layerY;
let date = new Date(start_date);
date.setDate(date.getDate() + x);
date = date.toISOString().split('T')[0];
let test = test_names[canvas.height - y][0];
let pixel = canvas.getContext('2d').getImageData(x, y, 1, 1).data;
console.log(pixel);
let info = document.getElementById('info');
info.innerText = `${date}, ${test}`;
info.style.background = pixel[0] > 0 ? '#F00' : (pixel[3] > 0 ? '#006400' : '#000');
info.style.display = 'block';
})
</script>
</body>
</html>

View File

@ -82,6 +82,8 @@ Comparison of EBS and EFS is from <b>Ramazan Polat</b>.<br/>
Results for Hetzner and Scaleway are from <b>Anthony Najjar Simon</b> (Panelbear).<br/>
Results for GCP are from <b>Vy Nguyen Tan</b>.<br/>
Results for ThinkPad P15 are from <b>Mikhail Shiryaev</b>.<br/>
Results for RockPi4 are from <b>Kirill Zholnay</b>.<br/>
Results for Xeon 6266C are from <b>David in Shanghai</b>.<br/>
</p>
</div>
</div>

View File

@ -0,0 +1,54 @@
[
{
"system": "Rock Pi 4, 4GiB, NVMe",
"system_full": "Rock Pi 4, 4GiB C, NVMe",
"time": "2021-12-23 00:00:00",
"kind": "desktop",
"result":
[
[0.007, 0.014, 0.005],
[0.229, 0.132, 0.215],
[0.489, 0.351, 0.306],
[0.879, 0.774, 0.768],
[1.034, 0.966, 0.879],
[2.491, 2.249, 2.493],
[0.379, 0.212, 0.213],
[0.227, 0.140, 0.152],
[3.944, 3.823, 3.805],
[5.272, 4.985, 5.069],
[2.356, 2.193, 2.254],
[2.819, 2.595, 2.568],
[9.124, 8.306, 8.529],
[11.857, 11.412, 11.290],
[9.796, 9.477, 9.610],
[8.846, 8.867, 8.909],
[null, null, null],
[null, null, null],
[null, null, null],
[1.293, 0.887, 0.980],
[15.018, 14.928, 14.748],
[19.179, 17.889, 18.021],
[45.524, 46.927, 46.909],
[23.904, 23.197, 23.511],
[5.264, 4.891, 4.936],
[4.211, 3.940, 4.047],
[5.113, 4.615, 4.783],
[17.910, 16.800, 16.410],
[23.537, 22.249, 22.172],
[16.549, 16.388, 16.337],
[9.562, 9.006, 9.260],
[17.097, 17.676, 17.585],
[null, null, null],
[null, null, null],
[null, null, null],
[null, null, null],
[1.668, 1.469, 1.342],
[0.463, 0.442, 0.353],
[0.486, 0.410, 0.346],
[2.190, 2.014, 1.878],
[0.263, 0.097, 0.201],
[0.173, 0.082, 0.139],
[0.188, 0.024, 0.016]
]
}
]

View File

@ -0,0 +1,56 @@
[
{
"system": "Xeon Gold 6266C, 3GHz, 4vCPU",
"system_full": "Xeon Gold 6266C, 3GHz, 4vCPU, 16GiB RAM, vda1 40GB",
"cpu_vendor": "Intel",
"cpu_model": "Xeon Gold 6266C",
"time": "2021-12-23 00:00:00",
"kind": "cloud",
"result":
[
[0.001, 0.001, 0.001],
[0.034, 0.023, 0.023],
[0.168, 0.105, 0.104],
[0.745, 0.162, 0.160],
[1.512, 0.328, 0.327],
[2.408, 1.162, 1.155],
[0.069, 0.052, 0.051],
[0.074, 0.027, 0.026],
[2.314, 1.833, 1.796],
[2.749, 2.014, 2.011],
[1.424, 0.618, 0.579],
[1.494, 0.681, 0.677],
[3.208, 2.457, 2.529],
[5.071, 3.329, 3.411],
[3.968, 3.289, 3.330],
[3.142, 2.925, 2.827],
[9.473, 9.034, 8.850],
[6.768, 6.256, 6.115],
[18.388, 17.790, 17.892],
[1.105, 0.195, 0.194],
[20.310, 3.459, 3.416],
[22.772, 3.811, 3.773],
[42.554, 8.738, 8.640],
[30.747, 4.013, 3.967],
[4.707, 0.973, 0.965],
[2.003, 0.845, 0.839],
[4.978, 0.991, 0.974],
[19.726, 3.293, 3.264],
[17.151, 5.171, 5.134],
[3.620, 3.600, 3.600],
[4.693, 2.172, 2.115],
[10.842, 2.686, 2.750],
[17.857, 17.086, 16.907],
[22.926, 13.070, 12.808],
[22.803, 12.727, 12.867],
[4.189, 3.888, 3.893],
[0.227, 0.176, 0.177],
[0.085, 0.068, 0.067],
[0.101, 0.064, 0.067],
[0.493, 0.438, 0.399],
[0.042, 0.022, 0.021],
[0.029, 0.017, 0.015],
[0.007, 0.005, 0.003]
]
}
]