mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into background_fetches_pool_size
This commit is contained in:
commit
34aaa710fa
1
.github/workflows/backport_branches.yml
vendored
1
.github/workflows/backport_branches.yml
vendored
@ -76,6 +76,7 @@ jobs:
|
||||
uses: ClickHouse/checkout@v1
|
||||
with:
|
||||
clear-repository: true
|
||||
fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags
|
||||
- name: Download changed aarch64 images
|
||||
uses: actions/download-artifact@v3
|
||||
with:
|
||||
|
1
.github/workflows/docs_check.yml
vendored
1
.github/workflows/docs_check.yml
vendored
@ -73,6 +73,7 @@ jobs:
|
||||
uses: ClickHouse/checkout@v1
|
||||
with:
|
||||
clear-repository: true
|
||||
fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags
|
||||
- name: Download changed aarch64 images
|
||||
uses: actions/download-artifact@v3
|
||||
with:
|
||||
|
1
.github/workflows/master.yml
vendored
1
.github/workflows/master.yml
vendored
@ -60,6 +60,7 @@ jobs:
|
||||
uses: ClickHouse/checkout@v1
|
||||
with:
|
||||
clear-repository: true
|
||||
fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags
|
||||
- name: Download changed aarch64 images
|
||||
uses: actions/download-artifact@v3
|
||||
with:
|
||||
|
1
.github/workflows/nightly.yml
vendored
1
.github/workflows/nightly.yml
vendored
@ -53,6 +53,7 @@ jobs:
|
||||
uses: ClickHouse/checkout@v1
|
||||
with:
|
||||
clear-repository: true
|
||||
fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags
|
||||
- name: Download changed aarch64 images
|
||||
uses: actions/download-artifact@v3
|
||||
with:
|
||||
|
1
.github/workflows/pull_request.yml
vendored
1
.github/workflows/pull_request.yml
vendored
@ -94,6 +94,7 @@ jobs:
|
||||
uses: ClickHouse/checkout@v1
|
||||
with:
|
||||
clear-repository: true
|
||||
fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags
|
||||
- name: Download changed aarch64 images
|
||||
uses: actions/download-artifact@v3
|
||||
with:
|
||||
|
1
.github/workflows/release_branches.yml
vendored
1
.github/workflows/release_branches.yml
vendored
@ -52,6 +52,7 @@ jobs:
|
||||
uses: ClickHouse/checkout@v1
|
||||
with:
|
||||
clear-repository: true
|
||||
fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags
|
||||
- name: Download changed aarch64 images
|
||||
uses: actions/download-artifact@v3
|
||||
with:
|
||||
|
2
contrib/libunwind
vendored
2
contrib/libunwind
vendored
@ -1 +1 @@
|
||||
Subproject commit e48aa13f67dc722511b5af33a32ba9b7748176b5
|
||||
Subproject commit 30cc1d3fd3655a5cfa0ab112fe320fb9fc0a8344
|
@ -30,7 +30,7 @@ It may lack support for new features.
|
||||
|
||||
## Usage {#cli_usage}
|
||||
|
||||
The client can be used in interactive and non-interactive (batch) mode.
|
||||
The client can be used in interactive and non-interactive (batch) mode.
|
||||
|
||||
### Gather your connection details
|
||||
<ConnectionDetails />
|
||||
@ -177,8 +177,8 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va
|
||||
- `--user, -u` – The username. Default value: default.
|
||||
- `--password` – The password. Default value: empty string.
|
||||
- `--ask-password` - Prompt the user to enter a password.
|
||||
- `--query, -q` – The query to process when using non-interactive mode. Cannot be used simultaneously with `--queries-file`.
|
||||
- `--queries-file` – file path with queries to execute. Cannot be used simultaneously with `--query`.
|
||||
- `--query, -q` – The query to process when using non-interactive mode. `--query` can be specified multiple times, e.g. `--query "SELECT 1" --query "SELECT 2"`. Cannot be used simultaneously with `--queries-file`.
|
||||
- `--queries-file` – file path with queries to execute. `--queries-file` can be specified multiple times, e.g. `--query queries1.sql --query queries2.sql`. Cannot be used simultaneously with `--query`.
|
||||
- `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`.
|
||||
- `--multiline, -m` – If specified, allow multiline queries (do not send the query on Enter).
|
||||
- `--database, -d` – Select the current default database. Default value: the current database from the server settings (‘default’ by default).
|
||||
|
@ -18,6 +18,8 @@ $ curl 'http://localhost:8123/'
|
||||
Ok.
|
||||
```
|
||||
|
||||
Also see: [HTTP response codes caveats](#http_response_codes_caveats).
|
||||
|
||||
Sometimes, `curl` command is not available on user operating systems. On Ubuntu or Debian, run `sudo apt install curl`. Please refer this [documentation](https://curl.se/download.html) to install it before running the examples.
|
||||
|
||||
Web UI can be accessed here: `http://localhost:8123/play`.
|
||||
@ -323,6 +325,27 @@ $ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&
|
||||
|
||||
Use buffering to avoid situations where a query processing error occurred after the response code and HTTP headers were sent to the client. In this situation, an error message is written at the end of the response body, and on the client-side, the error can only be detected at the parsing stage.
|
||||
|
||||
## HTTP response codes caveats {#http_response_codes_caveats}
|
||||
|
||||
Because of limitation of HTTP protocol, HTTP 200 response code does not guarantee that a query was successful.
|
||||
|
||||
Here is an example:
|
||||
|
||||
```
|
||||
curl -v -Ss "http://localhost:8123/?max_block_size=1&query=select+sleepEachRow(0.001),throwIf(number=2)from+numbers(5)"
|
||||
* Trying 127.0.0.1:8123...
|
||||
...
|
||||
< HTTP/1.1 200 OK
|
||||
...
|
||||
Code: 395. DB::Exception: Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(equals(number, 2) :: 1) -> throwIf(equals(number, 2))
|
||||
```
|
||||
|
||||
The reason for this behavior is the nature of the HTTP protocol. The HTTP header is sent first with an HTTP code of 200, followed by the HTTP body, and then the error is injected into the body as plain text.
|
||||
This behavior is independent of the format used, whether it's `Native`, `TSV`, or `JSON`; the error message will always be in the middle of the response stream.
|
||||
You can mitigate this problem by enabling `wait_end_of_query=1` ([Response Buffering](#response-buffering)). In this case, the sending of the HTTP header is delayed until the entire query is resolved.
|
||||
However, this does not completely solve the problem because the result must still fit within the `http_response_buffer_size`, and other settings like `send_progress_in_http_headers` can interfere with the delay of the header.
|
||||
The only way to catch all errors is to analyze the HTTP body before parsing it using the required format.
|
||||
|
||||
### Queries with Parameters {#cli-queries-with-parameters}
|
||||
|
||||
You can create a query with parameters and pass values for them from the corresponding HTTP request parameters. For more information, see [Queries with Parameters for CLI](../interfaces/cli.md#cli-queries-with-parameters).
|
||||
|
@ -11,6 +11,8 @@ ClickHouse runs sampling profiler that allows analyzing query execution. Using p
|
||||
|
||||
Query profiler is automatically enabled in ClickHouse Cloud and you can run a sample query as follows
|
||||
|
||||
:::note If you are running the following query in ClickHouse Cloud, make sure to change `FROM system.trace_log` to `FROM clusterAllReplicas(default, system.trace_log)` to select from all nodes of the cluster :::
|
||||
|
||||
``` sql
|
||||
SELECT
|
||||
count(),
|
||||
|
@ -4644,6 +4644,14 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars
|
||||
└─────────────────────┴──────────────────────────┘
|
||||
```
|
||||
|
||||
## partial_result_update_duration_ms
|
||||
|
||||
Interval (in milliseconds) for sending updates with partial data about the result table to the client (in interactive mode) during query execution. Setting to 0 disables partial results. Only supported for single-threaded GROUP BY without key, ORDER BY, LIMIT and OFFSET.
|
||||
|
||||
## max_rows_in_partial_result
|
||||
|
||||
Maximum rows to show in the partial result after every real-time update while the query runs (use partial result limit + OFFSET as a value in case of OFFSET in the query).
|
||||
|
||||
## validate_tcp_client_information {#validate-tcp-client-information}
|
||||
|
||||
Determines whether validation of client information enabled when query packet is received from a client using a TCP connection.
|
||||
|
@ -202,8 +202,8 @@ Arguments:
|
||||
- `-S`, `--structure` — table structure for input data.
|
||||
- `--input-format` — input format, `TSV` by default.
|
||||
- `-f`, `--file` — path to data, `stdin` by default.
|
||||
- `-q`, `--query` — queries to execute with `;` as delimiter. Cannot be used simultaneously with `--queries-file`.
|
||||
- `--queries-file` - file path with queries to execute. Cannot be used simultaneously with `--query`.
|
||||
- `-q`, `--query` — queries to execute with `;` as delimiter. `--query` can be specified multiple times, e.g. `--query "SELECT 1" --query "SELECT 2"`. Cannot be used simultaneously with `--queries-file`.
|
||||
- `--queries-file` - file path with queries to execute. `--queries-file` can be specified multiple times, e.g. `--query queries1.sql --query queries2.sql`. Cannot be used simultaneously with `--query`.
|
||||
- `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`.
|
||||
- `-N`, `--table` — table name where to put output data, `table` by default.
|
||||
- `--format`, `--output-format` — output format, `TSV` by default.
|
||||
|
@ -204,7 +204,7 @@ Other possible results:
|
||||
Query:
|
||||
|
||||
```sql
|
||||
SELECT detectLanguageMixed('Je pense que je ne parviendrai jamais à parler français comme un natif. Where there’s a will, there’s a way.');
|
||||
SELECT detectLanguage('Je pense que je ne parviendrai jamais à parler français comme un natif. Where there’s a will, there’s a way.');
|
||||
```
|
||||
|
||||
Result:
|
||||
|
@ -128,7 +128,7 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe
|
||||
- `--port` — порт для подключения, по умолчанию — 9000. Обратите внимание: для HTTP-интерфейса и нативного интерфейса используются разные порты.
|
||||
- `--user, -u` — имя пользователя, по умолчанию — ‘default’.
|
||||
- `--password` — пароль, по умолчанию — пустая строка.
|
||||
- `--query, -q` — запрос для выполнения, при использовании в неинтерактивном режиме.
|
||||
- `--query, -q` — запрос для выполнения, при использовании в неинтерактивном режиме. Допускается указание `--query` несколько раз (`--query "SELECT 1;" --query "SELECT 2;"...`).
|
||||
- `--queries-file` - путь к файлу с запросами для выполнения. Необходимо указать только одну из опций: `query` или `queries-file`.
|
||||
- `--database, -d` — выбрать текущую БД. Без указания значение берется из настроек сервера (по умолчанию — БД ‘default’).
|
||||
- `--multiline, -m` — если указано — разрешить многострочные запросы, не отправлять запрос по нажатию Enter.
|
||||
|
@ -116,7 +116,7 @@ $ clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM
|
||||
- `--port` – 连接的端口,默认值:9000。注意HTTP接口以及TCP原生接口使用的是不同端口。
|
||||
- `--user, -u` – 用户名。 默认值:`default`。
|
||||
- `--password` – 密码。 默认值:空字符串。
|
||||
- `--query, -q` – 使用非交互模式查询。
|
||||
- `--query, -q` – 使用非交互模式查询。 允许多次指定 `--query`(`--query "SELECT 1;" --query "SELECT 2;"...`)。
|
||||
- `--database, -d` – 默认当前操作的数据库. 默认值:服务端默认的配置(默认是`default`)。
|
||||
- `--multiline, -m` – 如果指定,允许多行语句查询(Enter仅代表换行,不代表查询语句完结)。
|
||||
- `--multiquery, -n` – 如果指定, 允许处理用`;`号分隔的多个查询,只在非交互模式下生效。
|
||||
|
@ -1189,7 +1189,7 @@ void Client::processOptions(const OptionsDescription & options_description,
|
||||
|
||||
void Client::processConfig()
|
||||
{
|
||||
if (config().has("query") && config().has("queries-file"))
|
||||
if (!queries.empty() && config().has("queries-file"))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time");
|
||||
|
||||
/// Batch mode is enabled if one of the following is true:
|
||||
@ -1200,9 +1200,9 @@ void Client::processConfig()
|
||||
/// - --queries-file command line option is present.
|
||||
/// The value of the option is used as file with query (or of multiple queries) to execute.
|
||||
|
||||
delayed_interactive = config().has("interactive") && (config().has("query") || config().has("queries-file"));
|
||||
delayed_interactive = config().has("interactive") && (!queries.empty() || config().has("queries-file"));
|
||||
if (stdin_is_a_tty
|
||||
&& (delayed_interactive || (!config().has("query") && queries_files.empty())))
|
||||
&& (delayed_interactive || (queries.empty() && queries_files.empty())))
|
||||
{
|
||||
is_interactive = true;
|
||||
}
|
||||
|
@ -319,7 +319,7 @@ static bool checkIfStdinIsRegularFile()
|
||||
|
||||
std::string LocalServer::getInitialCreateTableQuery()
|
||||
{
|
||||
if (!config().has("table-structure") && !config().has("table-file") && !config().has("table-data-format") && (!checkIfStdinIsRegularFile() || !config().has("query")))
|
||||
if (!config().has("table-structure") && !config().has("table-file") && !config().has("table-data-format") && (!checkIfStdinIsRegularFile() || queries.empty()))
|
||||
return {};
|
||||
|
||||
auto table_name = backQuoteIfNeed(config().getString("table-name", "table"));
|
||||
@ -461,7 +461,7 @@ try
|
||||
if (first_time)
|
||||
{
|
||||
|
||||
if (queries_files.empty() && !config().has("query"))
|
||||
if (queries_files.empty() && queries.empty())
|
||||
{
|
||||
std::cerr << "\033[31m" << "ClickHouse compiled in fuzzing mode." << "\033[0m" << std::endl;
|
||||
std::cerr << "\033[31m" << "You have to provide a query with --query or --queries-file option." << "\033[0m" << std::endl;
|
||||
@ -473,7 +473,7 @@ try
|
||||
#else
|
||||
is_interactive = stdin_is_a_tty
|
||||
&& (config().hasOption("interactive")
|
||||
|| (!config().has("query") && !config().has("table-structure") && queries_files.empty() && !config().has("table-file")));
|
||||
|| (queries.empty() && !config().has("table-structure") && queries_files.empty() && !config().has("table-file")));
|
||||
#endif
|
||||
if (!is_interactive)
|
||||
{
|
||||
@ -569,10 +569,10 @@ void LocalServer::updateLoggerLevel(const String & logs_level)
|
||||
|
||||
void LocalServer::processConfig()
|
||||
{
|
||||
if (config().has("query") && config().has("queries-file"))
|
||||
if (!queries.empty() && config().has("queries-file"))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time");
|
||||
|
||||
delayed_interactive = config().has("interactive") && (config().has("query") || config().has("queries-file"));
|
||||
delayed_interactive = config().has("interactive") && (!queries.empty() || config().has("queries-file"));
|
||||
if (is_interactive && !delayed_interactive)
|
||||
{
|
||||
if (config().has("multiquery"))
|
||||
|
@ -1,6 +1,9 @@
|
||||
#include <Backups/BackupUtils.h>
|
||||
#include <Backups/DDLAdjustingForBackupVisitor.h>
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
#include <Databases/DDLRenamingVisitor.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Common/setThreadName.h>
|
||||
|
||||
@ -95,4 +98,26 @@ AccessRightsElements getRequiredAccessToBackup(const ASTBackupQuery::Elements &
|
||||
return required_access;
|
||||
}
|
||||
|
||||
bool compareRestoredTableDef(const IAST & restored_table_create_query, const IAST & create_query_from_backup, const ContextPtr & global_context)
|
||||
{
|
||||
auto adjust_before_comparison = [&](const IAST & query) -> ASTPtr
|
||||
{
|
||||
auto new_query = query.clone();
|
||||
adjustCreateQueryForBackup(new_query, global_context, nullptr);
|
||||
ASTCreateQuery & create = typeid_cast<ASTCreateQuery &>(*new_query);
|
||||
create.setUUID({});
|
||||
create.if_not_exists = false;
|
||||
return new_query;
|
||||
};
|
||||
|
||||
ASTPtr query1 = adjust_before_comparison(restored_table_create_query);
|
||||
ASTPtr query2 = adjust_before_comparison(create_query_from_backup);
|
||||
return serializeAST(*query1) == serializeAST(*query2);
|
||||
}
|
||||
|
||||
bool compareRestoredDatabaseDef(const IAST & restored_database_create_query, const IAST & create_query_from_backup, const ContextPtr & global_context)
|
||||
{
|
||||
return compareRestoredTableDef(restored_database_create_query, create_query_from_backup, global_context);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTBackupQuery.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,8 +13,11 @@ class DDLRenamingMap;
|
||||
/// Initializes a DDLRenamingMap from a BACKUP or RESTORE query.
|
||||
DDLRenamingMap makeRenamingMapFromBackupQuery(const ASTBackupQuery::Elements & elements);
|
||||
|
||||
|
||||
/// Returns access required to execute BACKUP query.
|
||||
AccessRightsElements getRequiredAccessToBackup(const ASTBackupQuery::Elements & elements);
|
||||
|
||||
/// Checks the definition of a restored table - it must correspond to the definition from the backup.
|
||||
bool compareRestoredTableDef(const IAST & restored_table_create_query, const IAST & create_query_from_backup, const ContextPtr & global_context);
|
||||
bool compareRestoredDatabaseDef(const IAST & restored_database_create_query, const IAST & create_query_from_backup, const ContextPtr & global_context);
|
||||
|
||||
}
|
||||
|
@ -81,9 +81,6 @@ namespace
|
||||
|
||||
void visitCreateQuery(ASTCreateQuery & create, const DDLAdjustingForBackupVisitor::Data & data)
|
||||
{
|
||||
create.uuid = UUIDHelpers::Nil;
|
||||
create.to_inner_uuid = UUIDHelpers::Nil;
|
||||
|
||||
if (create.storage)
|
||||
visitStorage(*create.storage, data);
|
||||
}
|
||||
|
@ -7,6 +7,7 @@ namespace DB
|
||||
{
|
||||
class Exception;
|
||||
enum class UserDefinedSQLObjectType;
|
||||
class ASTCreateQuery;
|
||||
|
||||
/// Replicas use this class to coordinate what they're reading from a backup while executing RESTORE ON CLUSTER.
|
||||
/// There are two implementation of this interface: RestoreCoordinationLocal and RestoreCoordinationRemote.
|
||||
@ -40,10 +41,13 @@ public:
|
||||
/// The function returns false if user-defined function at a specified zk path are being already restored by another replica.
|
||||
virtual bool acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) = 0;
|
||||
|
||||
/// Generates a new UUID for a table. The same UUID must be used for a replicated table on each replica,
|
||||
/// (because otherwise the macro "{uuid}" in the ZooKeeper path will not work correctly).
|
||||
virtual void generateUUIDForTable(ASTCreateQuery & create_query) = 0;
|
||||
|
||||
/// This function is used to check if concurrent restores are running
|
||||
/// other than the restore passed to the function
|
||||
virtual bool hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const = 0;
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Backups/RestoreCoordinationLocal.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
@ -51,6 +52,39 @@ bool RestoreCoordinationLocal::acquireReplicatedSQLObjects(const String &, UserD
|
||||
return true;
|
||||
}
|
||||
|
||||
void RestoreCoordinationLocal::generateUUIDForTable(ASTCreateQuery & create_query)
|
||||
{
|
||||
String query_str = serializeAST(create_query);
|
||||
|
||||
auto find_in_map = [&]
|
||||
{
|
||||
auto it = create_query_uuids.find(query_str);
|
||||
if (it != create_query_uuids.end())
|
||||
{
|
||||
create_query.setUUID(it->second);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
};
|
||||
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
if (find_in_map())
|
||||
return;
|
||||
}
|
||||
|
||||
auto new_uuids = create_query.generateRandomUUID(/* always_generate_new_uuid= */ true);
|
||||
|
||||
String new_query_str = serializeAST(create_query);
|
||||
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
if (find_in_map())
|
||||
return;
|
||||
create_query_uuids[query_str] = new_uuids;
|
||||
}
|
||||
}
|
||||
|
||||
bool RestoreCoordinationLocal::hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const
|
||||
{
|
||||
if (num_active_restores > 1)
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Backups/IRestoreCoordination.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <mutex>
|
||||
#include <set>
|
||||
#include <unordered_set>
|
||||
@ -39,6 +40,10 @@ public:
|
||||
/// The function returns false if user-defined function at a specified zk path are being already restored by another replica.
|
||||
bool acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) override;
|
||||
|
||||
/// Generates a new UUID for a table. The same UUID must be used for a replicated table on each replica,
|
||||
/// (because otherwise the macro "{uuid}" in the ZooKeeper path will not work correctly).
|
||||
void generateUUIDForTable(ASTCreateQuery & create_query) override;
|
||||
|
||||
bool hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const override;
|
||||
|
||||
private:
|
||||
@ -46,6 +51,8 @@ private:
|
||||
|
||||
std::set<std::pair<String /* database_zk_path */, String /* table_name */>> acquired_tables_in_replicated_databases;
|
||||
std::unordered_set<String /* table_zk_path */> acquired_data_in_replicated_tables;
|
||||
std::unordered_map<String, ASTCreateQuery::UUIDs> create_query_uuids;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
};
|
||||
|
||||
|
@ -2,6 +2,8 @@
|
||||
#include <Backups/BackupCoordinationStage.h>
|
||||
#include <Backups/RestoreCoordinationRemote.h>
|
||||
#include <Backups/BackupCoordinationStageSync.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Functions/UserDefined/UserDefinedSQLObjectType.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
@ -87,6 +89,7 @@ void RestoreCoordinationRemote::createRootNodes()
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_tables_data_acquired", "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_access_storages_acquired", "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_sql_objects_acquired", "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/table_uuids", "", zkutil::CreateMode::Persistent));
|
||||
zk->tryMulti(ops, responses);
|
||||
});
|
||||
}
|
||||
@ -231,6 +234,33 @@ bool RestoreCoordinationRemote::acquireReplicatedSQLObjects(const String & loade
|
||||
return result;
|
||||
}
|
||||
|
||||
void RestoreCoordinationRemote::generateUUIDForTable(ASTCreateQuery & create_query)
|
||||
{
|
||||
String query_str = serializeAST(create_query);
|
||||
String new_uuids_str = create_query.generateRandomUUID(/* always_generate_new_uuid= */ true).toString();
|
||||
|
||||
auto holder = with_retries.createRetriesControlHolder("generateUUIDForTable");
|
||||
holder.retries_ctl.retryLoop(
|
||||
[&, &zk = holder.faulty_zookeeper]()
|
||||
{
|
||||
with_retries.renewZooKeeper(zk);
|
||||
|
||||
String path = zookeeper_path + "/table_uuids/" + escapeForFileName(query_str);
|
||||
Coordination::Error res = zk->tryCreate(path, new_uuids_str, zkutil::CreateMode::Persistent);
|
||||
|
||||
if (res == Coordination::Error::ZOK)
|
||||
return;
|
||||
|
||||
if (res == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
create_query.setUUID(ASTCreateQuery::UUIDs::fromString(zk->get(path)));
|
||||
return;
|
||||
}
|
||||
|
||||
zkutil::KeeperException::fromPath(res, path);
|
||||
});
|
||||
}
|
||||
|
||||
void RestoreCoordinationRemote::removeAllNodes()
|
||||
{
|
||||
/// Usually this function is called by the initiator when a restore operation is complete so we don't need the coordination anymore.
|
||||
|
@ -46,6 +46,10 @@ public:
|
||||
/// The function returns false if user-defined function at a specified zk path are being already restored by another replica.
|
||||
bool acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) override;
|
||||
|
||||
/// Generates a new UUID for a table. The same UUID must be used for a replicated table on each replica,
|
||||
/// (because otherwise the macro "{uuid}" in the ZooKeeper path will not work correctly).
|
||||
void generateUUIDForTable(ASTCreateQuery & create_query) override;
|
||||
|
||||
bool hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const override;
|
||||
|
||||
private:
|
||||
|
@ -571,12 +571,14 @@ void RestorerFromBackup::createDatabase(const String & database_name) const
|
||||
if (database_info.is_predefined_database)
|
||||
return;
|
||||
|
||||
auto create_database_query = database_info.create_database_query;
|
||||
if (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists)
|
||||
{
|
||||
create_database_query = create_database_query->clone();
|
||||
create_database_query->as<ASTCreateQuery &>().if_not_exists = true;
|
||||
}
|
||||
auto create_database_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(database_info.create_database_query->clone());
|
||||
|
||||
/// Generate a new UUID for a database.
|
||||
/// The generated UUID will be ignored if the database does not support UUIDs.
|
||||
restore_coordination->generateUUIDForTable(*create_database_query);
|
||||
|
||||
/// Add the clause `IF NOT EXISTS` if that is specified in the restore settings.
|
||||
create_database_query->if_not_exists = (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists);
|
||||
|
||||
LOG_TRACE(log, "Creating database {}: {}", backQuoteIfNeed(database_name), serializeAST(*create_database_query));
|
||||
|
||||
@ -605,17 +607,17 @@ void RestorerFromBackup::checkDatabase(const String & database_name)
|
||||
if (!restore_settings.allow_different_database_def && !database_info.is_predefined_database)
|
||||
{
|
||||
/// Check that the database's definition is the same as expected.
|
||||
ASTPtr create_database_query = database->getCreateDatabaseQuery();
|
||||
adjustCreateQueryForBackup(create_database_query, context->getGlobalContext(), nullptr);
|
||||
ASTPtr expected_create_query = database_info.create_database_query;
|
||||
if (serializeAST(*create_database_query) != serializeAST(*expected_create_query))
|
||||
|
||||
ASTPtr existing_database_def = database->getCreateDatabaseQuery();
|
||||
ASTPtr database_def_from_backup = database_info.create_database_query;
|
||||
if (!compareRestoredDatabaseDef(*existing_database_def, *database_def_from_backup, context->getGlobalContext()))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::CANNOT_RESTORE_DATABASE,
|
||||
"The database has a different definition: {} "
|
||||
"comparing to its definition in the backup: {}",
|
||||
serializeAST(*create_database_query),
|
||||
serializeAST(*expected_create_query));
|
||||
serializeAST(*existing_database_def),
|
||||
serializeAST(*database_def_from_backup));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -714,20 +716,23 @@ void RestorerFromBackup::createTable(const QualifiedTableName & table_name)
|
||||
if (table_info.is_predefined_table)
|
||||
return;
|
||||
|
||||
auto create_table_query = table_info.create_table_query;
|
||||
if (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists)
|
||||
{
|
||||
create_table_query = create_table_query->clone();
|
||||
create_table_query->as<ASTCreateQuery &>().if_not_exists = true;
|
||||
}
|
||||
auto create_table_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(table_info.create_table_query->clone());
|
||||
|
||||
/// Generate a new UUID for a table (the same table on different hosts must use the same UUID, `restore_coordination` will make it so).
|
||||
/// The generated UUID will be ignored if the database does not support UUIDs.
|
||||
restore_coordination->generateUUIDForTable(*create_table_query);
|
||||
|
||||
/// Add the clause `IF NOT EXISTS` if that is specified in the restore settings.
|
||||
create_table_query->if_not_exists = (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists);
|
||||
|
||||
LOG_TRACE(
|
||||
log, "Creating {}: {}", tableNameWithTypeToString(table_name.database, table_name.table, false), serializeAST(*create_table_query));
|
||||
|
||||
try
|
||||
{
|
||||
DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_name.database);
|
||||
table_info.database = database;
|
||||
if (!table_info.database)
|
||||
table_info.database = DatabaseCatalog::instance().getDatabase(table_name.database);
|
||||
DatabasePtr database = table_info.database;
|
||||
|
||||
/// Execute CREATE TABLE query (we call IDatabase::createTableRestoredFromBackup() to allow the database to do some
|
||||
/// database-specific things).
|
||||
@ -747,37 +752,33 @@ void RestorerFromBackup::createTable(const QualifiedTableName & table_name)
|
||||
void RestorerFromBackup::checkTable(const QualifiedTableName & table_name)
|
||||
{
|
||||
auto & table_info = table_infos.at(table_name);
|
||||
auto database = table_info.database;
|
||||
|
||||
try
|
||||
{
|
||||
if (!database)
|
||||
{
|
||||
database = DatabaseCatalog::instance().getDatabase(table_name.database);
|
||||
table_info.database = database;
|
||||
}
|
||||
|
||||
auto resolved_id = (table_name.database == DatabaseCatalog::TEMPORARY_DATABASE)
|
||||
? context->resolveStorageID(StorageID{"", table_name.table}, Context::ResolveExternal)
|
||||
: context->resolveStorageID(StorageID{table_name.database, table_name.table}, Context::ResolveGlobal);
|
||||
|
||||
if (!table_info.database)
|
||||
table_info.database = DatabaseCatalog::instance().getDatabase(table_name.database);
|
||||
DatabasePtr database = table_info.database;
|
||||
|
||||
StoragePtr storage = database->getTable(resolved_id.table_name, context);
|
||||
table_info.storage = storage;
|
||||
table_info.table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
|
||||
|
||||
if (!restore_settings.allow_different_table_def && !table_info.is_predefined_table)
|
||||
{
|
||||
ASTPtr create_table_query = database->getCreateTableQuery(resolved_id.table_name, context);
|
||||
adjustCreateQueryForBackup(create_table_query, context->getGlobalContext(), nullptr);
|
||||
ASTPtr expected_create_query = table_info.create_table_query;
|
||||
if (serializeAST(*create_table_query) != serializeAST(*expected_create_query))
|
||||
ASTPtr existing_table_def = database->getCreateTableQuery(resolved_id.table_name, context);
|
||||
ASTPtr table_def_from_backup = table_info.create_table_query;
|
||||
if (!compareRestoredTableDef(*existing_table_def, *table_def_from_backup, context->getGlobalContext()))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::CANNOT_RESTORE_TABLE,
|
||||
"The table has a different definition: {} "
|
||||
"comparing to its definition in the backup: {}",
|
||||
serializeAST(*create_table_query),
|
||||
serializeAST(*expected_create_query));
|
||||
serializeAST(*existing_table_def),
|
||||
serializeAST(*table_def_from_backup));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -441,7 +441,20 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query)
|
||||
if (!block)
|
||||
return;
|
||||
|
||||
processed_rows += block.rows();
|
||||
if (block.rows() == 0 && partial_result_mode == PartialResultMode::Active)
|
||||
{
|
||||
partial_result_mode = PartialResultMode::Inactive;
|
||||
if (is_interactive)
|
||||
{
|
||||
progress_indication.clearProgressOutput(*tty_buf);
|
||||
std::cout << "Full result:" << std::endl;
|
||||
progress_indication.writeProgress(*tty_buf);
|
||||
}
|
||||
}
|
||||
|
||||
if (partial_result_mode == PartialResultMode::Inactive)
|
||||
processed_rows += block.rows();
|
||||
|
||||
/// Even if all blocks are empty, we still need to initialize the output stream to write empty resultset.
|
||||
initOutputFormat(block, parsed_query);
|
||||
|
||||
@ -451,13 +464,20 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query)
|
||||
if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100))
|
||||
return;
|
||||
|
||||
if (!is_interactive && partial_result_mode == PartialResultMode::Active)
|
||||
return;
|
||||
|
||||
/// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker.
|
||||
if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout))
|
||||
progress_indication.clearProgressOutput(*tty_buf);
|
||||
|
||||
try
|
||||
{
|
||||
output_format->write(materializeBlock(block));
|
||||
if (partial_result_mode == PartialResultMode::Active)
|
||||
output_format->writePartialResult(materializeBlock(block));
|
||||
else
|
||||
output_format->write(materializeBlock(block));
|
||||
|
||||
written_first_block = true;
|
||||
}
|
||||
catch (const Exception &)
|
||||
@ -521,6 +541,9 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info)
|
||||
void ClientBase::initOutputFormat(const Block & block, ASTPtr parsed_query)
|
||||
try
|
||||
{
|
||||
if (partial_result_mode == PartialResultMode::NotInit)
|
||||
partial_result_mode = PartialResultMode::Active;
|
||||
|
||||
if (!output_format)
|
||||
{
|
||||
/// Ignore all results when fuzzing as they can be huge.
|
||||
@ -931,6 +954,14 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa
|
||||
|
||||
const auto & settings = global_context->getSettingsRef();
|
||||
const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1;
|
||||
bool has_partial_result_setting = settings.partial_result_update_duration_ms.totalMilliseconds() > 0;
|
||||
|
||||
if (has_partial_result_setting)
|
||||
{
|
||||
partial_result_mode = PartialResultMode::NotInit;
|
||||
if (is_interactive)
|
||||
std::cout << "Partial result:" << std::endl;
|
||||
}
|
||||
|
||||
int retries_left = 10;
|
||||
while (retries_left)
|
||||
@ -1736,6 +1767,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
|
||||
}
|
||||
|
||||
processed_rows = 0;
|
||||
partial_result_mode = PartialResultMode::Inactive;
|
||||
written_first_block = false;
|
||||
progress_indication.resetProgress();
|
||||
profile_events.watch.restart();
|
||||
@ -2495,23 +2527,34 @@ void ClientBase::runNonInteractive()
|
||||
return;
|
||||
}
|
||||
|
||||
String text;
|
||||
if (config().has("query"))
|
||||
if (!queries.empty())
|
||||
{
|
||||
text += config().getRawString("query"); /// Poco configuration should not process substitutions in form of ${...} inside query.
|
||||
for (const auto & query : queries)
|
||||
{
|
||||
if (query_fuzzer_runs)
|
||||
{
|
||||
if (!processWithFuzzing(query))
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!processQueryText(query))
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If 'query' parameter is not set, read a query from stdin.
|
||||
/// The query is read entirely into memory (streaming is disabled).
|
||||
ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||
String text;
|
||||
readStringUntilEOF(text, in);
|
||||
if (query_fuzzer_runs)
|
||||
processWithFuzzing(text);
|
||||
else
|
||||
processQueryText(text);
|
||||
}
|
||||
|
||||
if (query_fuzzer_runs)
|
||||
processWithFuzzing(text);
|
||||
else
|
||||
processQueryText(text);
|
||||
}
|
||||
|
||||
|
||||
@ -2680,8 +2723,8 @@ void ClientBase::init(int argc, char ** argv)
|
||||
stderr_is_a_tty = isatty(STDERR_FILENO);
|
||||
terminal_width = getTerminalWidth();
|
||||
|
||||
Arguments common_arguments{""}; /// 0th argument is ignored.
|
||||
std::vector<Arguments> external_tables_arguments;
|
||||
Arguments common_arguments = {""}; /// 0th argument is ignored.
|
||||
std::vector<Arguments> hosts_and_ports_arguments;
|
||||
|
||||
readArguments(argc, argv, common_arguments, external_tables_arguments, hosts_and_ports_arguments);
|
||||
@ -2699,7 +2742,6 @@ void ClientBase::init(int argc, char ** argv)
|
||||
}
|
||||
|
||||
|
||||
po::variables_map options;
|
||||
OptionsDescription options_description;
|
||||
options_description.main_description.emplace(createOptionsDescription("Main options", terminal_width));
|
||||
|
||||
@ -2711,9 +2753,8 @@ void ClientBase::init(int argc, char ** argv)
|
||||
|
||||
("config-file,C", po::value<std::string>(), "config-file path")
|
||||
|
||||
("query,q", po::value<std::string>(), "query")
|
||||
("queries-file", po::value<std::vector<std::string>>()->multitoken(),
|
||||
"file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)")
|
||||
("query,q", po::value<std::vector<std::string>>()->multitoken(), R"(query; can be specified multiple times (--query "SELECT 1" --query "SELECT 2"...))")
|
||||
("queries-file", po::value<std::vector<std::string>>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)")
|
||||
("multiquery,n", "If specified, multiple queries separated by semicolons can be listed after --query. For convenience, it is also possible to omit --query and pass the queries directly after --multiquery.")
|
||||
("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)")
|
||||
("database,d", po::value<std::string>(), "database")
|
||||
@ -2734,8 +2775,7 @@ void ClientBase::init(int argc, char ** argv)
|
||||
("log-level", po::value<std::string>(), "log level")
|
||||
("server_logs_file", po::value<std::string>(), "put server logs into specified file")
|
||||
|
||||
("suggestion_limit", po::value<int>()->default_value(10000),
|
||||
"Suggestion limit for how many databases, tables and columns to fetch.")
|
||||
("suggestion_limit", po::value<int>()->default_value(10000), "Suggestion limit for how many databases, tables and columns to fetch.")
|
||||
|
||||
("format,f", po::value<std::string>(), "default output format")
|
||||
("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command")
|
||||
@ -2773,6 +2813,7 @@ void ClientBase::init(int argc, char ** argv)
|
||||
std::transform(external_options.begin(), external_options.end(), std::back_inserter(cmd_options), getter);
|
||||
}
|
||||
|
||||
po::variables_map options;
|
||||
parseAndCheckOptions(options_description, options, common_arguments);
|
||||
po::notify(options);
|
||||
|
||||
@ -2800,7 +2841,7 @@ void ClientBase::init(int argc, char ** argv)
|
||||
if (options.count("time"))
|
||||
print_time_to_stderr = true;
|
||||
if (options.count("query"))
|
||||
config().setString("query", options["query"].as<std::string>());
|
||||
queries = options["query"].as<std::vector<std::string>>();
|
||||
if (options.count("query_id"))
|
||||
config().setString("query_id", options["query_id"].as<std::string>());
|
||||
if (options.count("database"))
|
||||
|
@ -202,6 +202,7 @@ protected:
|
||||
std::optional<Suggest> suggest;
|
||||
bool load_suggestions = false;
|
||||
|
||||
std::vector<String> queries; /// Queries passed via '--query'
|
||||
std::vector<String> queries_files; /// If not empty, queries will be read from these files
|
||||
std::vector<String> interleave_queries_files; /// If not empty, run queries from these files before processing every file from 'queries_files'.
|
||||
std::vector<String> cmd_options;
|
||||
@ -271,6 +272,21 @@ protected:
|
||||
size_t processed_rows = 0; /// How many rows have been read or written.
|
||||
bool print_num_processed_rows = false; /// Whether to print the number of processed rows at
|
||||
|
||||
enum class PartialResultMode: UInt8
|
||||
{
|
||||
/// Query doesn't show partial result before the first block with 0 rows.
|
||||
/// The first block with 0 rows initializes the output table format using its header.
|
||||
NotInit,
|
||||
|
||||
/// Query shows partial result after the first and before the second block with 0 rows.
|
||||
/// The second block with 0 rows indicates that that receiving blocks with partial result has been completed and next blocks will be with the full result.
|
||||
Active,
|
||||
|
||||
/// Query doesn't show partial result at all.
|
||||
Inactive,
|
||||
};
|
||||
PartialResultMode partial_result_mode = PartialResultMode::Inactive;
|
||||
|
||||
bool print_stack_trace = false;
|
||||
/// The last exception that was received from the server. Is used for the
|
||||
/// return code in batch mode.
|
||||
|
@ -309,6 +309,9 @@ class IColumn;
|
||||
\
|
||||
M(Bool, partial_result_on_first_cancel, false, "Allows query to return a partial result after cancel.", 0) \
|
||||
\
|
||||
M(Milliseconds, partial_result_update_duration_ms, 0, "Interval (in milliseconds) for sending updates with partial data about the result table to the client (in interactive mode) during query execution. Setting to 0 disables partial results. Only supported for single-threaded GROUP BY without key, ORDER BY, LIMIT and OFFSET.", 0) \
|
||||
M(UInt64, max_rows_in_partial_result, 10, "Maximum rows to show in the partial result after every real-time update while the query runs (use partial result limit + OFFSET as a value in case of OFFSET in the query).", 0) \
|
||||
\
|
||||
M(Bool, ignore_on_cluster_for_replicated_udf_queries, false, "Ignore ON CLUSTER clause for replicated UDF management queries.", 0) \
|
||||
M(Bool, ignore_on_cluster_for_replicated_access_entities_queries, false, "Ignore ON CLUSTER clause for replicated access entities management queries.", 0) \
|
||||
/** Settings for testing hedged requests */ \
|
||||
|
@ -58,6 +58,7 @@
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <Common/IPv6ToBinary.h>
|
||||
#include "DataTypes/IDataType.h"
|
||||
#include <Core/Types.h>
|
||||
|
||||
|
||||
@ -884,75 +885,179 @@ struct ConvertImpl<FromDataType, DataTypeString, Name, ConvertDefaultBehaviorTag
|
||||
|
||||
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/)
|
||||
{
|
||||
ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column);
|
||||
|
||||
const auto & col_with_type_and_name = columnGetNested(arguments[0]);
|
||||
const auto & type = static_cast<const FromDataType &>(*col_with_type_and_name.type);
|
||||
|
||||
const DateLUTImpl * time_zone = nullptr;
|
||||
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDate> || std::is_same_v<FromDataType, DataTypeDate32>)
|
||||
time_zone = &DateLUT::instance();
|
||||
/// For argument of Date or DateTime type, second argument with time zone could be specified.
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime> || std::is_same_v<FromDataType, DataTypeDateTime64>)
|
||||
if constexpr (IsDataTypeDateOrDateTime<FromDataType>)
|
||||
{
|
||||
auto non_null_args = createBlockWithNestedColumns(arguments);
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0);
|
||||
}
|
||||
auto datetime_arg = arguments[0];
|
||||
|
||||
if (const auto col_from = checkAndGetColumn<ColVecType>(col_with_type_and_name.column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
const DateLUTImpl * time_zone = nullptr;
|
||||
const ColumnConst * time_zone_column = nullptr;
|
||||
|
||||
const typename ColVecType::Container & vec_from = col_from->getData();
|
||||
ColumnString::Chars & data_to = col_to->getChars();
|
||||
ColumnString::Offsets & offsets_to = col_to->getOffsets();
|
||||
size_t size = vec_from.size();
|
||||
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDate>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDate32>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + col_from->getScale() + 1));
|
||||
else
|
||||
data_to.resize(size * 3); /// Arbitrary
|
||||
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> write_buffer(data_to);
|
||||
|
||||
if (null_map)
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
auto non_null_args = createBlockWithNestedColumns(arguments);
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0);
|
||||
}
|
||||
else /// When we have a column for timezone
|
||||
{
|
||||
datetime_arg.column = datetime_arg.column->convertToFullColumnIfConst();
|
||||
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDate> || std::is_same_v<FromDataType, DataTypeDate32>)
|
||||
time_zone = &DateLUT::instance();
|
||||
/// For argument of Date or DateTime type, second argument with time zone could be specified.
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime> || std::is_same_v<FromDataType, DataTypeDateTime64>)
|
||||
{
|
||||
bool is_ok = FormatImpl<FromDataType>::template execute<bool>(vec_from[i], write_buffer, &type, time_zone);
|
||||
null_map->getData()[i] |= !is_ok;
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
if ((time_zone_column = checkAndGetColumnConst<ColumnString>(arguments[1].column.get())))
|
||||
{
|
||||
auto non_null_args = createBlockWithNestedColumns(arguments);
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
const auto & col_with_type_and_name = columnGetNested(datetime_arg);
|
||||
|
||||
if (const auto col_from = checkAndGetColumn<ColVecType>(col_with_type_and_name.column.get()))
|
||||
{
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
const typename ColVecType::Container & vec_from = col_from->getData();
|
||||
ColumnString::Chars & data_to = col_to->getChars();
|
||||
ColumnString::Offsets & offsets_to = col_to->getOffsets();
|
||||
size_t size = vec_from.size();
|
||||
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDate>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDate32>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + col_from->getScale() + 1));
|
||||
else
|
||||
data_to.resize(size * 3); /// Arbitrary
|
||||
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> write_buffer(data_to);
|
||||
const auto & type = static_cast<const FromDataType &>(*col_with_type_and_name.type);
|
||||
|
||||
ColumnUInt8::MutablePtr null_map = copyNullMap(datetime_arg.column);
|
||||
|
||||
if (null_map)
|
||||
{
|
||||
FormatImpl<FromDataType>::template execute<void>(vec_from[i], write_buffer, &type, time_zone);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (!time_zone_column && arguments.size() > 1)
|
||||
{
|
||||
if (!arguments[1].column.get()->getDataAt(i).toString().empty())
|
||||
time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString());
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty");
|
||||
}
|
||||
bool is_ok = FormatImpl<FromDataType>::template execute<bool>(vec_from[i], write_buffer, &type, time_zone);
|
||||
null_map->getData()[i] |= !is_ok;
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (!time_zone_column && arguments.size() > 1)
|
||||
{
|
||||
if (!arguments[1].column.get()->getDataAt(i).toString().empty())
|
||||
time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString());
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty");
|
||||
}
|
||||
FormatImpl<FromDataType>::template execute<void>(vec_from[i], write_buffer, &type, time_zone);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
}
|
||||
}
|
||||
|
||||
write_buffer.finalize();
|
||||
|
||||
if (null_map)
|
||||
return ColumnNullable::create(std::move(col_to), std::move(null_map));
|
||||
return col_to;
|
||||
}
|
||||
|
||||
write_buffer.finalize();
|
||||
|
||||
if (null_map)
|
||||
return ColumnNullable::create(std::move(col_to), std::move(null_map));
|
||||
return col_to;
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
|
||||
arguments[0].column->getName(), Name::name);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
|
||||
arguments[0].column->getName(), Name::name);
|
||||
{
|
||||
ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column);
|
||||
|
||||
const auto & col_with_type_and_name = columnGetNested(arguments[0]);
|
||||
const auto & type = static_cast<const FromDataType &>(*col_with_type_and_name.type);
|
||||
|
||||
const DateLUTImpl * time_zone = nullptr;
|
||||
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDate> || std::is_same_v<FromDataType, DataTypeDate32>)
|
||||
time_zone = &DateLUT::instance();
|
||||
/// For argument of Date or DateTime type, second argument with time zone could be specified.
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime> || std::is_same_v<FromDataType, DataTypeDateTime64>)
|
||||
{
|
||||
auto non_null_args = createBlockWithNestedColumns(arguments);
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0);
|
||||
}
|
||||
|
||||
if (const auto col_from = checkAndGetColumn<ColVecType>(col_with_type_and_name.column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
const typename ColVecType::Container & vec_from = col_from->getData();
|
||||
ColumnString::Chars & data_to = col_to->getChars();
|
||||
ColumnString::Offsets & offsets_to = col_to->getOffsets();
|
||||
size_t size = vec_from.size();
|
||||
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDate>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDate32>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + col_from->getScale() + 1));
|
||||
else
|
||||
data_to.resize(size * 3); /// Arbitrary
|
||||
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> write_buffer(data_to);
|
||||
|
||||
if (null_map)
|
||||
{
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
bool is_ok = FormatImpl<FromDataType>::template execute<bool>(vec_from[i], write_buffer, &type, time_zone);
|
||||
null_map->getData()[i] |= !is_ok;
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
FormatImpl<FromDataType>::template execute<void>(vec_from[i], write_buffer, &type, time_zone);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
}
|
||||
}
|
||||
|
||||
write_buffer.finalize();
|
||||
|
||||
if (null_map)
|
||||
return ColumnNullable::create(std::move(col_to), std::move(null_map));
|
||||
return col_to;
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
|
||||
arguments[0].column->getName(), Name::name);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
@ -1867,7 +1972,7 @@ public:
|
||||
// toDateTime64(value, scale : Integer[, timezone: String])
|
||||
|| std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
{
|
||||
optional_args.push_back({"timezone", &isString<IDataType>, &isColumnConst, "const String"});
|
||||
optional_args.push_back({"timezone", &isString<IDataType>, nullptr, "String"});
|
||||
}
|
||||
|
||||
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
|
||||
@ -1931,7 +2036,9 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override
|
||||
{
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeString>)
|
||||
return {};
|
||||
else if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
return {2};
|
||||
return {1};
|
||||
}
|
||||
|
@ -746,7 +746,7 @@ public:
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
@ -855,17 +855,25 @@ public:
|
||||
template <typename DataType>
|
||||
ColumnPtr executeType(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const
|
||||
{
|
||||
auto * times = checkAndGetColumn<typename DataType::ColumnType>(arguments[0].column.get());
|
||||
auto non_const_datetime = arguments[0].column->convertToFullColumnIfConst();
|
||||
auto * times = checkAndGetColumn<typename DataType::ColumnType>(non_const_datetime.get());
|
||||
if (!times)
|
||||
return nullptr;
|
||||
|
||||
const ColumnConst * format_column = checkAndGetColumnConst<ColumnString>(arguments[1].column.get());
|
||||
if (!format_column)
|
||||
String format;
|
||||
if (const auto * format_column = checkAndGetColumnConst<ColumnString>(arguments[1].column.get()))
|
||||
format = format_column->getValue<String>();
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of second ('format') argument of function {}. Must be constant string.",
|
||||
arguments[1].column->getName(), getName());
|
||||
|
||||
String format = format_column->getValue<String>();
|
||||
const ColumnConst * const_time_zone_column = nullptr;
|
||||
const DateLUTImpl * time_zone = nullptr;
|
||||
if (arguments.size() == 2)
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(arguments, 2, 0);
|
||||
else if (arguments.size() > 2)
|
||||
const_time_zone_column = checkAndGetColumnConst<ColumnString>(arguments[2].column.get());
|
||||
|
||||
UInt32 scale [[maybe_unused]] = 0;
|
||||
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
|
||||
@ -893,15 +901,19 @@ public:
|
||||
String out_template;
|
||||
size_t out_template_size = parseFormat(format, instructions, scale, mysql_with_only_fixed_length_formatters, out_template);
|
||||
|
||||
const DateLUTImpl * time_zone_tmp = nullptr;
|
||||
if (castType(arguments[0].type.get(), [&]([[maybe_unused]] const auto & type) { return true; }))
|
||||
time_zone_tmp = &extractTimeZoneFromFunctionArguments(arguments, 2, 0);
|
||||
{
|
||||
if (const_time_zone_column)
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(arguments, 2, 0);
|
||||
}
|
||||
else if (std::is_same_v<DataType, DataTypeDateTime64> || std::is_same_v<DataType, DataTypeDateTime>)
|
||||
time_zone_tmp = &extractTimeZoneFromFunctionArguments(arguments, 2, 0);
|
||||
{
|
||||
if (const_time_zone_column)
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(arguments, 2, 0);
|
||||
}
|
||||
else
|
||||
time_zone_tmp = &DateLUT::instance();
|
||||
time_zone = &DateLUT::instance();
|
||||
|
||||
const DateLUTImpl & time_zone = *time_zone_tmp;
|
||||
const auto & vec = times->getData();
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
@ -941,6 +953,13 @@ public:
|
||||
auto * pos = begin;
|
||||
for (size_t i = 0; i < vec.size(); ++i)
|
||||
{
|
||||
if (!const_time_zone_column && arguments.size() > 2)
|
||||
{
|
||||
if (!arguments[2].column.get()->getDataAt(i).toString().empty())
|
||||
time_zone = &DateLUT::instance(arguments[2].column.get()->getDataAt(i).toString());
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty");
|
||||
}
|
||||
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
|
||||
{
|
||||
auto c = DecimalUtils::split(vec[i], scale);
|
||||
@ -954,12 +973,14 @@ public:
|
||||
}
|
||||
|
||||
for (auto & instruction : instructions)
|
||||
instruction.perform(pos, static_cast<Int64>(c.whole), c.fractional, scale, time_zone);
|
||||
{
|
||||
instruction.perform(pos, static_cast<Int64>(c.whole), c.fractional, scale, *time_zone);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto & instruction : instructions)
|
||||
instruction.perform(pos, static_cast<UInt32>(vec[i]), 0, 0, time_zone);
|
||||
instruction.perform(pos, static_cast<UInt32>(vec[i]), 0, 0, *time_zone);
|
||||
}
|
||||
*pos++ = '\0';
|
||||
|
||||
|
@ -238,8 +238,15 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf)
|
||||
++num_copied_chars;
|
||||
}
|
||||
|
||||
auto res = fast_float::from_chars(tmp_buf, tmp_buf + num_copied_chars, x);
|
||||
|
||||
fast_float::from_chars_result res;
|
||||
if constexpr (std::endian::native == std::endian::little)
|
||||
res = fast_float::from_chars(tmp_buf, tmp_buf + num_copied_chars, x);
|
||||
else
|
||||
{
|
||||
Float64 x64 = 0.0;
|
||||
res = fast_float::from_chars(tmp_buf, tmp_buf + num_copied_chars, x64);
|
||||
x = static_cast<T>(x64);
|
||||
}
|
||||
if (unlikely(res.ec != std::errc()))
|
||||
{
|
||||
if constexpr (throw_exception)
|
||||
|
@ -2272,6 +2272,29 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va
|
||||
return block;
|
||||
}
|
||||
|
||||
Block Aggregator::prepareBlockAndFillWithoutKeySnapshot(AggregatedDataVariants & data_variants) const
|
||||
{
|
||||
size_t rows = 1;
|
||||
bool final = true;
|
||||
|
||||
auto && out_cols
|
||||
= prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), data_variants.aggregates_pools, final, rows);
|
||||
auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols;
|
||||
|
||||
AggregatedDataWithoutKey & data = data_variants.without_key;
|
||||
|
||||
/// Always single-thread. It's safe to pass current arena from 'aggregates_pool'.
|
||||
for (size_t insert_i = 0; insert_i < params.aggregates_size; ++insert_i)
|
||||
aggregate_functions[insert_i]->insertResultInto(
|
||||
data + offsets_of_aggregate_states[insert_i],
|
||||
*final_aggregate_columns[insert_i],
|
||||
data_variants.aggregates_pool);
|
||||
|
||||
Block block = finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows);
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
template <bool return_single_block>
|
||||
Aggregator::ConvertToBlockRes<return_single_block>
|
||||
Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const
|
||||
|
@ -1210,6 +1210,7 @@ private:
|
||||
friend class ConvertingAggregatedToChunksSource;
|
||||
friend class ConvertingAggregatedToChunksWithMergingSource;
|
||||
friend class AggregatingInOrderTransform;
|
||||
friend class AggregatingPartialResultTransform;
|
||||
|
||||
/// Data structure of source blocks.
|
||||
Block header;
|
||||
@ -1391,6 +1392,7 @@ private:
|
||||
std::atomic<bool> * is_cancelled = nullptr) const;
|
||||
|
||||
Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const;
|
||||
Block prepareBlockAndFillWithoutKeySnapshot(AggregatedDataVariants & data_variants) const;
|
||||
BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const;
|
||||
|
||||
template <bool return_single_block>
|
||||
|
@ -982,7 +982,7 @@ void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t
|
||||
|
||||
auto file_cache = FileCacheFactory::instance().getByName(disk_ptr->getCacheName()).cache;
|
||||
if (!file_cache)
|
||||
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Cache '{}' is not found", file_cache->getBasePath());
|
||||
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Cache '{}' is not found", disk_ptr->getCacheName());
|
||||
|
||||
LOG_DEBUG(shared->log, "Using file cache ({}) for temporary files", file_cache->getBasePath());
|
||||
|
||||
|
@ -219,10 +219,12 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
else
|
||||
{
|
||||
bool is_on_cluster = getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
if (create.uuid != UUIDHelpers::Nil && !is_on_cluster)
|
||||
if (create.uuid != UUIDHelpers::Nil && !is_on_cluster && !internal)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "Ordinary database engine does not support UUID");
|
||||
|
||||
/// Ignore UUID if it's ON CLUSTER query
|
||||
/// The database doesn't support UUID so we'll ignore it. The UUID could be set here because of either
|
||||
/// a) the initiator of `ON CLUSTER` query generated it to ensure the same UUIDs are used on different hosts; or
|
||||
/// b) `RESTORE from backup` query generated it to ensure the same UUIDs are used on different hosts.
|
||||
create.uuid = UUIDHelpers::Nil;
|
||||
metadata_path = metadata_path / "metadata" / database_name_escaped;
|
||||
}
|
||||
@ -983,19 +985,6 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
|
||||
setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value);
|
||||
}
|
||||
|
||||
static void generateUUIDForTable(ASTCreateQuery & create)
|
||||
{
|
||||
if (create.uuid == UUIDHelpers::Nil)
|
||||
create.uuid = UUIDHelpers::generateV4();
|
||||
|
||||
/// If destination table (to_table_id) is not specified for materialized view,
|
||||
/// then MV will create inner table. We should generate UUID of inner table here,
|
||||
/// so it will be the same on all hosts if query in ON CLUSTER or database engine is Replicated.
|
||||
bool need_uuid_for_inner_table = !create.attach && create.is_materialized_view && !create.to_table_id;
|
||||
if (need_uuid_for_inner_table && create.to_inner_uuid == UUIDHelpers::Nil)
|
||||
create.to_inner_uuid = UUIDHelpers::generateV4();
|
||||
}
|
||||
|
||||
void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const
|
||||
{
|
||||
const auto * kind = create.is_dictionary ? "Dictionary" : "Table";
|
||||
@ -1028,17 +1017,26 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
|
||||
kind_upper, create.table);
|
||||
}
|
||||
|
||||
generateUUIDForTable(create);
|
||||
create.generateRandomUUID();
|
||||
}
|
||||
else
|
||||
{
|
||||
bool is_on_cluster = getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
bool has_uuid = create.uuid != UUIDHelpers::Nil || create.to_inner_uuid != UUIDHelpers::Nil;
|
||||
if (has_uuid && !is_on_cluster)
|
||||
if (has_uuid && !is_on_cluster && !internal)
|
||||
{
|
||||
/// We don't show the following error message either
|
||||
/// 1) if it's a secondary query (an initiator of a CREATE TABLE ON CLUSTER query
|
||||
/// doesn't know the exact database engines on replicas and generates an UUID, and then the replicas are free to ignore that UUID); or
|
||||
/// 2) if it's an internal query (for example RESTORE uses internal queries to create tables and it generates an UUID
|
||||
/// before creating a table to be possibly ignored if the database engine doesn't need it).
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"{} UUID specified, but engine of database {} is not Atomic", kind, create.getDatabase());
|
||||
}
|
||||
|
||||
/// Ignore UUID if it's ON CLUSTER query
|
||||
/// The database doesn't support UUID so we'll ignore it. The UUID could be set here because of either
|
||||
/// a) the initiator of `ON CLUSTER` query generated it to ensure the same UUIDs are used on different hosts; or
|
||||
/// b) `RESTORE from backup` query generated it to ensure the same UUIDs are used on different hosts.
|
||||
create.uuid = UUIDHelpers::Nil;
|
||||
create.to_inner_uuid = UUIDHelpers::Nil;
|
||||
}
|
||||
@ -1619,7 +1617,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, Cont
|
||||
|
||||
/// For CREATE query generate UUID on initiator, so it will be the same on all hosts.
|
||||
/// It will be ignored if database does not support UUIDs.
|
||||
generateUUIDForTable(create);
|
||||
create.generateRandomUUID();
|
||||
|
||||
/// For cross-replication cluster we cannot use UUID in replica path.
|
||||
String cluster_name_expanded = local_context->getMacros()->expand(cluster_name);
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -460,4 +462,49 @@ bool ASTCreateQuery::isParameterizedView() const
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
ASTCreateQuery::UUIDs::UUIDs(const ASTCreateQuery & query)
|
||||
: uuid(query.uuid)
|
||||
, to_inner_uuid(query.to_inner_uuid)
|
||||
{
|
||||
}
|
||||
|
||||
String ASTCreateQuery::UUIDs::toString() const
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
out << "{" << uuid << "," << to_inner_uuid << "}";
|
||||
return out.str();
|
||||
}
|
||||
|
||||
ASTCreateQuery::UUIDs ASTCreateQuery::UUIDs::fromString(const String & str)
|
||||
{
|
||||
ReadBufferFromString in{str};
|
||||
ASTCreateQuery::UUIDs res;
|
||||
in >> "{" >> res.uuid >> "," >> res.to_inner_uuid >> "}";
|
||||
return res;
|
||||
}
|
||||
|
||||
ASTCreateQuery::UUIDs ASTCreateQuery::generateRandomUUID(bool always_generate_new_uuid)
|
||||
{
|
||||
if (always_generate_new_uuid)
|
||||
setUUID({});
|
||||
|
||||
if (uuid == UUIDHelpers::Nil)
|
||||
uuid = UUIDHelpers::generateV4();
|
||||
|
||||
/// If destination table (to_table_id) is not specified for materialized view,
|
||||
/// then MV will create inner table. We should generate UUID of inner table here.
|
||||
bool need_uuid_for_inner_table = !attach && is_materialized_view && !to_table_id;
|
||||
if (need_uuid_for_inner_table && (to_inner_uuid == UUIDHelpers::Nil))
|
||||
to_inner_uuid = UUIDHelpers::generateV4();
|
||||
|
||||
return UUIDs{*this};
|
||||
}
|
||||
|
||||
void ASTCreateQuery::setUUID(const UUIDs & uuids)
|
||||
{
|
||||
uuid = uuids.uuid;
|
||||
to_inner_uuid = uuids.to_inner_uuid;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -146,6 +146,18 @@ public:
|
||||
|
||||
QueryKind getQueryKind() const override { return QueryKind::Create; }
|
||||
|
||||
struct UUIDs
|
||||
{
|
||||
UUID uuid = UUIDHelpers::Nil;
|
||||
UUID to_inner_uuid = UUIDHelpers::Nil;
|
||||
UUIDs() = default;
|
||||
explicit UUIDs(const ASTCreateQuery & query);
|
||||
String toString() const;
|
||||
static UUIDs fromString(const String & str);
|
||||
};
|
||||
UUIDs generateRandomUUID(bool always_generate_new_uuid = false);
|
||||
void setUUID(const UUIDs & uuids);
|
||||
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
|
||||
|
@ -14,7 +14,8 @@ namespace ErrorCodes
|
||||
extern const int POSITION_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns_)), num_rows(num_rows_)
|
||||
Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_)
|
||||
: columns(std::move(columns_)), num_rows(num_rows_)
|
||||
{
|
||||
checkNumRowsIsConsistent();
|
||||
}
|
||||
|
@ -75,7 +75,7 @@ void CompletedPipelineExecutor::execute()
|
||||
if (interactive_timeout_ms)
|
||||
{
|
||||
data = std::make_unique<Data>();
|
||||
data->executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element);
|
||||
data->executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms);
|
||||
data->executor->setReadProgressCallback(pipeline.getReadProgressCallback());
|
||||
|
||||
/// Avoid passing this to lambda, copy ptr to data instead.
|
||||
@ -105,7 +105,7 @@ void CompletedPipelineExecutor::execute()
|
||||
}
|
||||
else
|
||||
{
|
||||
PipelineExecutor executor(pipeline.processors, pipeline.process_list_element);
|
||||
PipelineExecutor executor(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms);
|
||||
executor.setReadProgressCallback(pipeline.getReadProgressCallback());
|
||||
executor.execute(pipeline.getNumThreads(), pipeline.getConcurrencyControl());
|
||||
}
|
||||
|
@ -260,7 +260,6 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue
|
||||
{
|
||||
pid = updated_processors.top();
|
||||
updated_processors.pop();
|
||||
|
||||
/// In this method we have ownership on node.
|
||||
auto & node = *nodes[pid];
|
||||
|
||||
|
@ -30,6 +30,12 @@ private:
|
||||
/// Callback for read progress.
|
||||
ReadProgressCallback * read_progress_callback = nullptr;
|
||||
|
||||
/// Timer that stops optimization of running local tasks instead of queuing them.
|
||||
/// It provides local progress for each IProcessor task, allowing the partial result of the request to be always sended to the user.
|
||||
Stopwatch watch;
|
||||
/// Time period that limits the maximum allowed duration for optimizing the scheduling of local tasks within the executor
|
||||
const UInt64 partial_result_duration_ms;
|
||||
|
||||
public:
|
||||
#ifndef NDEBUG
|
||||
/// Time for different processing stages.
|
||||
@ -62,8 +68,13 @@ public:
|
||||
void setException(std::exception_ptr exception_) { exception = exception_; }
|
||||
void rethrowExceptionIfHas();
|
||||
|
||||
explicit ExecutionThreadContext(size_t thread_number_, bool profile_processors_, bool trace_processors_, ReadProgressCallback * callback)
|
||||
bool needWatchRestartForPartialResultProgress() { return partial_result_duration_ms != 0 && partial_result_duration_ms < watch.elapsedMilliseconds(); }
|
||||
void restartWatch() { watch.restart(); }
|
||||
|
||||
explicit ExecutionThreadContext(size_t thread_number_, bool profile_processors_, bool trace_processors_, ReadProgressCallback * callback, UInt64 partial_result_duration_ms_)
|
||||
: read_progress_callback(callback)
|
||||
, watch(CLOCK_MONOTONIC)
|
||||
, partial_result_duration_ms(partial_result_duration_ms_)
|
||||
, thread_number(thread_number_)
|
||||
, profile_processors(profile_processors_)
|
||||
, trace_processors(trace_processors_)
|
||||
|
@ -108,8 +108,15 @@ void ExecutorTasks::pushTasks(Queue & queue, Queue & async_queue, ExecutionThrea
|
||||
{
|
||||
context.setTask(nullptr);
|
||||
|
||||
/// Take local task from queue if has one.
|
||||
if (!queue.empty() && !context.hasAsyncTasks())
|
||||
/// If sending partial results is allowed and local tasks scheduling optimization is repeated longer than the limit
|
||||
/// or new task need to send partial result later, skip optimization for this iteration.
|
||||
/// Otherwise take local task from queue if has one.
|
||||
if ((!queue.empty() && queue.front()->processor->isPartialResultProcessor())
|
||||
|| context.needWatchRestartForPartialResultProgress())
|
||||
{
|
||||
context.restartWatch();
|
||||
}
|
||||
else if (!queue.empty() && !context.hasAsyncTasks())
|
||||
{
|
||||
context.setTask(queue.front());
|
||||
queue.pop();
|
||||
@ -139,7 +146,7 @@ void ExecutorTasks::pushTasks(Queue & queue, Queue & async_queue, ExecutionThrea
|
||||
}
|
||||
}
|
||||
|
||||
void ExecutorTasks::init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback)
|
||||
void ExecutorTasks::init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback, UInt64 partial_result_duration_ms)
|
||||
{
|
||||
num_threads = num_threads_;
|
||||
use_threads = use_threads_;
|
||||
@ -151,7 +158,7 @@ void ExecutorTasks::init(size_t num_threads_, size_t use_threads_, bool profile_
|
||||
|
||||
executor_contexts.reserve(num_threads);
|
||||
for (size_t i = 0; i < num_threads; ++i)
|
||||
executor_contexts.emplace_back(std::make_unique<ExecutionThreadContext>(i, profile_processors, trace_processors, callback));
|
||||
executor_contexts.emplace_back(std::make_unique<ExecutionThreadContext>(i, profile_processors, trace_processors, callback, partial_result_duration_ms));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -58,7 +58,7 @@ public:
|
||||
void tryGetTask(ExecutionThreadContext & context);
|
||||
void pushTasks(Queue & queue, Queue & async_queue, ExecutionThreadContext & context);
|
||||
|
||||
void init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback);
|
||||
void init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback, UInt64 partial_result_duration_ms);
|
||||
void fill(Queue & queue);
|
||||
void upscale(size_t use_threads_);
|
||||
|
||||
|
@ -33,8 +33,9 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
PipelineExecutor::PipelineExecutor(std::shared_ptr<Processors> & processors, QueryStatusPtr elem)
|
||||
PipelineExecutor::PipelineExecutor(std::shared_ptr<Processors> & processors, QueryStatusPtr elem, UInt64 partial_result_duration_ms_)
|
||||
: process_list_element(std::move(elem))
|
||||
, partial_result_duration_ms(partial_result_duration_ms_)
|
||||
{
|
||||
if (process_list_element)
|
||||
{
|
||||
@ -328,7 +329,7 @@ void PipelineExecutor::initializeExecution(size_t num_threads, bool concurrency_
|
||||
Queue queue;
|
||||
graph->initializeExecution(queue);
|
||||
|
||||
tasks.init(num_threads, use_threads, profile_processors, trace_processors, read_progress_callback.get());
|
||||
tasks.init(num_threads, use_threads, profile_processors, trace_processors, read_progress_callback.get(), partial_result_duration_ms);
|
||||
tasks.fill(queue);
|
||||
|
||||
if (num_threads > 1)
|
||||
|
@ -33,7 +33,7 @@ public:
|
||||
/// During pipeline execution new processors can appear. They will be added to existing set.
|
||||
///
|
||||
/// Explicit graph representation is built in constructor. Throws if graph is not correct.
|
||||
explicit PipelineExecutor(std::shared_ptr<Processors> & processors, QueryStatusPtr elem);
|
||||
explicit PipelineExecutor(std::shared_ptr<Processors> & processors, QueryStatusPtr elem, UInt64 partial_result_duration_ms_ = 0);
|
||||
~PipelineExecutor();
|
||||
|
||||
/// Execute pipeline in multiple threads. Must be called once.
|
||||
@ -90,6 +90,9 @@ private:
|
||||
|
||||
ReadProgressCallbackPtr read_progress_callback;
|
||||
|
||||
/// Duration between sending partial result through the pipeline
|
||||
const UInt64 partial_result_duration_ms;
|
||||
|
||||
using Queue = std::queue<ExecutingGraph::Node *>;
|
||||
|
||||
void initializeExecution(size_t num_threads, bool concurrency_control); /// Initialize executor contexts and task_queue.
|
||||
|
@ -41,12 +41,13 @@ struct PullingAsyncPipelineExecutor::Data
|
||||
}
|
||||
};
|
||||
|
||||
PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_)
|
||||
PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_, bool has_partial_result_setting) : pipeline(pipeline_)
|
||||
{
|
||||
if (!pipeline.pulling())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PullingAsyncPipelineExecutor must be pulling");
|
||||
|
||||
lazy_format = std::make_shared<LazyOutputFormat>(pipeline.output->getHeader());
|
||||
lazy_format = std::make_shared<LazyOutputFormat>(pipeline.output->getHeader(), /*is_partial_result_protocol_active*/ has_partial_result_setting);
|
||||
|
||||
pipeline.complete(lazy_format);
|
||||
}
|
||||
|
||||
@ -103,7 +104,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds)
|
||||
if (!data)
|
||||
{
|
||||
data = std::make_unique<Data>();
|
||||
data->executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element);
|
||||
data->executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms);
|
||||
data->executor->setReadProgressCallback(pipeline.getReadProgressCallback());
|
||||
data->lazy_format = lazy_format.get();
|
||||
|
||||
|
@ -21,7 +21,7 @@ struct ProfileInfo;
|
||||
class PullingAsyncPipelineExecutor
|
||||
{
|
||||
public:
|
||||
explicit PullingAsyncPipelineExecutor(QueryPipeline & pipeline_);
|
||||
explicit PullingAsyncPipelineExecutor(QueryPipeline & pipeline_, bool has_partial_result_setting = false);
|
||||
~PullingAsyncPipelineExecutor();
|
||||
|
||||
/// Get structure of returned block or chunk.
|
||||
|
@ -44,7 +44,7 @@ bool PullingPipelineExecutor::pull(Chunk & chunk)
|
||||
{
|
||||
if (!executor)
|
||||
{
|
||||
executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element);
|
||||
executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms);
|
||||
executor->setReadProgressCallback(pipeline.getReadProgressCallback());
|
||||
}
|
||||
|
||||
|
@ -167,7 +167,7 @@ void PushingAsyncPipelineExecutor::start()
|
||||
started = true;
|
||||
|
||||
data = std::make_unique<Data>();
|
||||
data->executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element);
|
||||
data->executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms);
|
||||
data->executor->setReadProgressCallback(pipeline.getReadProgressCallback());
|
||||
data->source = pushing_source.get();
|
||||
|
||||
|
@ -87,7 +87,7 @@ void PushingPipelineExecutor::start()
|
||||
return;
|
||||
|
||||
started = true;
|
||||
executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element);
|
||||
executor = std::make_shared<PipelineExecutor>(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms);
|
||||
executor->setReadProgressCallback(pipeline.getReadProgressCallback());
|
||||
|
||||
if (!executor->executeStep(&input_wait_flag))
|
||||
|
@ -1,40 +1,89 @@
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
IOutputFormat::IOutputFormat(const Block & header_, WriteBuffer & out_)
|
||||
: IProcessor({header_, header_, header_}, {}), out(out_)
|
||||
IOutputFormat::IOutputFormat(const Block & header_, WriteBuffer & out_, bool is_partial_result_protocol_active_)
|
||||
: IProcessor({header_, header_, header_, header_}, {})
|
||||
, out(out_)
|
||||
, is_partial_result_protocol_active(is_partial_result_protocol_active_)
|
||||
{
|
||||
}
|
||||
|
||||
void IOutputFormat::setCurrentChunk(InputPort & input, PortKind kind)
|
||||
{
|
||||
current_chunk = input.pull(true);
|
||||
current_block_kind = kind;
|
||||
has_input = true;
|
||||
}
|
||||
|
||||
IOutputFormat::Status IOutputFormat::prepareMainAndPartialResult()
|
||||
{
|
||||
bool need_data = false;
|
||||
for (auto kind : {Main, PartialResult})
|
||||
{
|
||||
auto & input = getPort(kind);
|
||||
|
||||
if (input.isFinished())
|
||||
continue;
|
||||
|
||||
if (kind == PartialResult && main_input_activated)
|
||||
{
|
||||
input.close();
|
||||
continue;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
need_data = true;
|
||||
|
||||
if (!input.hasData())
|
||||
continue;
|
||||
|
||||
setCurrentChunk(input, kind);
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
if (need_data)
|
||||
return Status::NeedData;
|
||||
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
IOutputFormat::Status IOutputFormat::prepareTotalsAndExtremes()
|
||||
{
|
||||
for (auto kind : {Totals, Extremes})
|
||||
{
|
||||
auto & input = getPort(kind);
|
||||
|
||||
if (!input.isConnected() || input.isFinished())
|
||||
continue;
|
||||
|
||||
input.setNeeded();
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
setCurrentChunk(input, kind);
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
IOutputFormat::Status IOutputFormat::prepare()
|
||||
{
|
||||
if (has_input)
|
||||
return Status::Ready;
|
||||
|
||||
for (auto kind : {Main, Totals, Extremes})
|
||||
{
|
||||
auto & input = getPort(kind);
|
||||
auto status = prepareMainAndPartialResult();
|
||||
if (status != Status::Finished)
|
||||
return status;
|
||||
|
||||
if (kind != Main && !input.isConnected())
|
||||
continue;
|
||||
|
||||
if (input.isFinished())
|
||||
continue;
|
||||
|
||||
input.setNeeded();
|
||||
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
current_chunk = input.pull(true);
|
||||
current_block_kind = kind;
|
||||
has_input = true;
|
||||
return Status::Ready;
|
||||
}
|
||||
status = prepareTotalsAndExtremes();
|
||||
if (status != Status::Finished)
|
||||
return status;
|
||||
|
||||
finished = true;
|
||||
|
||||
@ -83,8 +132,18 @@ void IOutputFormat::work()
|
||||
case Main:
|
||||
result_rows += current_chunk.getNumRows();
|
||||
result_bytes += current_chunk.allocatedBytes();
|
||||
if (is_partial_result_protocol_active && !main_input_activated && current_chunk.hasRows())
|
||||
{
|
||||
/// Sending an empty block signals to the client that partial results are terminated,
|
||||
/// and only data from the main pipeline will be forwarded.
|
||||
consume(Chunk(current_chunk.cloneEmptyColumns(), 0));
|
||||
main_input_activated = true;
|
||||
}
|
||||
consume(std::move(current_chunk));
|
||||
break;
|
||||
case PartialResult:
|
||||
consumePartialResult(std::move(current_chunk));
|
||||
break;
|
||||
case Totals:
|
||||
writeSuffixIfNeeded();
|
||||
if (auto totals = prepareTotals(std::move(current_chunk)))
|
||||
@ -119,6 +178,15 @@ void IOutputFormat::write(const Block & block)
|
||||
flush();
|
||||
}
|
||||
|
||||
void IOutputFormat::writePartialResult(const Block & block)
|
||||
{
|
||||
writePrefixIfNeeded();
|
||||
consumePartialResult(Chunk(block.getColumns(), block.rows()));
|
||||
|
||||
if (auto_flush)
|
||||
flush();
|
||||
}
|
||||
|
||||
void IOutputFormat::finalize()
|
||||
{
|
||||
if (finalized)
|
||||
|
@ -23,9 +23,9 @@ class WriteBuffer;
|
||||
class IOutputFormat : public IProcessor
|
||||
{
|
||||
public:
|
||||
enum PortKind { Main = 0, Totals = 1, Extremes = 2 };
|
||||
enum PortKind { Main = 0, Totals = 1, Extremes = 2, PartialResult = 3 };
|
||||
|
||||
IOutputFormat(const Block & header_, WriteBuffer & out_);
|
||||
IOutputFormat(const Block & header_, WriteBuffer & out_, bool is_partial_result_protocol_active_ = false);
|
||||
|
||||
Status prepare() override;
|
||||
void work() override;
|
||||
@ -54,6 +54,7 @@ public:
|
||||
/// TODO: separate formats and processors.
|
||||
|
||||
void write(const Block & block);
|
||||
void writePartialResult(const Block & block);
|
||||
|
||||
void finalize();
|
||||
|
||||
@ -118,6 +119,7 @@ protected:
|
||||
virtual void consume(Chunk) = 0;
|
||||
virtual void consumeTotals(Chunk) {}
|
||||
virtual void consumeExtremes(Chunk) {}
|
||||
virtual void consumePartialResult(Chunk) {}
|
||||
virtual void finalizeImpl() {}
|
||||
virtual void finalizeBuffers() {}
|
||||
virtual void writePrefix() {}
|
||||
@ -166,6 +168,7 @@ protected:
|
||||
|
||||
Chunk current_chunk;
|
||||
PortKind current_block_kind = PortKind::Main;
|
||||
bool main_input_activated = false;
|
||||
bool has_input = false;
|
||||
bool finished = false;
|
||||
bool finalized = false;
|
||||
@ -180,9 +183,15 @@ protected:
|
||||
Statistics statistics;
|
||||
|
||||
private:
|
||||
void setCurrentChunk(InputPort & input, PortKind kind);
|
||||
IOutputFormat::Status prepareMainAndPartialResult();
|
||||
IOutputFormat::Status prepareTotalsAndExtremes();
|
||||
|
||||
size_t rows_read_before = 0;
|
||||
bool are_totals_written = false;
|
||||
|
||||
bool is_partial_result_protocol_active = false;
|
||||
|
||||
/// Counters for consumed chunks. Are used for QueryLog.
|
||||
size_t result_rows = 0;
|
||||
size_t result_bytes = 0;
|
||||
|
@ -134,7 +134,8 @@ void PrettyBlockOutputFormat::write(Chunk chunk, PortKind port_kind)
|
||||
{
|
||||
if (total_rows >= format_settings.pretty.max_rows)
|
||||
{
|
||||
total_rows += chunk.getNumRows();
|
||||
if (port_kind != PortKind::PartialResult)
|
||||
total_rows += chunk.getNumRows();
|
||||
return;
|
||||
}
|
||||
if (mono_block)
|
||||
@ -315,7 +316,8 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
|
||||
}
|
||||
writeString(bottom_separator_s, out);
|
||||
|
||||
total_rows += num_rows;
|
||||
if (port_kind != PortKind::PartialResult)
|
||||
total_rows += num_rows;
|
||||
}
|
||||
|
||||
|
||||
@ -388,6 +390,34 @@ void PrettyBlockOutputFormat::consumeExtremes(Chunk chunk)
|
||||
write(std::move(chunk), PortKind::Extremes);
|
||||
}
|
||||
|
||||
void PrettyBlockOutputFormat::clearLastLines(size_t lines_number)
|
||||
{
|
||||
/// http://en.wikipedia.org/wiki/ANSI_escape_code
|
||||
#define MOVE_TO_PREV_LINE "\033[A"
|
||||
#define CLEAR_TO_END_OF_LINE "\033[K"
|
||||
|
||||
static const char * clear_prev_line = MOVE_TO_PREV_LINE \
|
||||
CLEAR_TO_END_OF_LINE;
|
||||
|
||||
/// Move cursor to the beginning of line
|
||||
writeCString("\r", out);
|
||||
|
||||
for (size_t line = 0; line < lines_number; ++line)
|
||||
{
|
||||
writeCString(clear_prev_line, out);
|
||||
}
|
||||
}
|
||||
|
||||
void PrettyBlockOutputFormat::consumePartialResult(Chunk chunk)
|
||||
{
|
||||
if (prev_partial_block_rows > 0)
|
||||
/// number of rows + header line + footer line
|
||||
clearLastLines(prev_partial_block_rows + 2);
|
||||
|
||||
prev_partial_block_rows = chunk.getNumRows();
|
||||
write(std::move(chunk), PortKind::PartialResult);
|
||||
}
|
||||
|
||||
|
||||
void PrettyBlockOutputFormat::writeMonoChunkIfNeeded()
|
||||
{
|
||||
|
@ -28,7 +28,12 @@ protected:
|
||||
void consumeTotals(Chunk) override;
|
||||
void consumeExtremes(Chunk) override;
|
||||
|
||||
void clearLastLines(size_t lines_number);
|
||||
void consumePartialResult(Chunk) override;
|
||||
|
||||
size_t total_rows = 0;
|
||||
size_t prev_partial_block_rows = 0;
|
||||
|
||||
size_t row_number_width = 7; // "10000. "
|
||||
|
||||
const FormatSettings format_settings;
|
||||
@ -55,6 +60,7 @@ protected:
|
||||
void resetFormatterImpl() override
|
||||
{
|
||||
total_rows = 0;
|
||||
prev_partial_block_rows = 0;
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -194,7 +194,8 @@ void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind po
|
||||
|
||||
writeBottom(max_widths);
|
||||
|
||||
total_rows += num_rows;
|
||||
if (port_kind != PortKind::PartialResult)
|
||||
total_rows += num_rows;
|
||||
}
|
||||
|
||||
|
||||
|
@ -14,8 +14,8 @@ class LazyOutputFormat : public IOutputFormat
|
||||
{
|
||||
|
||||
public:
|
||||
explicit LazyOutputFormat(const Block & header)
|
||||
: IOutputFormat(header, out), queue(2) {}
|
||||
explicit LazyOutputFormat(const Block & header, bool is_partial_result_protocol_active = false)
|
||||
: IOutputFormat(header, out, is_partial_result_protocol_active), queue(2) {}
|
||||
|
||||
String getName() const override { return "LazyOutputFormat"; }
|
||||
|
||||
@ -49,6 +49,7 @@ protected:
|
||||
|
||||
void consumeTotals(Chunk chunk) override { totals = std::move(chunk); }
|
||||
void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); }
|
||||
void consumePartialResult(Chunk chunk) override { consume(std::move(chunk)); }
|
||||
|
||||
private:
|
||||
|
||||
|
@ -40,5 +40,10 @@ std::string IProcessor::statusToName(Status status)
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
ProcessorPtr IProcessor::getPartialResultProcessorPtr(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms)
|
||||
{
|
||||
return current_processor->getPartialResultProcessor(current_processor, partial_result_limit, partial_result_duration_ms);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -164,6 +164,8 @@ public:
|
||||
|
||||
static std::string statusToName(Status status);
|
||||
|
||||
static ProcessorPtr getPartialResultProcessorPtr(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms);
|
||||
|
||||
/** Method 'prepare' is responsible for all cheap ("instantaneous": O(1) of data volume, no wait) calculations.
|
||||
*
|
||||
* It may access input and output ports,
|
||||
@ -235,6 +237,22 @@ public:
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'expandPipeline' is not implemented for {} processor", getName());
|
||||
}
|
||||
|
||||
enum class PartialResultStatus
|
||||
{
|
||||
/// Processor currently doesn't support work with the partial result pipeline.
|
||||
NotSupported,
|
||||
|
||||
/// Processor can be skipped in the partial result pipeline.
|
||||
SkipSupported,
|
||||
|
||||
/// Processor creates a light-weight copy of itself in the partial result pipeline.
|
||||
/// The copy can create snapshots of the original processor or transform small blocks of data in the same way as the original processor
|
||||
FullSupported,
|
||||
};
|
||||
|
||||
virtual bool isPartialResultProcessor() const { return false; }
|
||||
virtual PartialResultStatus getPartialResultProcessorSupportStatus() const { return PartialResultStatus::NotSupported; }
|
||||
|
||||
/// In case if query was cancelled executor will wait till all processors finish their jobs.
|
||||
/// Generally, there is no reason to check this flag. However, it may be reasonable for long operations (e.g. i/o).
|
||||
bool isCancelled() const { return is_cancelled.load(std::memory_order_acquire); }
|
||||
@ -369,6 +387,11 @@ public:
|
||||
protected:
|
||||
virtual void onCancel() {}
|
||||
|
||||
virtual ProcessorPtr getPartialResultProcessor(const ProcessorPtr & /*current_processor*/, UInt64 /*partial_result_limit*/, UInt64 /*partial_result_duration_ms*/)
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getPartialResultProcessor' is not implemented for {} processor", getName());
|
||||
}
|
||||
|
||||
private:
|
||||
/// For:
|
||||
/// - elapsed_us
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Processors/LimitTransform.h>
|
||||
|
||||
#include <Processors/Transforms/LimitPartialResultTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -180,7 +180,6 @@ LimitTransform::Status LimitTransform::preparePair(PortsData & data)
|
||||
return Status::NeedData;
|
||||
|
||||
data.current_chunk = input.pull(true);
|
||||
|
||||
auto rows = data.current_chunk.getNumRows();
|
||||
|
||||
if (rows_before_limit_at_least && !data.input_port_has_counter)
|
||||
@ -367,5 +366,11 @@ bool LimitTransform::sortColumnsEqualAt(const ColumnRawPtrs & current_chunk_sort
|
||||
return true;
|
||||
}
|
||||
|
||||
ProcessorPtr LimitTransform::getPartialResultProcessor(const ProcessorPtr & /*current_processor*/, UInt64 partial_result_limit, UInt64 partial_result_duration_ms)
|
||||
{
|
||||
const auto & header = inputs.front().getHeader();
|
||||
return std::make_shared<LimitPartialResultTransform>(header, partial_result_limit, partial_result_duration_ms, limit, offset);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -55,6 +55,8 @@ private:
|
||||
ColumnRawPtrs extractSortColumns(const Columns & columns) const;
|
||||
bool sortColumnsEqualAt(const ColumnRawPtrs & current_chunk_sort_columns, UInt64 current_chunk_row_num) const;
|
||||
|
||||
ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override;
|
||||
|
||||
public:
|
||||
LimitTransform(
|
||||
const Block & header_, UInt64 limit_, UInt64 offset_, size_t num_streams = 1,
|
||||
@ -73,6 +75,8 @@ public:
|
||||
|
||||
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); }
|
||||
void setInputPortHasCounter(size_t pos) { ports_data[pos].input_port_has_counter = true; }
|
||||
|
||||
PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::FullSupported; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,7 +9,12 @@ namespace DB
|
||||
BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from)
|
||||
{
|
||||
BuildQueryPipelineSettings settings;
|
||||
settings.actions_settings = ExpressionActionsSettings::fromSettings(from->getSettingsRef(), CompileExpressions::yes);
|
||||
|
||||
const auto & context_settings = from->getSettingsRef();
|
||||
settings.partial_result_limit = context_settings.max_rows_in_partial_result;
|
||||
settings.partial_result_duration_ms = context_settings.partial_result_update_duration_ms.totalMilliseconds();
|
||||
|
||||
settings.actions_settings = ExpressionActionsSettings::fromSettings(context_settings, CompileExpressions::yes);
|
||||
settings.process_list_element = from->getProcessListElement();
|
||||
settings.progress_callback = from->getProgressCallback();
|
||||
return settings;
|
||||
|
@ -19,6 +19,9 @@ struct BuildQueryPipelineSettings
|
||||
QueryStatusPtr process_list_element;
|
||||
ProgressCallback progress_callback = nullptr;
|
||||
|
||||
UInt64 partial_result_limit = 0;
|
||||
UInt64 partial_result_duration_ms = 0;
|
||||
|
||||
const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; }
|
||||
static BuildQueryPipelineSettings fromContext(ContextPtr from);
|
||||
};
|
||||
|
@ -168,6 +168,8 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline(
|
||||
|
||||
QueryPipelineBuilderPtr last_pipeline;
|
||||
|
||||
bool has_partial_result_setting = build_pipeline_settings.partial_result_duration_ms > 0;
|
||||
|
||||
std::stack<Frame> stack;
|
||||
stack.push(Frame{.node = root});
|
||||
|
||||
@ -194,6 +196,9 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline(
|
||||
}
|
||||
else
|
||||
stack.push(Frame{.node = frame.node->children[next_child]});
|
||||
|
||||
if (has_partial_result_setting && last_pipeline && !last_pipeline->isPartialResultActive())
|
||||
last_pipeline->activatePartialResult(build_pipeline_settings.partial_result_limit, build_pipeline_settings.partial_result_duration_ms);
|
||||
}
|
||||
|
||||
last_pipeline->setProgressCallback(build_pipeline_settings.progress_callback);
|
||||
|
@ -23,12 +23,12 @@
|
||||
#include <Processors/Transforms/ReverseTransform.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <Storages/MergeTree/MergeTreeInOrderSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreePrefetchedReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolInOrder.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h>
|
||||
#include <Storages/MergeTree/MergeTreeSource.h>
|
||||
#include <Storages/MergeTree/MergeTreeThreadSelectProcessor.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
@ -251,7 +251,7 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
Poco::Logger * log_,
|
||||
MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_,
|
||||
bool enable_parallel_reading)
|
||||
: SourceStepWithFilter(DataStream{.header = IMergeTreeSelectAlgorithm::transformHeader(
|
||||
: SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader(
|
||||
storage_snapshot_->getSampleBlockForColumns(real_column_names_),
|
||||
getPrewhereInfoFromQueryInfo(query_info_),
|
||||
data_.getPartitionValueType(),
|
||||
@ -268,10 +268,11 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
, storage_snapshot(std::move(storage_snapshot_))
|
||||
, metadata_for_reading(storage_snapshot->getMetadataForQuery())
|
||||
, context(std::move(context_))
|
||||
, max_block_size(max_block_size_)
|
||||
, block_size{
|
||||
.max_block_size_rows = max_block_size_,
|
||||
.preferred_block_size_bytes = context->getSettingsRef().preferred_block_size_bytes,
|
||||
.preferred_max_column_in_block_size_bytes = context->getSettingsRef().preferred_max_column_in_block_size_bytes}
|
||||
, requested_num_streams(num_streams_)
|
||||
, preferred_block_size_bytes(context->getSettingsRef().preferred_block_size_bytes)
|
||||
, preferred_max_column_in_block_size_bytes(context->getSettingsRef().preferred_max_column_in_block_size_bytes)
|
||||
, sample_factor_column_queried(sample_factor_column_queried_)
|
||||
, max_block_numbers_to_read(std::move(max_block_numbers_to_read_))
|
||||
, log(log_)
|
||||
@ -281,7 +282,7 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
if (sample_factor_column_queried)
|
||||
{
|
||||
/// Only _sample_factor virtual column is added by ReadFromMergeTree
|
||||
/// Other virtual columns are added by MergeTreeBaseSelectProcessor.
|
||||
/// Other virtual columns are added by MergeTreeSelectProcessor.
|
||||
auto type = std::make_shared<DataTypeFloat64>();
|
||||
output_stream->header.insert({type->createColumn(), type, "_sample_factor"});
|
||||
}
|
||||
@ -325,50 +326,50 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
|
||||
RangesInDataParts parts_with_range,
|
||||
Names required_columns,
|
||||
size_t max_streams,
|
||||
size_t min_marks_for_concurrent_read,
|
||||
bool use_uncompressed_cache
|
||||
)
|
||||
PoolSettings pool_settings)
|
||||
{
|
||||
const auto & client_info = context->getClientInfo();
|
||||
|
||||
auto extension = ParallelReadingExtension
|
||||
{
|
||||
.all_callback = all_ranges_callback.value(),
|
||||
.callback = read_task_callback.value(),
|
||||
.count_participating_replicas = client_info.count_participating_replicas,
|
||||
.number_of_current_replica = client_info.number_of_current_replica,
|
||||
.columns_to_read = required_columns
|
||||
.columns_to_read = required_columns,
|
||||
};
|
||||
|
||||
/// We have a special logic for local replica. It has to read less data, because in some cases it should
|
||||
/// merge states of aggregate functions or do some other important stuff other than reading from Disk.
|
||||
min_marks_for_concurrent_read = static_cast<size_t>(min_marks_for_concurrent_read * context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier);
|
||||
pool_settings.min_marks_for_concurrent_read = static_cast<size_t>(pool_settings.min_marks_for_concurrent_read * context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier);
|
||||
size_t total_rows = parts_with_range.getRowsCountAllParts();
|
||||
|
||||
auto pool = std::make_shared<MergeTreeReadPoolParallelReplicas>(
|
||||
std::move(extension),
|
||||
std::move(parts_with_range),
|
||||
storage_snapshot,
|
||||
max_streams,
|
||||
extension,
|
||||
parts_with_range,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
min_marks_for_concurrent_read);
|
||||
pool_settings,
|
||||
context);
|
||||
|
||||
auto block_size_copy = block_size;
|
||||
block_size_copy.min_marks_to_read = pool_settings.min_marks_for_concurrent_read;
|
||||
|
||||
Pipes pipes;
|
||||
const auto & settings = context->getSettingsRef();
|
||||
size_t total_rows = parts_with_range.getRowsCountAllParts();
|
||||
|
||||
for (size_t i = 0; i < max_streams; ++i)
|
||||
for (size_t i = 0; i < pool_settings.threads; ++i)
|
||||
{
|
||||
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(
|
||||
i, pool, min_marks_for_concurrent_read, max_block_size,
|
||||
settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes,
|
||||
data, storage_snapshot, use_uncompressed_cache,
|
||||
prewhere_info, actions_settings, reader_settings, virt_column_names);
|
||||
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(i);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(algorithm));
|
||||
auto processor = std::make_unique<MergeTreeSelectProcessor>(
|
||||
pool, std::move(algorithm), data, prewhere_info,
|
||||
actions_settings, block_size_copy, reader_settings, virt_column_names);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(processor));
|
||||
|
||||
/// Set the approximate number of rows for the first source only
|
||||
/// In case of parallel processing on replicas do not set approximate rows at all.
|
||||
@ -387,12 +388,8 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
|
||||
Pipe ReadFromMergeTree::readFromPool(
|
||||
RangesInDataParts parts_with_range,
|
||||
Names required_columns,
|
||||
size_t max_streams,
|
||||
size_t min_marks_for_concurrent_read,
|
||||
bool use_uncompressed_cache)
|
||||
PoolSettings pool_settings)
|
||||
{
|
||||
Pipes pipes;
|
||||
size_t sum_marks = parts_with_range.getMarksCountAllParts();
|
||||
size_t total_rows = parts_with_range.getRowsCountAllParts();
|
||||
|
||||
if (query_info.limit > 0 && query_info.limit < total_rows)
|
||||
@ -403,11 +400,11 @@ Pipe ReadFromMergeTree::readFromPool(
|
||||
/// round min_marks_to_read up to nearest multiple of block_size expressed in marks
|
||||
/// If granularity is adaptive it doesn't make sense
|
||||
/// Maybe it will make sense to add settings `max_block_size_bytes`
|
||||
if (max_block_size && !data.canUseAdaptiveGranularity())
|
||||
if (block_size.max_block_size_rows && !data.canUseAdaptiveGranularity())
|
||||
{
|
||||
size_t fixed_index_granularity = data.getSettings()->index_granularity;
|
||||
min_marks_for_concurrent_read = (min_marks_for_concurrent_read * fixed_index_granularity + max_block_size - 1)
|
||||
/ max_block_size * max_block_size / fixed_index_granularity;
|
||||
pool_settings.min_marks_for_concurrent_read = (pool_settings.min_marks_for_concurrent_read * fixed_index_granularity + block_size.max_block_size_rows - 1)
|
||||
/ block_size.max_block_size_rows * block_size.max_block_size_rows / fixed_index_granularity;
|
||||
}
|
||||
|
||||
bool all_parts_are_remote = true;
|
||||
@ -421,34 +418,30 @@ Pipe ReadFromMergeTree::readFromPool(
|
||||
|
||||
MergeTreeReadPoolPtr pool;
|
||||
|
||||
if ((all_parts_are_remote && settings.allow_prefetched_read_pool_for_remote_filesystem
|
||||
&& MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.remote_fs_method))
|
||||
|| (all_parts_are_local && settings.allow_prefetched_read_pool_for_local_filesystem
|
||||
&& MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.local_fs_method)))
|
||||
bool allow_prefetched_remote = all_parts_are_remote
|
||||
&& settings.allow_prefetched_read_pool_for_remote_filesystem
|
||||
&& MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.remote_fs_method);
|
||||
|
||||
bool allow_prefetched_local = all_parts_are_local
|
||||
&& settings.allow_prefetched_read_pool_for_local_filesystem
|
||||
&& MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.local_fs_method);
|
||||
|
||||
if (allow_prefetched_remote || allow_prefetched_local)
|
||||
{
|
||||
pool = std::make_shared<MergeTreePrefetchedReadPool>(
|
||||
max_streams,
|
||||
sum_marks,
|
||||
min_marks_for_concurrent_read,
|
||||
std::move(parts_with_range),
|
||||
storage_snapshot,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
settings.preferred_block_size_bytes,
|
||||
reader_settings,
|
||||
context,
|
||||
use_uncompressed_cache,
|
||||
all_parts_are_remote,
|
||||
*data.getSettings());
|
||||
pool_settings,
|
||||
context);
|
||||
}
|
||||
else
|
||||
{
|
||||
pool = std::make_shared<MergeTreeReadPool>(
|
||||
max_streams,
|
||||
sum_marks,
|
||||
min_marks_for_concurrent_read,
|
||||
std::move(parts_with_range),
|
||||
storage_snapshot,
|
||||
prewhere_info,
|
||||
@ -456,22 +449,28 @@ Pipe ReadFromMergeTree::readFromPool(
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
context,
|
||||
false);
|
||||
pool_settings,
|
||||
context);
|
||||
}
|
||||
|
||||
auto * logger = &Poco::Logger::get(data.getLogName() + " (SelectExecutor)");
|
||||
LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, max_streams);
|
||||
LOG_DEBUG(log, "Reading approx. {} rows with {} streams", total_rows, pool_settings.threads);
|
||||
|
||||
for (size_t i = 0; i < max_streams; ++i)
|
||||
/// The reason why we change this setting is because MergeTreeReadPool takes the full task
|
||||
/// ignoring min_marks_to_read setting in case of remote disk (see MergeTreeReadPool::getTask).
|
||||
/// In this case, we won't limit the number of rows to read based on adaptive granularity settings.
|
||||
auto block_size_copy = block_size;
|
||||
block_size_copy.min_marks_to_read = pool_settings.min_marks_for_concurrent_read;
|
||||
|
||||
Pipes pipes;
|
||||
for (size_t i = 0; i < pool_settings.threads; ++i)
|
||||
{
|
||||
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(
|
||||
i, pool, min_marks_for_concurrent_read, max_block_size,
|
||||
settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes,
|
||||
data, storage_snapshot, use_uncompressed_cache,
|
||||
prewhere_info, actions_settings, reader_settings, virt_column_names);
|
||||
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(i);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(algorithm));
|
||||
auto processor = std::make_unique<MergeTreeSelectProcessor>(
|
||||
pool, std::move(algorithm), data, prewhere_info,
|
||||
actions_settings, block_size_copy, reader_settings, virt_column_names);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(processor));
|
||||
|
||||
if (i == 0)
|
||||
source->addTotalRowsApprox(total_rows);
|
||||
@ -485,17 +484,65 @@ Pipe ReadFromMergeTree::readFromPool(
|
||||
return pipe;
|
||||
}
|
||||
|
||||
template<typename Algorithm>
|
||||
ProcessorPtr ReadFromMergeTree::createSource(
|
||||
const RangesInDataPart & part,
|
||||
const Names & required_columns,
|
||||
bool use_uncompressed_cache,
|
||||
bool has_limit_below_one_block,
|
||||
MergeTreeInOrderReadPoolParallelReplicasPtr pool)
|
||||
Pipe ReadFromMergeTree::readInOrder(
|
||||
RangesInDataParts parts_with_ranges,
|
||||
Names required_columns,
|
||||
PoolSettings pool_settings,
|
||||
ReadType read_type,
|
||||
UInt64 limit)
|
||||
{
|
||||
auto total_rows = part.getRowsCount();
|
||||
if (query_info.limit > 0 && query_info.limit < total_rows)
|
||||
total_rows = query_info.limit;
|
||||
/// For reading in order it makes sense to read only
|
||||
/// one range per task to reduce number of read rows.
|
||||
bool has_limit_below_one_block = read_type != ReadType::Default && limit && limit < block_size.max_block_size_rows;
|
||||
MergeTreeReadPoolPtr pool;
|
||||
|
||||
if (is_parallel_reading_from_replicas)
|
||||
{
|
||||
const auto & client_info = context->getClientInfo();
|
||||
ParallelReadingExtension extension
|
||||
{
|
||||
.all_callback = all_ranges_callback.value(),
|
||||
.callback = read_task_callback.value(),
|
||||
.count_participating_replicas = client_info.count_participating_replicas,
|
||||
.number_of_current_replica = client_info.number_of_current_replica,
|
||||
.columns_to_read = required_columns,
|
||||
};
|
||||
|
||||
pool_settings.min_marks_for_concurrent_read = static_cast<size_t>(
|
||||
pool_settings.min_marks_for_concurrent_read * context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier);
|
||||
|
||||
CoordinationMode mode = read_type == ReadType::InOrder
|
||||
? CoordinationMode::WithOrder
|
||||
: CoordinationMode::ReverseOrder;
|
||||
|
||||
pool = std::make_shared<MergeTreeReadPoolParallelReplicasInOrder>(
|
||||
std::move(extension),
|
||||
mode,
|
||||
parts_with_ranges,
|
||||
storage_snapshot,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
pool_settings,
|
||||
context);
|
||||
}
|
||||
else
|
||||
{
|
||||
pool = std::make_shared<MergeTreeReadPoolInOrder>(
|
||||
has_limit_below_one_block,
|
||||
read_type,
|
||||
parts_with_ranges,
|
||||
storage_snapshot,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
pool_settings,
|
||||
context);
|
||||
}
|
||||
|
||||
/// Actually it means that parallel reading from replicas enabled
|
||||
/// and we have to collaborate with initiator.
|
||||
@ -504,37 +551,34 @@ ProcessorPtr ReadFromMergeTree::createSource(
|
||||
/// because we don't know actual amount of read rows in case when limit is set.
|
||||
bool set_rows_approx = !is_parallel_reading_from_replicas && !reader_settings.read_in_order;
|
||||
|
||||
auto algorithm = std::make_unique<Algorithm>(
|
||||
data, storage_snapshot, part.data_part, part.alter_conversions, max_block_size, preferred_block_size_bytes,
|
||||
preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info,
|
||||
actions_settings, reader_settings, pool, virt_column_names, part.part_index_in_query, has_limit_below_one_block);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(algorithm));
|
||||
|
||||
if (set_rows_approx)
|
||||
source->addTotalRowsApprox(total_rows);
|
||||
|
||||
return source;
|
||||
}
|
||||
|
||||
Pipe ReadFromMergeTree::readInOrder(
|
||||
RangesInDataParts parts_with_range,
|
||||
Names required_columns,
|
||||
ReadType read_type,
|
||||
bool use_uncompressed_cache,
|
||||
UInt64 limit,
|
||||
MergeTreeInOrderReadPoolParallelReplicasPtr pool)
|
||||
{
|
||||
Pipes pipes;
|
||||
/// For reading in order it makes sense to read only
|
||||
/// one range per task to reduce number of read rows.
|
||||
bool has_limit_below_one_block = read_type != ReadType::Default && limit && limit < max_block_size;
|
||||
|
||||
for (const auto & part : parts_with_range)
|
||||
for (size_t i = 0; i < parts_with_ranges.size(); ++i)
|
||||
{
|
||||
auto source = read_type == ReadType::InReverseOrder
|
||||
? createSource<MergeTreeReverseSelectAlgorithm>(part, required_columns, use_uncompressed_cache, has_limit_below_one_block, pool)
|
||||
: createSource<MergeTreeInOrderSelectAlgorithm>(part, required_columns, use_uncompressed_cache, has_limit_below_one_block, pool);
|
||||
const auto & part_with_ranges = parts_with_ranges[i];
|
||||
|
||||
UInt64 total_rows = part_with_ranges.getRowsCount();
|
||||
if (query_info.limit > 0 && query_info.limit < total_rows)
|
||||
total_rows = query_info.limit;
|
||||
|
||||
LOG_TRACE(log, "Reading {} ranges in{}order from part {}, approx. {} rows starting from {}",
|
||||
part_with_ranges.ranges.size(),
|
||||
read_type == ReadType::InReverseOrder ? " reverse " : " ",
|
||||
part_with_ranges.data_part->name, total_rows,
|
||||
part_with_ranges.data_part->index_granularity.getMarkStartingRow(part_with_ranges.ranges.front().begin));
|
||||
|
||||
MergeTreeSelectAlgorithmPtr algorithm;
|
||||
if (read_type == ReadType::InReverseOrder)
|
||||
algorithm = std::make_unique<MergeTreeInReverseOrderSelectAlgorithm>(i);
|
||||
else
|
||||
algorithm = std::make_unique<MergeTreeInOrderSelectAlgorithm>(i);
|
||||
|
||||
auto processor = std::make_unique<MergeTreeSelectProcessor>(
|
||||
pool, std::move(algorithm), data, prewhere_info,
|
||||
actions_settings, block_size, reader_settings, virt_column_names);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(processor));
|
||||
if (set_rows_approx)
|
||||
source->addTotalRowsApprox(total_rows);
|
||||
|
||||
pipes.emplace_back(std::move(source));
|
||||
}
|
||||
@ -553,16 +597,33 @@ Pipe ReadFromMergeTree::readInOrder(
|
||||
}
|
||||
|
||||
Pipe ReadFromMergeTree::read(
|
||||
RangesInDataParts parts_with_range, Names required_columns, ReadType read_type,
|
||||
size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache)
|
||||
RangesInDataParts parts_with_range,
|
||||
Names required_columns,
|
||||
ReadType read_type,
|
||||
size_t max_streams,
|
||||
size_t min_marks_for_concurrent_read,
|
||||
bool use_uncompressed_cache)
|
||||
{
|
||||
const auto & settings = context->getSettingsRef();
|
||||
size_t sum_marks = parts_with_range.getMarksCountAllParts();
|
||||
|
||||
PoolSettings pool_settings
|
||||
{
|
||||
.threads = max_streams,
|
||||
.sum_marks = sum_marks,
|
||||
.min_marks_for_concurrent_read = min_marks_for_concurrent_read,
|
||||
.preferred_block_size_bytes = settings.preferred_block_size_bytes,
|
||||
.use_uncompressed_cache = use_uncompressed_cache,
|
||||
.use_const_size_tasks_for_remote_reading = settings.merge_tree_use_const_size_tasks_for_remote_reading,
|
||||
};
|
||||
|
||||
if (read_type == ReadType::ParallelReplicas)
|
||||
return readFromPoolParallelReplicas(parts_with_range, required_columns, max_streams, min_marks_for_concurrent_read, use_uncompressed_cache);
|
||||
return readFromPoolParallelReplicas(std::move(parts_with_range), std::move(required_columns), std::move(pool_settings));
|
||||
|
||||
if (read_type == ReadType::Default && max_streams > 1)
|
||||
return readFromPool(parts_with_range, required_columns, max_streams, min_marks_for_concurrent_read, use_uncompressed_cache);
|
||||
return readFromPool(std::move(parts_with_range), std::move(required_columns), std::move(pool_settings));
|
||||
|
||||
auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache, /*limit */0, /*pool*/nullptr);
|
||||
auto pipe = readInOrder(parts_with_range, required_columns, pool_settings, read_type, /*limit=*/ 0);
|
||||
|
||||
/// Use ConcatProcessor to concat sources together.
|
||||
/// It is needed to read in parts order (and so in PK order) if single thread is used.
|
||||
@ -585,7 +646,6 @@ struct PartRangesReadInfo
|
||||
size_t index_granularity_bytes = 0;
|
||||
size_t max_marks_to_use_cache = 0;
|
||||
size_t min_marks_for_concurrent_read = 0;
|
||||
|
||||
bool use_uncompressed_cache = false;
|
||||
|
||||
PartRangesReadInfo(
|
||||
@ -663,8 +723,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
|
||||
|
||||
auto read_type = is_parallel_reading_from_replicas ? ReadType::ParallelReplicas : ReadType::Default;
|
||||
|
||||
return read(std::move(parts_with_ranges), column_names, read_type,
|
||||
num_streams, info.min_marks_for_concurrent_read, info.use_uncompressed_cache);
|
||||
return read(std::move(parts_with_ranges),
|
||||
column_names,
|
||||
read_type,
|
||||
num_streams,
|
||||
info.min_marks_for_concurrent_read,
|
||||
info.use_uncompressed_cache);
|
||||
}
|
||||
|
||||
static ActionsDAGPtr createProjection(const Block & header)
|
||||
@ -715,7 +779,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
|
||||
/// Let's split ranges to avoid reading much data.
|
||||
auto split_ranges
|
||||
= [rows_granularity = data_settings->index_granularity, my_max_block_size = max_block_size](const auto & ranges, int direction)
|
||||
= [rows_granularity = data_settings->index_granularity, my_max_block_size = block_size.max_block_size_rows]
|
||||
(const auto & ranges, int direction)
|
||||
{
|
||||
MarkRanges new_ranges;
|
||||
const size_t max_marks_in_range = (my_max_block_size + rows_granularity - 1) / rows_granularity;
|
||||
@ -762,109 +827,94 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
const size_t min_marks_per_stream = (info.sum_marks - 1) / num_streams + 1;
|
||||
bool need_preliminary_merge = (parts_with_ranges.size() > settings.read_in_order_two_level_merge_threshold);
|
||||
|
||||
std::vector<RangesInDataParts> splitted_parts_and_ranges;
|
||||
splitted_parts_and_ranges.reserve(num_streams);
|
||||
const auto read_type = input_order_info->direction == 1 ? ReadType::InOrder : ReadType::InReverseOrder;
|
||||
|
||||
const auto read_type = input_order_info->direction == 1
|
||||
? ReadFromMergeTree::ReadType::InOrder
|
||||
: ReadFromMergeTree::ReadType::InReverseOrder;
|
||||
|
||||
MergeTreeInOrderReadPoolParallelReplicasPtr pool;
|
||||
|
||||
if (is_parallel_reading_from_replicas)
|
||||
PoolSettings pool_settings
|
||||
{
|
||||
const auto & client_info = context->getClientInfo();
|
||||
auto extension = ParallelReadingExtension
|
||||
{
|
||||
.all_callback = all_ranges_callback.value(),
|
||||
.callback = read_task_callback.value(),
|
||||
.count_participating_replicas = client_info.count_participating_replicas,
|
||||
.number_of_current_replica = client_info.number_of_current_replica,
|
||||
.columns_to_read = column_names
|
||||
};
|
||||
|
||||
auto min_marks_for_concurrent_read = info.min_marks_for_concurrent_read;
|
||||
min_marks_for_concurrent_read = static_cast<size_t>(min_marks_for_concurrent_read * settings.parallel_replicas_single_task_marks_count_multiplier);
|
||||
|
||||
pool = std::make_shared<MergeTreeInOrderReadPoolParallelReplicas>(
|
||||
parts_with_ranges,
|
||||
extension,
|
||||
read_type == ReadFromMergeTree::ReadType::InOrder ? CoordinationMode::WithOrder : CoordinationMode::ReverseOrder,
|
||||
min_marks_for_concurrent_read);
|
||||
}
|
||||
|
||||
|
||||
for (size_t i = 0; i < num_streams && !parts_with_ranges.empty(); ++i)
|
||||
{
|
||||
size_t need_marks = min_marks_per_stream;
|
||||
RangesInDataParts new_parts;
|
||||
|
||||
/// Loop over parts.
|
||||
/// We will iteratively take part or some subrange of a part from the back
|
||||
/// and assign a stream to read from it.
|
||||
while (need_marks > 0 && !parts_with_ranges.empty())
|
||||
{
|
||||
RangesInDataPart part = parts_with_ranges.back();
|
||||
parts_with_ranges.pop_back();
|
||||
size_t & marks_in_part = info.sum_marks_in_parts.back();
|
||||
|
||||
/// We will not take too few rows from a part.
|
||||
if (marks_in_part >= info.min_marks_for_concurrent_read && need_marks < info.min_marks_for_concurrent_read)
|
||||
need_marks = info.min_marks_for_concurrent_read;
|
||||
|
||||
/// Do not leave too few rows in the part.
|
||||
if (marks_in_part > need_marks && marks_in_part - need_marks < info.min_marks_for_concurrent_read)
|
||||
need_marks = marks_in_part;
|
||||
|
||||
MarkRanges ranges_to_get_from_part;
|
||||
|
||||
/// We take full part if it contains enough marks or
|
||||
/// if we know limit and part contains less than 'limit' rows.
|
||||
bool take_full_part = marks_in_part <= need_marks || (input_order_info->limit && input_order_info->limit < part.getRowsCount());
|
||||
|
||||
/// We take the whole part if it is small enough.
|
||||
if (take_full_part)
|
||||
{
|
||||
ranges_to_get_from_part = part.ranges;
|
||||
|
||||
need_marks -= marks_in_part;
|
||||
info.sum_marks_in_parts.pop_back();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Loop through ranges in part. Take enough ranges to cover "need_marks".
|
||||
while (need_marks > 0)
|
||||
{
|
||||
if (part.ranges.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected end of ranges while spreading marks among streams");
|
||||
|
||||
MarkRange & range = part.ranges.front();
|
||||
|
||||
const size_t marks_in_range = range.end - range.begin;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks);
|
||||
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range);
|
||||
range.begin += marks_to_get_from_range;
|
||||
marks_in_part -= marks_to_get_from_range;
|
||||
need_marks -= marks_to_get_from_range;
|
||||
if (range.begin == range.end)
|
||||
part.ranges.pop_front();
|
||||
}
|
||||
parts_with_ranges.emplace_back(part);
|
||||
}
|
||||
|
||||
ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction);
|
||||
new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part));
|
||||
}
|
||||
|
||||
splitted_parts_and_ranges.emplace_back(std::move(new_parts));
|
||||
}
|
||||
.min_marks_for_concurrent_read = info.min_marks_for_concurrent_read,
|
||||
.preferred_block_size_bytes = settings.preferred_block_size_bytes,
|
||||
.use_uncompressed_cache = info.use_uncompressed_cache,
|
||||
};
|
||||
|
||||
Pipes pipes;
|
||||
for (auto & item : splitted_parts_and_ranges)
|
||||
/// For parallel replicas the split will be performed on the initiator side.
|
||||
if (is_parallel_reading_from_replicas)
|
||||
{
|
||||
pipes.emplace_back(readInOrder(std::move(item), column_names, read_type,
|
||||
info.use_uncompressed_cache, input_order_info->limit, pool));
|
||||
pipes.emplace_back(readInOrder(std::move(parts_with_ranges), column_names, pool_settings, read_type, input_order_info->limit));
|
||||
}
|
||||
else
|
||||
{
|
||||
std::vector<RangesInDataParts> splitted_parts_and_ranges;
|
||||
splitted_parts_and_ranges.reserve(num_streams);
|
||||
|
||||
for (size_t i = 0; i < num_streams && !parts_with_ranges.empty(); ++i)
|
||||
{
|
||||
size_t need_marks = min_marks_per_stream;
|
||||
RangesInDataParts new_parts;
|
||||
|
||||
/// Loop over parts.
|
||||
/// We will iteratively take part or some subrange of a part from the back
|
||||
/// and assign a stream to read from it.
|
||||
while (need_marks > 0 && !parts_with_ranges.empty())
|
||||
{
|
||||
RangesInDataPart part = parts_with_ranges.back();
|
||||
parts_with_ranges.pop_back();
|
||||
size_t & marks_in_part = info.sum_marks_in_parts.back();
|
||||
|
||||
/// We will not take too few rows from a part.
|
||||
if (marks_in_part >= info.min_marks_for_concurrent_read && need_marks < info.min_marks_for_concurrent_read)
|
||||
need_marks = info.min_marks_for_concurrent_read;
|
||||
|
||||
/// Do not leave too few rows in the part.
|
||||
if (marks_in_part > need_marks && marks_in_part - need_marks < info.min_marks_for_concurrent_read)
|
||||
need_marks = marks_in_part;
|
||||
|
||||
MarkRanges ranges_to_get_from_part;
|
||||
|
||||
/// We take full part if it contains enough marks or
|
||||
/// if we know limit and part contains less than 'limit' rows.
|
||||
bool take_full_part = marks_in_part <= need_marks || (input_order_info->limit && input_order_info->limit < part.getRowsCount());
|
||||
|
||||
/// We take the whole part if it is small enough.
|
||||
if (take_full_part)
|
||||
{
|
||||
ranges_to_get_from_part = part.ranges;
|
||||
|
||||
need_marks -= marks_in_part;
|
||||
info.sum_marks_in_parts.pop_back();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Loop through ranges in part. Take enough ranges to cover "need_marks".
|
||||
while (need_marks > 0)
|
||||
{
|
||||
if (part.ranges.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected end of ranges while spreading marks among streams");
|
||||
|
||||
MarkRange & range = part.ranges.front();
|
||||
|
||||
const size_t marks_in_range = range.end - range.begin;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks);
|
||||
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range);
|
||||
range.begin += marks_to_get_from_range;
|
||||
marks_in_part -= marks_to_get_from_range;
|
||||
need_marks -= marks_to_get_from_range;
|
||||
if (range.begin == range.end)
|
||||
part.ranges.pop_front();
|
||||
}
|
||||
parts_with_ranges.emplace_back(part);
|
||||
}
|
||||
|
||||
ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction);
|
||||
new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part));
|
||||
}
|
||||
|
||||
splitted_parts_and_ranges.emplace_back(std::move(new_parts));
|
||||
}
|
||||
|
||||
for (auto && item : splitted_parts_and_ranges)
|
||||
pipes.emplace_back(readInOrder(std::move(item), column_names, pool_settings, read_type, input_order_info->limit));
|
||||
}
|
||||
|
||||
Block pipe_header;
|
||||
@ -898,7 +948,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
if (pipe.numOutputPorts() > 1)
|
||||
{
|
||||
auto transform = std::make_shared<MergingSortedTransform>(
|
||||
pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
pipe.getHeader(), pipe.numOutputPorts(), sort_description, block_size.max_block_size_rows, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
|
||||
pipe.addTransform(std::move(transform));
|
||||
}
|
||||
@ -931,7 +981,7 @@ static void addMergingFinal(
|
||||
const SortDescription & sort_description,
|
||||
MergeTreeData::MergingParams merging_params,
|
||||
Names partition_key_columns,
|
||||
size_t max_block_size)
|
||||
size_t max_block_size_rows)
|
||||
{
|
||||
const auto & header = pipe.getHeader();
|
||||
size_t num_outputs = pipe.numOutputPorts();
|
||||
@ -944,31 +994,31 @@ static void addMergingFinal(
|
||||
{
|
||||
case MergeTreeData::MergingParams::Ordinary:
|
||||
return std::make_shared<MergingSortedTransform>(header, num_outputs,
|
||||
sort_description, max_block_size, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
return std::make_shared<CollapsingSortedTransform>(header, num_outputs,
|
||||
sort_description, merging_params.sign_column, true, max_block_size, /*max_block_size_bytes=*/0);
|
||||
sort_description, merging_params.sign_column, true, max_block_size_rows, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Summing:
|
||||
return std::make_shared<SummingSortedTransform>(header, num_outputs,
|
||||
sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size, /*max_block_size_bytes=*/0);
|
||||
sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size_rows, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
return std::make_shared<AggregatingSortedTransform>(header, num_outputs,
|
||||
sort_description, max_block_size, /*max_block_size_bytes=*/0);
|
||||
sort_description, max_block_size_rows, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Replacing:
|
||||
return std::make_shared<ReplacingSortedTransform>(header, num_outputs,
|
||||
sort_description, merging_params.is_deleted_column, merging_params.version_column, max_block_size, /*max_block_size_bytes=*/0, /*out_row_sources_buf_*/ nullptr, /*use_average_block_sizes*/ false, /*cleanup*/ !merging_params.is_deleted_column.empty());
|
||||
sort_description, merging_params.is_deleted_column, merging_params.version_column, max_block_size_rows, /*max_block_size_bytes=*/0, /*out_row_sources_buf_*/ nullptr, /*use_average_block_sizes*/ false, /*cleanup*/ !merging_params.is_deleted_column.empty());
|
||||
|
||||
case MergeTreeData::MergingParams::VersionedCollapsing:
|
||||
return std::make_shared<VersionedCollapsingTransform>(header, num_outputs,
|
||||
sort_description, merging_params.sign_column, max_block_size, /*max_block_size_bytes=*/0);
|
||||
sort_description, merging_params.sign_column, max_block_size_rows, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
return std::make_shared<GraphiteRollupSortedTransform>(header, num_outputs,
|
||||
sort_description, max_block_size, /*max_block_size_bytes=*/0, merging_params.graphite_params, now);
|
||||
sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, merging_params.graphite_params, now);
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
@ -1064,11 +1114,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
return this->read(
|
||||
std::move(parts),
|
||||
column_names,
|
||||
ReadFromMergeTree::ReadType::InOrder,
|
||||
ReadType::InOrder,
|
||||
1 /* num_streams */,
|
||||
0 /* min_marks_for_concurrent_read */,
|
||||
info.use_uncompressed_cache);
|
||||
};
|
||||
|
||||
pipes = buildPipesForReadingByPKRanges(
|
||||
metadata_for_reading->getPrimaryKey(),
|
||||
sorting_expr,
|
||||
@ -1080,7 +1131,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
else
|
||||
{
|
||||
pipes.emplace_back(read(
|
||||
std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, num_streams, 0, info.use_uncompressed_cache));
|
||||
std::move(new_parts), column_names, ReadType::InOrder, num_streams, 0, info.use_uncompressed_cache));
|
||||
|
||||
pipes.back().addSimpleTransform([sorting_expr](const Block & header)
|
||||
{ return std::make_shared<ExpressionTransform>(header, sorting_expr); });
|
||||
@ -1121,7 +1172,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
sort_description,
|
||||
data.merging_params,
|
||||
partition_key_columns,
|
||||
max_block_size);
|
||||
block_size.max_block_size_rows);
|
||||
|
||||
partition_pipes.emplace_back(Pipe::unitePipes(std::move(pipes)));
|
||||
}
|
||||
@ -1141,7 +1192,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts)
|
||||
num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size());
|
||||
|
||||
auto pipe = read(std::move(lonely_parts), column_names, ReadFromMergeTree::ReadType::Default,
|
||||
auto pipe = read(std::move(lonely_parts), column_names, ReadType::Default,
|
||||
num_streams_for_lonely_parts, min_marks_for_concurrent_read, info.use_uncompressed_cache);
|
||||
|
||||
/// Drop temporary columns, added by 'sorting_key_expr'
|
||||
@ -1605,11 +1656,13 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info
|
||||
{
|
||||
query_info.prewhere_info = prewhere_info_value;
|
||||
prewhere_info = prewhere_info_value;
|
||||
output_stream = DataStream{.header = IMergeTreeSelectAlgorithm::transformHeader(
|
||||
|
||||
output_stream = DataStream{.header = MergeTreeSelectProcessor::transformHeader(
|
||||
storage_snapshot->getSampleBlockForColumns(real_column_names),
|
||||
prewhere_info_value,
|
||||
data.getPartitionValueType(),
|
||||
virt_column_names)};
|
||||
|
||||
updateSortDescriptionForOutputStream(
|
||||
*output_stream,
|
||||
storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(),
|
||||
|
@ -87,25 +87,7 @@ public:
|
||||
};
|
||||
|
||||
using IndexStats = std::vector<IndexStat>;
|
||||
|
||||
enum class ReadType
|
||||
{
|
||||
/// By default, read will use MergeTreeReadPool and return pipe with num_streams outputs.
|
||||
/// If num_streams == 1, will read without pool, in order specified in parts.
|
||||
Default,
|
||||
/// Read in sorting key order.
|
||||
/// Returned pipe will have the number of ports equals to parts.size().
|
||||
/// Parameter num_streams_ is ignored in this case.
|
||||
/// User should add MergingSorted itself if needed.
|
||||
InOrder,
|
||||
/// The same as InOrder, but in reverse order.
|
||||
/// For every part, read ranges and granules from end to begin. Also add ReverseTransform.
|
||||
InReverseOrder,
|
||||
/// A special type of reading where every replica
|
||||
/// talks to a remote coordinator (which is located on the initiator node)
|
||||
/// and who spreads marks and parts across them.
|
||||
ParallelReplicas,
|
||||
};
|
||||
using ReadType = MergeTreeReadType;
|
||||
|
||||
struct AnalysisResult
|
||||
{
|
||||
@ -113,7 +95,7 @@ public:
|
||||
MergeTreeDataSelectSamplingData sampling;
|
||||
IndexStats index_stats;
|
||||
Names column_names_to_read;
|
||||
ReadFromMergeTree::ReadType read_type = ReadFromMergeTree::ReadType::Default;
|
||||
ReadType read_type = ReadType::Default;
|
||||
UInt64 total_parts = 0;
|
||||
UInt64 parts_before_pk = 0;
|
||||
UInt64 selected_parts = 0;
|
||||
@ -223,7 +205,7 @@ public:
|
||||
|
||||
const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; }
|
||||
const MergeTreeData & getMergeTreeData() const { return data; }
|
||||
size_t getMaxBlockSize() const { return max_block_size; }
|
||||
size_t getMaxBlockSize() const { return block_size.max_block_size_rows; }
|
||||
size_t getNumStreams() const { return requested_num_streams; }
|
||||
bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; }
|
||||
|
||||
@ -271,12 +253,10 @@ private:
|
||||
StorageMetadataPtr metadata_for_reading;
|
||||
|
||||
ContextPtr context;
|
||||
const MergeTreeReadTask::BlockSizeParams block_size;
|
||||
|
||||
const size_t max_block_size;
|
||||
size_t requested_num_streams;
|
||||
size_t output_streams_limit = 0;
|
||||
const size_t preferred_block_size_bytes;
|
||||
const size_t preferred_max_column_in_block_size_bytes;
|
||||
const bool sample_factor_column_queried;
|
||||
|
||||
/// Used for aggregation optimisation (see DB::QueryPlanOptimizations::tryAggregateEachPartitionIndependently).
|
||||
@ -292,16 +272,14 @@ private:
|
||||
UInt64 selected_rows = 0;
|
||||
UInt64 selected_marks = 0;
|
||||
|
||||
using PoolSettings = MergeTreeReadPoolBase::PoolSettings;
|
||||
|
||||
Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache);
|
||||
Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache);
|
||||
Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache);
|
||||
Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache, UInt64 limit, MergeTreeInOrderReadPoolParallelReplicasPtr pool);
|
||||
Pipe readFromPool(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings);
|
||||
Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings);
|
||||
Pipe readInOrder(RangesInDataParts parts_with_ranges, Names required_columns, PoolSettings pool_settings, ReadType read_type, UInt64 limit);
|
||||
|
||||
template<typename TSource>
|
||||
ProcessorPtr createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache, bool has_limit_below_one_block, MergeTreeInOrderReadPoolParallelReplicasPtr pool);
|
||||
|
||||
Pipe spreadMarkRanges(
|
||||
RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection);
|
||||
Pipe spreadMarkRanges(RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection);
|
||||
|
||||
Pipe groupStreamsByPartition(AnalysisResult & result, ActionsDAGPtr & result_projection);
|
||||
|
||||
|
@ -27,6 +27,8 @@ public:
|
||||
size_t max_bytes_before_external_sort = 0;
|
||||
TemporaryDataOnDiskScopePtr tmp_data = nullptr;
|
||||
size_t min_free_disk_space = 0;
|
||||
UInt64 partial_result_limit = 0;
|
||||
UInt64 partial_result_duration_ms = 0;
|
||||
|
||||
explicit Settings(const Context & context);
|
||||
explicit Settings(size_t max_block_size_);
|
||||
|
@ -0,0 +1,47 @@
|
||||
#include <Processors/Transforms/AggregatingPartialResultTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
AggregatingPartialResultTransform::AggregatingPartialResultTransform(
|
||||
const Block & input_header, const Block & output_header, AggregatingTransformPtr aggregating_transform_,
|
||||
UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_)
|
||||
: PartialResultTransform(input_header, output_header, partial_result_limit_, partial_result_duration_ms_)
|
||||
, aggregating_transform(std::move(aggregating_transform_))
|
||||
, transform_aggregator(input_header, aggregating_transform->params->params)
|
||||
{}
|
||||
|
||||
void AggregatingPartialResultTransform::transformPartialResult(Chunk & chunk)
|
||||
{
|
||||
auto & params = aggregating_transform->params->params;
|
||||
|
||||
bool no_more_keys = false;
|
||||
AggregatedDataVariants variants;
|
||||
ColumnRawPtrs key_columns(params.keys_size);
|
||||
Aggregator::AggregateColumns aggregate_columns(params.aggregates_size);
|
||||
|
||||
const UInt64 num_rows = chunk.getNumRows();
|
||||
transform_aggregator.executeOnBlock(chunk.detachColumns(), 0, num_rows, variants, key_columns, aggregate_columns, no_more_keys);
|
||||
|
||||
auto transformed_block = transform_aggregator.convertToBlocks(variants, /*final*/ true, /*max_threads*/ 1).front();
|
||||
|
||||
chunk = convertToChunk(transformed_block);
|
||||
}
|
||||
|
||||
PartialResultTransform::ShaphotResult AggregatingPartialResultTransform::getRealProcessorSnapshot()
|
||||
{
|
||||
std::lock_guard lock(aggregating_transform->snapshot_mutex);
|
||||
if (aggregating_transform->is_generate_initialized)
|
||||
return {{}, SnaphotStatus::Stopped};
|
||||
|
||||
if (aggregating_transform->variants.empty())
|
||||
return {{}, SnaphotStatus::NotReady};
|
||||
|
||||
auto & snapshot_aggregator = aggregating_transform->params->aggregator;
|
||||
auto & snapshot_variants = aggregating_transform->many_data->variants;
|
||||
auto block = snapshot_aggregator.prepareBlockAndFillWithoutKeySnapshot(*snapshot_variants.at(0));
|
||||
|
||||
return {convertToChunk(block), SnaphotStatus::Ready};
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,29 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
#include <Processors/Transforms/PartialResultTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class AggregatingPartialResultTransform : public PartialResultTransform
|
||||
{
|
||||
public:
|
||||
using AggregatingTransformPtr = std::shared_ptr<AggregatingTransform>;
|
||||
|
||||
AggregatingPartialResultTransform(
|
||||
const Block & input_header, const Block & output_header, AggregatingTransformPtr aggregating_transform_,
|
||||
UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_);
|
||||
|
||||
String getName() const override { return "AggregatingPartialResultTransform"; }
|
||||
|
||||
void transformPartialResult(Chunk & chunk) override;
|
||||
ShaphotResult getRealProcessorSnapshot() override;
|
||||
|
||||
private:
|
||||
AggregatingTransformPtr aggregating_transform;
|
||||
Aggregator transform_aggregator;
|
||||
};
|
||||
|
||||
}
|
@ -1,3 +1,4 @@
|
||||
#include <Processors/Transforms/AggregatingPartialResultTransform.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
|
||||
#include <Formats/NativeReader.h>
|
||||
@ -657,6 +658,8 @@ void AggregatingTransform::consume(Chunk chunk)
|
||||
src_rows += num_rows;
|
||||
src_bytes += chunk.bytes();
|
||||
|
||||
std::lock_guard lock(snapshot_mutex);
|
||||
|
||||
if (params->params.only_merge)
|
||||
{
|
||||
auto block = getInputs().front().getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
@ -676,6 +679,7 @@ void AggregatingTransform::initGenerate()
|
||||
if (is_generate_initialized)
|
||||
return;
|
||||
|
||||
std::lock_guard lock(snapshot_mutex);
|
||||
is_generate_initialized = true;
|
||||
|
||||
/// If there was no data, and we aggregate without keys, and we must return single row with the result of empty aggregation.
|
||||
@ -806,4 +810,12 @@ void AggregatingTransform::initGenerate()
|
||||
}
|
||||
}
|
||||
|
||||
ProcessorPtr AggregatingTransform::getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms)
|
||||
{
|
||||
const auto & input_header = inputs.front().getHeader();
|
||||
const auto & output_header = outputs.front().getHeader();
|
||||
auto aggregating_processor = std::dynamic_pointer_cast<AggregatingTransform>(current_processor);
|
||||
return std::make_shared<AggregatingPartialResultTransform>(input_header, output_header, std::move(aggregating_processor), partial_result_limit, partial_result_duration_ms);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -170,9 +170,23 @@ public:
|
||||
void work() override;
|
||||
Processors expandPipeline() override;
|
||||
|
||||
PartialResultStatus getPartialResultProcessorSupportStatus() const override
|
||||
{
|
||||
/// Currently AggregatingPartialResultTransform support only single-thread aggregation without key.
|
||||
|
||||
/// TODO: check that insert results from aggregator.prepareBlockAndFillWithoutKey return values without
|
||||
/// changing of the aggregator state when aggregation with keys will be supported in AggregatingPartialResultTransform.
|
||||
bool is_partial_result_supported = params->params.keys_size == 0 /// Aggregation without key.
|
||||
&& many_data->variants.size() == 1; /// Use only one stream for aggregation.
|
||||
|
||||
return is_partial_result_supported ? PartialResultStatus::FullSupported : PartialResultStatus::NotSupported;
|
||||
}
|
||||
|
||||
protected:
|
||||
void consume(Chunk chunk);
|
||||
|
||||
ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override;
|
||||
|
||||
private:
|
||||
/// To read the data that was flushed into the temporary data file.
|
||||
Processors processors;
|
||||
@ -212,6 +226,13 @@ private:
|
||||
|
||||
bool is_consume_started = false;
|
||||
|
||||
friend class AggregatingPartialResultTransform;
|
||||
/// The mutex protects variables that are used for creating a snapshot of the current processor.
|
||||
/// The current implementation of AggregatingPartialResultTransform uses the 'is_generate_initialized' variable to check
|
||||
/// whether the processor has started sending data through the main pipeline, and the corresponding partial result processor should stop creating snapshots.
|
||||
/// Additionally, the mutex protects the 'params->aggregator' and 'many_data->variants' variables, which are used to get data from them for a snapshot.
|
||||
std::mutex snapshot_mutex;
|
||||
|
||||
void initGenerate();
|
||||
};
|
||||
|
||||
|
@ -25,6 +25,12 @@ void ExpressionTransform::transform(Chunk & chunk)
|
||||
chunk.setColumns(block.getColumns(), num_rows);
|
||||
}
|
||||
|
||||
ProcessorPtr ExpressionTransform::getPartialResultProcessor(const ProcessorPtr & /*current_processor*/, UInt64 /*partial_result_limit*/, UInt64 /*partial_result_duration_ms*/)
|
||||
{
|
||||
const auto & header = getInputPort().getHeader();
|
||||
return std::make_shared<ExpressionTransform>(header, expression);
|
||||
}
|
||||
|
||||
ConvertingTransform::ConvertingTransform(const Block & header_, ExpressionActionsPtr expression_)
|
||||
: ExceptionKeepingTransform(header_, ExpressionTransform::transformHeader(header_, expression_->getActionsDAG()))
|
||||
, expression(std::move(expression_))
|
||||
|
@ -26,10 +26,15 @@ public:
|
||||
|
||||
static Block transformHeader(Block header, const ActionsDAG & expression);
|
||||
|
||||
PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::FullSupported; }
|
||||
|
||||
protected:
|
||||
void transform(Chunk & chunk) override;
|
||||
|
||||
ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override;
|
||||
|
||||
private:
|
||||
|
||||
ExpressionActionsPtr expression;
|
||||
};
|
||||
|
||||
|
42
src/Processors/Transforms/LimitPartialResultTransform.cpp
Normal file
42
src/Processors/Transforms/LimitPartialResultTransform.cpp
Normal file
@ -0,0 +1,42 @@
|
||||
#include <Processors/LimitTransform.h>
|
||||
#include <Processors/Transforms/LimitPartialResultTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
LimitPartialResultTransform::LimitPartialResultTransform(
|
||||
const Block & header,
|
||||
UInt64 partial_result_limit_,
|
||||
UInt64 partial_result_duration_ms_,
|
||||
UInt64 limit_,
|
||||
UInt64 offset_)
|
||||
: PartialResultTransform(header, partial_result_limit_, partial_result_duration_ms_)
|
||||
, limit(limit_)
|
||||
, offset(offset_)
|
||||
{}
|
||||
|
||||
void LimitPartialResultTransform::transformPartialResult(Chunk & chunk)
|
||||
{
|
||||
UInt64 num_rows = chunk.getNumRows();
|
||||
if (num_rows < offset || limit == 0)
|
||||
{
|
||||
chunk = {};
|
||||
return;
|
||||
}
|
||||
|
||||
UInt64 length = std::min(limit, num_rows - offset);
|
||||
|
||||
/// Check if some rows should be removed
|
||||
if (length < num_rows)
|
||||
{
|
||||
UInt64 num_columns = chunk.getNumColumns();
|
||||
auto columns = chunk.detachColumns();
|
||||
|
||||
for (UInt64 i = 0; i < num_columns; ++i)
|
||||
columns[i] = columns[i]->cut(offset, length);
|
||||
|
||||
chunk.setColumns(std::move(columns), length);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
36
src/Processors/Transforms/LimitPartialResultTransform.h
Normal file
36
src/Processors/Transforms/LimitPartialResultTransform.h
Normal file
@ -0,0 +1,36 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Transforms/PartialResultTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class LimitTransform;
|
||||
|
||||
/// Currently support only single thread implementation with one input and one output ports
|
||||
class LimitPartialResultTransform : public PartialResultTransform
|
||||
{
|
||||
public:
|
||||
using LimitTransformPtr = std::shared_ptr<LimitTransform>;
|
||||
|
||||
LimitPartialResultTransform(
|
||||
const Block & header,
|
||||
UInt64 partial_result_limit_,
|
||||
UInt64 partial_result_duration_ms_,
|
||||
UInt64 limit_,
|
||||
UInt64 offset_);
|
||||
|
||||
String getName() const override { return "LimitPartialResultTransform"; }
|
||||
|
||||
void transformPartialResult(Chunk & chunk) override;
|
||||
/// LimitsTransform doesn't have a state which can be snapshoted
|
||||
ShaphotResult getRealProcessorSnapshot() override { return {{}, SnaphotStatus::Stopped}; }
|
||||
|
||||
private:
|
||||
UInt64 limit;
|
||||
UInt64 offset;
|
||||
|
||||
LimitTransformPtr limit_transform;
|
||||
};
|
||||
|
||||
}
|
@ -1,4 +1,5 @@
|
||||
#include <Processors/Transforms/LimitsCheckingTransform.h>
|
||||
#include <Processors/Transforms/PartialResultTransform.h>
|
||||
#include <Access/EnabledQuota.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -33,6 +33,8 @@ public:
|
||||
|
||||
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) { quota = quota_; }
|
||||
|
||||
PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::SkipSupported; }
|
||||
|
||||
protected:
|
||||
void transform(Chunk & chunk) override;
|
||||
|
||||
|
@ -0,0 +1,48 @@
|
||||
#include <Processors/Transforms/MergeSortingPartialResultTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
MergeSortingPartialResultTransform::MergeSortingPartialResultTransform(
|
||||
const Block & header, MergeSortingTransformPtr merge_sorting_transform_,
|
||||
UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_)
|
||||
: PartialResultTransform(header, partial_result_limit_, partial_result_duration_ms_)
|
||||
, merge_sorting_transform(std::move(merge_sorting_transform_))
|
||||
{}
|
||||
|
||||
PartialResultTransform::ShaphotResult MergeSortingPartialResultTransform::getRealProcessorSnapshot()
|
||||
{
|
||||
std::lock_guard lock(merge_sorting_transform->snapshot_mutex);
|
||||
if (merge_sorting_transform->generated_prefix)
|
||||
return {{}, SnaphotStatus::Stopped};
|
||||
|
||||
if (merge_sorting_transform->chunks.empty())
|
||||
return {{}, SnaphotStatus::NotReady};
|
||||
|
||||
/// Sort all input data
|
||||
merge_sorting_transform->remerge();
|
||||
/// Add a copy of the first `partial_result_limit` rows to a generated_chunk
|
||||
/// to send it later as a partial result in the next prepare stage of the current processor
|
||||
auto generated_columns = merge_sorting_transform->chunks[0].cloneEmptyColumns();
|
||||
size_t total_rows = 0;
|
||||
for (const auto & merged_chunk : merge_sorting_transform->chunks)
|
||||
{
|
||||
size_t rows = std::min(merged_chunk.getNumRows(), partial_result_limit - total_rows);
|
||||
if (rows == 0)
|
||||
break;
|
||||
|
||||
for (size_t position = 0; position < generated_columns.size(); ++position)
|
||||
{
|
||||
auto column = merged_chunk.getColumns()[position];
|
||||
generated_columns[position]->insertRangeFrom(*column, 0, rows);
|
||||
}
|
||||
|
||||
total_rows += rows;
|
||||
}
|
||||
|
||||
auto partial_result = Chunk(std::move(generated_columns), total_rows, merge_sorting_transform->chunks[0].getChunkInfo());
|
||||
merge_sorting_transform->enrichChunkWithConstants(partial_result);
|
||||
return {std::move(partial_result), SnaphotStatus::Ready};
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,28 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Transforms/MergeSortingTransform.h>
|
||||
#include <Processors/Transforms/PartialResultTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeSortingPartialResultTransform : public PartialResultTransform
|
||||
{
|
||||
public:
|
||||
using MergeSortingTransformPtr = std::shared_ptr<MergeSortingTransform>;
|
||||
|
||||
MergeSortingPartialResultTransform(
|
||||
const Block & header, MergeSortingTransformPtr merge_sorting_transform_,
|
||||
UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_);
|
||||
|
||||
String getName() const override { return "MergeSortingPartialResultTransform"; }
|
||||
|
||||
/// MergeSortingTransform always receives chunks in a sorted state, so transformation is not needed
|
||||
void transformPartialResult(Chunk & /*chunk*/) override {}
|
||||
ShaphotResult getRealProcessorSnapshot() override;
|
||||
|
||||
private:
|
||||
MergeSortingTransformPtr merge_sorting_transform;
|
||||
};
|
||||
|
||||
}
|
@ -1,4 +1,5 @@
|
||||
#include <Processors/Transforms/MergeSortingTransform.h>
|
||||
#include <Processors/Transforms/MergeSortingPartialResultTransform.h>
|
||||
#include <Processors/IAccumulatingTransform.h>
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
@ -136,6 +137,8 @@ void MergeSortingTransform::consume(Chunk chunk)
|
||||
|
||||
/// If there were only const columns in sort description, then there is no need to sort.
|
||||
/// Return the chunk as is.
|
||||
std::lock_guard lock(snapshot_mutex);
|
||||
|
||||
if (description.empty())
|
||||
{
|
||||
generated_chunk = std::move(chunk);
|
||||
@ -213,6 +216,8 @@ void MergeSortingTransform::serialize()
|
||||
|
||||
void MergeSortingTransform::generate()
|
||||
{
|
||||
std::lock_guard lock(snapshot_mutex);
|
||||
|
||||
if (!generated_prefix)
|
||||
{
|
||||
size_t num_tmp_files = tmp_data ? tmp_data->getStreams().size() : 0;
|
||||
@ -273,4 +278,11 @@ void MergeSortingTransform::remerge()
|
||||
sum_bytes_in_blocks = new_sum_bytes_in_blocks;
|
||||
}
|
||||
|
||||
ProcessorPtr MergeSortingTransform::getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms)
|
||||
{
|
||||
const auto & header = inputs.front().getHeader();
|
||||
auto merge_sorting_processor = std::dynamic_pointer_cast<MergeSortingTransform>(current_processor);
|
||||
return std::make_shared<MergeSortingPartialResultTransform>(header, std::move(merge_sorting_processor), partial_result_limit, partial_result_duration_ms);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -33,6 +33,8 @@ public:
|
||||
|
||||
String getName() const override { return "MergeSortingTransform"; }
|
||||
|
||||
PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::FullSupported; }
|
||||
|
||||
protected:
|
||||
void consume(Chunk chunk) override;
|
||||
void serialize() override;
|
||||
@ -40,6 +42,8 @@ protected:
|
||||
|
||||
Processors expandPipeline() override;
|
||||
|
||||
ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override;
|
||||
|
||||
private:
|
||||
size_t max_bytes_before_remerge;
|
||||
double remerge_lowered_memory_bytes_ratio;
|
||||
@ -59,6 +63,13 @@ private:
|
||||
void remerge();
|
||||
|
||||
ProcessorPtr external_merging_sorted;
|
||||
|
||||
friend class MergeSortingPartialResultTransform;
|
||||
/// The mutex protects variables that are used for creating a snapshot of the current processor.
|
||||
/// The current implementation of MergeSortingPartialResultTransform uses the 'generated_prefix' variable to check
|
||||
/// whether the processor has started sending data through the main pipeline, and the corresponding partial result processor should stop creating snapshots.
|
||||
/// Additionally, the mutex protects the 'chunks' variable and all variables in the 'remerge' function, which is used to transition 'chunks' to a sorted state.
|
||||
std::mutex snapshot_mutex;
|
||||
};
|
||||
|
||||
}
|
||||
|
80
src/Processors/Transforms/PartialResultTransform.cpp
Normal file
80
src/Processors/Transforms/PartialResultTransform.cpp
Normal file
@ -0,0 +1,80 @@
|
||||
#include <Processors/Transforms/PartialResultTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
PartialResultTransform::PartialResultTransform(const Block & header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_)
|
||||
: PartialResultTransform(header, header, partial_result_limit_, partial_result_duration_ms_) {}
|
||||
|
||||
PartialResultTransform::PartialResultTransform(const Block & input_header, const Block & output_header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_)
|
||||
: IProcessor({input_header}, {output_header})
|
||||
, input(inputs.front())
|
||||
, output(outputs.front())
|
||||
, partial_result_limit(partial_result_limit_)
|
||||
, partial_result_duration_ms(partial_result_duration_ms_)
|
||||
, watch(CLOCK_MONOTONIC)
|
||||
{}
|
||||
|
||||
IProcessor::Status PartialResultTransform::prepare()
|
||||
{
|
||||
if (output.isFinished())
|
||||
{
|
||||
input.close();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (finished_getting_snapshots)
|
||||
{
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (!output.canPush())
|
||||
{
|
||||
input.setNotNeeded();
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
/// If input data from previous partial result processor is finished then
|
||||
/// PartialResultTransform ready to create snapshots and send them as a partial result
|
||||
if (input.isFinished())
|
||||
{
|
||||
if (partial_result.snapshot_status == SnaphotStatus::Ready)
|
||||
{
|
||||
partial_result.snapshot_status = SnaphotStatus::NotReady;
|
||||
output.push(std::move(partial_result.chunk));
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
partial_result.chunk = input.pull();
|
||||
transformPartialResult(partial_result.chunk);
|
||||
if (partial_result.chunk.getNumRows() > 0)
|
||||
{
|
||||
output.push(std::move(partial_result.chunk));
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
return Status::NeedData;
|
||||
}
|
||||
|
||||
void PartialResultTransform::work()
|
||||
{
|
||||
if (partial_result_duration_ms < watch.elapsedMilliseconds())
|
||||
{
|
||||
partial_result = getRealProcessorSnapshot();
|
||||
if (partial_result.snapshot_status == SnaphotStatus::Stopped)
|
||||
finished_getting_snapshots = true;
|
||||
|
||||
watch.restart();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
57
src/Processors/Transforms/PartialResultTransform.h
Normal file
57
src/Processors/Transforms/PartialResultTransform.h
Normal file
@ -0,0 +1,57 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/IProcessor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Processors of this type are used to construct an auxiliary pipeline with processors corresponding to those in the main pipeline.
|
||||
/// These processors work in two modes:
|
||||
/// 1) Creating a snapshot of the corresponding processor from the main pipeline once per partial_result_duration_ms (period in milliseconds), and then sending the snapshot through the partial result pipeline.
|
||||
/// 2) Transforming small blocks of data in the same way as the original processor and sending the transformed data through the partial result pipeline.
|
||||
/// All processors of this type rely on the invariant that a new block from the previous processor of the partial result pipeline overwrites information about the previous block of the same previous processor.
|
||||
class PartialResultTransform : public IProcessor
|
||||
{
|
||||
public:
|
||||
PartialResultTransform(const Block & header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_);
|
||||
PartialResultTransform(const Block & input_header, const Block & output_header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_);
|
||||
|
||||
String getName() const override { return "PartialResultTransform"; }
|
||||
|
||||
Status prepare() override;
|
||||
void work() override;
|
||||
|
||||
bool isPartialResultProcessor() const override { return true; }
|
||||
|
||||
protected:
|
||||
enum class SnaphotStatus
|
||||
{
|
||||
NotReady, // Waiting for data from the previous partial result processor or awaiting a timer before creating the snapshot.
|
||||
Ready, // Current partial result processor has received a snapshot from the processor in the main pipeline.
|
||||
Stopped, // The processor from the main pipeline has started sending data, and the pipeline for partial results should use data from the next processors of the main pipeline.
|
||||
};
|
||||
|
||||
struct ShaphotResult
|
||||
{
|
||||
Chunk chunk;
|
||||
SnaphotStatus snapshot_status;
|
||||
};
|
||||
|
||||
InputPort & input;
|
||||
OutputPort & output;
|
||||
|
||||
UInt64 partial_result_limit;
|
||||
UInt64 partial_result_duration_ms;
|
||||
|
||||
ShaphotResult partial_result = {{}, SnaphotStatus::NotReady};
|
||||
|
||||
bool finished_getting_snapshots = false;
|
||||
|
||||
virtual void transformPartialResult(Chunk & /*chunk*/) = 0;
|
||||
virtual ShaphotResult getRealProcessorSnapshot() = 0; // { return {{}, SnaphotStatus::Stopped}; }
|
||||
|
||||
private:
|
||||
Stopwatch watch;
|
||||
};
|
||||
|
||||
}
|
@ -12,6 +12,7 @@
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <QueryPipeline/ReadProgressCallback.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -167,12 +168,9 @@ Pipe::Pipe(ProcessorPtr source)
|
||||
{
|
||||
checkSource(*source);
|
||||
|
||||
if (collected_processors)
|
||||
collected_processors->emplace_back(source);
|
||||
|
||||
output_ports.push_back(&source->getOutputs().front());
|
||||
header = output_ports.front()->getHeader();
|
||||
processors->emplace_back(std::move(source));
|
||||
addProcessor(std::move(source));
|
||||
max_parallel_streams = 1;
|
||||
}
|
||||
|
||||
@ -319,6 +317,16 @@ Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors, bool allow
|
||||
res.processors->insert(res.processors->end(), pipe.processors->begin(), pipe.processors->end());
|
||||
res.output_ports.insert(res.output_ports.end(), pipe.output_ports.begin(), pipe.output_ports.end());
|
||||
|
||||
if (res.isPartialResultActive() && pipe.isPartialResultActive())
|
||||
{
|
||||
res.partial_result_ports.insert(
|
||||
res.partial_result_ports.end(),
|
||||
pipe.partial_result_ports.begin(),
|
||||
pipe.partial_result_ports.end());
|
||||
}
|
||||
else
|
||||
res.dropPartialResult();
|
||||
|
||||
res.max_parallel_streams += pipe.max_parallel_streams;
|
||||
|
||||
if (pipe.totals_port)
|
||||
@ -352,11 +360,11 @@ void Pipe::addSource(ProcessorPtr source)
|
||||
else
|
||||
assertBlocksHaveEqualStructure(header, source_header, "Pipes");
|
||||
|
||||
if (collected_processors)
|
||||
collected_processors->emplace_back(source);
|
||||
|
||||
output_ports.push_back(&source->getOutputs().front());
|
||||
processors->emplace_back(std::move(source));
|
||||
if (isPartialResultActive())
|
||||
partial_result_ports.push_back(nullptr);
|
||||
|
||||
addProcessor(std::move(source));
|
||||
|
||||
max_parallel_streams = std::max<size_t>(max_parallel_streams, output_ports.size());
|
||||
}
|
||||
@ -374,11 +382,9 @@ void Pipe::addTotalsSource(ProcessorPtr source)
|
||||
|
||||
assertBlocksHaveEqualStructure(header, source_header, "Pipes");
|
||||
|
||||
if (collected_processors)
|
||||
collected_processors->emplace_back(source);
|
||||
|
||||
totals_port = &source->getOutputs().front();
|
||||
processors->emplace_back(std::move(source));
|
||||
|
||||
addProcessor(std::move(source));
|
||||
}
|
||||
|
||||
void Pipe::addExtremesSource(ProcessorPtr source)
|
||||
@ -394,11 +400,20 @@ void Pipe::addExtremesSource(ProcessorPtr source)
|
||||
|
||||
assertBlocksHaveEqualStructure(header, source_header, "Pipes");
|
||||
|
||||
if (collected_processors)
|
||||
collected_processors->emplace_back(source);
|
||||
|
||||
extremes_port = &source->getOutputs().front();
|
||||
processors->emplace_back(std::move(source));
|
||||
|
||||
addProcessor(std::move(source));
|
||||
}
|
||||
|
||||
void Pipe::activatePartialResult(UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_)
|
||||
{
|
||||
if (is_partial_result_active)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Partial result for Pipe should be initialized only once");
|
||||
|
||||
is_partial_result_active = true;
|
||||
partial_result_limit = partial_result_limit_;
|
||||
partial_result_duration_ms = partial_result_duration_ms_;
|
||||
partial_result_ports.assign(output_ports.size(), nullptr);
|
||||
}
|
||||
|
||||
static void dropPort(OutputPort *& port, Processors & processors, Processors * collected_processors)
|
||||
@ -426,6 +441,15 @@ void Pipe::dropExtremes()
|
||||
dropPort(extremes_port, *processors, collected_processors);
|
||||
}
|
||||
|
||||
void Pipe::dropPartialResult()
|
||||
{
|
||||
for (auto & port : partial_result_ports)
|
||||
dropPort(port, *processors, collected_processors);
|
||||
|
||||
is_partial_result_active = false;
|
||||
partial_result_ports.clear();
|
||||
}
|
||||
|
||||
void Pipe::addTransform(ProcessorPtr transform)
|
||||
{
|
||||
addTransform(std::move(transform), static_cast<OutputPort *>(nullptr), static_cast<OutputPort *>(nullptr));
|
||||
@ -456,6 +480,8 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort
|
||||
if (extremes)
|
||||
extremes_port = extremes;
|
||||
|
||||
addPartialResultTransform(transform);
|
||||
|
||||
size_t next_output = 0;
|
||||
for (auto & input : inputs)
|
||||
{
|
||||
@ -506,10 +532,7 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort
|
||||
if (extremes_port)
|
||||
assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes");
|
||||
|
||||
if (collected_processors)
|
||||
collected_processors->emplace_back(transform);
|
||||
|
||||
processors->emplace_back(std::move(transform));
|
||||
addProcessor(std::move(transform));
|
||||
|
||||
max_parallel_streams = std::max<size_t>(max_parallel_streams, output_ports.size());
|
||||
}
|
||||
@ -546,6 +569,8 @@ void Pipe::addTransform(ProcessorPtr transform, InputPort * totals, InputPort *
|
||||
extremes_port = nullptr;
|
||||
}
|
||||
|
||||
addPartialResultTransform(transform);
|
||||
|
||||
bool found_totals = false;
|
||||
bool found_extremes = false;
|
||||
|
||||
@ -595,14 +620,104 @@ void Pipe::addTransform(ProcessorPtr transform, InputPort * totals, InputPort *
|
||||
if (extremes_port)
|
||||
assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes");
|
||||
|
||||
if (collected_processors)
|
||||
collected_processors->emplace_back(transform);
|
||||
|
||||
processors->emplace_back(std::move(transform));
|
||||
addProcessor(std::move(transform));
|
||||
|
||||
max_parallel_streams = std::max<size_t>(max_parallel_streams, output_ports.size());
|
||||
}
|
||||
|
||||
void Pipe::addPartialResultSimpleTransform(const ProcessorPtr & transform, size_t partial_result_port_id)
|
||||
{
|
||||
if (isPartialResultActive())
|
||||
{
|
||||
auto & partial_result_port = partial_result_ports[partial_result_port_id];
|
||||
auto partial_result_status = transform->getPartialResultProcessorSupportStatus();
|
||||
|
||||
if (partial_result_status == IProcessor::PartialResultStatus::NotSupported)
|
||||
dropPort(partial_result_port, *processors, collected_processors);
|
||||
|
||||
if (partial_result_status != IProcessor::PartialResultStatus::FullSupported)
|
||||
return;
|
||||
|
||||
auto partial_result_transform = IProcessor::getPartialResultProcessorPtr(transform, partial_result_limit, partial_result_duration_ms);
|
||||
|
||||
connectPartialResultPort(partial_result_port, partial_result_transform->getInputs().front());
|
||||
|
||||
partial_result_port = &partial_result_transform->getOutputs().front();
|
||||
|
||||
addProcessor(std::move(partial_result_transform));
|
||||
}
|
||||
}
|
||||
|
||||
void Pipe::addPartialResultTransform(const ProcessorPtr & transform)
|
||||
{
|
||||
if (isPartialResultActive())
|
||||
{
|
||||
size_t new_outputs_size = transform->getOutputs().size();
|
||||
auto partial_result_status = transform->getPartialResultProcessorSupportStatus();
|
||||
|
||||
if (partial_result_status == IProcessor::PartialResultStatus::SkipSupported && new_outputs_size != partial_result_ports.size())
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cannot skip transform {} in the partial result part of the Pipe because it has {} output ports, but the partial result part expects {} output ports",
|
||||
transform->getName(),
|
||||
new_outputs_size,
|
||||
partial_result_ports.size());
|
||||
|
||||
if (partial_result_status == IProcessor::PartialResultStatus::NotSupported)
|
||||
{
|
||||
for (auto & partial_result_port : partial_result_ports)
|
||||
dropPort(partial_result_port, *processors, collected_processors);
|
||||
|
||||
partial_result_ports.assign(new_outputs_size, nullptr);
|
||||
}
|
||||
|
||||
if (partial_result_status != IProcessor::PartialResultStatus::FullSupported)
|
||||
return;
|
||||
|
||||
auto partial_result_transform = IProcessor::getPartialResultProcessorPtr(transform, partial_result_limit, partial_result_duration_ms);
|
||||
auto & inputs = partial_result_transform->getInputs();
|
||||
|
||||
if (inputs.size() != partial_result_ports.size())
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cannot add partial result transform {} to Pipe because it has {} input ports, but {} expected",
|
||||
partial_result_transform->getName(),
|
||||
inputs.size(),
|
||||
partial_result_ports.size());
|
||||
|
||||
size_t next_port = 0;
|
||||
for (auto & input : inputs)
|
||||
{
|
||||
connectPartialResultPort(partial_result_ports[next_port], input);
|
||||
++next_port;
|
||||
}
|
||||
|
||||
partial_result_ports.assign(new_outputs_size, nullptr);
|
||||
|
||||
next_port = 0;
|
||||
for (auto & new_partial_result_port : partial_result_transform->getOutputs())
|
||||
{
|
||||
partial_result_ports[next_port] = &new_partial_result_port;
|
||||
++next_port;
|
||||
}
|
||||
|
||||
addProcessor(std::move(partial_result_transform));
|
||||
}
|
||||
}
|
||||
|
||||
void Pipe::connectPartialResultPort(OutputPort * partial_result_port, InputPort & partial_result_transform_port)
|
||||
{
|
||||
if (partial_result_port == nullptr)
|
||||
{
|
||||
auto source = std::make_shared<NullSource>(getHeader());
|
||||
partial_result_port = &source->getPort();
|
||||
|
||||
addProcessor(std::move(source));
|
||||
}
|
||||
|
||||
connect(*partial_result_port, partial_result_transform_port);
|
||||
}
|
||||
|
||||
void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter)
|
||||
{
|
||||
if (output_ports.empty())
|
||||
@ -610,7 +725,7 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter)
|
||||
|
||||
Block new_header;
|
||||
|
||||
auto add_transform = [&](OutputPort *& port, StreamType stream_type)
|
||||
auto add_transform = [&](OutputPort *& port, size_t partial_result_port_id, StreamType stream_type)
|
||||
{
|
||||
if (!port)
|
||||
return;
|
||||
@ -646,19 +761,22 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter)
|
||||
{
|
||||
connect(*port, transform->getInputs().front());
|
||||
port = &transform->getOutputs().front();
|
||||
if (stream_type == StreamType::Main)
|
||||
addPartialResultSimpleTransform(transform, partial_result_port_id);
|
||||
|
||||
if (collected_processors)
|
||||
collected_processors->emplace_back(transform);
|
||||
|
||||
processors->emplace_back(std::move(transform));
|
||||
addProcessor(std::move(transform));
|
||||
}
|
||||
};
|
||||
|
||||
size_t partial_result_port_id = 0;
|
||||
for (auto & port : output_ports)
|
||||
add_transform(port, StreamType::Main);
|
||||
{
|
||||
add_transform(port, partial_result_port_id, StreamType::Main);
|
||||
++partial_result_port_id;
|
||||
}
|
||||
|
||||
add_transform(totals_port, StreamType::Totals);
|
||||
add_transform(extremes_port, StreamType::Extremes);
|
||||
add_transform(totals_port, 0, StreamType::Totals);
|
||||
add_transform(extremes_port, 0, StreamType::Extremes);
|
||||
|
||||
header = std::move(new_header);
|
||||
}
|
||||
@ -679,6 +797,7 @@ void Pipe::addChains(std::vector<Chain> chains)
|
||||
|
||||
dropTotals();
|
||||
dropExtremes();
|
||||
dropPartialResult();
|
||||
|
||||
size_t max_parallel_streams_for_chains = 0;
|
||||
|
||||
@ -697,18 +816,21 @@ void Pipe::addChains(std::vector<Chain> chains)
|
||||
|
||||
auto added_processors = Chain::getProcessors(std::move(chains[i]));
|
||||
for (auto & transform : added_processors)
|
||||
{
|
||||
if (collected_processors)
|
||||
collected_processors->emplace_back(transform);
|
||||
|
||||
processors->emplace_back(std::move(transform));
|
||||
}
|
||||
addProcessor(std::move(transform));
|
||||
}
|
||||
|
||||
header = std::move(new_header);
|
||||
max_parallel_streams = std::max(max_parallel_streams, max_parallel_streams_for_chains);
|
||||
}
|
||||
|
||||
void Pipe::addProcessor(ProcessorPtr processor)
|
||||
{
|
||||
if (collected_processors)
|
||||
collected_processors->emplace_back(processor);
|
||||
|
||||
processors->emplace_back(std::move(processor));
|
||||
}
|
||||
|
||||
void Pipe::resize(size_t num_streams, bool force, bool strict)
|
||||
{
|
||||
if (output_ports.empty())
|
||||
@ -769,6 +891,9 @@ void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter)
|
||||
add_transform(totals_port, StreamType::Totals);
|
||||
add_transform(extremes_port, StreamType::Extremes);
|
||||
|
||||
for (auto & port : partial_result_ports)
|
||||
add_transform(port, StreamType::PartialResult);
|
||||
|
||||
output_ports.clear();
|
||||
header.clear();
|
||||
}
|
||||
@ -778,6 +903,9 @@ void Pipe::transform(const Transformer & transformer, bool check_ports)
|
||||
if (output_ports.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot transform empty Pipe");
|
||||
|
||||
/// TODO: Add functionality to work with partial result ports in transformer.
|
||||
dropPartialResult();
|
||||
|
||||
auto new_processors = transformer(output_ports);
|
||||
|
||||
/// Create hash table with new processors.
|
||||
|
@ -48,6 +48,9 @@ public:
|
||||
OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; }
|
||||
OutputPort * getTotalsPort() const { return totals_port; }
|
||||
OutputPort * getExtremesPort() const { return extremes_port; }
|
||||
OutputPort * getPartialResultPort(size_t pos) const { return partial_result_ports.empty() ? nullptr : partial_result_ports[pos]; }
|
||||
|
||||
bool isPartialResultActive() { return is_partial_result_active; }
|
||||
|
||||
/// Add processor to list, add it output ports to output_ports.
|
||||
/// Processor shouldn't have input ports, output ports shouldn't be connected.
|
||||
@ -58,9 +61,13 @@ public:
|
||||
void addTotalsSource(ProcessorPtr source);
|
||||
void addExtremesSource(ProcessorPtr source);
|
||||
|
||||
/// Drop totals and extremes (create NullSink for them).
|
||||
/// Activate sending partial result during main pipeline execution
|
||||
void activatePartialResult(UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_);
|
||||
|
||||
/// Drop totals, extremes and partial result (create NullSink for them).
|
||||
void dropTotals();
|
||||
void dropExtremes();
|
||||
void dropPartialResult();
|
||||
|
||||
/// Add processor to list. It should have size() input ports with compatible header.
|
||||
/// Output ports should have same headers.
|
||||
@ -69,11 +76,16 @@ public:
|
||||
void addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes);
|
||||
void addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes);
|
||||
|
||||
void addPartialResultTransform(const ProcessorPtr & transform);
|
||||
void addPartialResultSimpleTransform(const ProcessorPtr & transform, size_t partial_result_port_id);
|
||||
void connectPartialResultPort(OutputPort * partial_result_port, InputPort & partial_result_transform_port);
|
||||
|
||||
enum class StreamType
|
||||
{
|
||||
Main = 0, /// Stream for query data. There may be several streams of this type.
|
||||
Totals, /// Stream for totals. No more than one.
|
||||
Extremes, /// Stream for extremes. No more than one.
|
||||
PartialResult, /// Stream for partial result data. There may be several streams of this type.
|
||||
};
|
||||
|
||||
using ProcessorGetter = std::function<ProcessorPtr(const Block & header)>;
|
||||
@ -109,10 +121,17 @@ private:
|
||||
Block header;
|
||||
std::shared_ptr<Processors> processors;
|
||||
|
||||
/// Output ports. Totals and extremes are allowed to be empty.
|
||||
/// If the variable is true, then each time a processor is added pipe will try
|
||||
/// to add processor which will send partial result from original processor
|
||||
bool is_partial_result_active = false;
|
||||
UInt64 partial_result_limit = 0;
|
||||
UInt64 partial_result_duration_ms = 0;
|
||||
|
||||
/// Output ports. Totals, extremes and partial results are allowed to be empty.
|
||||
OutputPortRawPtrs output_ports;
|
||||
OutputPort * totals_port = nullptr;
|
||||
OutputPort * extremes_port = nullptr;
|
||||
OutputPortRawPtrs partial_result_ports;
|
||||
|
||||
/// It is the max number of processors which can be executed in parallel for each step.
|
||||
/// Usually, it's the same as the number of output ports.
|
||||
@ -128,6 +147,8 @@ private:
|
||||
static Pipe unitePipes(Pipes pipes, Processors * collected_processors, bool allow_empty_header);
|
||||
void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter);
|
||||
|
||||
void addProcessor(ProcessorPtr processor);
|
||||
|
||||
friend class QueryPipelineBuilder;
|
||||
friend class QueryPipeline;
|
||||
};
|
||||
|
@ -66,7 +66,8 @@ static void checkPulling(
|
||||
Processors & processors,
|
||||
OutputPort * output,
|
||||
OutputPort * totals,
|
||||
OutputPort * extremes)
|
||||
OutputPort * extremes,
|
||||
OutputPort * partial_result)
|
||||
{
|
||||
if (!output || output->isConnected())
|
||||
throw Exception(
|
||||
@ -83,9 +84,15 @@ static void checkPulling(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cannot create pulling QueryPipeline because its extremes port is connected");
|
||||
|
||||
if (partial_result && partial_result->isConnected())
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cannot create pulling QueryPipeline because its partial_result port is connected");
|
||||
|
||||
bool found_output = false;
|
||||
bool found_totals = false;
|
||||
bool found_extremes = false;
|
||||
bool found_partial_result = false;
|
||||
for (const auto & processor : processors)
|
||||
{
|
||||
for (const auto & in : processor->getInputs())
|
||||
@ -99,6 +106,8 @@ static void checkPulling(
|
||||
found_totals = true;
|
||||
else if (extremes && &out == extremes)
|
||||
found_extremes = true;
|
||||
else if (partial_result && &out == partial_result)
|
||||
found_partial_result = true;
|
||||
else
|
||||
checkOutput(out, processor);
|
||||
}
|
||||
@ -116,6 +125,10 @@ static void checkPulling(
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cannot create pulling QueryPipeline because its extremes port does not belong to any processor");
|
||||
if (partial_result && !found_partial_result)
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cannot create pulling QueryPipeline because its partial result port does not belong to any processor");
|
||||
}
|
||||
|
||||
static void checkCompleted(Processors & processors)
|
||||
@ -318,17 +331,20 @@ QueryPipeline::QueryPipeline(
|
||||
std::shared_ptr<Processors> processors_,
|
||||
OutputPort * output_,
|
||||
OutputPort * totals_,
|
||||
OutputPort * extremes_)
|
||||
OutputPort * extremes_,
|
||||
OutputPort * partial_result_)
|
||||
: resources(std::move(resources_))
|
||||
, processors(std::move(processors_))
|
||||
, output(output_)
|
||||
, totals(totals_)
|
||||
, extremes(extremes_)
|
||||
, partial_result(partial_result_)
|
||||
{
|
||||
checkPulling(*processors, output, totals, extremes);
|
||||
checkPulling(*processors, output, totals, extremes, partial_result);
|
||||
}
|
||||
|
||||
QueryPipeline::QueryPipeline(Pipe pipe)
|
||||
: partial_result_duration_ms(pipe.partial_result_duration_ms)
|
||||
{
|
||||
if (pipe.numOutputPorts() > 0)
|
||||
{
|
||||
@ -336,9 +352,10 @@ QueryPipeline::QueryPipeline(Pipe pipe)
|
||||
output = pipe.getOutputPort(0);
|
||||
totals = pipe.getTotalsPort();
|
||||
extremes = pipe.getExtremesPort();
|
||||
partial_result = pipe.getPartialResultPort(0);
|
||||
|
||||
processors = std::move(pipe.processors);
|
||||
checkPulling(*processors, output, totals, extremes);
|
||||
checkPulling(*processors, output, totals, extremes, partial_result);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -370,6 +387,7 @@ QueryPipeline::QueryPipeline(std::shared_ptr<IOutputFormat> format)
|
||||
auto & format_main = format->getPort(IOutputFormat::PortKind::Main);
|
||||
auto & format_totals = format->getPort(IOutputFormat::PortKind::Totals);
|
||||
auto & format_extremes = format->getPort(IOutputFormat::PortKind::Extremes);
|
||||
auto & format_partial_result = format->getPort(IOutputFormat::PortKind::PartialResult);
|
||||
|
||||
if (!totals)
|
||||
{
|
||||
@ -385,12 +403,21 @@ QueryPipeline::QueryPipeline(std::shared_ptr<IOutputFormat> format)
|
||||
processors->emplace_back(std::move(source));
|
||||
}
|
||||
|
||||
if (!partial_result)
|
||||
{
|
||||
auto source = std::make_shared<NullSource>(format_partial_result.getHeader());
|
||||
partial_result = &source->getPort();
|
||||
processors->emplace_back(std::move(source));
|
||||
}
|
||||
|
||||
connect(*totals, format_totals);
|
||||
connect(*extremes, format_extremes);
|
||||
connect(*partial_result, format_partial_result);
|
||||
|
||||
input = &format_main;
|
||||
totals = nullptr;
|
||||
extremes = nullptr;
|
||||
partial_result = nullptr;
|
||||
|
||||
output_format = format.get();
|
||||
|
||||
@ -418,6 +445,7 @@ void QueryPipeline::complete(std::shared_ptr<ISink> sink)
|
||||
|
||||
drop(totals, *processors);
|
||||
drop(extremes, *processors);
|
||||
drop(partial_result, *processors);
|
||||
|
||||
connect(*output, sink->getPort());
|
||||
processors->emplace_back(std::move(sink));
|
||||
@ -433,6 +461,7 @@ void QueryPipeline::complete(Chain chain)
|
||||
|
||||
drop(totals, *processors);
|
||||
drop(extremes, *processors);
|
||||
drop(partial_result, *processors);
|
||||
|
||||
processors->reserve(processors->size() + chain.getProcessors().size() + 1);
|
||||
for (auto processor : chain.getProcessors())
|
||||
@ -458,6 +487,7 @@ void QueryPipeline::complete(Pipe pipe)
|
||||
pipe.resize(1);
|
||||
pipe.dropExtremes();
|
||||
pipe.dropTotals();
|
||||
pipe.dropPartialResult();
|
||||
connect(*pipe.getOutputPort(0), *input);
|
||||
input = nullptr;
|
||||
|
||||
@ -486,11 +516,13 @@ void QueryPipeline::complete(std::shared_ptr<IOutputFormat> format)
|
||||
addMaterializing(output, *processors);
|
||||
addMaterializing(totals, *processors);
|
||||
addMaterializing(extremes, *processors);
|
||||
addMaterializing(partial_result, *processors);
|
||||
}
|
||||
|
||||
auto & format_main = format->getPort(IOutputFormat::PortKind::Main);
|
||||
auto & format_totals = format->getPort(IOutputFormat::PortKind::Totals);
|
||||
auto & format_extremes = format->getPort(IOutputFormat::PortKind::Extremes);
|
||||
auto & format_partial_result = format->getPort(IOutputFormat::PortKind::PartialResult);
|
||||
|
||||
if (!totals)
|
||||
{
|
||||
@ -506,13 +538,22 @@ void QueryPipeline::complete(std::shared_ptr<IOutputFormat> format)
|
||||
processors->emplace_back(std::move(source));
|
||||
}
|
||||
|
||||
if (!partial_result)
|
||||
{
|
||||
auto source = std::make_shared<NullSource>(format_partial_result.getHeader());
|
||||
partial_result = &source->getPort();
|
||||
processors->emplace_back(std::move(source));
|
||||
}
|
||||
|
||||
connect(*output, format_main);
|
||||
connect(*totals, format_totals);
|
||||
connect(*extremes, format_extremes);
|
||||
connect(*partial_result, format_partial_result);
|
||||
|
||||
output = nullptr;
|
||||
totals = nullptr;
|
||||
extremes = nullptr;
|
||||
partial_result = nullptr;
|
||||
|
||||
initRowsBeforeLimit(format.get());
|
||||
output_format = format.get();
|
||||
@ -684,6 +725,7 @@ void QueryPipeline::convertStructureTo(const ColumnsWithTypeAndName & columns)
|
||||
addExpression(output, actions, *processors);
|
||||
addExpression(totals, actions, *processors);
|
||||
addExpression(extremes, actions, *processors);
|
||||
addExpression(partial_result, actions, *processors);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadProgressCallback> QueryPipeline::getReadProgressCallback() const
|
||||
|
@ -75,7 +75,8 @@ public:
|
||||
std::shared_ptr<Processors> processors_,
|
||||
OutputPort * output_,
|
||||
OutputPort * totals_ = nullptr,
|
||||
OutputPort * extremes_ = nullptr);
|
||||
OutputPort * extremes_ = nullptr,
|
||||
OutputPort * partial_result_ = nullptr);
|
||||
|
||||
bool initialized() const { return !processors->empty(); }
|
||||
/// When initialized, exactly one of the following is true.
|
||||
@ -154,6 +155,7 @@ private:
|
||||
OutputPort * output = nullptr;
|
||||
OutputPort * totals = nullptr;
|
||||
OutputPort * extremes = nullptr;
|
||||
OutputPort * partial_result = nullptr;
|
||||
|
||||
QueryStatusPtr process_list_element;
|
||||
|
||||
@ -162,6 +164,9 @@ private:
|
||||
size_t num_threads = 0;
|
||||
bool concurrency_control = false;
|
||||
|
||||
UInt64 partial_result_limit = 0;
|
||||
UInt64 partial_result_duration_ms = 0;
|
||||
|
||||
friend class PushingPipelineExecutor;
|
||||
friend class PullingPipelineExecutor;
|
||||
friend class PushingAsyncPipelineExecutor;
|
||||
|
@ -110,6 +110,16 @@ void QueryPipelineBuilder::init(QueryPipeline & pipeline)
|
||||
pipe.header = {};
|
||||
}
|
||||
|
||||
if (pipeline.partial_result)
|
||||
{
|
||||
/// Set partial result ports only after activation because when activated, it is set to nullptr
|
||||
pipe.activatePartialResult(pipeline.partial_result_limit, pipeline.partial_result_duration_ms);
|
||||
pipe.partial_result_ports = {pipeline.partial_result};
|
||||
}
|
||||
|
||||
if (!pipeline.partial_result)
|
||||
pipe.dropPartialResult();
|
||||
|
||||
pipe.totals_port = pipeline.totals;
|
||||
pipe.extremes_port = pipeline.extremes;
|
||||
pipe.max_parallel_streams = pipeline.num_threads;
|
||||
@ -352,6 +362,10 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesYShaped
|
||||
left->checkInitializedAndNotCompleted();
|
||||
right->checkInitializedAndNotCompleted();
|
||||
|
||||
/// TODO: Support joining of partial results from different pipelines.
|
||||
left->pipe.dropPartialResult();
|
||||
right->pipe.dropPartialResult();
|
||||
|
||||
left->pipe.dropExtremes();
|
||||
right->pipe.dropExtremes();
|
||||
if (left->getNumStreams() != 1 || right->getNumStreams() != 1)
|
||||
@ -364,6 +378,7 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesYShaped
|
||||
|
||||
auto joining = std::make_shared<MergeJoinTransform>(join, inputs, out_header, max_block_size);
|
||||
|
||||
/// TODO: Support partial results in merge pipelines after joining support above.
|
||||
return mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors);
|
||||
}
|
||||
|
||||
@ -384,6 +399,10 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
|
||||
left->pipe.dropExtremes();
|
||||
right->pipe.dropExtremes();
|
||||
|
||||
/// TODO: Support joining of partial results from different pipelines.
|
||||
left->pipe.dropPartialResult();
|
||||
right->pipe.dropPartialResult();
|
||||
|
||||
left->pipe.collected_processors = collected_processors;
|
||||
|
||||
/// Collect the NEW processors for the right pipeline.
|
||||
@ -634,7 +653,7 @@ PipelineExecutorPtr QueryPipelineBuilder::execute()
|
||||
if (!isCompleted())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot execute pipeline because it is not completed");
|
||||
|
||||
return std::make_shared<PipelineExecutor>(pipe.processors, process_list_element);
|
||||
return std::make_shared<PipelineExecutor>(pipe.processors, process_list_element, pipe.partial_result_duration_ms);
|
||||
}
|
||||
|
||||
Pipe QueryPipelineBuilder::getPipe(QueryPipelineBuilder pipeline, QueryPlanResourceHolder & resources)
|
||||
|
@ -85,6 +85,12 @@ public:
|
||||
/// Pipeline will be completed after this transformation.
|
||||
void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter);
|
||||
|
||||
/// Activate building separate pipeline for sending partial result.
|
||||
void activatePartialResult(UInt64 partial_result_limit, UInt64 partial_result_duration_ms) { pipe.activatePartialResult(partial_result_limit, partial_result_duration_ms); }
|
||||
|
||||
/// Check if building of a pipeline for sending partial result active.
|
||||
bool isPartialResultActive() { return pipe.isPartialResultActive(); }
|
||||
|
||||
/// Add totals which returns one chunk with single row with defaults.
|
||||
void addDefaultTotals();
|
||||
|
||||
|
@ -885,7 +885,8 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
|
||||
std::unique_lock progress_lock(task_callback_mutex, std::defer_lock);
|
||||
|
||||
{
|
||||
PullingAsyncPipelineExecutor executor(pipeline);
|
||||
bool has_partial_result_setting = query_context->getSettingsRef().partial_result_update_duration_ms.totalMilliseconds() > 0;
|
||||
PullingAsyncPipelineExecutor executor(pipeline, has_partial_result_setting);
|
||||
CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread};
|
||||
|
||||
Block block;
|
||||
|
@ -3,24 +3,27 @@
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Core/Block.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadTask.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct MergeTreeReadTask;
|
||||
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
|
||||
|
||||
|
||||
/// The interface that determines how tasks for reading (MergeTreeReadTask)
|
||||
/// are distributed among data parts with ranges.
|
||||
class IMergeTreeReadPool : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
virtual ~IMergeTreeReadPool() = default;
|
||||
|
||||
virtual String getName() const = 0;
|
||||
virtual Block getHeader() const = 0;
|
||||
|
||||
virtual MergeTreeReadTaskPtr getTask(size_t thread) = 0;
|
||||
/// Returns true if tasks are returned in the same order as the order of ranges passed to pool
|
||||
virtual bool preservesOrderOfRanges() const = 0;
|
||||
|
||||
/// task_idx is an implementation defined identifier that helps
|
||||
/// to get required task. E.g. it may be number of thread in case of Default reading type or an index of a part in case of InOrder/InReverseOrder reading type.
|
||||
virtual MergeTreeReadTaskPtr getTask(size_t task_idx, MergeTreeReadTask * previous_task) = 0;
|
||||
virtual void profileFeedback(ReadBufferFromFileBase::ProfileInfo info) = 0;
|
||||
};
|
||||
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Storages/MergeTree/MergeTreeReaderStream.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPartInfoForReader.h>
|
||||
|
||||
|
@ -1,696 +0,0 @@
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Columns/FilterDescription.h>
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
#include <city.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event WaitPrefetchTaskMicroseconds;
|
||||
};
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int QUERY_WAS_CANCELLED;
|
||||
}
|
||||
|
||||
static void injectNonConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
const Names & virtual_columns);
|
||||
|
||||
static void injectPartConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
MergeTreeReadTask * task,
|
||||
const DataTypePtr & partition_value_type,
|
||||
const Names & virtual_columns);
|
||||
|
||||
|
||||
IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm(
|
||||
Block header,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
UInt64 max_block_size_rows_,
|
||||
UInt64 preferred_block_size_bytes_,
|
||||
UInt64 preferred_max_column_in_block_size_bytes_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
bool use_uncompressed_cache_,
|
||||
const Names & virt_column_names_)
|
||||
: storage(storage_)
|
||||
, storage_snapshot(storage_snapshot_)
|
||||
, prewhere_info(prewhere_info_)
|
||||
, actions_settings(actions_settings_)
|
||||
, prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps))
|
||||
, max_block_size_rows(max_block_size_rows_)
|
||||
, preferred_block_size_bytes(preferred_block_size_bytes_)
|
||||
, preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes_)
|
||||
, reader_settings(reader_settings_)
|
||||
, use_uncompressed_cache(use_uncompressed_cache_)
|
||||
, virt_column_names(virt_column_names_)
|
||||
, partition_value_type(storage.getPartitionValueType())
|
||||
, owned_uncompressed_cache(use_uncompressed_cache ? storage.getContext()->getUncompressedCache() : nullptr)
|
||||
, owned_mark_cache(storage.getContext()->getMarkCache())
|
||||
{
|
||||
header_without_const_virtual_columns = applyPrewhereActions(std::move(header), prewhere_info);
|
||||
size_t non_const_columns_offset = header_without_const_virtual_columns.columns();
|
||||
injectNonConstVirtualColumns(0, header_without_const_virtual_columns, virt_column_names);
|
||||
|
||||
for (size_t col_num = non_const_columns_offset; col_num < header_without_const_virtual_columns.columns(); ++col_num)
|
||||
non_const_virtual_column_names.emplace_back(header_without_const_virtual_columns.getByPosition(col_num).name);
|
||||
|
||||
result_header = header_without_const_virtual_columns;
|
||||
injectPartConstVirtualColumns(0, result_header, nullptr, partition_value_type, virt_column_names);
|
||||
|
||||
if (!prewhere_actions.steps.empty())
|
||||
LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions.steps.size(), prewhere_actions.dumpConditions());
|
||||
|
||||
if (prewhere_info)
|
||||
LOG_TEST(log, "Original PREWHERE DAG:\n{}\nPREWHERE actions:\n{}",
|
||||
(prewhere_info->prewhere_actions ? prewhere_info->prewhere_actions->dumpDAG(): std::string("<nullptr>")),
|
||||
(!prewhere_actions.steps.empty() ? prewhere_actions.dump() : std::string("<nullptr>")));
|
||||
}
|
||||
|
||||
bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere);
|
||||
|
||||
PrewhereExprInfo IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps)
|
||||
{
|
||||
PrewhereExprInfo prewhere_actions;
|
||||
if (prewhere_info)
|
||||
{
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
PrewhereExprStep row_level_filter_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter, actions_settings),
|
||||
.filter_column_name = prewhere_info->row_level_column_name,
|
||||
.remove_filter_column = true,
|
||||
.need_filter = true,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
prewhere_actions.steps.emplace_back(std::make_shared<PrewhereExprStep>(std::move(row_level_filter_step)));
|
||||
}
|
||||
|
||||
if (!enable_multiple_prewhere_read_steps ||
|
||||
!tryBuildPrewhereSteps(prewhere_info, actions_settings, prewhere_actions))
|
||||
{
|
||||
PrewhereExprStep prewhere_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings),
|
||||
.filter_column_name = prewhere_info->prewhere_column_name,
|
||||
.remove_filter_column = prewhere_info->remove_prewhere_column,
|
||||
.need_filter = prewhere_info->need_filter,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
prewhere_actions.steps.emplace_back(std::make_shared<PrewhereExprStep>(std::move(prewhere_step)));
|
||||
}
|
||||
}
|
||||
|
||||
return prewhere_actions;
|
||||
}
|
||||
|
||||
|
||||
bool IMergeTreeSelectAlgorithm::getNewTask()
|
||||
{
|
||||
if (getNewTaskImpl())
|
||||
{
|
||||
finalizeNewTask();
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
ChunkAndProgress IMergeTreeSelectAlgorithm::read()
|
||||
{
|
||||
while (!is_cancelled)
|
||||
{
|
||||
try
|
||||
{
|
||||
if ((!task || task->isFinished()) && !getNewTask())
|
||||
break;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// See MergeTreeBaseSelectProcessor::getTaskFromBuffer()
|
||||
if (e.code() == ErrorCodes::QUERY_WAS_CANCELLED)
|
||||
break;
|
||||
throw;
|
||||
}
|
||||
|
||||
auto res = readFromPart();
|
||||
|
||||
if (res.row_count)
|
||||
{
|
||||
injectVirtualColumns(res.block, res.row_count, task.get(), partition_value_type, virt_column_names);
|
||||
|
||||
/// Reorder the columns according to result_header
|
||||
Columns ordered_columns;
|
||||
ordered_columns.reserve(result_header.columns());
|
||||
for (size_t i = 0; i < result_header.columns(); ++i)
|
||||
{
|
||||
auto name = result_header.getByPosition(i).name;
|
||||
ordered_columns.push_back(res.block.getByName(name).column);
|
||||
}
|
||||
|
||||
return ChunkAndProgress{
|
||||
.chunk = Chunk(ordered_columns, res.row_count),
|
||||
.num_read_rows = res.num_read_rows,
|
||||
.num_read_bytes = res.num_read_bytes,
|
||||
.is_finished = false};
|
||||
}
|
||||
else
|
||||
{
|
||||
return {Chunk(), res.num_read_rows, res.num_read_bytes, false};
|
||||
}
|
||||
}
|
||||
|
||||
return {Chunk(), 0, 0, true};
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask(
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback)
|
||||
{
|
||||
if (!task)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no task");
|
||||
|
||||
if (task->reader.valid())
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
|
||||
reader = task->reader.get();
|
||||
}
|
||||
else
|
||||
{
|
||||
reader = task->data_part->getReader(
|
||||
task->task_columns.columns, storage_snapshot, task->mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
task->alter_conversions, reader_settings, value_size_map, profile_callback);
|
||||
}
|
||||
|
||||
if (!task->pre_reader_for_step.empty())
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
|
||||
pre_reader_for_step.clear();
|
||||
for (auto & pre_reader : task->pre_reader_for_step)
|
||||
pre_reader_for_step.push_back(pre_reader.get());
|
||||
}
|
||||
else
|
||||
{
|
||||
initializeMergeTreePreReadersForPart(
|
||||
task->data_part, task->alter_conversions,
|
||||
task->task_columns, task->mark_ranges,
|
||||
value_size_map, profile_callback);
|
||||
}
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForPart(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const MarkRanges & mark_ranges,
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback)
|
||||
{
|
||||
reader = data_part->getReader(
|
||||
task_columns.columns, storage_snapshot, mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
alter_conversions, reader_settings, value_size_map, profile_callback);
|
||||
|
||||
initializeMergeTreePreReadersForPart(
|
||||
data_part, alter_conversions, task_columns,
|
||||
mark_ranges, value_size_map, profile_callback);
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeMergeTreePreReadersForPart(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const MarkRanges & mark_ranges,
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback)
|
||||
{
|
||||
pre_reader_for_step.clear();
|
||||
|
||||
/// Add lightweight delete filtering step
|
||||
if (reader_settings.apply_deleted_mask && data_part->hasLightweightDelete())
|
||||
{
|
||||
pre_reader_for_step.push_back(
|
||||
data_part->getReader(
|
||||
{LightweightDeleteDescription::FILTER_COLUMN}, storage_snapshot,
|
||||
mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
alter_conversions, reader_settings, value_size_map, profile_callback));
|
||||
}
|
||||
|
||||
for (const auto & pre_columns_per_step : task_columns.pre_columns)
|
||||
{
|
||||
pre_reader_for_step.push_back(
|
||||
data_part->getReader(
|
||||
pre_columns_per_step, storage_snapshot, mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
alter_conversions, reader_settings, value_size_map, profile_callback));
|
||||
}
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeRangeReaders(MergeTreeReadTask & current_task)
|
||||
{
|
||||
return initializeRangeReadersImpl(
|
||||
current_task.range_reader, current_task.pre_range_readers, prewhere_actions,
|
||||
reader.get(), current_task.data_part->hasLightweightDelete(), reader_settings,
|
||||
pre_reader_for_step, lightweight_delete_filter_step, non_const_virtual_column_names);
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeRangeReadersImpl(
|
||||
MergeTreeRangeReader & range_reader,
|
||||
std::deque<MergeTreeRangeReader> & pre_range_readers,
|
||||
const PrewhereExprInfo & prewhere_actions,
|
||||
IMergeTreeReader * reader,
|
||||
bool has_lightweight_delete,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
const std::vector<std::unique_ptr<IMergeTreeReader>> & pre_reader_for_step,
|
||||
const PrewhereExprStep & lightweight_delete_filter_step,
|
||||
const Names & non_const_virtual_column_names)
|
||||
{
|
||||
MergeTreeRangeReader * prev_reader = nullptr;
|
||||
bool last_reader = false;
|
||||
size_t pre_readers_shift = 0;
|
||||
|
||||
/// Add filtering step with lightweight delete mask
|
||||
if (reader_settings.apply_deleted_mask && has_lightweight_delete)
|
||||
{
|
||||
MergeTreeRangeReader pre_range_reader(pre_reader_for_step[0].get(), prev_reader, &lightweight_delete_filter_step, last_reader, non_const_virtual_column_names);
|
||||
pre_range_readers.push_back(std::move(pre_range_reader));
|
||||
prev_reader = &pre_range_readers.back();
|
||||
pre_readers_shift++;
|
||||
}
|
||||
|
||||
if (prewhere_actions.steps.size() + pre_readers_shift != pre_reader_for_step.size())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"PREWHERE steps count mismatch, actions: {}, readers: {}",
|
||||
prewhere_actions.steps.size(), pre_reader_for_step.size());
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < prewhere_actions.steps.size(); ++i)
|
||||
{
|
||||
last_reader = reader->getColumns().empty() && (i + 1 == prewhere_actions.steps.size());
|
||||
|
||||
MergeTreeRangeReader current_reader(
|
||||
pre_reader_for_step[i + pre_readers_shift].get(),
|
||||
prev_reader, prewhere_actions.steps[i].get(),
|
||||
last_reader, non_const_virtual_column_names);
|
||||
|
||||
pre_range_readers.push_back(std::move(current_reader));
|
||||
prev_reader = &pre_range_readers.back();
|
||||
}
|
||||
|
||||
if (!last_reader)
|
||||
{
|
||||
range_reader = MergeTreeRangeReader(reader, prev_reader, nullptr, true, non_const_virtual_column_names);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If all columns are read by pre_range_readers than move last pre_range_reader into range_reader
|
||||
range_reader = std::move(pre_range_readers.back());
|
||||
pre_range_readers.pop_back();
|
||||
}
|
||||
}
|
||||
|
||||
static UInt64 estimateNumRows(const MergeTreeReadTask & current_task, UInt64 current_preferred_block_size_bytes,
|
||||
UInt64 current_max_block_size_rows, UInt64 current_preferred_max_column_in_block_size_bytes, double min_filtration_ratio, size_t min_marks_to_read)
|
||||
{
|
||||
const MergeTreeRangeReader & current_reader = current_task.range_reader;
|
||||
|
||||
if (!current_task.size_predictor)
|
||||
return static_cast<size_t>(current_max_block_size_rows);
|
||||
|
||||
/// Calculates number of rows will be read using preferred_block_size_bytes.
|
||||
/// Can't be less than avg_index_granularity.
|
||||
size_t rows_to_read = current_task.size_predictor->estimateNumRows(current_preferred_block_size_bytes);
|
||||
if (!rows_to_read)
|
||||
return rows_to_read;
|
||||
auto total_row_in_current_granule = current_reader.numRowsInCurrentGranule();
|
||||
rows_to_read = std::max(total_row_in_current_granule, rows_to_read);
|
||||
|
||||
if (current_preferred_max_column_in_block_size_bytes)
|
||||
{
|
||||
/// Calculates number of rows will be read using preferred_max_column_in_block_size_bytes.
|
||||
auto rows_to_read_for_max_size_column
|
||||
= current_task.size_predictor->estimateNumRowsForMaxSizeColumn(current_preferred_max_column_in_block_size_bytes);
|
||||
double filtration_ratio = std::max(min_filtration_ratio, 1.0 - current_task.size_predictor->filtered_rows_ratio);
|
||||
auto rows_to_read_for_max_size_column_with_filtration
|
||||
= static_cast<size_t>(rows_to_read_for_max_size_column / filtration_ratio);
|
||||
|
||||
/// If preferred_max_column_in_block_size_bytes is used, number of rows to read can be less than current_index_granularity.
|
||||
rows_to_read = std::min(rows_to_read, rows_to_read_for_max_size_column_with_filtration);
|
||||
}
|
||||
|
||||
auto unread_rows_in_current_granule = current_reader.numPendingRowsInCurrentGranule();
|
||||
if (unread_rows_in_current_granule >= rows_to_read)
|
||||
return rows_to_read;
|
||||
|
||||
const MergeTreeIndexGranularity & index_granularity = current_task.data_part->index_granularity;
|
||||
|
||||
return index_granularity.countMarksForRows(current_reader.currentMark(), rows_to_read, current_reader.numReadRowsInCurrentGranule(), min_marks_to_read);
|
||||
}
|
||||
|
||||
|
||||
IMergeTreeSelectAlgorithm::BlockAndProgress IMergeTreeSelectAlgorithm::readFromPartImpl()
|
||||
{
|
||||
if (task->size_predictor)
|
||||
task->size_predictor->startBlock();
|
||||
|
||||
const UInt64 current_max_block_size_rows = max_block_size_rows;
|
||||
const UInt64 current_preferred_block_size_bytes = preferred_block_size_bytes;
|
||||
const UInt64 current_preferred_max_column_in_block_size_bytes = preferred_max_column_in_block_size_bytes;
|
||||
const double min_filtration_ratio = 0.00001;
|
||||
|
||||
UInt64 recommended_rows = estimateNumRows(*task, current_preferred_block_size_bytes,
|
||||
current_max_block_size_rows, current_preferred_max_column_in_block_size_bytes, min_filtration_ratio, min_marks_to_read);
|
||||
UInt64 rows_to_read = std::max(static_cast<UInt64>(1), std::min(current_max_block_size_rows, recommended_rows));
|
||||
|
||||
auto read_result = task->range_reader.read(rows_to_read, task->mark_ranges);
|
||||
|
||||
/// All rows were filtered. Repeat.
|
||||
if (read_result.num_rows == 0)
|
||||
read_result.columns.clear();
|
||||
|
||||
const auto & sample_block = task->range_reader.getSampleBlock();
|
||||
if (read_result.num_rows != 0 && sample_block.columns() != read_result.columns.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent number of columns got from MergeTreeRangeReader. "
|
||||
"Have {} in sample block and {} columns in list",
|
||||
toString(sample_block.columns()), toString(read_result.columns.size()));
|
||||
|
||||
/// TODO: check columns have the same types as in header.
|
||||
|
||||
UInt64 num_filtered_rows = read_result.numReadRows() - read_result.num_rows;
|
||||
|
||||
size_t num_read_rows = read_result.numReadRows();
|
||||
size_t num_read_bytes = read_result.numBytesRead();
|
||||
|
||||
if (task->size_predictor)
|
||||
{
|
||||
task->size_predictor->updateFilteredRowsRation(read_result.numReadRows(), num_filtered_rows);
|
||||
|
||||
if (!read_result.columns.empty())
|
||||
task->size_predictor->update(sample_block, read_result.columns, read_result.num_rows);
|
||||
}
|
||||
|
||||
Block block;
|
||||
if (read_result.num_rows != 0)
|
||||
block = sample_block.cloneWithColumns(read_result.columns);
|
||||
|
||||
BlockAndProgress res = {
|
||||
.block = std::move(block),
|
||||
.row_count = read_result.num_rows,
|
||||
.num_read_rows = num_read_rows,
|
||||
.num_read_bytes = num_read_bytes };
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
IMergeTreeSelectAlgorithm::BlockAndProgress IMergeTreeSelectAlgorithm::readFromPart()
|
||||
{
|
||||
if (!task->range_reader.isInitialized())
|
||||
initializeRangeReaders(*task);
|
||||
|
||||
return readFromPartImpl();
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
struct VirtualColumnsInserter
|
||||
{
|
||||
explicit VirtualColumnsInserter(Block & block_) : block(block_) {}
|
||||
|
||||
bool columnExists(const String & name) const { return block.has(name); }
|
||||
|
||||
void insertUInt8Column(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeUInt8>(), name});
|
||||
}
|
||||
|
||||
void insertUInt64Column(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeUInt64>(), name});
|
||||
}
|
||||
|
||||
void insertUUIDColumn(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeUUID>(), name});
|
||||
}
|
||||
|
||||
void insertLowCardinalityColumn(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), name});
|
||||
}
|
||||
|
||||
void insertPartitionValueColumn(
|
||||
size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name)
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = partition_value_type->createColumnConst(rows, Tuple(partition_value.begin(), partition_value.end()))
|
||||
->convertToFullColumnIfConst();
|
||||
else
|
||||
column = partition_value_type->createColumn();
|
||||
|
||||
block.insert({column, partition_value_type, name});
|
||||
}
|
||||
|
||||
Block & block;
|
||||
};
|
||||
}
|
||||
|
||||
/// Adds virtual columns that are not const for all rows
|
||||
static void injectNonConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
const Names & virtual_columns)
|
||||
{
|
||||
VirtualColumnsInserter inserter(block);
|
||||
for (const auto & virtual_column_name : virtual_columns)
|
||||
{
|
||||
if (virtual_column_name == "_part_offset")
|
||||
{
|
||||
if (!rows)
|
||||
{
|
||||
inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!inserter.columnExists(virtual_column_name))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Column {} must have been filled part reader",
|
||||
virtual_column_name);
|
||||
}
|
||||
}
|
||||
|
||||
if (virtual_column_name == LightweightDeleteDescription::FILTER_COLUMN.name)
|
||||
{
|
||||
/// If _row_exists column isn't present in the part then fill it here with 1s
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumnConst(rows, 1)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumn();
|
||||
|
||||
inserter.insertUInt8Column(column, virtual_column_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Adds virtual columns that are const for the whole part
|
||||
static void injectPartConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
MergeTreeReadTask * task,
|
||||
const DataTypePtr & partition_value_type,
|
||||
const Names & virtual_columns)
|
||||
{
|
||||
VirtualColumnsInserter inserter(block);
|
||||
/// add virtual columns
|
||||
/// Except _sample_factor, which is added from the outside.
|
||||
if (!virtual_columns.empty())
|
||||
{
|
||||
if (unlikely(rows && !task))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot insert virtual columns to non-empty chunk without specified task.");
|
||||
|
||||
const IMergeTreeDataPart * part = nullptr;
|
||||
if (rows)
|
||||
{
|
||||
part = task->data_part.get();
|
||||
if (part->isProjectionPart())
|
||||
part = part->getParentPart();
|
||||
}
|
||||
for (const auto & virtual_column_name : virtual_columns)
|
||||
{
|
||||
if (virtual_column_name == "_part")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}
|
||||
.createColumnConst(rows, part->name)
|
||||
->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn();
|
||||
|
||||
inserter.insertLowCardinalityColumn(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_part_index")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeUInt64().createColumnConst(rows, task->part_index_in_query)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeUInt64().createColumn();
|
||||
|
||||
inserter.insertUInt64Column(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_part_uuid")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeUUID().createColumnConst(rows, part->uuid)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeUUID().createColumn();
|
||||
|
||||
inserter.insertUUIDColumn(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_partition_id")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}
|
||||
.createColumnConst(rows, part->info.partition_id)
|
||||
->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn();
|
||||
|
||||
inserter.insertLowCardinalityColumn(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_partition_value")
|
||||
{
|
||||
if (rows)
|
||||
inserter.insertPartitionValueColumn(rows, part->partition.value, partition_value_type, virtual_column_name);
|
||||
else
|
||||
inserter.insertPartitionValueColumn(rows, {}, partition_value_type, virtual_column_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::injectVirtualColumns(
|
||||
Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns)
|
||||
{
|
||||
/// First add non-const columns that are filled by the range reader and then const columns that we will fill ourselves.
|
||||
/// Note that the order is important: virtual columns filled by the range reader must go first
|
||||
injectNonConstVirtualColumns(row_count, block, virtual_columns);
|
||||
injectPartConstVirtualColumns(row_count, block, task, partition_value_type, virtual_columns);
|
||||
}
|
||||
|
||||
Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info)
|
||||
{
|
||||
if (prewhere_info)
|
||||
{
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
block = prewhere_info->row_level_filter->updateHeader(std::move(block));
|
||||
auto & row_level_column = block.getByName(prewhere_info->row_level_column_name);
|
||||
if (!row_level_column.type->canBeUsedInBooleanContext())
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}",
|
||||
row_level_column.type->getName());
|
||||
}
|
||||
|
||||
block.erase(prewhere_info->row_level_column_name);
|
||||
}
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
block = prewhere_info->prewhere_actions->updateHeader(std::move(block));
|
||||
|
||||
auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name);
|
||||
if (!prewhere_column.type->canBeUsedInBooleanContext())
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}",
|
||||
prewhere_column.type->getName());
|
||||
}
|
||||
|
||||
if (prewhere_info->remove_prewhere_column)
|
||||
{
|
||||
block.erase(prewhere_info->prewhere_column_name);
|
||||
}
|
||||
else if (prewhere_info->need_filter)
|
||||
{
|
||||
WhichDataType which(removeNullable(recursiveRemoveLowCardinality(prewhere_column.type)));
|
||||
|
||||
if (which.isNativeInt() || which.isNativeUInt())
|
||||
prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst();
|
||||
else if (which.isFloat())
|
||||
prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1.0f)->convertToFullColumnIfConst();
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER,
|
||||
"Illegal type {} of column for filter",
|
||||
prewhere_column.type->getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
Block IMergeTreeSelectAlgorithm::transformHeader(
|
||||
Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns)
|
||||
{
|
||||
auto transformed = applyPrewhereActions(std::move(block), prewhere_info);
|
||||
injectVirtualColumns(transformed, 0, nullptr, partition_value_type, virtual_columns);
|
||||
return transformed;
|
||||
}
|
||||
|
||||
std::unique_ptr<MergeTreeBlockSizePredictor> IMergeTreeSelectAlgorithm::getSizePredictor(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const Block & sample_block)
|
||||
{
|
||||
const auto & required_column_names = task_columns.columns.getNames();
|
||||
NameSet complete_column_names(required_column_names.begin(), required_column_names.end());
|
||||
for (const auto & pre_columns_per_step : task_columns.pre_columns)
|
||||
{
|
||||
const auto & required_pre_column_names = pre_columns_per_step.getNames();
|
||||
complete_column_names.insert(required_pre_column_names.begin(), required_pre_column_names.end());
|
||||
}
|
||||
|
||||
return std::make_unique<MergeTreeBlockSizePredictor>(
|
||||
data_part, Names(complete_column_names.begin(), complete_column_names.end()), sample_block);
|
||||
}
|
||||
|
||||
|
||||
IMergeTreeSelectAlgorithm::~IMergeTreeSelectAlgorithm() = default;
|
||||
|
||||
}
|
@ -1,217 +0,0 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Processors/Chunk.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergeTreeReader;
|
||||
class UncompressedCache;
|
||||
class MarkCache;
|
||||
struct PrewhereExprInfo;
|
||||
|
||||
struct ChunkAndProgress
|
||||
{
|
||||
Chunk chunk;
|
||||
size_t num_read_rows = 0;
|
||||
size_t num_read_bytes = 0;
|
||||
/// Explicitly indicate that we have read all data.
|
||||
/// This is needed to occasionally return empty chunk to indicate the progress while the rows are filtered out in PREWHERE.
|
||||
bool is_finished = false;
|
||||
};
|
||||
|
||||
struct ParallelReadingExtension
|
||||
{
|
||||
MergeTreeAllRangesCallback all_callback;
|
||||
MergeTreeReadTaskCallback callback;
|
||||
size_t count_participating_replicas{0};
|
||||
size_t number_of_current_replica{0};
|
||||
/// This is needed to estimate the number of bytes
|
||||
/// between a pair of marks to perform one request
|
||||
/// over the network for a 1Gb of data.
|
||||
Names columns_to_read;
|
||||
};
|
||||
|
||||
/// Base class for MergeTreeThreadSelectAlgorithm and MergeTreeSelectAlgorithm
|
||||
class IMergeTreeSelectAlgorithm
|
||||
{
|
||||
public:
|
||||
IMergeTreeSelectAlgorithm(
|
||||
Block header,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings,
|
||||
UInt64 max_block_size_rows_,
|
||||
UInt64 preferred_block_size_bytes_,
|
||||
UInt64 preferred_max_column_in_block_size_bytes_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
bool use_uncompressed_cache_,
|
||||
const Names & virt_column_names_ = {});
|
||||
|
||||
virtual ~IMergeTreeSelectAlgorithm();
|
||||
|
||||
static Block transformHeader(
|
||||
Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns);
|
||||
|
||||
static std::unique_ptr<MergeTreeBlockSizePredictor> getSizePredictor(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const Block & sample_block);
|
||||
|
||||
Block getHeader() const { return result_header; }
|
||||
|
||||
ChunkAndProgress read();
|
||||
|
||||
void cancel() { is_cancelled = true; }
|
||||
|
||||
const MergeTreeReaderSettings & getSettings() const { return reader_settings; }
|
||||
|
||||
virtual std::string getName() const = 0;
|
||||
|
||||
static PrewhereExprInfo getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps);
|
||||
|
||||
protected:
|
||||
/// This struct allow to return block with no columns but with non-zero number of rows similar to Chunk
|
||||
struct BlockAndProgress
|
||||
{
|
||||
Block block;
|
||||
size_t row_count = 0;
|
||||
size_t num_read_rows = 0;
|
||||
size_t num_read_bytes = 0;
|
||||
};
|
||||
|
||||
/// Creates new this->task and return a flag whether it was successful or not
|
||||
virtual bool getNewTaskImpl() = 0;
|
||||
/// Creates new readers for a task it is needed. These methods are separate, because
|
||||
/// in case of parallel reading from replicas the whole task could be denied by a coodinator
|
||||
/// or it could modified somehow.
|
||||
virtual void finalizeNewTask() = 0;
|
||||
|
||||
size_t estimateMaxBatchSizeForHugeRanges();
|
||||
|
||||
/// Closes readers and unlock part locks
|
||||
virtual void finish() = 0;
|
||||
|
||||
virtual BlockAndProgress readFromPart();
|
||||
|
||||
BlockAndProgress readFromPartImpl();
|
||||
|
||||
/// Used for filling header with no rows as well as block with data
|
||||
static void
|
||||
injectVirtualColumns(Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns);
|
||||
|
||||
protected:
|
||||
static void initializeRangeReadersImpl(
|
||||
MergeTreeRangeReader & range_reader,
|
||||
std::deque<MergeTreeRangeReader> & pre_range_readers,
|
||||
const PrewhereExprInfo & prewhere_actions,
|
||||
IMergeTreeReader * reader,
|
||||
bool has_lightweight_delete,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
const std::vector<std::unique_ptr<IMergeTreeReader>> & pre_reader_for_step,
|
||||
const PrewhereExprStep & lightweight_delete_filter_step,
|
||||
const Names & non_const_virtual_column_names);
|
||||
|
||||
/// Sets up data readers for each step of prewhere and where
|
||||
void initializeMergeTreeReadersForCurrentTask(
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback);
|
||||
|
||||
void initializeMergeTreeReadersForPart(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const MarkRanges & mark_ranges,
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback);
|
||||
|
||||
/// Sets up range readers corresponding to data readers
|
||||
void initializeRangeReaders(MergeTreeReadTask & task);
|
||||
|
||||
const MergeTreeData & storage;
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
|
||||
/// This step is added when the part has lightweight delete mask
|
||||
const PrewhereExprStep lightweight_delete_filter_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = nullptr,
|
||||
.filter_column_name = LightweightDeleteDescription::FILTER_COLUMN.name,
|
||||
.remove_filter_column = true,
|
||||
.need_filter = true,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
PrewhereInfoPtr prewhere_info;
|
||||
ExpressionActionsSettings actions_settings;
|
||||
PrewhereExprInfo prewhere_actions;
|
||||
|
||||
UInt64 max_block_size_rows;
|
||||
UInt64 preferred_block_size_bytes;
|
||||
UInt64 preferred_max_column_in_block_size_bytes;
|
||||
|
||||
MergeTreeReaderSettings reader_settings;
|
||||
|
||||
bool use_uncompressed_cache;
|
||||
|
||||
Names virt_column_names;
|
||||
|
||||
/// These columns will be filled by the merge tree range reader
|
||||
Names non_const_virtual_column_names;
|
||||
|
||||
DataTypePtr partition_value_type;
|
||||
|
||||
/// This header is used for chunks from readFromPart().
|
||||
Block header_without_const_virtual_columns;
|
||||
/// A result of getHeader(). A chunk which this header is returned from read().
|
||||
Block result_header;
|
||||
|
||||
UncompressedCachePtr owned_uncompressed_cache;
|
||||
MarkCachePtr owned_mark_cache;
|
||||
|
||||
using MergeTreeReaderPtr = std::unique_ptr<IMergeTreeReader>;
|
||||
MergeTreeReaderPtr reader;
|
||||
std::vector<MergeTreeReaderPtr> pre_reader_for_step;
|
||||
|
||||
MergeTreeReadTaskPtr task;
|
||||
|
||||
/// This setting is used in base algorithm only to additionally limit the number of granules to read.
|
||||
/// It is changed in ctor of MergeTreeThreadSelectAlgorithm.
|
||||
///
|
||||
/// The reason why we have it here is because MergeTreeReadPool takes the full task
|
||||
/// ignoring min_marks_to_read setting in case of remote disk (see MergeTreeReadPool::getTask).
|
||||
/// In this case, we won't limit the number of rows to read based on adaptive granularity settings.
|
||||
///
|
||||
/// Big reading tasks are better for remote disk and prefetches.
|
||||
/// So, for now it's easier to limit max_rows_to_read.
|
||||
/// Somebody need to refactor this later.
|
||||
size_t min_marks_to_read = 0;
|
||||
|
||||
private:
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeBaseSelectProcessor");
|
||||
|
||||
std::atomic<bool> is_cancelled{false};
|
||||
|
||||
bool getNewTask();
|
||||
|
||||
/// Initialize pre readers.
|
||||
void initializeMergeTreePreReadersForPart(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const MarkRanges & mark_ranges,
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback);
|
||||
|
||||
static Block applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info);
|
||||
};
|
||||
|
||||
using MergeTreeSelectAlgorithmPtr = std::unique_ptr<IMergeTreeSelectAlgorithm>;
|
||||
|
||||
}
|
@ -6,7 +6,7 @@
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
@ -16,6 +16,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -136,43 +137,6 @@ NameSet injectRequiredColumns(
|
||||
return injected_columns;
|
||||
}
|
||||
|
||||
|
||||
MergeTreeReadTask::MergeTreeReadTask(
|
||||
const DataPartPtr & data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
const MarkRanges & mark_ranges_,
|
||||
size_t part_index_in_query_,
|
||||
const NameSet & column_name_set_,
|
||||
const MergeTreeReadTaskColumns & task_columns_,
|
||||
MergeTreeBlockSizePredictorPtr size_predictor_,
|
||||
Priority priority_,
|
||||
std::future<MergeTreeReaderPtr> reader_,
|
||||
std::vector<std::future<MergeTreeReaderPtr>> && pre_reader_for_step_)
|
||||
: data_part{data_part_}
|
||||
, alter_conversions{alter_conversions_}
|
||||
, mark_ranges{mark_ranges_}
|
||||
, part_index_in_query{part_index_in_query_}
|
||||
, column_name_set{column_name_set_}
|
||||
, task_columns{task_columns_}
|
||||
, size_predictor{size_predictor_}
|
||||
, reader(std::move(reader_))
|
||||
, pre_reader_for_step(std::move(pre_reader_for_step_))
|
||||
, priority(priority_)
|
||||
{
|
||||
}
|
||||
|
||||
MergeTreeReadTask::~MergeTreeReadTask()
|
||||
{
|
||||
if (reader.valid())
|
||||
reader.wait();
|
||||
|
||||
for (const auto & pre_reader : pre_reader_for_step)
|
||||
{
|
||||
if (pre_reader.valid())
|
||||
pre_reader.wait();
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeBlockSizePredictor::MergeTreeBlockSizePredictor(
|
||||
const DataPartPtr & data_part_, const Names & columns, const Block & sample_block)
|
||||
: data_part(data_part_)
|
||||
@ -195,9 +159,8 @@ void MergeTreeBlockSizePredictor::initialize(const Block & sample_block, const C
|
||||
for (size_t pos = 0; pos < num_columns; ++pos)
|
||||
{
|
||||
const auto & column_with_type_and_name = sample_block.getByPosition(pos);
|
||||
const String & column_name = column_with_type_and_name.name;
|
||||
const ColumnPtr & column_data = from_update ? columns[pos]
|
||||
: column_with_type_and_name.column;
|
||||
const auto & column_name = column_with_type_and_name.name;
|
||||
const auto & column_data = from_update ? columns[pos] : column_with_type_and_name.column;
|
||||
|
||||
if (!from_update && !names_set.contains(column_name))
|
||||
continue;
|
||||
@ -246,7 +209,6 @@ void MergeTreeBlockSizePredictor::startBlock()
|
||||
info.size_bytes = 0;
|
||||
}
|
||||
|
||||
|
||||
/// TODO: add last_read_row_in_part parameter to take into account gaps between adjacent ranges
|
||||
void MergeTreeBlockSizePredictor::update(const Block & sample_block, const Columns & columns, size_t num_rows, double decay)
|
||||
{
|
||||
@ -296,7 +258,7 @@ void MergeTreeBlockSizePredictor::update(const Block & sample_block, const Colum
|
||||
}
|
||||
|
||||
|
||||
MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
MergeTreeReadTask::Columns getReadTaskColumns(
|
||||
const IMergeTreeDataPartInfoForReader & data_part_info_for_reader,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const Names & required_columns,
|
||||
@ -317,7 +279,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
injectRequiredColumns(
|
||||
data_part_info_for_reader, storage_snapshot, with_subcolumns, column_to_read_after_prewhere);
|
||||
|
||||
MergeTreeReadTaskColumns result;
|
||||
MergeTreeReadTask::Columns result;
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::All)
|
||||
.withExtendedObjects()
|
||||
.withSystemColumns();
|
||||
@ -365,7 +327,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions(
|
||||
auto prewhere_actions = MergeTreeSelectProcessor::getPrewhereActions(
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings.enable_multiple_prewhere_read_steps);
|
||||
@ -387,16 +349,4 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
std::string MergeTreeReadTaskColumns::dump() const
|
||||
{
|
||||
WriteBufferFromOwnString s;
|
||||
for (size_t i = 0; i < pre_columns.size(); ++i)
|
||||
{
|
||||
s << "STEP " << i << ": " << pre_columns[i].toString() << "\n";
|
||||
}
|
||||
s << "COLUMNS: " << columns.toString() << "\n";
|
||||
return s.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,29 +2,15 @@
|
||||
|
||||
#include <optional>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadTask.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeTreeData;
|
||||
struct MergeTreeReadTask;
|
||||
struct MergeTreeReaderSettings;
|
||||
struct MergeTreeBlockSizePredictor;
|
||||
class IMergeTreeDataPartInfoForReader;
|
||||
|
||||
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
|
||||
using MergeTreeBlockSizePredictorPtr = std::shared_ptr<MergeTreeBlockSizePredictor>;
|
||||
|
||||
class IMergeTreeDataPart;
|
||||
using DataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
|
||||
|
||||
|
||||
/** If some of the requested columns are not in the part,
|
||||
* then find out which columns may need to be read further,
|
||||
* so that you can calculate the DEFAULT expression for these columns.
|
||||
@ -36,64 +22,7 @@ NameSet injectRequiredColumns(
|
||||
bool with_subcolumns,
|
||||
Names & columns);
|
||||
|
||||
struct MergeTreeReadTaskColumns
|
||||
{
|
||||
/// column names to read during WHERE
|
||||
NamesAndTypesList columns;
|
||||
/// column names to read during each PREWHERE step
|
||||
std::vector<NamesAndTypesList> pre_columns;
|
||||
|
||||
std::string dump() const;
|
||||
};
|
||||
|
||||
/// A batch of work for MergeTreeThreadSelectProcessor
|
||||
struct MergeTreeReadTask
|
||||
{
|
||||
/// Data part which should be read while performing this task
|
||||
DataPartPtr data_part;
|
||||
/// Alter converversionss that should be applied on-fly for part.
|
||||
AlterConversionsPtr alter_conversions;
|
||||
/// Ranges to read from `data_part`.
|
||||
MarkRanges mark_ranges;
|
||||
/// for virtual `part_index` virtual column
|
||||
size_t part_index_in_query;
|
||||
/// used to determine whether column should be filtered during PREWHERE or WHERE
|
||||
const NameSet & column_name_set;
|
||||
/// column names to read during PREWHERE and WHERE
|
||||
const MergeTreeReadTaskColumns & task_columns;
|
||||
/// Used to satistfy preferred_block_size_bytes limitation
|
||||
MergeTreeBlockSizePredictorPtr size_predictor;
|
||||
/// Used to save current range processing status
|
||||
MergeTreeRangeReader range_reader;
|
||||
/// Range readers for multiple filtering steps: row level security, PREWHERE etc.
|
||||
/// NOTE: we take references to elements and push_back new elements, that's why it is a deque but not a vector
|
||||
std::deque<MergeTreeRangeReader> pre_range_readers;
|
||||
|
||||
using MergeTreeReaderPtr = std::unique_ptr<IMergeTreeReader>;
|
||||
std::future<MergeTreeReaderPtr> reader;
|
||||
std::vector<std::future<MergeTreeReaderPtr>> pre_reader_for_step;
|
||||
|
||||
Priority priority;
|
||||
|
||||
bool isFinished() const { return mark_ranges.empty() && range_reader.isCurrentRangeFinished(); }
|
||||
|
||||
MergeTreeReadTask(
|
||||
const DataPartPtr & data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
const MarkRanges & mark_ranges_,
|
||||
size_t part_index_in_query_,
|
||||
const NameSet & column_name_set_,
|
||||
const MergeTreeReadTaskColumns & task_columns_,
|
||||
MergeTreeBlockSizePredictorPtr size_predictor_,
|
||||
Priority priority_ = {},
|
||||
std::future<MergeTreeReaderPtr> reader_ = {},
|
||||
std::vector<std::future<MergeTreeReaderPtr>> && pre_reader_for_step_ = {});
|
||||
|
||||
~MergeTreeReadTask();
|
||||
};
|
||||
|
||||
|
||||
MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
MergeTreeReadTask::Columns getReadTaskColumns(
|
||||
const IMergeTreeDataPartInfoForReader & data_part_info_for_reader,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const Names & required_columns,
|
||||
@ -119,7 +48,6 @@ struct MergeTreeBlockSizePredictor
|
||||
return block_size_bytes;
|
||||
}
|
||||
|
||||
|
||||
/// Predicts what number of rows should be read to exhaust byte quota per column
|
||||
inline size_t estimateNumRowsForMaxSizeColumn(size_t bytes_quota) const
|
||||
{
|
||||
@ -153,7 +81,6 @@ struct MergeTreeBlockSizePredictor
|
||||
static double calculateDecay() { return 1. - std::pow(TARGET_WEIGHT, 1. / NUM_UPDATES_TO_TARGET_WEIGHT); }
|
||||
|
||||
protected:
|
||||
|
||||
DataPartPtr data_part;
|
||||
|
||||
struct ColumnInfo
|
||||
|
@ -68,7 +68,7 @@
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/Freeze.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
|
||||
|
@ -1,76 +0,0 @@
|
||||
#include <Storages/MergeTree/MergeTreeInOrderSelectProcessor.h>
|
||||
#include "Storages/MergeTree/RangesInDataPart.h"
|
||||
#include <Storages/MergeTree/IntersectionsIndexes.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||
}
|
||||
|
||||
bool MergeTreeInOrderSelectAlgorithm::getNewTaskImpl()
|
||||
try
|
||||
{
|
||||
if (all_mark_ranges.empty())
|
||||
return false;
|
||||
|
||||
if (!reader)
|
||||
initializeReaders();
|
||||
|
||||
MarkRanges mark_ranges_for_task;
|
||||
|
||||
if (!pool)
|
||||
{
|
||||
/// If we need to read few rows, set one range per task to reduce number of read data.
|
||||
if (has_limit_below_one_block)
|
||||
{
|
||||
mark_ranges_for_task = MarkRanges{};
|
||||
mark_ranges_for_task.emplace_front(std::move(all_mark_ranges.front()));
|
||||
all_mark_ranges.pop_front();
|
||||
}
|
||||
else
|
||||
{
|
||||
mark_ranges_for_task = std::move(all_mark_ranges);
|
||||
all_mark_ranges.clear();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto description = RangesInDataPartDescription{
|
||||
.info = data_part->info,
|
||||
/// We just ignore all the distribution done before
|
||||
/// Everything will be done on coordinator side
|
||||
.ranges = {},
|
||||
};
|
||||
|
||||
mark_ranges_for_task = pool->getNewTask(description);
|
||||
|
||||
if (mark_ranges_for_task.empty())
|
||||
return false;
|
||||
}
|
||||
|
||||
auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr
|
||||
: getSizePredictor(data_part, task_columns, sample_block);
|
||||
|
||||
task = std::make_unique<MergeTreeReadTask>(
|
||||
data_part,
|
||||
alter_conversions,
|
||||
mark_ranges_for_task,
|
||||
part_index_in_query,
|
||||
column_name_set,
|
||||
task_columns,
|
||||
std::move(size_predictor));
|
||||
|
||||
return true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Suspicion of the broken part. A part is added to the queue for verification.
|
||||
if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
|
||||
storage.reportBrokenPart(data_part);
|
||||
throw;
|
||||
}
|
||||
|
||||
}
|
@ -1,33 +0,0 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/// Used to read data from single part with select query in order of primary key.
|
||||
/// Cares about PREWHERE, virtual columns, indexes etc.
|
||||
/// To read data from multiple parts, Storage (MergeTree) creates multiple such objects.
|
||||
class MergeTreeInOrderSelectAlgorithm final : public MergeTreeSelectAlgorithm
|
||||
{
|
||||
public:
|
||||
template <typename... Args>
|
||||
explicit MergeTreeInOrderSelectAlgorithm(Args &&... args)
|
||||
: MergeTreeSelectAlgorithm{std::forward<Args>(args)...}
|
||||
{
|
||||
LOG_TRACE(log, "Reading {} ranges in order from part {}, approx. {} rows starting from {}",
|
||||
all_mark_ranges.size(), data_part->name, total_rows,
|
||||
data_part->index_granularity.getMarkStartingRow(all_mark_ranges.front().begin));
|
||||
}
|
||||
|
||||
String getName() const override { return "MergeTreeInOrder"; }
|
||||
|
||||
private:
|
||||
bool getNewTaskImpl() override;
|
||||
void finalizeNewTask() override {}
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeInOrderSelectProcessor");
|
||||
};
|
||||
|
||||
}
|
@ -5,7 +5,6 @@
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreePrefetchedReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
@ -14,10 +13,10 @@
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event MergeTreePrefetchedReadPoolInit;
|
||||
extern const Event WaitPrefetchTaskMicroseconds;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
@ -29,145 +28,124 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
bool MergeTreePrefetchedReadPool::TaskHolder::operator<(const TaskHolder & other) const
|
||||
{
|
||||
chassert(task->priority >= 0);
|
||||
chassert(other.task->priority >= 0);
|
||||
/// With default std::priority_queue, top() returns largest element.
|
||||
/// So closest to 0 will be on top with this comparator.
|
||||
return task->priority > other.task->priority; /// Less is better.
|
||||
}
|
||||
|
||||
MergeTreePrefetchedReadPool::PrefetechedReaders::PrefetechedReaders(
|
||||
MergeTreeReadTask::Readers readers_,
|
||||
Priority priority_,
|
||||
MergeTreePrefetchedReadPool & pool_)
|
||||
: is_valid(true)
|
||||
, readers(std::move(readers_))
|
||||
{
|
||||
prefetch_futures.push_back(pool_.createPrefetchedFuture(readers.main.get(), priority_));
|
||||
|
||||
for (const auto & reader : readers.prewhere)
|
||||
prefetch_futures.push_back(pool_.createPrefetchedFuture(reader.get(), priority_));
|
||||
}
|
||||
|
||||
void MergeTreePrefetchedReadPool::PrefetechedReaders::wait()
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
|
||||
for (auto & prefetch_future : prefetch_futures)
|
||||
prefetch_future.wait();
|
||||
}
|
||||
|
||||
MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetechedReaders::get()
|
||||
{
|
||||
SCOPE_EXIT({ is_valid = false; });
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
|
||||
for (auto & prefetch_future : prefetch_futures)
|
||||
prefetch_future.get();
|
||||
|
||||
return std::move(readers);
|
||||
}
|
||||
|
||||
MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool(
|
||||
size_t threads,
|
||||
size_t sum_marks_,
|
||||
size_t min_marks_for_concurrent_read_,
|
||||
RangesInDataParts && parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
ContextPtr context_,
|
||||
bool use_uncompressed_cache_,
|
||||
bool is_remote_read_,
|
||||
const MergeTreeSettings & storage_settings_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get("MergeTreePrefetchedReadPool(" + (parts_.empty() ? "" : parts_.front().data_part->storage.getStorageID().getNameForLogs()) + ")"))
|
||||
, header(storage_snapshot_->getSampleBlockForColumns(column_names_))
|
||||
, mark_cache(context_->getGlobalContext()->getMarkCache().get())
|
||||
, uncompressed_cache(use_uncompressed_cache_ ? context_->getGlobalContext()->getUncompressedCache().get() : nullptr)
|
||||
, profile_callback([this](ReadBufferFromFileBase::ProfileInfo info_) { profileFeedback(info_); })
|
||||
, index_granularity_bytes(storage_settings_.index_granularity_bytes)
|
||||
, fixed_index_granularity(storage_settings_.index_granularity)
|
||||
, storage_snapshot(storage_snapshot_)
|
||||
, column_names(column_names_)
|
||||
, virtual_column_names(virtual_column_names_)
|
||||
, prewhere_info(prewhere_info_)
|
||||
, actions_settings(actions_settings_)
|
||||
, reader_settings(reader_settings_)
|
||||
, is_remote_read(is_remote_read_)
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_)
|
||||
: MergeTreeReadPoolBase(
|
||||
std::move(parts_),
|
||||
storage_snapshot_,
|
||||
prewhere_info_,
|
||||
actions_settings_,
|
||||
reader_settings_,
|
||||
column_names_,
|
||||
virtual_column_names_,
|
||||
settings_,
|
||||
context_)
|
||||
, WithContext(context_)
|
||||
, prefetch_threadpool(getContext()->getPrefetchThreadpool())
|
||||
, log(&Poco::Logger::get("MergeTreePrefetchedReadPool(" + (parts_.empty() ? "" : parts_.front().data_part->storage.getStorageID().getNameForLogs()) + ")"))
|
||||
{
|
||||
/// Tasks creation might also create a lost of readers - check they do not
|
||||
/// do any time consuming operations in ctor.
|
||||
ProfileEventTimeIncrement<Milliseconds> watch(ProfileEvents::MergeTreePrefetchedReadPoolInit);
|
||||
|
||||
parts_infos = getPartsInfos(parts_, preferred_block_size_bytes_);
|
||||
threads_tasks = createThreadsTasks(threads, sum_marks_, min_marks_for_concurrent_read_);
|
||||
fillPerPartStatistics();
|
||||
fillPerThreadTasks(pool_settings.threads, pool_settings.sum_marks);
|
||||
}
|
||||
|
||||
struct MergeTreePrefetchedReadPool::PartInfo
|
||||
std::future<void> MergeTreePrefetchedReadPool::createPrefetchedFuture(IMergeTreeReader * reader, Priority priority)
|
||||
{
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
AlterConversionsPtr alter_conversions;
|
||||
size_t part_index_in_query;
|
||||
size_t sum_marks = 0;
|
||||
MarkRanges ranges;
|
||||
|
||||
NameSet column_name_set;
|
||||
MergeTreeReadTaskColumns task_columns;
|
||||
MergeTreeBlockSizePredictorPtr size_predictor;
|
||||
|
||||
size_t approx_size_of_mark = 0;
|
||||
size_t prefetch_step_marks = 0;
|
||||
|
||||
size_t estimated_memory_usage_for_single_prefetch = 0;
|
||||
size_t required_readers_num = 0;
|
||||
};
|
||||
|
||||
std::future<MergeTreeReaderPtr> MergeTreePrefetchedReadPool::createPrefetchedReader(
|
||||
const IMergeTreeDataPart & data_part,
|
||||
const NamesAndTypesList & columns,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MarkRanges & required_ranges,
|
||||
Priority priority) const
|
||||
{
|
||||
auto reader = data_part.getReader(
|
||||
columns, storage_snapshot, required_ranges,
|
||||
uncompressed_cache, mark_cache, alter_conversions, reader_settings,
|
||||
IMergeTreeReader::ValueSizeMap{}, profile_callback);
|
||||
|
||||
/// In order to make a prefetch we need to wait for marks to be loaded. But we just created
|
||||
/// a reader (which starts loading marks in its constructor), then if we do prefetch right
|
||||
/// after creating a reader, it will be very inefficient. We can do prefetch for all parts
|
||||
/// only inside this MergeTreePrefetchedReadPool, where read tasks are created and distributed,
|
||||
/// and we cannot block either, therefore make prefetch inside the pool and put the future
|
||||
/// into the read task (MergeTreeReadTask). When a thread calls getTask(), it will wait for
|
||||
/// it (if not yet ready) after getting the task.
|
||||
auto task = [=, my_reader = std::move(reader), context = getContext()]() mutable -> MergeTreeReaderPtr &&
|
||||
/// into the thread task. When a thread calls getTask(), it will wait for it is not ready yet.
|
||||
auto task = [=, context = getContext()]() mutable
|
||||
{
|
||||
/// For async read metrics in system.query_log.
|
||||
PrefetchIncrement watch(context->getAsyncReadCounters());
|
||||
|
||||
my_reader->prefetchBeginOfRange(priority);
|
||||
return std::move(my_reader);
|
||||
reader->prefetchBeginOfRange(priority);
|
||||
};
|
||||
return scheduleFromThreadPool<IMergeTreeDataPart::MergeTreeReaderPtr>(std::move(task), prefetch_threadpool, "ReadPrepare", priority);
|
||||
|
||||
return scheduleFromThreadPool<void>(std::move(task), prefetch_threadpool, "ReadPrepare", priority);
|
||||
}
|
||||
|
||||
void MergeTreePrefetchedReadPool::createPrefetchedReaderForTask(MergeTreeReadTask & task) const
|
||||
void MergeTreePrefetchedReadPool::createPrefetchedReadersForTask(ThreadTask & task)
|
||||
{
|
||||
if (task.reader.valid())
|
||||
if (task.readers_future.valid())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Task already has a reader");
|
||||
|
||||
task.reader = createPrefetchedReader(*task.data_part, task.task_columns.columns, task.alter_conversions, task.mark_ranges, task.priority);
|
||||
|
||||
if (reader_settings.apply_deleted_mask && task.data_part->hasLightweightDelete())
|
||||
{
|
||||
auto pre_reader = createPrefetchedReader(*task.data_part, {LightweightDeleteDescription::FILTER_COLUMN}, task.alter_conversions, task.mark_ranges, task.priority);
|
||||
task.pre_reader_for_step.push_back(std::move(pre_reader));
|
||||
}
|
||||
|
||||
for (const auto & pre_columns_per_step : task.task_columns.pre_columns)
|
||||
{
|
||||
auto pre_reader = createPrefetchedReader(*task.data_part, pre_columns_per_step, task.alter_conversions, task.mark_ranges, task.priority);
|
||||
task.pre_reader_for_step.push_back(std::move(pre_reader));
|
||||
}
|
||||
auto extras = getExtras();
|
||||
auto readers = MergeTreeReadTask::createReaders(task.read_info, extras, task.ranges);
|
||||
task.readers_future = PrefetechedReaders(std::move(readers), task.priority, *this);
|
||||
}
|
||||
|
||||
bool MergeTreePrefetchedReadPool::TaskHolder::operator <(const TaskHolder & other) const
|
||||
{
|
||||
chassert(task->priority >= 0);
|
||||
chassert(other.task->priority >= 0);
|
||||
return task->priority > other.task->priority; /// Less is better.
|
||||
/// With default std::priority_queue, top() returns largest element.
|
||||
/// So closest to 0 will be on top with this comparator.
|
||||
}
|
||||
|
||||
void MergeTreePrefetchedReadPool::startPrefetches() const
|
||||
void MergeTreePrefetchedReadPool::startPrefetches()
|
||||
{
|
||||
if (prefetch_queue.empty())
|
||||
return;
|
||||
|
||||
[[maybe_unused]] TaskHolder prev(nullptr, 0);
|
||||
[[maybe_unused]] TaskHolder prev;
|
||||
[[maybe_unused]] const Priority highest_priority{reader_settings.read_settings.priority.value + 1};
|
||||
assert(prefetch_queue.top().task->priority == highest_priority);
|
||||
|
||||
while (!prefetch_queue.empty())
|
||||
{
|
||||
const auto & top = prefetch_queue.top();
|
||||
createPrefetchedReaderForTask(*top.task);
|
||||
createPrefetchedReadersForTask(*top.task);
|
||||
#ifndef NDEBUG
|
||||
if (prev.task)
|
||||
{
|
||||
assert(top.task->priority >= highest_priority);
|
||||
if (prev.thread_id == top.thread_id)
|
||||
{
|
||||
assert(prev.task->priority < top.task->priority);
|
||||
}
|
||||
}
|
||||
prev = top;
|
||||
#endif
|
||||
@ -175,11 +153,11 @@ void MergeTreePrefetchedReadPool::startPrefetches() const
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::getTask(size_t thread)
|
||||
MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::getTask(size_t task_idx, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (threads_tasks.empty())
|
||||
if (per_thread_tasks.empty())
|
||||
return nullptr;
|
||||
|
||||
if (!started_prefetches)
|
||||
@ -188,112 +166,129 @@ MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::getTask(size_t thread)
|
||||
startPrefetches();
|
||||
}
|
||||
|
||||
auto it = threads_tasks.find(thread);
|
||||
if (it == threads_tasks.end())
|
||||
{
|
||||
ThreadsTasks::iterator non_prefetched_tasks_to_steal = threads_tasks.end();
|
||||
ThreadsTasks::iterator prefetched_tasks_to_steal = threads_tasks.end();
|
||||
int64_t best_prefetched_task_priority = -1;
|
||||
|
||||
/// There is no point stealing in order (like in MergeTreeReadPool, where tasks can be stolen
|
||||
/// only from the next thread). Even if we steal task from the next thread, which reads from
|
||||
/// the same part as we just read, it might seem that we can reuse our own reader, do some
|
||||
/// seek avoiding and it will have a good result as we avoided seek (new request). But it is
|
||||
/// not so, because this next task will most likely have its own reader a prefetch already on
|
||||
/// the fly. (Not to mention that in fact we cannot reuse our own reader if initially we did
|
||||
/// not accounted this range into range request to object storage).
|
||||
for (auto thread_tasks_it = threads_tasks.begin(); thread_tasks_it != threads_tasks.end(); ++thread_tasks_it)
|
||||
{
|
||||
/// Prefer to steal tasks which have an initialized reader (with prefetched data). Thus we avoid
|
||||
/// losing a prefetch by creating our own reader (or resusing our own reader if the part
|
||||
/// is the same as last read by this thread).
|
||||
auto & thread_tasks = thread_tasks_it->second;
|
||||
auto task_it = std::find_if(
|
||||
thread_tasks.begin(), thread_tasks.end(),
|
||||
[](const auto & task) { return task->reader.valid(); });
|
||||
|
||||
if (task_it == thread_tasks.end())
|
||||
{
|
||||
/// The follow back to non-prefetched task should lie on the thread which
|
||||
/// has more tasks than others.
|
||||
if (non_prefetched_tasks_to_steal == threads_tasks.end()
|
||||
|| non_prefetched_tasks_to_steal->second.size() < thread_tasks.size())
|
||||
non_prefetched_tasks_to_steal = thread_tasks_it;
|
||||
}
|
||||
/// Try to steal task with the best (lowest) priority (because it will be executed faster).
|
||||
else if (prefetched_tasks_to_steal == threads_tasks.end()
|
||||
|| (*task_it)->priority < best_prefetched_task_priority)
|
||||
{
|
||||
best_prefetched_task_priority = (*task_it)->priority;
|
||||
chassert(best_prefetched_task_priority >= 0);
|
||||
prefetched_tasks_to_steal = thread_tasks_it;
|
||||
}
|
||||
}
|
||||
|
||||
if (prefetched_tasks_to_steal != threads_tasks.end())
|
||||
{
|
||||
auto & thread_tasks = prefetched_tasks_to_steal->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
auto task_it = std::find_if(
|
||||
thread_tasks.begin(), thread_tasks.end(),
|
||||
[](const auto & task) { return task->reader.valid(); });
|
||||
assert(task_it != thread_tasks.end());
|
||||
|
||||
auto task = std::move(*task_it);
|
||||
thread_tasks.erase(task_it);
|
||||
|
||||
if (thread_tasks.empty())
|
||||
threads_tasks.erase(prefetched_tasks_to_steal);
|
||||
|
||||
return task;
|
||||
}
|
||||
|
||||
/// TODO: it also makes sense to first try to steal from the next thread if it has ranges
|
||||
/// from the same part as current thread last read - to reuse the reader.
|
||||
|
||||
if (non_prefetched_tasks_to_steal != threads_tasks.end())
|
||||
{
|
||||
auto & thread_tasks = non_prefetched_tasks_to_steal->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
/// Get second half of the tasks.
|
||||
const size_t total_tasks = thread_tasks.size();
|
||||
const size_t half = total_tasks / 2;
|
||||
auto half_it = thread_tasks.begin() + half;
|
||||
assert(half_it != thread_tasks.end());
|
||||
|
||||
/// Give them to current thread, as current thread's tasks list is empty.
|
||||
auto & current_thread_tasks = threads_tasks[thread];
|
||||
current_thread_tasks.insert(
|
||||
current_thread_tasks.end(), make_move_iterator(half_it), make_move_iterator(thread_tasks.end()));
|
||||
|
||||
/// Erase them from the thread from which we steal.
|
||||
thread_tasks.resize(half);
|
||||
if (thread_tasks.empty())
|
||||
threads_tasks.erase(non_prefetched_tasks_to_steal);
|
||||
|
||||
auto task = std::move(current_thread_tasks.front());
|
||||
current_thread_tasks.erase(current_thread_tasks.begin());
|
||||
if (current_thread_tasks.empty())
|
||||
threads_tasks.erase(thread);
|
||||
|
||||
return task;
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
auto it = per_thread_tasks.find(task_idx);
|
||||
if (it == per_thread_tasks.end())
|
||||
return stealTask(task_idx, previous_task);
|
||||
|
||||
auto & thread_tasks = it->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
auto task = std::move(thread_tasks.front());
|
||||
auto thread_task = std::move(thread_tasks.front());
|
||||
thread_tasks.pop_front();
|
||||
|
||||
if (thread_tasks.empty())
|
||||
threads_tasks.erase(it);
|
||||
per_thread_tasks.erase(it);
|
||||
|
||||
return task;
|
||||
return createTask(*thread_task, previous_task);
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::stealTask(size_t thread, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
auto non_prefetched_tasks_to_steal = per_thread_tasks.end();
|
||||
auto prefetched_tasks_to_steal = per_thread_tasks.end();
|
||||
int64_t best_prefetched_task_priority = -1;
|
||||
|
||||
/// There is no point stealing in order (like in MergeTreeReadPool, where tasks can be stolen
|
||||
/// only from the next thread). Even if we steal task from the next thread, which reads from
|
||||
/// the same part as we just read, it might seem that we can reuse our own reader, do some
|
||||
/// seek avoiding and it will have a good result as we avoided seek (new request). But it is
|
||||
/// not so, because this next task will most likely have its own reader a prefetch already on
|
||||
/// the fly. (Not to mention that in fact we cannot reuse our own reader if initially we did
|
||||
/// not accounted this range into range request to object storage).
|
||||
for (auto thread_tasks_it = per_thread_tasks.begin(); thread_tasks_it != per_thread_tasks.end(); ++thread_tasks_it)
|
||||
{
|
||||
/// Prefer to steal tasks which have an initialized reader (with prefetched data). Thus we avoid
|
||||
/// losing a prefetch by creating our own reader (or resusing our own reader if the part
|
||||
/// is the same as last read by this thread).
|
||||
auto & thread_tasks = thread_tasks_it->second;
|
||||
|
||||
auto task_it = std::find_if(
|
||||
thread_tasks.begin(), thread_tasks.end(),
|
||||
[](const auto & task) { return task->readers_future.valid(); });
|
||||
|
||||
if (task_it == thread_tasks.end())
|
||||
{
|
||||
/// The follow back to non-prefetched task should lie on the thread which
|
||||
/// has more tasks than others.
|
||||
if (non_prefetched_tasks_to_steal == per_thread_tasks.end()
|
||||
|| non_prefetched_tasks_to_steal->second.size() < thread_tasks.size())
|
||||
non_prefetched_tasks_to_steal = thread_tasks_it;
|
||||
}
|
||||
/// Try to steal task with the best (lowest) priority (because it will be executed faster).
|
||||
else if (prefetched_tasks_to_steal == per_thread_tasks.end()
|
||||
|| (*task_it)->priority < best_prefetched_task_priority)
|
||||
{
|
||||
best_prefetched_task_priority = (*task_it)->priority;
|
||||
chassert(best_prefetched_task_priority >= 0);
|
||||
prefetched_tasks_to_steal = thread_tasks_it;
|
||||
}
|
||||
}
|
||||
|
||||
if (prefetched_tasks_to_steal != per_thread_tasks.end())
|
||||
{
|
||||
auto & thread_tasks = prefetched_tasks_to_steal->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
auto task_it = std::find_if(
|
||||
thread_tasks.begin(), thread_tasks.end(),
|
||||
[](const auto & task) { return task->readers_future.valid(); });
|
||||
|
||||
assert(task_it != thread_tasks.end());
|
||||
auto thread_task = std::move(*task_it);
|
||||
thread_tasks.erase(task_it);
|
||||
|
||||
if (thread_tasks.empty())
|
||||
per_thread_tasks.erase(prefetched_tasks_to_steal);
|
||||
|
||||
return createTask(*thread_task, previous_task);
|
||||
}
|
||||
|
||||
/// TODO: it also makes sense to first try to steal from the next thread if it has ranges
|
||||
/// from the same part as current thread last read - to reuse the reader.
|
||||
if (non_prefetched_tasks_to_steal != per_thread_tasks.end())
|
||||
{
|
||||
auto & thread_tasks = non_prefetched_tasks_to_steal->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
/// Get second half of the tasks.
|
||||
const size_t total_tasks = thread_tasks.size();
|
||||
const size_t half = total_tasks / 2;
|
||||
auto half_it = thread_tasks.begin() + half;
|
||||
assert(half_it != thread_tasks.end());
|
||||
|
||||
/// Give them to current thread, as current thread's tasks list is empty.
|
||||
auto & current_thread_tasks = per_thread_tasks[thread];
|
||||
current_thread_tasks.insert(
|
||||
current_thread_tasks.end(), make_move_iterator(half_it), make_move_iterator(thread_tasks.end()));
|
||||
|
||||
/// Erase them from the thread from which we steal.
|
||||
thread_tasks.resize(half);
|
||||
if (thread_tasks.empty())
|
||||
per_thread_tasks.erase(non_prefetched_tasks_to_steal);
|
||||
|
||||
auto thread_task = std::move(current_thread_tasks.front());
|
||||
current_thread_tasks.erase(current_thread_tasks.begin());
|
||||
if (current_thread_tasks.empty())
|
||||
per_thread_tasks.erase(thread);
|
||||
|
||||
return createTask(*thread_task, previous_task);
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::createTask(ThreadTask & task, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
if (task.readers_future.valid())
|
||||
{
|
||||
auto size_predictor = task.read_info->shared_size_predictor
|
||||
? std::make_unique<MergeTreeBlockSizePredictor>(*task.read_info->shared_size_predictor)
|
||||
: nullptr;
|
||||
|
||||
return std::make_unique<MergeTreeReadTask>(task.read_info, task.readers_future.get(), task.ranges, std::move(size_predictor));
|
||||
}
|
||||
|
||||
return MergeTreeReadPoolBase::createTask(task.read_info, task.ranges, previous_task);
|
||||
}
|
||||
|
||||
size_t getApproximateSizeOfGranule(const IMergeTreeDataPart & part, const Names & columns_to_read)
|
||||
@ -304,154 +299,111 @@ size_t getApproximateSizeOfGranule(const IMergeTreeDataPart & part, const Names
|
||||
return columns_size.data_compressed / part.getMarksCount();
|
||||
}
|
||||
|
||||
MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInfos(
|
||||
const RangesInDataParts & parts, size_t preferred_block_size_bytes) const
|
||||
void MergeTreePrefetchedReadPool::fillPerPartStatistics()
|
||||
{
|
||||
PartsInfos result;
|
||||
Block sample_block = storage_snapshot->metadata->getSampleBlock();
|
||||
per_part_statistics.clear();
|
||||
per_part_statistics.reserve(parts_ranges.size());
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
const bool predict_block_size_bytes = preferred_block_size_bytes > 0;
|
||||
|
||||
for (const auto & part : parts)
|
||||
for (size_t i = 0; i < parts_ranges.size(); ++i)
|
||||
{
|
||||
auto part_info = std::make_unique<PartInfo>();
|
||||
|
||||
part_info->data_part = part.data_part;
|
||||
part_info->alter_conversions = part.alter_conversions;
|
||||
part_info->part_index_in_query = part.part_index_in_query;
|
||||
part_info->ranges = part.ranges;
|
||||
std::sort(part_info->ranges.begin(), part_info->ranges.end());
|
||||
|
||||
LoadedMergeTreeDataPartInfoForReader part_reader_info(part.data_part, part_info->alter_conversions);
|
||||
auto & part_stat = per_part_statistics.emplace_back();
|
||||
const auto & read_info = *per_part_infos[i];
|
||||
|
||||
/// Sum up total size of all mark ranges in a data part.
|
||||
for (const auto & range : part.ranges)
|
||||
part_info->sum_marks += range.end - range.begin;
|
||||
for (const auto & range : parts_ranges[i].ranges)
|
||||
part_stat.sum_marks += range.end - range.begin;
|
||||
|
||||
const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info
|
||||
? prewhere_info->prewhere_actions->getRequiredColumnsNames()
|
||||
: column_names;
|
||||
part_info->approx_size_of_mark = getApproximateSizeOfGranule(*part_info->data_part, columns);
|
||||
|
||||
const auto task_columns = getReadTaskColumns(
|
||||
part_reader_info,
|
||||
storage_snapshot,
|
||||
column_names,
|
||||
virtual_column_names,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
/* with_subcolumns */ true);
|
||||
part_stat.approx_size_of_mark = getApproximateSizeOfGranule(*read_info.data_part, columns);
|
||||
|
||||
part_info->size_predictor = !predict_block_size_bytes
|
||||
? nullptr
|
||||
: IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block);
|
||||
|
||||
/// Will be used to distinguish between PREWHERE and WHERE columns when applying filter.
|
||||
const auto & required_column_names = task_columns.columns.getNames();
|
||||
part_info->column_name_set = {required_column_names.begin(), required_column_names.end()};
|
||||
part_info->task_columns = task_columns;
|
||||
auto update_stat_for_column = [&](const auto & column_name)
|
||||
{
|
||||
size_t column_size = read_info.data_part->getColumnSize(column_name).data_compressed;
|
||||
part_stat.estimated_memory_usage_for_single_prefetch += std::min<size_t>(column_size, settings.prefetch_buffer_size);
|
||||
++part_stat.required_readers_num;
|
||||
};
|
||||
|
||||
/// adjustBufferSize(), which is done in MergeTreeReaderStream and MergeTreeReaderCompact,
|
||||
/// lowers buffer size if file size (or required read range) is less. So we know that the
|
||||
/// settings.prefetch_buffer_size will be lowered there, therefore we account it here as well.
|
||||
/// But here we make a more approximate lowering (because we do not have loaded marks yet),
|
||||
/// while in adjustBufferSize it will be presize.
|
||||
for (const auto & col : task_columns.columns)
|
||||
{
|
||||
const auto col_size = part.data_part->getColumnSize(col.name).data_compressed;
|
||||
part_info->estimated_memory_usage_for_single_prefetch += std::min<size_t>(col_size, settings.prefetch_buffer_size);
|
||||
++part_info->required_readers_num;
|
||||
}
|
||||
if (reader_settings.apply_deleted_mask && part.data_part->hasLightweightDelete())
|
||||
{
|
||||
const auto col_size = part.data_part->getColumnSize(
|
||||
LightweightDeleteDescription::FILTER_COLUMN.name).data_compressed;
|
||||
part_info->estimated_memory_usage_for_single_prefetch += std::min<size_t>(col_size, settings.prefetch_buffer_size);
|
||||
++part_info->required_readers_num;
|
||||
}
|
||||
if (prewhere_info)
|
||||
{
|
||||
for (const auto & cols : task_columns.pre_columns)
|
||||
{
|
||||
for (const auto & col : cols)
|
||||
{
|
||||
const size_t col_size = part.data_part->getColumnSize(col.name).data_compressed;
|
||||
part_info->estimated_memory_usage_for_single_prefetch += std::min<size_t>(col_size, settings.prefetch_buffer_size);
|
||||
++part_info->required_readers_num;
|
||||
}
|
||||
}
|
||||
}
|
||||
for (const auto & column : read_info.task_columns.columns)
|
||||
update_stat_for_column(column.name);
|
||||
|
||||
result.push_back(std::move(part_info));
|
||||
if (reader_settings.apply_deleted_mask && read_info.data_part->hasLightweightDelete())
|
||||
update_stat_for_column(LightweightDeleteDescription::FILTER_COLUMN.name);
|
||||
|
||||
for (const auto & pre_columns : read_info.task_columns.pre_columns)
|
||||
for (const auto & column : pre_columns)
|
||||
update_stat_for_column(column.name);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThreadsTasks(
|
||||
size_t threads, size_t sum_marks, size_t /* min_marks_for_concurrent_read */) const
|
||||
void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_marks)
|
||||
{
|
||||
if (parts_infos.empty())
|
||||
return {};
|
||||
if (per_part_infos.empty())
|
||||
return;
|
||||
|
||||
const auto & context = getContext();
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
size_t total_size_approx = 0;
|
||||
for (const auto & part : parts_infos)
|
||||
{
|
||||
total_size_approx += part->sum_marks * part->approx_size_of_mark;
|
||||
}
|
||||
for (const auto & part : per_part_statistics)
|
||||
total_size_approx += part.sum_marks * part.approx_size_of_mark;
|
||||
|
||||
size_t min_prefetch_step_marks = 0;
|
||||
|
||||
for (const auto & part : parts_infos)
|
||||
for (size_t i = 0; i < per_part_infos.size(); ++i)
|
||||
{
|
||||
auto & part_stat = per_part_statistics[i];
|
||||
|
||||
if (settings.filesystem_prefetch_step_marks)
|
||||
{
|
||||
part->prefetch_step_marks = settings.filesystem_prefetch_step_marks;
|
||||
part_stat.prefetch_step_marks = settings.filesystem_prefetch_step_marks;
|
||||
}
|
||||
else if (settings.filesystem_prefetch_step_bytes && part->approx_size_of_mark)
|
||||
else if (settings.filesystem_prefetch_step_bytes && part_stat.approx_size_of_mark)
|
||||
{
|
||||
part->prefetch_step_marks = std::max<size_t>(
|
||||
1, static_cast<size_t>(std::round(static_cast<double>(settings.filesystem_prefetch_step_bytes) / part->approx_size_of_mark)));
|
||||
part_stat.prefetch_step_marks = std::max<size_t>(
|
||||
1, static_cast<size_t>(std::round(static_cast<double>(settings.filesystem_prefetch_step_bytes) / part_stat.approx_size_of_mark)));
|
||||
}
|
||||
|
||||
/// This limit is important to avoid spikes of slow aws getObject requests when parallelizing within one file.
|
||||
/// (The default is taken from here https://docs.aws.amazon.com/whitepapers/latest/s3-optimizing-performance-best-practices/use-byte-range-fetches.html).
|
||||
if (part->approx_size_of_mark
|
||||
if (part_stat.approx_size_of_mark
|
||||
&& settings.filesystem_prefetch_min_bytes_for_single_read_task
|
||||
&& part->approx_size_of_mark < settings.filesystem_prefetch_min_bytes_for_single_read_task)
|
||||
&& part_stat.approx_size_of_mark < settings.filesystem_prefetch_min_bytes_for_single_read_task)
|
||||
{
|
||||
const size_t min_prefetch_step_marks_by_total_cols = static_cast<size_t>(
|
||||
std::ceil(static_cast<double>(settings.filesystem_prefetch_min_bytes_for_single_read_task) / part->approx_size_of_mark));
|
||||
std::ceil(static_cast<double>(settings.filesystem_prefetch_min_bytes_for_single_read_task) / part_stat.approx_size_of_mark));
|
||||
|
||||
/// At least one task to start working on it right now and another one to prefetch in the meantime.
|
||||
const size_t new_min_prefetch_step_marks = std::min<size_t>(min_prefetch_step_marks_by_total_cols, sum_marks / threads / 2);
|
||||
if (min_prefetch_step_marks < new_min_prefetch_step_marks)
|
||||
{
|
||||
LOG_DEBUG(log, "Increasing min prefetch step from {} to {}", min_prefetch_step_marks, new_min_prefetch_step_marks);
|
||||
|
||||
min_prefetch_step_marks = new_min_prefetch_step_marks;
|
||||
}
|
||||
}
|
||||
|
||||
if (part->prefetch_step_marks < min_prefetch_step_marks)
|
||||
if (part_stat.prefetch_step_marks < min_prefetch_step_marks)
|
||||
{
|
||||
LOG_DEBUG(log, "Increasing prefetch step from {} to {}", part->prefetch_step_marks, min_prefetch_step_marks);
|
||||
|
||||
part->prefetch_step_marks = min_prefetch_step_marks;
|
||||
LOG_DEBUG(log, "Increasing prefetch step from {} to {}", part_stat.prefetch_step_marks, min_prefetch_step_marks);
|
||||
part_stat.prefetch_step_marks = min_prefetch_step_marks;
|
||||
}
|
||||
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Part: {}, sum_marks: {}, approx mark size: {}, prefetch_step_bytes: {}, prefetch_step_marks: {}, (ranges: {})",
|
||||
part->data_part->name,
|
||||
part->sum_marks,
|
||||
part->approx_size_of_mark,
|
||||
parts_ranges[i].data_part->name,
|
||||
part_stat.sum_marks,
|
||||
part_stat.approx_size_of_mark,
|
||||
settings.filesystem_prefetch_step_bytes,
|
||||
part->prefetch_step_marks,
|
||||
toString(part->ranges));
|
||||
part_stat.prefetch_step_marks,
|
||||
toString(parts_ranges[i].ranges));
|
||||
}
|
||||
|
||||
const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1;
|
||||
@ -469,13 +421,24 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr
|
||||
size_t allowed_memory_usage = settings.filesystem_prefetch_max_memory_usage;
|
||||
if (!allowed_memory_usage)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `filesystem_prefetch_max_memory_usage` must be non-zero");
|
||||
|
||||
std::optional<size_t> allowed_prefetches_num = settings.filesystem_prefetches_limit
|
||||
? std::optional<size_t>(settings.filesystem_prefetches_limit)
|
||||
: std::nullopt;
|
||||
|
||||
ThreadsTasks result_threads_tasks;
|
||||
per_thread_tasks.clear();
|
||||
size_t total_tasks = 0;
|
||||
for (size_t i = 0, part_idx = 0; i < threads && part_idx < parts_infos.size(); ++i)
|
||||
|
||||
/// Make a copy to modify ranges.
|
||||
std::vector<MarkRanges> per_part_ranges;
|
||||
per_part_ranges.reserve(parts_ranges.size());
|
||||
for (const auto & part_with_ranges : parts_ranges)
|
||||
{
|
||||
auto & part_ranges = per_part_ranges.emplace_back(part_with_ranges.ranges);
|
||||
std::sort(part_ranges.begin(), part_ranges.end());
|
||||
}
|
||||
|
||||
for (size_t i = 0, part_idx = 0; i < threads && part_idx < per_part_infos.size(); ++i)
|
||||
{
|
||||
int64_t need_marks = min_marks_per_thread;
|
||||
|
||||
@ -486,119 +449,102 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr
|
||||
/// reads from pool which are from reader.
|
||||
Priority priority{reader_settings.read_settings.priority.value + 1};
|
||||
|
||||
while (need_marks > 0 && part_idx < parts_infos.size())
|
||||
while (need_marks > 0 && part_idx < per_part_infos.size())
|
||||
{
|
||||
auto & part = *parts_infos[part_idx];
|
||||
size_t & marks_in_part = part.sum_marks;
|
||||
auto & part_stat = per_part_statistics[part_idx];
|
||||
auto & part_ranges = per_part_ranges[part_idx];
|
||||
|
||||
if (marks_in_part == 0)
|
||||
if (part_stat.sum_marks == 0)
|
||||
{
|
||||
++part_idx;
|
||||
continue;
|
||||
}
|
||||
|
||||
MarkRanges ranges_to_get_from_part;
|
||||
size_t marks_to_get_from_part = std::min<size_t>(need_marks, marks_in_part);
|
||||
size_t marks_to_get_from_part = std::min<size_t>(need_marks, part_stat.sum_marks);
|
||||
|
||||
/// Split by prefetch step even if !allow_prefetch below. Because it will allow
|
||||
/// to make a better distribution of tasks which did not fill into memory limit
|
||||
/// or prefetches limit through tasks stealing.
|
||||
if (part.prefetch_step_marks)
|
||||
if (part_stat.prefetch_step_marks)
|
||||
{
|
||||
marks_to_get_from_part = std::min<size_t>(marks_to_get_from_part, part.prefetch_step_marks);
|
||||
marks_to_get_from_part = std::min<size_t>(marks_to_get_from_part, part_stat.prefetch_step_marks);
|
||||
}
|
||||
|
||||
if (marks_in_part == marks_to_get_from_part)
|
||||
if (part_stat.sum_marks == marks_to_get_from_part)
|
||||
{
|
||||
ranges_to_get_from_part = part.ranges;
|
||||
ranges_to_get_from_part = part_ranges;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (part.sum_marks < marks_to_get_from_part)
|
||||
if (part_stat.sum_marks < marks_to_get_from_part)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Requested {} marks from part {}, but part has only {} marks",
|
||||
marks_to_get_from_part, part.data_part->name, part.sum_marks);
|
||||
marks_to_get_from_part, per_part_infos[part_idx]->data_part->name, part_stat.sum_marks);
|
||||
}
|
||||
|
||||
size_t get_marks_num = marks_to_get_from_part;
|
||||
while (get_marks_num > 0)
|
||||
size_t num_marks_to_get = marks_to_get_from_part;
|
||||
while (num_marks_to_get > 0)
|
||||
{
|
||||
MarkRange & range = part.ranges.front();
|
||||
MarkRange & range = part_ranges.front();
|
||||
const size_t marks_in_range = range.end - range.begin;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, get_marks_num);
|
||||
get_marks_num -= marks_to_get_from_range;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, num_marks_to_get);
|
||||
num_marks_to_get -= marks_to_get_from_range;
|
||||
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range);
|
||||
range.begin += marks_to_get_from_range;
|
||||
|
||||
if (range.begin == range.end)
|
||||
{
|
||||
part.ranges.pop_front();
|
||||
part_ranges.pop_front();
|
||||
}
|
||||
else if (!get_marks_num && part.prefetch_step_marks && range.end - range.begin < part.prefetch_step_marks)
|
||||
else if (!num_marks_to_get && part_stat.prefetch_step_marks && range.end - range.begin < part_stat.prefetch_step_marks)
|
||||
{
|
||||
/// We already have `get_marks_num` marks, but current mark range has
|
||||
/// We already have `num_marks_to_get` marks, but current mark range has
|
||||
/// less than `prefetch_step_marks` marks, then add them too.
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.end);
|
||||
marks_to_get_from_part += range.end - range.begin;
|
||||
part.ranges.pop_front();
|
||||
part_ranges.pop_front();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
need_marks -= marks_to_get_from_part;
|
||||
sum_marks -= marks_to_get_from_part;
|
||||
marks_in_part -= marks_to_get_from_part;
|
||||
|
||||
auto curr_task_size_predictor = !part.size_predictor ? nullptr
|
||||
: std::make_unique<MergeTreeBlockSizePredictor>(*part.size_predictor); /// make a copy
|
||||
|
||||
auto read_task = std::make_unique<MergeTreeReadTask>(
|
||||
part.data_part,
|
||||
part.alter_conversions,
|
||||
ranges_to_get_from_part,
|
||||
part.part_index_in_query,
|
||||
part.column_name_set,
|
||||
part.task_columns,
|
||||
std::move(curr_task_size_predictor));
|
||||
|
||||
read_task->priority = priority;
|
||||
part_stat.sum_marks -= marks_to_get_from_part;
|
||||
|
||||
bool allow_prefetch = false;
|
||||
if (allowed_memory_usage
|
||||
&& (allowed_prefetches_num.has_value() == false || allowed_prefetches_num.value() > 0))
|
||||
&& (!allowed_prefetches_num.has_value() || allowed_prefetches_num.value() > 0))
|
||||
{
|
||||
allow_prefetch = part.estimated_memory_usage_for_single_prefetch <= allowed_memory_usage
|
||||
&& (allowed_prefetches_num.has_value() == false
|
||||
|| part.required_readers_num <= allowed_prefetches_num.value());
|
||||
allow_prefetch = part_stat.estimated_memory_usage_for_single_prefetch <= allowed_memory_usage
|
||||
&& (!allowed_prefetches_num.has_value() || part_stat.required_readers_num <= allowed_prefetches_num.value());
|
||||
|
||||
if (allow_prefetch)
|
||||
{
|
||||
allowed_memory_usage -= part.estimated_memory_usage_for_single_prefetch;
|
||||
allowed_memory_usage -= part_stat.estimated_memory_usage_for_single_prefetch;
|
||||
if (allowed_prefetches_num.has_value())
|
||||
*allowed_prefetches_num -= part.required_readers_num;
|
||||
*allowed_prefetches_num -= part_stat.required_readers_num;
|
||||
}
|
||||
}
|
||||
|
||||
auto thread_task = std::make_unique<ThreadTask>(per_part_infos[part_idx], ranges_to_get_from_part, priority);
|
||||
if (allow_prefetch)
|
||||
{
|
||||
prefetch_queue.emplace(TaskHolder(read_task.get(), i));
|
||||
}
|
||||
++priority.value;
|
||||
prefetch_queue.emplace(TaskHolder{thread_task.get(), i});
|
||||
|
||||
result_threads_tasks[i].push_back(std::move(read_task));
|
||||
per_thread_tasks[i].push_back(std::move(thread_task));
|
||||
|
||||
++priority.value;
|
||||
++total_tasks;
|
||||
}
|
||||
}
|
||||
|
||||
LOG_TEST(log, "Result tasks {} for {} threads: {}", total_tasks, threads, dumpTasks(result_threads_tasks));
|
||||
|
||||
return result_threads_tasks;
|
||||
LOG_TEST(log, "Result tasks {} for {} threads: {}", total_tasks, threads, dumpTasks(per_thread_tasks));
|
||||
}
|
||||
|
||||
std::string MergeTreePrefetchedReadPool::dumpTasks(const ThreadsTasks & tasks)
|
||||
std::string MergeTreePrefetchedReadPool::dumpTasks(const TasksPerThread & tasks)
|
||||
{
|
||||
WriteBufferFromOwnString result;
|
||||
for (const auto & [thread_id, thread_tasks] : tasks)
|
||||
@ -611,9 +557,9 @@ std::string MergeTreePrefetchedReadPool::dumpTasks(const ThreadsTasks & tasks)
|
||||
{
|
||||
result << '\t';
|
||||
result << ++no << ": ";
|
||||
result << "reader: " << task->reader.valid() << ", ";
|
||||
result << "part: " << task->data_part->name << ", ";
|
||||
result << "ranges: " << toString(task->mark_ranges);
|
||||
result << "reader future: " << task->readers_future.valid() << ", ";
|
||||
result << "part: " << task->read_info->data_part->name << ", ";
|
||||
result << "ranges: " << toString(task->ranges);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user