mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 18:12:02 +00:00
Merge remote-tracking branch 'blessed/master' into 02294_decimal_second_errors
This commit is contained in:
commit
a9f6dfa74d
@ -249,10 +249,13 @@ stop_logs_replication
|
||||
successfuly_saved=0
|
||||
for table in query_log zookeeper_log trace_log transactions_info_log
|
||||
do
|
||||
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst || successfuly_saved=$((successfuly_saved+$?))
|
||||
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst
|
||||
successfuly_saved=$?
|
||||
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst || successfuly_saved=$((successfuly_saved+$?))
|
||||
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst || successfuly_saved=$((successfuly_saved+$?))
|
||||
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst
|
||||
successfuly_saved=$((successfuly_saved | $?))
|
||||
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst
|
||||
successfuly_saved=$((successfuly_saved | $?))
|
||||
fi
|
||||
done
|
||||
|
||||
|
@ -11,6 +11,7 @@ sidebar_label: 2023
|
||||
* Remove the `status_info` configuration option and dictionaries status from the default Prometheus handler. [#54090](https://github.com/ClickHouse/ClickHouse/pull/54090) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* The experimental parts metadata cache is removed from the codebase. [#54215](https://github.com/ClickHouse/ClickHouse/pull/54215) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Disable setting `input_format_json_try_infer_numbers_from_strings` by default, so we don't try to infer numbers from strings in JSON formats by default to avoid possible parsing errors when sample data contains strings that looks like a number. [#55099](https://github.com/ClickHouse/ClickHouse/pull/55099) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* IPv6 bloom filter indexes created prior to March 2023 are not compatible with current version and have to be rebuilt. [#54200](https://github.com/ClickHouse/ClickHouse/pull/54200) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
|
||||
|
||||
#### New Feature
|
||||
* Added new type of authentication based on SSH keys. It works only for Native TCP protocol. [#41109](https://github.com/ClickHouse/ClickHouse/pull/41109) ([George Gamezardashvili](https://github.com/InfJoker)).
|
||||
|
@ -39,8 +39,8 @@ If you need to update rows frequently, we recommend using the [`ReplacingMergeTr
|
||||
``` sql
|
||||
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
(
|
||||
name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY],
|
||||
name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY],
|
||||
name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY] [SETTINGS (name = value, ...)],
|
||||
name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY] [SETTINGS (name = value, ...)],
|
||||
...
|
||||
INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1],
|
||||
INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2],
|
||||
@ -56,7 +56,7 @@ ORDER BY expr
|
||||
[DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ]
|
||||
[WHERE conditions]
|
||||
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ]
|
||||
[SETTINGS name=value, ...]
|
||||
[SETTINGS name = value, ...]
|
||||
```
|
||||
|
||||
For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md).
|
||||
@ -620,7 +620,7 @@ The `TTL` clause can’t be used for key columns.
|
||||
#### Creating a table with `TTL`:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE example_table
|
||||
CREATE TABLE tab
|
||||
(
|
||||
d DateTime,
|
||||
a Int TTL d + INTERVAL 1 MONTH,
|
||||
@ -635,7 +635,7 @@ ORDER BY d;
|
||||
#### Adding TTL to a column of an existing table
|
||||
|
||||
``` sql
|
||||
ALTER TABLE example_table
|
||||
ALTER TABLE tab
|
||||
MODIFY COLUMN
|
||||
c String TTL d + INTERVAL 1 DAY;
|
||||
```
|
||||
@ -643,7 +643,7 @@ ALTER TABLE example_table
|
||||
#### Altering TTL of the column
|
||||
|
||||
``` sql
|
||||
ALTER TABLE example_table
|
||||
ALTER TABLE tab
|
||||
MODIFY COLUMN
|
||||
c String TTL d + INTERVAL 1 MONTH;
|
||||
```
|
||||
@ -681,7 +681,7 @@ If a column is not part of the `GROUP BY` expression and is not set explicitly i
|
||||
#### Creating a table with `TTL`:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE example_table
|
||||
CREATE TABLE tab
|
||||
(
|
||||
d DateTime,
|
||||
a Int
|
||||
@ -697,7 +697,7 @@ TTL d + INTERVAL 1 MONTH DELETE,
|
||||
#### Altering `TTL` of the table:
|
||||
|
||||
``` sql
|
||||
ALTER TABLE example_table
|
||||
ALTER TABLE tab
|
||||
MODIFY TTL d + INTERVAL 1 DAY;
|
||||
```
|
||||
|
||||
@ -1366,7 +1366,7 @@ In this sample configuration:
|
||||
The statistic declaration is in the columns section of the `CREATE` query for tables from the `*MergeTree*` Family when we enable `set allow_experimental_statistic = 1`.
|
||||
|
||||
``` sql
|
||||
CREATE TABLE example_table
|
||||
CREATE TABLE tab
|
||||
(
|
||||
a Int64 STATISTIC(tdigest),
|
||||
b Float64
|
||||
@ -1378,8 +1378,8 @@ ORDER BY a
|
||||
We can also manipulate statistics with `ALTER` statements.
|
||||
|
||||
```sql
|
||||
ALTER TABLE example_table ADD STATISTIC b TYPE tdigest;
|
||||
ALTER TABLE example_table DROP STATISTIC a TYPE tdigest;
|
||||
ALTER TABLE tab ADD STATISTIC b TYPE tdigest;
|
||||
ALTER TABLE tab DROP STATISTIC a TYPE tdigest;
|
||||
```
|
||||
|
||||
These lightweight statistics aggregate information about distribution of values in columns.
|
||||
@ -1390,3 +1390,42 @@ They can be used for query optimization when we enable `set allow_statistic_opti
|
||||
- `tdigest`
|
||||
|
||||
Stores distribution of values from numeric columns in [TDigest](https://github.com/tdunning/t-digest) sketch.
|
||||
|
||||
## Column-level Settings {#column-level-settings}
|
||||
|
||||
Certain MergeTree settings can be override at column level:
|
||||
|
||||
- `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table.
|
||||
- `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark.
|
||||
|
||||
Example:
|
||||
|
||||
```sql
|
||||
CREATE TABLE tab
|
||||
(
|
||||
id Int64,
|
||||
document String SETTINGS (min_compress_block_size = 16777216, max_compress_block_size = 16777216)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
```
|
||||
|
||||
Column-level settings can be modified or removed using [ALTER MODIFY COLUMN](/docs/en/sql-reference/statements/alter/column.md), for example:
|
||||
|
||||
- Remove `SETTINGS` from column declaration:
|
||||
|
||||
```sql
|
||||
ALTER TABLE tab MODIFY COLUMN document REMOVE SETTINGS;
|
||||
```
|
||||
|
||||
- Modify a setting:
|
||||
|
||||
```sql
|
||||
ALTER TABLE tab MODIFY COLUMN document MODIFY SETTING min_compress_block_size = 8192;
|
||||
```
|
||||
|
||||
- Reset one or more settings, also removes the setting declaration in the column expression of the table's CREATE query.
|
||||
|
||||
```sql
|
||||
ALTER TABLE tab MODIFY COLUMN document RESET SETTING min_compress_block_size;
|
||||
```
|
||||
|
@ -10,7 +10,7 @@ Columns:
|
||||
- `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query.
|
||||
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date.
|
||||
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time.
|
||||
- `name` ([String](../../sql-reference/data-types/string.md)) — Metric name.
|
||||
- `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name.
|
||||
- `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value.
|
||||
|
||||
**Example**
|
||||
|
@ -23,10 +23,11 @@ The following actions are supported:
|
||||
- [RENAME COLUMN](#rename-column) — Renames an existing column.
|
||||
- [CLEAR COLUMN](#clear-column) — Resets column values.
|
||||
- [COMMENT COLUMN](#comment-column) — Adds a text comment to the column.
|
||||
- [MODIFY COLUMN](#modify-column) — Changes column’s type, default expression and TTL.
|
||||
- [MODIFY COLUMN](#modify-column) — Changes column’s type, default expression, TTL, and column settings.
|
||||
- [MODIFY COLUMN REMOVE](#modify-column-remove) — Removes one of the column properties.
|
||||
- [MODIFY COLUMN MODIFY SETTING](#modify-column-modify-setting) - Changes column settings.
|
||||
- [MODIFY COLUMN RESET SETTING](#modify-column-reset-setting) - Reset column settings.
|
||||
- [MATERIALIZE COLUMN](#materialize-column) — Materializes the column in the parts where the column is missing.
|
||||
|
||||
These actions are described in detail below.
|
||||
|
||||
## ADD COLUMN
|
||||
@ -75,7 +76,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified,
|
||||
|
||||
Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly.
|
||||
|
||||
:::tip
|
||||
:::tip
|
||||
You can’t delete a column if it is referenced by [materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error.
|
||||
:::
|
||||
|
||||
@ -208,7 +209,7 @@ The `ALTER` query for changing columns is replicated. The instructions are saved
|
||||
|
||||
## MODIFY COLUMN REMOVE
|
||||
|
||||
Removes one of the column properties: `DEFAULT`, `ALIAS`, `MATERIALIZED`, `CODEC`, `COMMENT`, `TTL`.
|
||||
Removes one of the column properties: `DEFAULT`, `ALIAS`, `MATERIALIZED`, `CODEC`, `COMMENT`, `TTL`, `SETTING`.
|
||||
|
||||
Syntax:
|
||||
|
||||
@ -228,6 +229,43 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL;
|
||||
|
||||
- [REMOVE TTL](ttl.md).
|
||||
|
||||
|
||||
## MODIFY COLUMN MODIFY SETTING
|
||||
|
||||
Modify a column setting.
|
||||
|
||||
Syntax:
|
||||
|
||||
```sql
|
||||
ALTER TABLE table_name MODIFY COLUMN MODIFY SETTING name=value,...;
|
||||
```
|
||||
|
||||
**Example**
|
||||
|
||||
Modify column's `max_compress_block_size` to `1MB`:
|
||||
|
||||
```sql
|
||||
ALTER TABLE table_name MODIFY COLUMN MODIFY SETTING max_compress_block_size = 1048576;
|
||||
```
|
||||
|
||||
## MODIFY COLUMN RESET SETTING
|
||||
|
||||
Reset a column setting, also removes the setting declaration in the column expression of the table's CREATE query.
|
||||
|
||||
Syntax:
|
||||
|
||||
```sql
|
||||
ALTER TABLE table_name MODIFY COLUMN RESET SETTING name,...;
|
||||
```
|
||||
|
||||
**Example**
|
||||
|
||||
Remove column setting `max_compress_block_size` to `1MB`:
|
||||
|
||||
```sql
|
||||
ALTER TABLE table_name MODIFY COLUMN REMOVE SETTING max_compress_block_size;
|
||||
```
|
||||
|
||||
## MATERIALIZE COLUMN
|
||||
|
||||
Materializes or updates a column with an expression for a default value (`DEFAULT` or `MATERIALIZED`).
|
||||
|
@ -112,7 +112,7 @@ Note that:
|
||||
For the query to run successfully, the following conditions must be met:
|
||||
|
||||
- Both tables must have the same structure.
|
||||
- Both tables must have the same order by key and the same primary key.
|
||||
- Both tables must have the same partition key, the same order by key and the same primary key.
|
||||
- Both tables must have the same indices and projections.
|
||||
- Both tables must have the same storage policy.
|
||||
|
||||
|
@ -1900,6 +1900,7 @@ try
|
||||
|
||||
/// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread.
|
||||
async_metrics.start();
|
||||
global_context->setAsynchronousMetrics(&async_metrics);
|
||||
|
||||
main_config_reloader->start();
|
||||
access_control.startPeriodicReloading();
|
||||
|
@ -170,6 +170,7 @@ enum class AccessType
|
||||
M(SYSTEM_RELOAD_MODEL, "SYSTEM RELOAD MODELS, RELOAD MODEL, RELOAD MODELS", GLOBAL, SYSTEM_RELOAD) \
|
||||
M(SYSTEM_RELOAD_FUNCTION, "SYSTEM RELOAD FUNCTIONS, RELOAD FUNCTION, RELOAD FUNCTIONS", GLOBAL, SYSTEM_RELOAD) \
|
||||
M(SYSTEM_RELOAD_EMBEDDED_DICTIONARIES, "RELOAD EMBEDDED DICTIONARIES", GLOBAL, SYSTEM_RELOAD) /* implicitly enabled by the grant SYSTEM_RELOAD_DICTIONARY ON *.* */\
|
||||
M(SYSTEM_RELOAD_ASYNCHRONOUS_METRICS, "RELOAD ASYNCHRONOUS METRICS", GLOBAL, SYSTEM_RELOAD) \
|
||||
M(SYSTEM_RELOAD, "", GROUP, SYSTEM) \
|
||||
M(SYSTEM_RESTART_DISK, "SYSTEM RESTART DISK", GLOBAL, SYSTEM) \
|
||||
M(SYSTEM_MERGES, "SYSTEM STOP MERGES, SYSTEM START MERGES, STOP MERGES, START MERGES", TABLE, SYSTEM) \
|
||||
|
@ -10,14 +10,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Rewrites `sum(column +/- literal)` into two individual functions
|
||||
* `sum(column)` and `literal * count(column)`.
|
||||
* sum(column + literal) -> sum(column) + literal * count(column)
|
||||
* sum(literal + column) -> literal * count(column) + sum(column)
|
||||
* sum(column - literal) -> sum(column) - literal * count(column)
|
||||
* sum(literal - column) -> literal * count(column) - sum(column)
|
||||
*/
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -29,6 +21,9 @@ public:
|
||||
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions)
|
||||
return;
|
||||
|
||||
static const std::unordered_set<String> func_supported = {
|
||||
"plus",
|
||||
"minus"
|
||||
|
@ -5,6 +5,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/**
|
||||
* Rewrites `sum(column +/- literal)` into two individual functions
|
||||
* `sum(column)` and `literal * count(column)`.
|
||||
* sum(column + literal) -> sum(column) + literal * count(column)
|
||||
* sum(literal + column) -> literal * count(column) + sum(column)
|
||||
* sum(column - literal) -> sum(column) - literal * count(column)
|
||||
* sum(literal - column) -> literal * count(column) - sum(column)
|
||||
*/
|
||||
class RewriteSumFunctionWithSumAndCountPass final : public IQueryTreePass
|
||||
{
|
||||
public:
|
||||
|
@ -98,7 +98,7 @@ AsynchronousMetrics::AsynchronousMetrics(
|
||||
}
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
void AsynchronousMetrics::openSensors()
|
||||
void AsynchronousMetrics::openSensors() TSA_REQUIRES(data_mutex)
|
||||
{
|
||||
LOG_TRACE(log, "Scanning /sys/class/thermal");
|
||||
|
||||
@ -136,7 +136,7 @@ void AsynchronousMetrics::openSensors()
|
||||
}
|
||||
}
|
||||
|
||||
void AsynchronousMetrics::openBlockDevices()
|
||||
void AsynchronousMetrics::openBlockDevices() TSA_REQUIRES(data_mutex)
|
||||
{
|
||||
LOG_TRACE(log, "Scanning /sys/block");
|
||||
|
||||
@ -163,7 +163,7 @@ void AsynchronousMetrics::openBlockDevices()
|
||||
}
|
||||
}
|
||||
|
||||
void AsynchronousMetrics::openEDAC()
|
||||
void AsynchronousMetrics::openEDAC() TSA_REQUIRES(data_mutex)
|
||||
{
|
||||
LOG_TRACE(log, "Scanning /sys/devices/system/edac");
|
||||
|
||||
@ -194,7 +194,7 @@ void AsynchronousMetrics::openEDAC()
|
||||
}
|
||||
}
|
||||
|
||||
void AsynchronousMetrics::openSensorsChips()
|
||||
void AsynchronousMetrics::openSensorsChips() TSA_REQUIRES(data_mutex)
|
||||
{
|
||||
LOG_TRACE(log, "Scanning /sys/class/hwmon");
|
||||
|
||||
@ -281,7 +281,7 @@ void AsynchronousMetrics::stop()
|
||||
try
|
||||
{
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
std::lock_guard lock(thread_mutex);
|
||||
quit = true;
|
||||
}
|
||||
|
||||
@ -306,11 +306,14 @@ AsynchronousMetrics::~AsynchronousMetrics()
|
||||
|
||||
AsynchronousMetricValues AsynchronousMetrics::getValues() const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
std::lock_guard lock(data_mutex);
|
||||
return values;
|
||||
}
|
||||
|
||||
static auto get_next_update_time(std::chrono::seconds update_period)
|
||||
namespace
|
||||
{
|
||||
|
||||
auto get_next_update_time(std::chrono::seconds update_period)
|
||||
{
|
||||
using namespace std::chrono;
|
||||
|
||||
@ -334,6 +337,8 @@ static auto get_next_update_time(std::chrono::seconds update_period)
|
||||
return time_next;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void AsynchronousMetrics::run()
|
||||
{
|
||||
setThreadName("AsyncMetrics");
|
||||
@ -344,9 +349,9 @@ void AsynchronousMetrics::run()
|
||||
|
||||
{
|
||||
// Wait first, so that the first metric collection is also on even time.
|
||||
std::unique_lock lock{mutex};
|
||||
std::unique_lock lock(thread_mutex);
|
||||
if (wait_cond.wait_until(lock, next_update_time,
|
||||
[this] { return quit; }))
|
||||
[this] TSA_REQUIRES(thread_mutex) { return quit; }))
|
||||
{
|
||||
break;
|
||||
}
|
||||
@ -364,6 +369,9 @@ void AsynchronousMetrics::run()
|
||||
}
|
||||
|
||||
#if USE_JEMALLOC
|
||||
namespace
|
||||
{
|
||||
|
||||
uint64_t updateJemallocEpoch()
|
||||
{
|
||||
uint64_t value = 0;
|
||||
@ -373,7 +381,7 @@ uint64_t updateJemallocEpoch()
|
||||
}
|
||||
|
||||
template <typename Value>
|
||||
static Value saveJemallocMetricImpl(
|
||||
Value saveJemallocMetricImpl(
|
||||
AsynchronousMetricValues & values,
|
||||
const std::string & jemalloc_full_name,
|
||||
const std::string & clickhouse_full_name)
|
||||
@ -386,7 +394,7 @@ static Value saveJemallocMetricImpl(
|
||||
}
|
||||
|
||||
template<typename Value>
|
||||
static Value saveJemallocMetric(AsynchronousMetricValues & values,
|
||||
Value saveJemallocMetric(AsynchronousMetricValues & values,
|
||||
const std::string & metric_name)
|
||||
{
|
||||
return saveJemallocMetricImpl<Value>(values,
|
||||
@ -395,13 +403,15 @@ static Value saveJemallocMetric(AsynchronousMetricValues & values,
|
||||
}
|
||||
|
||||
template<typename Value>
|
||||
static Value saveAllArenasMetric(AsynchronousMetricValues & values,
|
||||
Value saveAllArenasMetric(AsynchronousMetricValues & values,
|
||||
const std::string & metric_name)
|
||||
{
|
||||
return saveJemallocMetricImpl<Value>(values,
|
||||
fmt::format("stats.arenas.{}.{}", MALLCTL_ARENAS_ALL, metric_name),
|
||||
fmt::format("jemalloc.arenas.all.{}", metric_name));
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
||||
|
||||
|
||||
@ -547,21 +557,23 @@ AsynchronousMetrics::NetworkInterfaceStatValues::operator-(const AsynchronousMet
|
||||
#endif
|
||||
|
||||
|
||||
void AsynchronousMetrics::update(TimePoint update_time)
|
||||
void AsynchronousMetrics::update(TimePoint update_time, bool force_update)
|
||||
{
|
||||
Stopwatch watch;
|
||||
|
||||
AsynchronousMetricValues new_values;
|
||||
|
||||
std::lock_guard lock(data_mutex);
|
||||
|
||||
auto current_time = std::chrono::system_clock::now();
|
||||
auto time_after_previous_update = current_time - previous_update_time;
|
||||
auto time_since_previous_update = current_time - previous_update_time;
|
||||
previous_update_time = update_time;
|
||||
|
||||
double update_interval = 0.;
|
||||
if (first_run)
|
||||
update_interval = update_period.count();
|
||||
else
|
||||
update_interval = std::chrono::duration_cast<std::chrono::microseconds>(time_after_previous_update).count() / 1e6;
|
||||
update_interval = std::chrono::duration_cast<std::chrono::microseconds>(time_since_previous_update).count() / 1e6;
|
||||
new_values["AsynchronousMetricsUpdateInterval"] = { update_interval, "Metrics update interval" };
|
||||
|
||||
/// This is also a good indicator of system responsiveness.
|
||||
@ -815,7 +827,7 @@ void AsynchronousMetrics::update(TimePoint update_time)
|
||||
if (-1 == hz)
|
||||
throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ");
|
||||
|
||||
double multiplier = 1.0 / hz / (std::chrono::duration_cast<std::chrono::nanoseconds>(time_after_previous_update).count() / 1e9);
|
||||
double multiplier = 1.0 / hz / (std::chrono::duration_cast<std::chrono::nanoseconds>(time_since_previous_update).count() / 1e9);
|
||||
size_t num_cpus = 0;
|
||||
|
||||
ProcStatValuesOther current_other_values{};
|
||||
@ -1572,7 +1584,7 @@ void AsynchronousMetrics::update(TimePoint update_time)
|
||||
|
||||
/// Add more metrics as you wish.
|
||||
|
||||
updateImpl(new_values, update_time, current_time);
|
||||
updateImpl(update_time, current_time, force_update, first_run, new_values);
|
||||
|
||||
new_values["AsynchronousMetricsCalculationTimeSpent"] = { watch.elapsedSeconds(), "Time in seconds spent for calculation of asynchronous metrics (this is the overhead of asynchronous metrics)." };
|
||||
|
||||
@ -1581,7 +1593,6 @@ void AsynchronousMetrics::update(TimePoint update_time)
|
||||
first_run = false;
|
||||
|
||||
// Finally, update the current metrics.
|
||||
std::lock_guard lock(mutex);
|
||||
values = new_values;
|
||||
}
|
||||
|
||||
|
@ -56,8 +56,13 @@ struct ProtocolServerMetrics
|
||||
*/
|
||||
class AsynchronousMetrics
|
||||
{
|
||||
protected:
|
||||
using Duration = std::chrono::seconds;
|
||||
using TimePoint = std::chrono::system_clock::time_point;
|
||||
|
||||
public:
|
||||
using ProtocolServerMetricsFunc = std::function<std::vector<ProtocolServerMetrics>()>;
|
||||
|
||||
AsynchronousMetrics(
|
||||
int update_period_seconds,
|
||||
const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
|
||||
@ -69,62 +74,66 @@ public:
|
||||
|
||||
void stop();
|
||||
|
||||
void update(TimePoint update_time, bool force_update = false);
|
||||
|
||||
/// Returns copy of all values.
|
||||
AsynchronousMetricValues getValues() const;
|
||||
|
||||
protected:
|
||||
using Duration = std::chrono::seconds;
|
||||
using TimePoint = std::chrono::system_clock::time_point;
|
||||
|
||||
const Duration update_period;
|
||||
|
||||
/// Some values are incremental and we have to calculate the difference.
|
||||
/// On first run we will only collect the values to subtract later.
|
||||
bool first_run = true;
|
||||
TimePoint previous_update_time;
|
||||
|
||||
Poco::Logger * log;
|
||||
private:
|
||||
virtual void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) = 0;
|
||||
virtual void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) = 0;
|
||||
virtual void logImpl(AsynchronousMetricValues &) {}
|
||||
|
||||
ProtocolServerMetricsFunc protocol_server_metrics_func;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
std::unique_ptr<ThreadFromGlobalPool> thread;
|
||||
|
||||
mutable std::mutex thread_mutex;
|
||||
std::condition_variable wait_cond;
|
||||
bool quit {false};
|
||||
AsynchronousMetricValues values;
|
||||
bool quit TSA_GUARDED_BY(thread_mutex) = false;
|
||||
|
||||
mutable std::mutex data_mutex;
|
||||
|
||||
/// Some values are incremental and we have to calculate the difference.
|
||||
/// On first run we will only collect the values to subtract later.
|
||||
bool first_run TSA_GUARDED_BY(data_mutex) = true;
|
||||
TimePoint previous_update_time TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
AsynchronousMetricValues values TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
#if defined(OS_LINUX) || defined(OS_FREEBSD)
|
||||
MemoryStatisticsOS memory_stat;
|
||||
MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex);
|
||||
#endif
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
std::optional<ReadBufferFromFilePRead> meminfo;
|
||||
std::optional<ReadBufferFromFilePRead> loadavg;
|
||||
std::optional<ReadBufferFromFilePRead> proc_stat;
|
||||
std::optional<ReadBufferFromFilePRead> cpuinfo;
|
||||
std::optional<ReadBufferFromFilePRead> file_nr;
|
||||
std::optional<ReadBufferFromFilePRead> uptime;
|
||||
std::optional<ReadBufferFromFilePRead> net_dev;
|
||||
std::optional<ReadBufferFromFilePRead> meminfo TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> loadavg TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> proc_stat TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> cpuinfo TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> file_nr TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> uptime TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> net_dev TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
std::optional<ReadBufferFromFilePRead> cgroupmem_limit_in_bytes;
|
||||
std::optional<ReadBufferFromFilePRead> cgroupmem_usage_in_bytes;
|
||||
std::optional<ReadBufferFromFilePRead> cgroupcpu_cfs_period;
|
||||
std::optional<ReadBufferFromFilePRead> cgroupcpu_cfs_quota;
|
||||
std::optional<ReadBufferFromFilePRead> cgroupcpu_max;
|
||||
std::optional<ReadBufferFromFilePRead> cgroupmem_limit_in_bytes TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> cgroupmem_usage_in_bytes TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> cgroupcpu_cfs_period TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> cgroupcpu_cfs_quota TSA_GUARDED_BY(data_mutex);
|
||||
std::optional<ReadBufferFromFilePRead> cgroupcpu_max TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
std::vector<std::unique_ptr<ReadBufferFromFilePRead>> thermal;
|
||||
std::vector<std::unique_ptr<ReadBufferFromFilePRead>> thermal TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
std::unordered_map<String /* device name */,
|
||||
std::unordered_map<String /* label name */,
|
||||
std::unique_ptr<ReadBufferFromFilePRead>>> hwmon_devices;
|
||||
std::unique_ptr<ReadBufferFromFilePRead>>> hwmon_devices TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
std::vector<std::pair<
|
||||
std::unique_ptr<ReadBufferFromFilePRead> /* correctable errors */,
|
||||
std::unique_ptr<ReadBufferFromFilePRead> /* uncorrectable errors */>> edac;
|
||||
std::unique_ptr<ReadBufferFromFilePRead> /* uncorrectable errors */>> edac TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
std::unordered_map<String /* device name */, std::unique_ptr<ReadBufferFromFilePRead>> block_devs;
|
||||
std::unordered_map<String /* device name */, std::unique_ptr<ReadBufferFromFilePRead>> block_devs TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
/// TODO: socket statistics.
|
||||
|
||||
@ -154,9 +163,9 @@ private:
|
||||
ProcStatValuesOther operator-(const ProcStatValuesOther & other) const;
|
||||
};
|
||||
|
||||
ProcStatValuesCPU proc_stat_values_all_cpus{};
|
||||
ProcStatValuesOther proc_stat_values_other{};
|
||||
std::vector<ProcStatValuesCPU> proc_stat_values_per_cpu;
|
||||
ProcStatValuesCPU proc_stat_values_all_cpus TSA_GUARDED_BY(data_mutex) {};
|
||||
ProcStatValuesOther proc_stat_values_other TSA_GUARDED_BY(data_mutex) {};
|
||||
std::vector<ProcStatValuesCPU> proc_stat_values_per_cpu TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
/// https://www.kernel.org/doc/Documentation/block/stat.txt
|
||||
struct BlockDeviceStatValues
|
||||
@ -181,7 +190,7 @@ private:
|
||||
BlockDeviceStatValues operator-(const BlockDeviceStatValues & other) const;
|
||||
};
|
||||
|
||||
std::unordered_map<String /* device name */, BlockDeviceStatValues> block_device_stats;
|
||||
std::unordered_map<String /* device name */, BlockDeviceStatValues> block_device_stats TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
struct NetworkInterfaceStatValues
|
||||
{
|
||||
@ -197,9 +206,9 @@ private:
|
||||
NetworkInterfaceStatValues operator-(const NetworkInterfaceStatValues & other) const;
|
||||
};
|
||||
|
||||
std::unordered_map<String /* device name */, NetworkInterfaceStatValues> network_interface_stats;
|
||||
std::unordered_map<String /* device name */, NetworkInterfaceStatValues> network_interface_stats TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
Stopwatch block_devices_rescan_delay;
|
||||
Stopwatch block_devices_rescan_delay TSA_GUARDED_BY(data_mutex);
|
||||
|
||||
void openSensors();
|
||||
void openBlockDevices();
|
||||
@ -207,10 +216,7 @@ private:
|
||||
void openEDAC();
|
||||
#endif
|
||||
|
||||
std::unique_ptr<ThreadFromGlobalPool> thread;
|
||||
|
||||
void run();
|
||||
void update(TimePoint update_time);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -121,7 +121,7 @@ KeeperAsynchronousMetrics::~KeeperAsynchronousMetrics()
|
||||
stop();
|
||||
}
|
||||
|
||||
void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint /*update_time*/, TimePoint /*current_time*/)
|
||||
void KeeperAsynchronousMetrics::updateImpl(TimePoint /*update_time*/, TimePoint /*current_time*/, bool /*force_update*/, bool /*first_run*/, AsynchronousMetricValues & new_values)
|
||||
{
|
||||
#if USE_NURAFT
|
||||
{
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
private:
|
||||
ContextPtr context;
|
||||
|
||||
void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override;
|
||||
void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) override;
|
||||
};
|
||||
|
||||
|
||||
|
@ -208,6 +208,11 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server
|
||||
return sm_commit_exec_in_progress_;
|
||||
}
|
||||
|
||||
void setServingRequest(bool value)
|
||||
{
|
||||
serving_req_ = value;
|
||||
}
|
||||
|
||||
using nuraft::raft_server::raft_server;
|
||||
|
||||
// peers are initially marked as responding because at least one cycle
|
||||
@ -687,6 +692,14 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
if (req.get_type() != nuraft::msg_type::append_entries_request)
|
||||
break;
|
||||
|
||||
if (req.log_entries().empty())
|
||||
break;
|
||||
|
||||
/// committing/preprocessing of local logs can take some time
|
||||
/// and we don't want election to start during that time so we
|
||||
/// set serving requests to avoid elections on timeout
|
||||
raft_instance->setServingRequest(true);
|
||||
SCOPE_EXIT(raft_instance->setServingRequest(false));
|
||||
/// maybe we got snapshot installed
|
||||
if (state_machine->last_commit_index() >= last_log_idx_on_disk && !raft_instance->isCommitInProgress())
|
||||
preprocess_logs();
|
||||
|
@ -97,7 +97,8 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
|
||||
{"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"},
|
||||
{"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"},
|
||||
{"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"},
|
||||
{"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}}},
|
||||
{"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"},
|
||||
{"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}}},
|
||||
{"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."},
|
||||
{"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"},
|
||||
{"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"},
|
||||
|
@ -4,9 +4,9 @@
|
||||
#include <Disks/DiskFactory.h>
|
||||
#include <IO/FileEncryptionCommon.h>
|
||||
#include <IO/ReadBufferFromEncryptedFile.h>
|
||||
#include <IO/ReadBufferFromFileDecorator.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferFromEncryptedFile.h>
|
||||
#include <IO/ReadBufferFromEmptyFile.h>
|
||||
#include <boost/algorithm/hex.hpp>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -374,7 +374,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
|
||||
{
|
||||
/// File is empty, that's a normal case, see DiskEncrypted::truncateFile().
|
||||
/// There is no header so we just return `ReadBufferFromString("")`.
|
||||
return std::make_unique<ReadBufferFromFileDecorator>(std::make_unique<ReadBufferFromString>(std::string_view{}), wrapped_path);
|
||||
return std::make_unique<ReadBufferFromEmptyFile>(wrapped_path);
|
||||
}
|
||||
auto encryption_settings = current_settings.get();
|
||||
FileEncryption::Header header = readHeader(*buffer);
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <boost/algorithm/hex.hpp>
|
||||
#include <IO/ReadBufferFromEncryptedFile.h>
|
||||
#include <IO/ReadBufferFromFileDecorator.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferFromEncryptedFile.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
@ -560,8 +560,9 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment)
|
||||
ProfileEvents::FileSegmentPredownloadMicroseconds, predownload_watch.elapsedMicroseconds());
|
||||
});
|
||||
|
||||
OpenTelemetry::SpanHolder span{
|
||||
fmt::format("CachedOnDiskReadBufferFromFile::predownload(key={}, size={})", file_segment.key().toString(), bytes_to_predownload)};
|
||||
OpenTelemetry::SpanHolder span("CachedOnDiskReadBufferFromFile::predownload");
|
||||
span.addAttribute("clickhouse.key", file_segment.key().toString());
|
||||
span.addAttribute("clickhouse.size", bytes_to_predownload);
|
||||
|
||||
if (bytes_to_predownload)
|
||||
{
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "ReadBufferFromRemoteFSGather.h"
|
||||
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
|
||||
#include <Disks/IO/CachedOnDiskReadBufferFromFile.h>
|
||||
#include <Disks/ObjectStorages/Cached/CachedObjectStorage.h>
|
||||
@ -61,7 +61,7 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(
|
||||
current_object = blobs_to_read.front();
|
||||
}
|
||||
|
||||
SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object)
|
||||
std::unique_ptr<ReadBufferFromFileBase> ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object)
|
||||
{
|
||||
if (current_buf && !with_cache)
|
||||
{
|
||||
@ -78,7 +78,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
|
||||
if (with_cache)
|
||||
{
|
||||
auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path);
|
||||
return std::make_shared<CachedOnDiskReadBufferFromFile>(
|
||||
return std::make_unique<CachedOnDiskReadBufferFromFile>(
|
||||
object_path,
|
||||
cache_key,
|
||||
settings.remote_fs_cache,
|
||||
|
@ -53,7 +53,7 @@ public:
|
||||
bool isContentCached(size_t offset, size_t size) override;
|
||||
|
||||
private:
|
||||
SeekableReadBufferPtr createImplementationBuffer(const StoredObject & object);
|
||||
std::unique_ptr<ReadBufferFromFileBase> createImplementationBuffer(const StoredObject & object);
|
||||
|
||||
bool nextImpl() override;
|
||||
|
||||
@ -80,7 +80,7 @@ private:
|
||||
|
||||
StoredObject current_object;
|
||||
size_t current_buf_idx = 0;
|
||||
SeekableReadBufferPtr current_buf;
|
||||
std::unique_ptr<ReadBufferFromFileBase> current_buf;
|
||||
|
||||
Poco::Logger * log;
|
||||
};
|
||||
|
@ -39,7 +39,7 @@ std::unique_ptr<ReadBufferFromFileBase> createReadBufferFromFileBase(
|
||||
size_t alignment)
|
||||
{
|
||||
if (file_size.has_value() && !*file_size)
|
||||
return std::make_unique<ReadBufferFromEmptyFile>();
|
||||
return std::make_unique<ReadBufferFromEmptyFile>(filename);
|
||||
|
||||
size_t estimated_size = 0;
|
||||
if (read_hint.has_value())
|
||||
|
@ -531,7 +531,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
|
||||
const bool file_can_be_empty = !file_size.has_value() || *file_size == 0;
|
||||
|
||||
if (storage_objects.empty() && file_can_be_empty)
|
||||
return std::make_unique<ReadBufferFromEmptyFile>();
|
||||
return std::make_unique<ReadBufferFromEmptyFile>(path);
|
||||
|
||||
return object_storage->readObjects(
|
||||
storage_objects,
|
||||
|
@ -228,7 +228,12 @@ public:
|
||||
|
||||
off_t getPosition() override
|
||||
{
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive");
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition is not supported when reading from archive");
|
||||
}
|
||||
|
||||
size_t getFileOffsetOfBufferEnd() const override
|
||||
{
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getFileOffsetOfBufferEnd is not supported when reading from archive");
|
||||
}
|
||||
|
||||
String getFileName() const override { return handle.getFileName(); }
|
||||
|
@ -15,6 +15,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_UNPACK_ARCHIVE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int SEEK_POSITION_OUT_OF_BOUND;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int CANNOT_SEEK_THROUGH_FILE;
|
||||
}
|
||||
|
||||
@ -252,6 +253,11 @@ public:
|
||||
checkResult(err);
|
||||
}
|
||||
|
||||
size_t getFileOffsetOfBufferEnd() const override
|
||||
{
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getFileOffsetOfBufferEnd is not supported when reading from zip archive");
|
||||
}
|
||||
|
||||
off_t seek(off_t off, int whence) override
|
||||
{
|
||||
off_t current_pos = getPosition();
|
||||
|
@ -4,8 +4,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
BoundedReadBuffer::BoundedReadBuffer(std::unique_ptr<SeekableReadBuffer> impl_)
|
||||
: ReadBufferFromFileDecorator(std::move(impl_))
|
||||
BoundedReadBuffer::BoundedReadBuffer(std::unique_ptr<ReadBufferFromFileBase> impl_) : impl(std::move(impl_))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <IO/ReadBufferFromFileDecorator.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -7,10 +7,10 @@ namespace DB
|
||||
|
||||
/// A buffer which allows to make an underlying buffer as right bounded,
|
||||
/// e.g. the buffer cannot return data beyond offset specified in `setReadUntilPosition`.
|
||||
class BoundedReadBuffer : public ReadBufferFromFileDecorator
|
||||
class BoundedReadBuffer : public ReadBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
explicit BoundedReadBuffer(std::unique_ptr<SeekableReadBuffer> impl_);
|
||||
explicit BoundedReadBuffer(std::unique_ptr<ReadBufferFromFileBase> impl_);
|
||||
|
||||
bool supportsRightBoundedReads() const override { return true; }
|
||||
|
||||
@ -23,6 +23,8 @@ public:
|
||||
off_t seek(off_t off, int whence) override;
|
||||
|
||||
size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; }
|
||||
String getFileName() const override { return impl->getFileName(); }
|
||||
size_t getFileSize() override { return impl->getFileSize(); }
|
||||
|
||||
/// file_offset_of_buffer_end can differ from impl's file_offset_of_buffer_end
|
||||
/// because of resizing of the tail. => Need to also override getPosition() as
|
||||
@ -30,6 +32,8 @@ public:
|
||||
off_t getPosition() override;
|
||||
|
||||
private:
|
||||
std::unique_ptr<ReadBufferFromFileBase> impl;
|
||||
|
||||
std::optional<size_t> read_until_position;
|
||||
/// atomic because can be used in log or exception messages while being updated.
|
||||
std::atomic<size_t> file_offset_of_buffer_end = 0;
|
||||
|
@ -18,7 +18,6 @@ public:
|
||||
|
||||
/// Returns adjusted position, i.e. returns `3` if the position in the nested buffer is `start_offset + 3`.
|
||||
off_t getPosition() override;
|
||||
|
||||
off_t seek(off_t off, int whence) override;
|
||||
|
||||
private:
|
||||
|
@ -92,6 +92,11 @@ size_t MMapReadBufferFromFileDescriptor::getFileSize()
|
||||
return getSizeFromFileDescriptor(getFD(), getFileName());
|
||||
}
|
||||
|
||||
size_t MMapReadBufferFromFileDescriptor::getFileOffsetOfBufferEnd() const
|
||||
{
|
||||
return mapped.getOffset() + mapped.getLength();
|
||||
}
|
||||
|
||||
size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &)
|
||||
{
|
||||
if (offset >= mapped.getLength())
|
||||
|
@ -36,6 +36,8 @@ public:
|
||||
|
||||
std::string getFileName() const override;
|
||||
|
||||
size_t getFileOffsetOfBufferEnd() const override;
|
||||
|
||||
int getFD() const;
|
||||
|
||||
size_t getFileSize() override;
|
||||
|
@ -76,4 +76,9 @@ off_t MMapReadBufferFromFileWithCache::seek(off_t offset, int whence)
|
||||
return new_pos;
|
||||
}
|
||||
|
||||
size_t MMapReadBufferFromFileWithCache::getFileOffsetOfBufferEnd() const
|
||||
{
|
||||
return mapped->getOffset() + mapped->getLength();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
off_t getPosition() override;
|
||||
std::string getFileName() const override;
|
||||
off_t seek(off_t offset, int whence) override;
|
||||
|
||||
size_t getFileOffsetOfBufferEnd() const override;
|
||||
bool isRegularLocalFile(size_t * /* out_view_offset */) override { return true; }
|
||||
|
||||
private:
|
||||
|
@ -14,12 +14,18 @@ namespace DB
|
||||
/// - ThreadPoolReader
|
||||
class ReadBufferFromEmptyFile : public ReadBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
explicit ReadBufferFromEmptyFile(const String & file_name_) : file_name(file_name_) {}
|
||||
|
||||
private:
|
||||
String file_name;
|
||||
|
||||
bool nextImpl() override { return false; }
|
||||
std::string getFileName() const override { return "<empty>"; }
|
||||
std::string getFileName() const override { return file_name; }
|
||||
off_t seek(off_t /*off*/, int /*whence*/) override { return 0; }
|
||||
off_t getPosition() override { return 0; }
|
||||
size_t getFileSize() override { return 0; }
|
||||
size_t getFileOffsetOfBufferEnd() const override { return 0; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -101,6 +101,18 @@ bool ReadBufferFromEncryptedFile::nextImpl()
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t ReadBufferFromEncryptedFile::getFileSize()
|
||||
{
|
||||
size_t size = in->getFileSize();
|
||||
return size > FileEncryption::Header::kSize ? size - FileEncryption::Header::kSize : size;
|
||||
}
|
||||
|
||||
size_t ReadBufferFromEncryptedFile::getFileOffsetOfBufferEnd() const
|
||||
{
|
||||
size_t file_offset = in->getFileOffsetOfBufferEnd();
|
||||
return file_offset > FileEncryption::Header::kSize ? file_offset - FileEncryption::Header::kSize : file_offset;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -27,10 +27,10 @@ public:
|
||||
std::string getFileName() const override { return in->getFileName(); }
|
||||
|
||||
void setReadUntilPosition(size_t position) override { in->setReadUntilPosition(position + FileEncryption::Header::kSize); }
|
||||
|
||||
void setReadUntilEnd() override { in->setReadUntilEnd(); }
|
||||
|
||||
size_t getFileSize() override { return in->getFileSize(); }
|
||||
size_t getFileSize() override;
|
||||
size_t getFileOffsetOfBufferEnd() const override;
|
||||
|
||||
private:
|
||||
bool nextImpl() override;
|
||||
|
@ -60,6 +60,12 @@ public:
|
||||
/// file offset and what getPosition() returns.
|
||||
virtual bool isRegularLocalFile(size_t * /* out_view_offset */ = nullptr) { return false; }
|
||||
|
||||
/// NOTE: This method should be thread-safe against seek(), since it can be
|
||||
/// used in CachedOnDiskReadBufferFromFile from multiple threads (because
|
||||
/// it first releases the buffer, and then do logging, and so other thread
|
||||
/// can already call seek() which will lead to data-race).
|
||||
virtual size_t getFileOffsetOfBufferEnd() const = 0;
|
||||
|
||||
protected:
|
||||
std::optional<size_t> file_size;
|
||||
ProfileCallback profile_callback;
|
||||
|
@ -1,60 +0,0 @@
|
||||
#include <IO/ReadBufferFromFileDecorator.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ReadBufferFromFileDecorator::ReadBufferFromFileDecorator(std::unique_ptr<SeekableReadBuffer> impl_)
|
||||
: ReadBufferFromFileDecorator(std::move(impl_), "")
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
ReadBufferFromFileDecorator::ReadBufferFromFileDecorator(std::unique_ptr<SeekableReadBuffer> impl_, const String & file_name_)
|
||||
: impl(std::move(impl_)), file_name(file_name_)
|
||||
{
|
||||
swap(*impl);
|
||||
}
|
||||
|
||||
|
||||
std::string ReadBufferFromFileDecorator::getFileName() const
|
||||
{
|
||||
if (!file_name.empty())
|
||||
return file_name;
|
||||
|
||||
return getFileNameFromReadBuffer(*impl);
|
||||
}
|
||||
|
||||
|
||||
off_t ReadBufferFromFileDecorator::getPosition()
|
||||
{
|
||||
swap(*impl);
|
||||
auto position = impl->getPosition();
|
||||
swap(*impl);
|
||||
return position;
|
||||
}
|
||||
|
||||
|
||||
off_t ReadBufferFromFileDecorator::seek(off_t off, int whence)
|
||||
{
|
||||
swap(*impl);
|
||||
auto result = impl->seek(off, whence);
|
||||
swap(*impl);
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
bool ReadBufferFromFileDecorator::nextImpl()
|
||||
{
|
||||
swap(*impl);
|
||||
auto result = impl->next();
|
||||
swap(*impl);
|
||||
return result;
|
||||
}
|
||||
|
||||
size_t ReadBufferFromFileDecorator::getFileSize()
|
||||
{
|
||||
return getFileSizeFromReadBuffer(*impl);
|
||||
}
|
||||
|
||||
}
|
@ -1,37 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Delegates all reads to underlying buffer. Doesn't have own memory.
|
||||
class ReadBufferFromFileDecorator : public ReadBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
explicit ReadBufferFromFileDecorator(std::unique_ptr<SeekableReadBuffer> impl_);
|
||||
ReadBufferFromFileDecorator(std::unique_ptr<SeekableReadBuffer> impl_, const String & file_name_);
|
||||
|
||||
std::string getFileName() const override;
|
||||
|
||||
off_t getPosition() override;
|
||||
|
||||
off_t seek(off_t off, int whence) override;
|
||||
|
||||
bool nextImpl() override;
|
||||
|
||||
bool isWithFileSize() const { return dynamic_cast<const WithFileSize *>(impl.get()) != nullptr; }
|
||||
|
||||
const ReadBuffer & getWrappedReadBuffer() const { return *impl; }
|
||||
|
||||
ReadBuffer & getWrappedReadBuffer() { return *impl; }
|
||||
|
||||
size_t getFileSize() override;
|
||||
|
||||
protected:
|
||||
std::unique_ptr<SeekableReadBuffer> impl;
|
||||
String file_name;
|
||||
};
|
||||
|
||||
}
|
@ -20,7 +20,6 @@ public:
|
||||
: SeekableReadBuffer(const_cast<char *>(str.data()), str.size(), 0) {}
|
||||
|
||||
off_t seek(off_t off, int whence) override;
|
||||
|
||||
off_t getPosition() override;
|
||||
};
|
||||
|
||||
|
@ -44,12 +44,6 @@ public:
|
||||
|
||||
virtual String getInfoForLog() { return ""; }
|
||||
|
||||
/// NOTE: This method should be thread-safe against seek(), since it can be
|
||||
/// used in CachedOnDiskReadBufferFromFile from multiple threads (because
|
||||
/// it first releases the buffer, and then do logging, and so other thread
|
||||
/// can already call seek() which will lead to data-race).
|
||||
virtual size_t getFileOffsetOfBufferEnd() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFileOffsetOfBufferEnd() not implemented"); }
|
||||
|
||||
/// If true, setReadUntilPosition() guarantees that eof will be reported at the given position.
|
||||
virtual bool supportsRightBoundedReads() const { return false; }
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/CompressedReadBufferWrapper.h>
|
||||
#include <IO/ParallelReadBuffer.h>
|
||||
#include <IO/ReadBufferFromFileDecorator.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
|
||||
namespace DB
|
||||
@ -17,23 +16,15 @@ template <typename T>
|
||||
static size_t getFileSize(T & in)
|
||||
{
|
||||
if (auto * with_file_size = dynamic_cast<WithFileSize *>(&in))
|
||||
{
|
||||
return with_file_size->getFileSize();
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size");
|
||||
}
|
||||
|
||||
size_t getFileSizeFromReadBuffer(ReadBuffer & in)
|
||||
{
|
||||
if (auto * delegate = dynamic_cast<ReadBufferFromFileDecorator *>(&in))
|
||||
{
|
||||
return getFileSize(delegate->getWrappedReadBuffer());
|
||||
}
|
||||
else if (auto * compressed = dynamic_cast<CompressedReadBufferWrapper *>(&in))
|
||||
{
|
||||
if (auto * compressed = dynamic_cast<CompressedReadBufferWrapper *>(&in))
|
||||
return getFileSize(compressed->getWrappedReadBuffer());
|
||||
}
|
||||
|
||||
return getFileSize(in);
|
||||
}
|
||||
@ -52,11 +43,7 @@ std::optional<size_t> tryGetFileSizeFromReadBuffer(ReadBuffer & in)
|
||||
|
||||
bool isBufferWithFileSize(const ReadBuffer & in)
|
||||
{
|
||||
if (const auto * delegate = dynamic_cast<const ReadBufferFromFileDecorator *>(&in))
|
||||
{
|
||||
return delegate->isWithFileSize();
|
||||
}
|
||||
else if (const auto * compressed = dynamic_cast<const CompressedReadBufferWrapper *>(&in))
|
||||
if (const auto * compressed = dynamic_cast<const CompressedReadBufferWrapper *>(&in))
|
||||
{
|
||||
return isBufferWithFileSize(compressed->getWrappedReadBuffer());
|
||||
}
|
||||
@ -66,11 +53,7 @@ bool isBufferWithFileSize(const ReadBuffer & in)
|
||||
|
||||
size_t getDataOffsetMaybeCompressed(const ReadBuffer & in)
|
||||
{
|
||||
if (const auto * delegate = dynamic_cast<const ReadBufferFromFileDecorator *>(&in))
|
||||
{
|
||||
return getDataOffsetMaybeCompressed(delegate->getWrappedReadBuffer());
|
||||
}
|
||||
else if (const auto * compressed = dynamic_cast<const CompressedReadBufferWrapper *>(&in))
|
||||
if (const auto * compressed = dynamic_cast<const CompressedReadBufferWrapper *>(&in))
|
||||
{
|
||||
return getDataOffsetMaybeCompressed(compressed->getWrappedReadBuffer());
|
||||
}
|
||||
|
@ -178,6 +178,11 @@ void WriteBufferFromS3::preFinalize()
|
||||
|
||||
void WriteBufferFromS3::finalizeImpl()
|
||||
{
|
||||
OpenTelemetry::SpanHolder span("WriteBufferFromS3::finalizeImpl");
|
||||
span.addAttribute("clickhouse.s3_bucket", bucket);
|
||||
span.addAttribute("clickhouse.s3_key", key);
|
||||
span.addAttribute("clickhouse.total_size", total_size);
|
||||
|
||||
LOG_TRACE(limitedLog, "finalizeImpl WriteBufferFromS3. {}.", getShortLogDetails());
|
||||
|
||||
if (!is_prefinalized)
|
||||
@ -188,6 +193,8 @@ void WriteBufferFromS3::finalizeImpl()
|
||||
|
||||
task_tracker->waitAll();
|
||||
|
||||
span.addAttributeIfNotZero("clickhouse.multipart_upload_parts", multipart_tags.size());
|
||||
|
||||
if (!multipart_upload_id.empty())
|
||||
{
|
||||
completeMultipartUpload();
|
||||
|
@ -424,7 +424,9 @@ void FileSegment::write(const char * from, size_t size, size_t offset)
|
||||
|
||||
FileSegment::State FileSegment::wait(size_t offset)
|
||||
{
|
||||
OpenTelemetry::SpanHolder span{fmt::format("FileSegment::wait({})", key().toString())};
|
||||
OpenTelemetry::SpanHolder span("FileSegment::wait");
|
||||
span.addAttribute("clickhouse.key", key().toString());
|
||||
span.addAttribute("clickhouse.offset", offset);
|
||||
|
||||
auto lock = lockFileSegment();
|
||||
|
||||
|
@ -322,7 +322,6 @@ void executeQuery(
|
||||
|
||||
void executeQueryWithParallelReplicas(
|
||||
QueryPlan & query_plan,
|
||||
const StorageID & main_table,
|
||||
SelectStreamFactory & stream_factory,
|
||||
const ASTPtr & query_ast,
|
||||
ContextPtr context,
|
||||
@ -414,7 +413,6 @@ void executeQueryWithParallelReplicas(
|
||||
std::move(coordinator),
|
||||
stream_factory.header,
|
||||
stream_factory.processed_stage,
|
||||
main_table,
|
||||
new_context,
|
||||
getThrottler(new_context),
|
||||
std::move(scalars),
|
||||
|
@ -70,7 +70,6 @@ void executeQuery(
|
||||
|
||||
void executeQueryWithParallelReplicas(
|
||||
QueryPlan & query_plan,
|
||||
const StorageID & main_table,
|
||||
SelectStreamFactory & stream_factory,
|
||||
const ASTPtr & query_ast,
|
||||
ContextPtr context,
|
||||
|
@ -290,6 +290,7 @@ struct ContextSharedPart : boost::noncopyable
|
||||
mutable QueryCachePtr query_cache TSA_GUARDED_BY(mutex); /// Cache of query results.
|
||||
mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices.
|
||||
mutable MMappedFileCachePtr mmap_cache TSA_GUARDED_BY(mutex); /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads.
|
||||
AsynchronousMetrics * asynchronous_metrics TSA_GUARDED_BY(mutex) = nullptr; /// Points to asynchronous metrics
|
||||
ProcessList process_list; /// Executing queries at the moment.
|
||||
SessionTracker session_tracker;
|
||||
GlobalOvercommitTracker global_overcommit_tracker;
|
||||
@ -1644,6 +1645,11 @@ void Context::addQueryAccessInfo(const QualifiedProjectionName & qualified_proje
|
||||
"{}.{}", qualified_projection_name.storage_id.getFullTableName(), backQuoteIfNeed(qualified_projection_name.projection_name)));
|
||||
}
|
||||
|
||||
Context::QueryFactoriesInfo Context::getQueryFactoriesInfo() const
|
||||
{
|
||||
return query_factories_info;
|
||||
}
|
||||
|
||||
void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const
|
||||
{
|
||||
if (isGlobalContext())
|
||||
@ -2860,6 +2866,18 @@ void Context::clearCaches() const
|
||||
/// Intentionally not clearing the query cache which is transactionally inconsistent by design.
|
||||
}
|
||||
|
||||
void Context::setAsynchronousMetrics(AsynchronousMetrics * asynchronous_metrics_)
|
||||
{
|
||||
std::lock_guard lock(shared->mutex);
|
||||
shared->asynchronous_metrics = asynchronous_metrics_;
|
||||
}
|
||||
|
||||
AsynchronousMetrics * Context::getAsynchronousMetrics() const
|
||||
{
|
||||
SharedLockGuard lock(shared->mutex);
|
||||
return shared->asynchronous_metrics;
|
||||
}
|
||||
|
||||
ThreadPool & Context::getPrefetchThreadpool() const
|
||||
{
|
||||
callOnce(shared->prefetch_threadpool_initialized, [&] {
|
||||
|
@ -70,6 +70,7 @@ class IUserDefinedSQLObjectsStorage;
|
||||
class InterserverCredentials;
|
||||
using InterserverCredentialsPtr = std::shared_ptr<const InterserverCredentials>;
|
||||
class InterserverIOHandler;
|
||||
class AsynchronousMetrics;
|
||||
class BackgroundSchedulePool;
|
||||
class MergeList;
|
||||
class MovesList;
|
||||
@ -373,25 +374,6 @@ protected:
|
||||
|
||||
QueryFactoriesInfo(QueryFactoriesInfo && rhs) = delete;
|
||||
|
||||
QueryFactoriesInfo & operator=(QueryFactoriesInfo rhs)
|
||||
{
|
||||
swap(rhs);
|
||||
return *this;
|
||||
}
|
||||
|
||||
void swap(QueryFactoriesInfo & rhs)
|
||||
{
|
||||
std::swap(aggregate_functions, rhs.aggregate_functions);
|
||||
std::swap(aggregate_function_combinators, rhs.aggregate_function_combinators);
|
||||
std::swap(database_engines, rhs.database_engines);
|
||||
std::swap(data_type_families, rhs.data_type_families);
|
||||
std::swap(dictionaries, rhs.dictionaries);
|
||||
std::swap(formats, rhs.formats);
|
||||
std::swap(functions, rhs.functions);
|
||||
std::swap(storages, rhs.storages);
|
||||
std::swap(table_functions, rhs.table_functions);
|
||||
}
|
||||
|
||||
std::unordered_set<std::string> aggregate_functions;
|
||||
std::unordered_set<std::string> aggregate_function_combinators;
|
||||
std::unordered_set<std::string> database_engines;
|
||||
@ -727,7 +709,7 @@ public:
|
||||
TableFunction
|
||||
};
|
||||
|
||||
const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; }
|
||||
QueryFactoriesInfo getQueryFactoriesInfo() const;
|
||||
void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const;
|
||||
|
||||
/// For table functions s3/file/url/hdfs/input we can use structure from
|
||||
@ -1014,6 +996,9 @@ public:
|
||||
|
||||
/// -----------------------------------------------------------------------------------------------------
|
||||
|
||||
void setAsynchronousMetrics(AsynchronousMetrics * asynchronous_metrics_);
|
||||
AsynchronousMetrics * getAsynchronousMetrics() const;
|
||||
|
||||
ThreadPool & getPrefetchThreadpool() const;
|
||||
|
||||
/// Note: prefetchThreadpool is different from threadpoolReader
|
||||
|
@ -11,6 +11,8 @@
|
||||
#include <Common/atomicRename.h>
|
||||
#include <Common/PoolId.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
#include <base/hex.h>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
@ -463,6 +465,14 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
|
||||
column_declaration->children.push_back(column_declaration->ttl);
|
||||
}
|
||||
|
||||
if (!column.settings.empty())
|
||||
{
|
||||
auto settings = std::make_shared<ASTSetQuery>();
|
||||
settings->is_standalone = false;
|
||||
settings->changes = column.settings;
|
||||
column_declaration->settings = std::move(settings);
|
||||
}
|
||||
|
||||
columns_list->children.push_back(column_declaration_ptr);
|
||||
}
|
||||
|
||||
@ -670,6 +680,12 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
if (col_decl.ttl)
|
||||
column.ttl = col_decl.ttl;
|
||||
|
||||
if (col_decl.settings)
|
||||
{
|
||||
column.settings = col_decl.settings->as<ASTSetQuery &>().changes;
|
||||
MergeTreeColumnSettings::validate(column.settings);
|
||||
}
|
||||
|
||||
res.add(std::move(column));
|
||||
}
|
||||
|
||||
|
@ -561,6 +561,14 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
getContext()->checkAccess(AccessType::SYSTEM_RELOAD_USERS);
|
||||
system_context->getAccessControl().reload(AccessControl::ReloadMode::ALL);
|
||||
break;
|
||||
case Type::RELOAD_ASYNCHRONOUS_METRICS:
|
||||
{
|
||||
getContext()->checkAccess(AccessType::SYSTEM_RELOAD_ASYNCHRONOUS_METRICS);
|
||||
auto * asynchronous_metrics = system_context->getAsynchronousMetrics();
|
||||
if (asynchronous_metrics)
|
||||
asynchronous_metrics->update(std::chrono::system_clock::now(), /*force_update*/ true);
|
||||
break;
|
||||
}
|
||||
case Type::STOP_MERGES:
|
||||
startStopAction(ActionLocks::PartsMerge, false);
|
||||
break;
|
||||
@ -1225,6 +1233,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
|
||||
required_access.emplace_back(AccessType::SYSTEM_RELOAD_USERS);
|
||||
break;
|
||||
}
|
||||
case Type::RELOAD_ASYNCHRONOUS_METRICS:
|
||||
{
|
||||
required_access.emplace_back(AccessType::SYSTEM_RELOAD_ASYNCHRONOUS_METRICS);
|
||||
break;
|
||||
}
|
||||
case Type::STOP_MERGES:
|
||||
case Type::START_MERGES:
|
||||
{
|
||||
|
@ -1,17 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <Core/Range.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/FieldToDataType.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/applyFunction.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/IAST.h>
|
||||
@ -37,8 +33,6 @@ public:
|
||||
ASTIdentifier * identifier = nullptr;
|
||||
DataTypePtr arg_data_type = {};
|
||||
|
||||
Range range = Range::createWholeUniverse();
|
||||
|
||||
void reject() { monotonicity.is_monotonic = false; }
|
||||
bool isRejected() const { return !monotonicity.is_monotonic; }
|
||||
|
||||
@ -103,30 +97,13 @@ public:
|
||||
if (data.isRejected())
|
||||
return;
|
||||
|
||||
/// Monotonicity check only works for functions that contain at most two arguments and one of them must be a constant.
|
||||
if (!ast_function.arguments)
|
||||
/// TODO: monotonicity for functions of several arguments
|
||||
if (!ast_function.arguments || ast_function.arguments->children.size() != 1)
|
||||
{
|
||||
data.reject();
|
||||
return;
|
||||
}
|
||||
|
||||
auto arguments_size = ast_function.arguments->children.size();
|
||||
|
||||
if (arguments_size == 0 || arguments_size > 2)
|
||||
{
|
||||
data.reject();
|
||||
return;
|
||||
}
|
||||
else if (arguments_size == 2)
|
||||
{
|
||||
/// If the function has two arguments, then one of them must be a constant.
|
||||
if (!ast_function.arguments->children[0]->as<ASTLiteral>() && !ast_function.arguments->children[1]->as<ASTLiteral>())
|
||||
{
|
||||
data.reject();
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
if (!data.canOptimize(ast_function))
|
||||
{
|
||||
data.reject();
|
||||
@ -147,33 +124,14 @@ public:
|
||||
return;
|
||||
}
|
||||
|
||||
auto function_arguments = getFunctionArguments(ast_function, data);
|
||||
|
||||
auto function_base = function->build(function_arguments);
|
||||
ColumnsWithTypeAndName args;
|
||||
args.emplace_back(data.arg_data_type, "tmp");
|
||||
auto function_base = function->build(args);
|
||||
|
||||
if (function_base && function_base->hasInformationAboutMonotonicity())
|
||||
{
|
||||
bool is_positive = data.monotonicity.is_positive;
|
||||
data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, data.range.left, data.range.right);
|
||||
|
||||
auto & key_range = data.range;
|
||||
|
||||
/// If we apply function to open interval, we can get empty intervals in result.
|
||||
/// E.g. for ('2020-01-03', '2020-01-20') after applying 'toYYYYMM' we will get ('202001', '202001').
|
||||
/// To avoid this we make range left and right included.
|
||||
/// Any function that treats NULL specially is not monotonic.
|
||||
/// Thus we can safely use isNull() as an -Inf/+Inf indicator here.
|
||||
if (!key_range.left.isNull())
|
||||
{
|
||||
key_range.left = applyFunction(function_base, data.arg_data_type, key_range.left);
|
||||
key_range.left_included = true;
|
||||
}
|
||||
|
||||
if (!key_range.right.isNull())
|
||||
{
|
||||
key_range.right = applyFunction(function_base, data.arg_data_type, key_range.right);
|
||||
key_range.right_included = true;
|
||||
}
|
||||
data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, Field(), Field());
|
||||
|
||||
if (!is_positive)
|
||||
data.monotonicity.is_positive = !data.monotonicity.is_positive;
|
||||
@ -185,53 +143,13 @@ public:
|
||||
|
||||
static bool needChildVisit(const ASTPtr & parent, const ASTPtr &)
|
||||
{
|
||||
/// Multi-argument functions with all but one constant arguments can be monotonic.
|
||||
/// Currently we check monotonicity only for single-argument functions.
|
||||
/// Although, multi-argument functions with all but one constant arguments can also be monotonic.
|
||||
if (const auto * func = typeid_cast<const ASTFunction *>(parent.get()))
|
||||
return func->arguments->children.size() <= 2;
|
||||
return func->arguments->children.size() < 2;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
static ColumnWithTypeAndName extractLiteralColumnAndTypeFromAstLiteral(const ASTLiteral * literal)
|
||||
{
|
||||
ColumnWithTypeAndName result;
|
||||
|
||||
result.type = applyVisitor(FieldToDataType(), literal->value);
|
||||
result.column = result.type->createColumnConst(0, literal->value);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
static ColumnsWithTypeAndName getFunctionArguments(const ASTFunction & ast_function, const Data & data)
|
||||
{
|
||||
ColumnsWithTypeAndName args;
|
||||
|
||||
auto arguments_size = ast_function.arguments->children.size();
|
||||
|
||||
chassert(arguments_size == 1 || arguments_size == 2);
|
||||
|
||||
if (arguments_size == 2)
|
||||
{
|
||||
if (ast_function.arguments->children[0]->as<ASTLiteral>())
|
||||
{
|
||||
const auto * literal = ast_function.arguments->children[0]->as<ASTLiteral>();
|
||||
args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal));
|
||||
args.emplace_back(data.arg_data_type, "tmp");
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto * literal = ast_function.arguments->children[1]->as<ASTLiteral>();
|
||||
args.emplace_back(data.arg_data_type, "tmp");
|
||||
args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal));
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
args.emplace_back(data.arg_data_type, "tmp");
|
||||
}
|
||||
|
||||
return args;
|
||||
}
|
||||
};
|
||||
|
||||
using MonotonicityCheckVisitor = ConstInDepthNodeVisitor<MonotonicityCheckMatcher, false>;
|
||||
|
@ -69,7 +69,7 @@ ServerAsynchronousMetrics::~ServerAsynchronousMetrics()
|
||||
stop();
|
||||
}
|
||||
|
||||
void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time)
|
||||
void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values)
|
||||
{
|
||||
if (auto mark_cache = getContext()->getMarkCache())
|
||||
{
|
||||
@ -377,7 +377,7 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
|
||||
}
|
||||
#endif
|
||||
|
||||
updateHeavyMetricsIfNeeded(current_time, update_time, new_values);
|
||||
updateHeavyMetricsIfNeeded(current_time, update_time, force_update, first_run, new_values);
|
||||
}
|
||||
|
||||
void ServerAsynchronousMetrics::logImpl(AsynchronousMetricValues & new_values)
|
||||
@ -421,19 +421,19 @@ void ServerAsynchronousMetrics::updateDetachedPartsStats()
|
||||
detached_parts_stats = current_values;
|
||||
}
|
||||
|
||||
void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values)
|
||||
void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values)
|
||||
{
|
||||
const auto time_after_previous_update = current_time - heavy_metric_previous_update_time;
|
||||
const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run;
|
||||
const auto time_since_previous_update = current_time - heavy_metric_previous_update_time;
|
||||
const bool update_heavy_metrics = (time_since_previous_update >= heavy_metric_update_period) || force_update || first_run;
|
||||
|
||||
Stopwatch watch;
|
||||
if (update_heavy_metric)
|
||||
if (update_heavy_metrics)
|
||||
{
|
||||
heavy_metric_previous_update_time = update_time;
|
||||
if (first_run)
|
||||
heavy_update_interval = heavy_metric_update_period.count();
|
||||
else
|
||||
heavy_update_interval = std::chrono::duration_cast<std::chrono::microseconds>(time_after_previous_update).count() / 1e6;
|
||||
heavy_update_interval = std::chrono::duration_cast<std::chrono::microseconds>(time_since_previous_update).count() / 1e6;
|
||||
|
||||
/// Test shows that listing 100000 entries consuming around 0.15 sec.
|
||||
updateDetachedPartsStats();
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
~ServerAsynchronousMetrics() override;
|
||||
|
||||
private:
|
||||
void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override;
|
||||
void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) override;
|
||||
void logImpl(AsynchronousMetricValues & new_values) override;
|
||||
|
||||
const Duration heavy_metric_update_period;
|
||||
@ -34,7 +34,7 @@ private:
|
||||
DetachedPartsStats detached_parts_stats{};
|
||||
|
||||
void updateDetachedPartsStats();
|
||||
void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values);
|
||||
void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -752,7 +752,8 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result,
|
||||
}
|
||||
|
||||
/// Rewrite sum(column +/- literal) function with sum(column) +/- literal * count(column).
|
||||
rewriteSumFunctionWithSumAndCount(query, tables_with_columns);
|
||||
if (settings.optimize_arithmetic_operations_in_aggregate_functions)
|
||||
rewriteSumFunctionWithSumAndCount(query, tables_with_columns);
|
||||
|
||||
/// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, etc.
|
||||
optimizeDateFilters(select_query, tables_with_columns, context);
|
||||
|
@ -1,43 +0,0 @@
|
||||
#include <Interpreters/applyFunction.h>
|
||||
|
||||
#include <Core/Range.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static Field applyFunctionForField(const FunctionBasePtr & func, const DataTypePtr & arg_type, const Field & arg_value)
|
||||
{
|
||||
ColumnsWithTypeAndName columns{
|
||||
{arg_type->createColumnConst(1, arg_value), arg_type, "x"},
|
||||
};
|
||||
|
||||
auto col = func->execute(columns, func->getResultType(), 1);
|
||||
return (*col)[0];
|
||||
}
|
||||
|
||||
FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field)
|
||||
{
|
||||
/// Fallback for fields without block reference.
|
||||
if (field.isExplicit())
|
||||
return applyFunctionForField(func, current_type, field);
|
||||
|
||||
String result_name = "_" + func->getName() + "_" + toString(field.column_idx);
|
||||
const auto & columns = field.columns;
|
||||
size_t result_idx = columns->size();
|
||||
|
||||
for (size_t i = 0; i < result_idx; ++i)
|
||||
if ((*columns)[i].name == result_name)
|
||||
result_idx = i;
|
||||
|
||||
if (result_idx == columns->size())
|
||||
{
|
||||
ColumnsWithTypeAndName args{(*columns)[field.column_idx]};
|
||||
field.columns->emplace_back(ColumnWithTypeAndName{nullptr, func->getResultType(), result_name});
|
||||
(*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size());
|
||||
}
|
||||
|
||||
return {field.columns, field.row_idx, result_idx};
|
||||
}
|
||||
|
||||
}
|
@ -1,16 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct FieldRef;
|
||||
|
||||
class IFunctionBase;
|
||||
class IDataType;
|
||||
|
||||
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
||||
|
||||
FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field);
|
||||
}
|
@ -257,7 +257,7 @@ addStatusInfoToQueryLogElement(QueryLogElement & element, const QueryStatusInfo
|
||||
element.query_projections.insert(access_info.projections.begin(), access_info.projections.end());
|
||||
element.query_views.insert(access_info.views.begin(), access_info.views.end());
|
||||
|
||||
const auto & factories_info = context_ptr->getQueryFactoriesInfo();
|
||||
const auto factories_info = context_ptr->getQueryFactoriesInfo();
|
||||
element.used_aggregate_functions = factories_info.aggregate_functions;
|
||||
element.used_aggregate_function_combinators = factories_info.aggregate_function_combinators;
|
||||
element.used_database_engines = factories_info.database_engines;
|
||||
|
@ -57,70 +57,83 @@ ASTPtr ASTColumnDeclaration::clone() const
|
||||
res->children.push_back(res->collation);
|
||||
}
|
||||
|
||||
if (settings)
|
||||
{
|
||||
res->settings = settings->clone();
|
||||
res->children.push_back(res->settings);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
frame.need_parens = false;
|
||||
|
||||
/// We have to always backquote column names to avoid ambiguouty with INDEX and other declarations in CREATE query.
|
||||
settings.ostr << backQuote(name);
|
||||
format_settings.ostr << backQuote(name);
|
||||
|
||||
if (type)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
format_settings.ostr << ' ';
|
||||
|
||||
FormatStateStacked type_frame = frame;
|
||||
type_frame.indent = 0;
|
||||
|
||||
type->formatImpl(settings, state, type_frame);
|
||||
type->formatImpl(format_settings, state, type_frame);
|
||||
}
|
||||
|
||||
if (null_modifier)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "")
|
||||
<< (*null_modifier ? "" : "NOT ") << "NULL" << (settings.hilite ? hilite_none : "");
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "")
|
||||
<< (*null_modifier ? "" : "NOT ") << "NULL" << (format_settings.hilite ? hilite_none : "");
|
||||
}
|
||||
|
||||
if (default_expression)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << default_specifier << (settings.hilite ? hilite_none : "");
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << default_specifier << (format_settings.hilite ? hilite_none : "");
|
||||
if (!ephemeral_default)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
default_expression->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ';
|
||||
default_expression->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
if (comment)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' ';
|
||||
comment->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COMMENT" << (format_settings.hilite ? hilite_none : "") << ' ';
|
||||
comment->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (codec)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
codec->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ';
|
||||
codec->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (stat_type)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
stat_type->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ';
|
||||
stat_type->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (ttl)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "TTL" << (settings.hilite ? hilite_none : "") << ' ';
|
||||
ttl->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "TTL" << (format_settings.hilite ? hilite_none : "") << ' ';
|
||||
ttl->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (collation)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COLLATE" << (settings.hilite ? hilite_none : "") << ' ';
|
||||
collation->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COLLATE" << (format_settings.hilite ? hilite_none : "") << ' ';
|
||||
collation->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (settings)
|
||||
{
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' ' << '(';
|
||||
settings->formatImpl(format_settings, state, frame);
|
||||
format_settings.ostr << ')';
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -22,12 +22,13 @@ public:
|
||||
ASTPtr stat_type;
|
||||
ASTPtr ttl;
|
||||
ASTPtr collation;
|
||||
ASTPtr settings;
|
||||
bool primary_key_specifier = false;
|
||||
|
||||
String getID(char delim) const override { return "ColumnDeclaration" + (delim + name); }
|
||||
|
||||
ASTPtr clone() const override;
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
void formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -67,6 +67,7 @@ public:
|
||||
RELOAD_EMBEDDED_DICTIONARIES,
|
||||
RELOAD_CONFIG,
|
||||
RELOAD_USERS,
|
||||
RELOAD_ASYNCHRONOUS_METRICS,
|
||||
RESTART_DISK,
|
||||
STOP_MERGES,
|
||||
START_MERGES,
|
||||
|
@ -111,6 +111,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_comment("COMMENT");
|
||||
ParserKeyword s_codec("CODEC");
|
||||
ParserKeyword s_ttl("TTL");
|
||||
ParserKeyword s_settings("SETTINGS");
|
||||
|
||||
ParserKeyword s_remove_ttl("REMOVE TTL");
|
||||
ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY");
|
||||
@ -725,9 +726,21 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
command->remove_property = "CODEC";
|
||||
else if (s_ttl.ignore(pos, expected))
|
||||
command->remove_property = "TTL";
|
||||
else if (s_settings.ignore(pos, expected))
|
||||
command->remove_property = "SETTINGS";
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else if (s_modify_setting.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_settings.parse(pos, command_settings_changes, expected))
|
||||
return false;
|
||||
}
|
||||
else if (s_reset_setting.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_reset_setting.parse(pos, command_settings_resets, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (s_first.ignore(pos, expected))
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ParserDataType.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
#include <Poco/String.h>
|
||||
|
||||
namespace DB
|
||||
@ -120,8 +121,6 @@ using ParserCompoundColumnDeclaration = IParserColumnDeclaration<ParserCompoundI
|
||||
template <typename NameParser>
|
||||
bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
NameParser name_parser;
|
||||
ParserDataType type_parser;
|
||||
ParserKeyword s_default{"DEFAULT"};
|
||||
ParserKeyword s_null{"NULL"};
|
||||
ParserKeyword s_not{"NOT"};
|
||||
@ -134,9 +133,15 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
ParserKeyword s_stat{"STATISTIC"};
|
||||
ParserKeyword s_ttl{"TTL"};
|
||||
ParserKeyword s_remove{"REMOVE"};
|
||||
ParserKeyword s_modify_setting("MODIFY SETTING");
|
||||
ParserKeyword s_reset_setting("RESET SETTING");
|
||||
ParserKeyword s_settings("SETTINGS");
|
||||
ParserKeyword s_type{"TYPE"};
|
||||
ParserKeyword s_collate{"COLLATE"};
|
||||
ParserKeyword s_primary_key{"PRIMARY KEY"};
|
||||
|
||||
NameParser name_parser;
|
||||
ParserDataType type_parser;
|
||||
ParserExpression expr_parser;
|
||||
ParserStringLiteral string_literal_parser;
|
||||
ParserLiteral literal_parser;
|
||||
@ -144,6 +149,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
ParserCollation collation_parser;
|
||||
ParserStatisticType stat_type_parser;
|
||||
ParserExpression expression_parser;
|
||||
ParserSetQuery settings_parser(true);
|
||||
|
||||
/// mandatory column name
|
||||
ASTPtr name;
|
||||
@ -155,11 +161,12 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
|
||||
/// This keyword may occur only in MODIFY COLUMN query. We check it here
|
||||
/// because ParserDataType parses types as an arbitrary identifiers and
|
||||
/// doesn't check that parsed string is existing data type. In this way
|
||||
/// REMOVE keyword can be parsed as data type and further parsing will fail.
|
||||
/// So we just check this keyword and in case of success return column
|
||||
/// declaration with name only.
|
||||
if (!require_type && s_remove.checkWithoutMoving(pos, expected))
|
||||
/// doesn't check that parsed string is existing data type. In this way,
|
||||
/// REMOVE, MODIFY SETTING, or RESET SETTING can be parsed as data type
|
||||
/// and further parsing will fail. So we just check these keyword and in
|
||||
/// case of success return column declaration with name only.
|
||||
if (!require_type
|
||||
&& (s_remove.checkWithoutMoving(pos, expected) || s_modify_setting.checkWithoutMoving(pos, expected) || s_reset_setting.checkWithoutMoving(pos, expected)))
|
||||
{
|
||||
if (!check_keywords_after_name)
|
||||
return false;
|
||||
@ -181,6 +188,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
ASTPtr stat_type_expression;
|
||||
ASTPtr ttl_expression;
|
||||
ASTPtr collation_expression;
|
||||
ASTPtr settings;
|
||||
bool primary_key_specifier = false;
|
||||
|
||||
auto null_check_without_moving = [&]() -> bool
|
||||
@ -321,6 +329,28 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
primary_key_specifier = true;
|
||||
}
|
||||
|
||||
auto old_pos = pos;
|
||||
if (s_settings.ignore(pos, expected))
|
||||
{
|
||||
/// When the keyword `SETTINGS` appear here, it can be a column settings declaration or query settings
|
||||
/// For example:
|
||||
/// - Column settings: `ALTER TABLE xx MODIFY COLUMN yy <new_type> SETTINGS (name = value)`
|
||||
/// - Query settings: ` ALTER TABLE xx MODIFY COLUMN yy <new_type> SETTINGS mutation_sync = 2`
|
||||
/// So after parsing keyword `SETTINGS`, we check if it's followed by an `(` then it's the column
|
||||
/// settings, otherwise it's the query settings and we need to move `pos` back to origin position.
|
||||
ParserToken parser_opening_bracket(TokenType::OpeningRoundBracket);
|
||||
if (parser_opening_bracket.ignore(pos, expected))
|
||||
{
|
||||
if (!settings_parser.parse(pos, settings, expected))
|
||||
return false;
|
||||
ParserToken parser_closing_bracket(TokenType::ClosingRoundBracket);
|
||||
if (!parser_closing_bracket.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
pos = old_pos;
|
||||
}
|
||||
|
||||
node = column_declaration;
|
||||
|
||||
if (type)
|
||||
@ -351,6 +381,12 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
column_declaration->children.push_back(std::move(codec_expression));
|
||||
}
|
||||
|
||||
if (settings)
|
||||
{
|
||||
column_declaration->settings = settings;
|
||||
column_declaration->children.push_back(std::move(settings));
|
||||
}
|
||||
|
||||
if (stat_type_expression)
|
||||
{
|
||||
column_declaration->stat_type = stat_type_expression;
|
||||
@ -362,6 +398,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
column_declaration->ttl = ttl_expression;
|
||||
column_declaration->children.push_back(std::move(ttl_expression));
|
||||
}
|
||||
|
||||
if (collation_expression)
|
||||
{
|
||||
column_declaration->collation = collation_expression;
|
||||
|
@ -3,11 +3,6 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
String queryToStringNullable(const ASTPtr & query)
|
||||
{
|
||||
return query ? queryToString(query) : "";
|
||||
}
|
||||
|
||||
String queryToString(const ASTPtr & query)
|
||||
{
|
||||
return queryToString(*query);
|
||||
|
@ -6,5 +6,4 @@ namespace DB
|
||||
{
|
||||
String queryToString(const ASTPtr & query);
|
||||
String queryToString(const IAST & query);
|
||||
String queryToStringNullable(const ASTPtr & query);
|
||||
}
|
||||
|
@ -1391,7 +1391,7 @@ void Planner::buildPlanForQueryNode()
|
||||
}
|
||||
}
|
||||
|
||||
if (query_context->canUseTaskBasedParallelReplicas() || !settings.parallel_replicas_custom_key.value.empty())
|
||||
if (!settings.parallel_replicas_custom_key.value.empty())
|
||||
{
|
||||
/// Check support for JOIN for parallel replicas with custom key
|
||||
if (planner_context->getTableExpressionNodeToData().size() > 1)
|
||||
|
@ -357,7 +357,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
|
||||
ParallelReplicasReadingCoordinatorPtr coordinator_,
|
||||
Block header_,
|
||||
QueryProcessingStage::Enum stage_,
|
||||
StorageID main_table_,
|
||||
ContextMutablePtr context_,
|
||||
ThrottlerPtr throttler_,
|
||||
Scalars scalars_,
|
||||
@ -369,7 +368,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
|
||||
, query_ast(query_ast_)
|
||||
, coordinator(std::move(coordinator_))
|
||||
, stage(std::move(stage_))
|
||||
, main_table(std::move(main_table_))
|
||||
, context(context_)
|
||||
, throttler(throttler_)
|
||||
, scalars(scalars_)
|
||||
|
@ -76,7 +76,6 @@ public:
|
||||
ParallelReplicasReadingCoordinatorPtr coordinator_,
|
||||
Block header_,
|
||||
QueryProcessingStage::Enum stage_,
|
||||
StorageID main_table_,
|
||||
ContextMutablePtr context_,
|
||||
ThrottlerPtr throttler_,
|
||||
Scalars scalars_,
|
||||
@ -99,7 +98,6 @@ private:
|
||||
ASTPtr query_ast;
|
||||
ParallelReplicasReadingCoordinatorPtr coordinator;
|
||||
QueryProcessingStage::Enum stage;
|
||||
StorageID main_table;
|
||||
ContextMutablePtr context;
|
||||
ThrottlerPtr throttler;
|
||||
Scalars scalars;
|
||||
|
@ -36,6 +36,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
|
||||
#include <ranges>
|
||||
|
||||
@ -74,6 +75,8 @@ AlterCommand::RemoveProperty removePropertyFromString(const String & property)
|
||||
return AlterCommand::RemoveProperty::CODEC;
|
||||
else if (property == "TTL")
|
||||
return AlterCommand::RemoveProperty::TTL;
|
||||
else if (property == "SETTINGS")
|
||||
return AlterCommand::RemoveProperty::SETTINGS;
|
||||
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot remove unknown property '{}'", property);
|
||||
}
|
||||
@ -173,6 +176,25 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
if (ast_col_decl.codec)
|
||||
command.codec = ast_col_decl.codec;
|
||||
|
||||
if (ast_col_decl.settings)
|
||||
command.settings_changes = ast_col_decl.settings->as<ASTSetQuery &>().changes;
|
||||
|
||||
/// At most only one of ast_col_decl.settings or command_ast->settings_changes is non-null
|
||||
if (command_ast->settings_changes)
|
||||
{
|
||||
command.settings_changes = command_ast->settings_changes->as<ASTSetQuery &>().changes;
|
||||
command.append_column_setting = true;
|
||||
}
|
||||
|
||||
if (command_ast->settings_resets)
|
||||
{
|
||||
for (const ASTPtr & identifier_ast : command_ast->settings_resets->children)
|
||||
{
|
||||
const auto & identifier = identifier_ast->as<ASTIdentifier &>();
|
||||
command.settings_resets.emplace(identifier.name());
|
||||
}
|
||||
}
|
||||
|
||||
if (command_ast->column)
|
||||
command.after_column = getIdentifierName(command_ast->column);
|
||||
|
||||
@ -501,6 +523,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
{
|
||||
column.ttl.reset();
|
||||
}
|
||||
else if (to_remove == RemoveProperty::SETTINGS)
|
||||
{
|
||||
column.settings.clear();
|
||||
}
|
||||
else
|
||||
{
|
||||
if (codec)
|
||||
@ -515,6 +541,22 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
if (data_type)
|
||||
column.type = data_type;
|
||||
|
||||
if (!settings_changes.empty())
|
||||
{
|
||||
MergeTreeColumnSettings::validate(settings_changes);
|
||||
if (append_column_setting)
|
||||
for (const auto & change : settings_changes)
|
||||
column.settings.setSetting(change.name, change.value);
|
||||
else
|
||||
column.settings = settings_changes;
|
||||
}
|
||||
|
||||
if (!settings_resets.empty())
|
||||
{
|
||||
for (const auto & setting : settings_resets)
|
||||
column.settings.removeSetting(setting);
|
||||
}
|
||||
|
||||
/// User specified default expression or changed
|
||||
/// datatype. We have to replace default.
|
||||
if (default_expression || data_type)
|
||||
@ -1357,7 +1399,6 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Column {} doesn't have COMMENT, cannot remove it",
|
||||
backQuote(column_name));
|
||||
|
||||
}
|
||||
|
||||
modified_columns.emplace(column_name);
|
||||
|
@ -64,7 +64,8 @@ struct AlterCommand
|
||||
/// Other properties
|
||||
COMMENT,
|
||||
CODEC,
|
||||
TTL
|
||||
TTL,
|
||||
SETTINGS
|
||||
};
|
||||
|
||||
Type type = UNKNOWN;
|
||||
@ -137,10 +138,10 @@ struct AlterCommand
|
||||
/// For ADD and MODIFY
|
||||
ASTPtr codec = nullptr;
|
||||
|
||||
/// For MODIFY SETTING
|
||||
/// For MODIFY SETTING or MODIFY COLUMN MODIFY SETTING
|
||||
SettingsChanges settings_changes;
|
||||
|
||||
/// For RESET SETTING
|
||||
/// For RESET SETTING or MODIFY COLUMN RESET SETTING
|
||||
std::set<String> settings_resets;
|
||||
|
||||
/// For MODIFY_QUERY
|
||||
@ -155,6 +156,9 @@ struct AlterCommand
|
||||
/// What to remove from column (or TTL)
|
||||
RemoveProperty to_remove = RemoveProperty::NO_PROPERTY;
|
||||
|
||||
/// Is this MODIFY COLUMN MODIFY SETTING or MODIFY COLUMN column with settings declaration)
|
||||
bool append_column_setting = false;
|
||||
|
||||
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
||||
|
||||
void apply(StorageInMemoryMetadata & metadata, ContextPtr context) const;
|
||||
|
@ -24,6 +24,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include "Parsers/ASTSetQuery.h"
|
||||
#include <Core/Defines.h>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
@ -72,6 +73,7 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const
|
||||
&& default_desc == other.default_desc
|
||||
&& stat == other.stat
|
||||
&& ast_to_str(codec) == ast_to_str(other.codec)
|
||||
&& settings == other.settings
|
||||
&& ast_to_str(ttl) == ast_to_str(other.ttl);
|
||||
}
|
||||
|
||||
@ -104,6 +106,18 @@ void ColumnDescription::writeText(WriteBuffer & buf) const
|
||||
writeEscapedString(queryToString(codec), buf);
|
||||
}
|
||||
|
||||
if (!settings.empty())
|
||||
{
|
||||
writeChar('\t', buf);
|
||||
DB::writeText("SETTINGS ", buf);
|
||||
DB::writeText("(", buf);
|
||||
ASTSetQuery ast;
|
||||
ast.is_standalone = false;
|
||||
ast.changes = settings;
|
||||
writeEscapedString(queryToString(ast), buf);
|
||||
DB::writeText(")", buf);
|
||||
}
|
||||
|
||||
if (stat)
|
||||
{
|
||||
writeChar('\t', buf);
|
||||
@ -154,6 +168,9 @@ void ColumnDescription::readText(ReadBuffer & buf)
|
||||
|
||||
if (col_ast->ttl)
|
||||
ttl = col_ast->ttl;
|
||||
|
||||
if (col_ast->settings)
|
||||
settings = col_ast->settings->as<ASTSetQuery &>().changes;
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description");
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Core/NamesAndAliases.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Storages/ColumnDefault.h>
|
||||
#include <Common/SettingsChanges.h>
|
||||
#include <Storages/StatisticsDescription.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
@ -83,6 +84,7 @@ struct ColumnDescription
|
||||
ColumnDefault default_desc;
|
||||
String comment;
|
||||
ASTPtr codec;
|
||||
SettingsChanges settings;
|
||||
ASTPtr ttl;
|
||||
std::optional<StatisticDescription> stat;
|
||||
|
||||
|
@ -37,7 +37,7 @@ namespace ErrorCodes
|
||||
|
||||
AsynchronousReadBufferFromHDFS::AsynchronousReadBufferFromHDFS(
|
||||
IAsynchronousReader & reader_, const ReadSettings & settings_, std::shared_ptr<ReadBufferFromHDFS> impl_)
|
||||
: BufferWithOwnMemory<SeekableReadBuffer>(settings_.remote_fs_buffer_size)
|
||||
: ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0)
|
||||
, reader(reader_)
|
||||
, base_priority(settings_.priority)
|
||||
, impl(std::move(impl_))
|
||||
|
@ -21,7 +21,7 @@ namespace DB
|
||||
|
||||
class IAsynchronousReader;
|
||||
|
||||
class AsynchronousReadBufferFromHDFS : public BufferWithOwnMemory<SeekableReadBuffer>, public WithFileName, public WithFileSize
|
||||
class AsynchronousReadBufferFromHDFS : public ReadBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
AsynchronousReadBufferFromHDFS(
|
||||
|
@ -81,7 +81,6 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par
|
||||
auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key);
|
||||
size_t minmax_idx_size = minmax_column_types.size();
|
||||
|
||||
hyperrectangle.clear();
|
||||
hyperrectangle.reserve(minmax_idx_size);
|
||||
for (size_t i = 0; i < minmax_idx_size; ++i)
|
||||
{
|
||||
@ -105,39 +104,6 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
Block IMergeTreeDataPart::MinMaxIndex::getBlock(const MergeTreeData & data) const
|
||||
{
|
||||
if (!initialized)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to get block from uninitialized MinMax index.");
|
||||
|
||||
Block block;
|
||||
|
||||
const auto metadata_snapshot = data.getInMemoryMetadataPtr();
|
||||
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||
|
||||
const auto minmax_column_names = data.getMinMaxColumnsNames(partition_key);
|
||||
const auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key);
|
||||
const auto minmax_idx_size = minmax_column_types.size();
|
||||
|
||||
for (size_t i = 0; i < minmax_idx_size; ++i)
|
||||
{
|
||||
const auto & data_type = minmax_column_types[i];
|
||||
const auto & column_name = minmax_column_names[i];
|
||||
|
||||
const auto column = data_type->createColumn();
|
||||
|
||||
const auto min_val = hyperrectangle.at(i).left;
|
||||
const auto max_val = hyperrectangle.at(i).right;
|
||||
|
||||
column->insert(min_val);
|
||||
column->insert(max_val);
|
||||
|
||||
block.insert(ColumnWithTypeAndName(column->getPtr(), data_type, column_name));
|
||||
}
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::store(
|
||||
const MergeTreeData & data, IDataPartStorage & part_storage, Checksums & out_checksums) const
|
||||
{
|
||||
@ -219,7 +185,8 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other)
|
||||
|
||||
if (!initialized)
|
||||
{
|
||||
*this = other;
|
||||
hyperrectangle = other.hyperrectangle;
|
||||
initialized = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -336,7 +336,6 @@ public:
|
||||
}
|
||||
|
||||
void load(const MergeTreeData & data, const PartMetadataManagerPtr & manager);
|
||||
Block getBlock(const MergeTreeData & data) const;
|
||||
|
||||
using WrittenFiles = std::vector<std::unique_ptr<WriteBufferFromFileBase>>;
|
||||
|
||||
|
@ -1,37 +1,36 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnSet.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Storages/MergeTree/BoolMask.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/FieldToDataType.h>
|
||||
#include <DataTypes/Utils.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Functions/CastOverloadResolver.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/indexHint.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <DataTypes/Utils.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/applyFunction.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Storages/MergeTree/BoolMask.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/indexHint.h>
|
||||
#include <Functions/CastOverloadResolver.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/MortonUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/ColumnSet.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <cassert>
|
||||
@ -837,6 +836,21 @@ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants
|
||||
return node.tryGetConstant(out_value, out_type);
|
||||
}
|
||||
|
||||
|
||||
static Field applyFunctionForField(
|
||||
const FunctionBasePtr & func,
|
||||
const DataTypePtr & arg_type,
|
||||
const Field & arg_value)
|
||||
{
|
||||
ColumnsWithTypeAndName columns
|
||||
{
|
||||
{ arg_type->createColumnConst(1, arg_value), arg_type, "x" },
|
||||
};
|
||||
|
||||
auto col = func->execute(columns, func->getResultType(), 1);
|
||||
return (*col)[0];
|
||||
}
|
||||
|
||||
/// The case when arguments may have types different than in the primary key.
|
||||
static std::pair<Field, DataTypePtr> applyFunctionForFieldOfUnknownType(
|
||||
const FunctionBasePtr & func,
|
||||
@ -876,6 +890,33 @@ static std::pair<Field, DataTypePtr> applyBinaryFunctionForFieldOfUnknownType(
|
||||
return {std::move(result), std::move(return_type)};
|
||||
}
|
||||
|
||||
|
||||
static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field)
|
||||
{
|
||||
/// Fallback for fields without block reference.
|
||||
if (field.isExplicit())
|
||||
return applyFunctionForField(func, current_type, field);
|
||||
|
||||
String result_name = "_" + func->getName() + "_" + toString(field.column_idx);
|
||||
const auto & columns = field.columns;
|
||||
size_t result_idx = columns->size();
|
||||
|
||||
for (size_t i = 0; i < result_idx; ++i)
|
||||
{
|
||||
if ((*columns)[i].name == result_name)
|
||||
result_idx = i;
|
||||
}
|
||||
|
||||
if (result_idx == columns->size())
|
||||
{
|
||||
ColumnsWithTypeAndName args{(*columns)[field.column_idx]};
|
||||
field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name});
|
||||
(*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size());
|
||||
}
|
||||
|
||||
return {field.columns, field.row_idx, result_idx};
|
||||
}
|
||||
|
||||
/** When table's key has expression with these functions from a column,
|
||||
* and when a column in a query is compared with a constant, such as:
|
||||
* CREATE TABLE (x String) ORDER BY toDate(x)
|
||||
|
@ -8,6 +8,21 @@
|
||||
#include <Backups/BackupEntryWrappedWith.h>
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/RestorerFromBackup.h>
|
||||
#include <Common/Config/ConfigHelper.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Increment.h>
|
||||
#include <Common/ProfileEventsScope.h>
|
||||
#include <Common/SimpleIncrement.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/noexcept_scope.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
@ -28,20 +43,19 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/inplaceBlockConversions.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTNameTypePair.h>
|
||||
#include <Parsers/ASTPartition.h>
|
||||
@ -50,41 +64,26 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/QueryPlan/QueryIdHolder.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/BlockNumberColumn.h>
|
||||
#include <Storages/Freeze.h>
|
||||
#include <Storages/MergeTree/ActiveDataPartSet.h>
|
||||
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartBuilder.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartCloner.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
|
||||
#include <Storages/Statistics/Estimator.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <Storages/MutationCommands.h>
|
||||
#include <Storages/MergeTree/ActiveDataPartSet.h>
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Common/Config/ConfigHelper.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Increment.h>
|
||||
#include <Common/ProfileEventsScope.h>
|
||||
#include <Common/SimpleIncrement.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/noexcept_scope.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
@ -198,50 +197,6 @@ namespace ErrorCodes
|
||||
extern const int LIMIT_EXCEEDED;
|
||||
}
|
||||
|
||||
static size_t getPartitionAstFieldsCount(const ASTPartition & partition_ast, ASTPtr partition_value_ast)
|
||||
{
|
||||
if (partition_ast.fields_count.has_value())
|
||||
return *partition_ast.fields_count;
|
||||
|
||||
if (partition_value_ast->as<ASTLiteral>())
|
||||
return 1;
|
||||
|
||||
const auto * tuple_ast = partition_value_ast->as<ASTFunction>();
|
||||
|
||||
if (!tuple_ast)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID());
|
||||
}
|
||||
|
||||
if (tuple_ast->name != "tuple")
|
||||
{
|
||||
if (!isFunctionCast(tuple_ast))
|
||||
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name);
|
||||
|
||||
if (tuple_ast->arguments->as<ASTExpressionList>()->children.empty())
|
||||
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name);
|
||||
|
||||
auto first_arg = tuple_ast->arguments->as<ASTExpressionList>()->children.at(0);
|
||||
if (const auto * inner_tuple = first_arg->as<ASTFunction>(); inner_tuple && inner_tuple->name == "tuple")
|
||||
{
|
||||
const auto * arguments_ast = tuple_ast->arguments->as<ASTExpressionList>();
|
||||
return arguments_ast ? arguments_ast->children.size() : 0;
|
||||
}
|
||||
else if (const auto * inner_literal_tuple = first_arg->as<ASTLiteral>(); inner_literal_tuple)
|
||||
{
|
||||
return inner_literal_tuple->value.getType() == Field::Types::Tuple ? inner_literal_tuple->value.safeGet<Tuple>().size() : 1;
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto * arguments_ast = tuple_ast->arguments->as<ASTExpressionList>();
|
||||
return arguments_ast ? arguments_ast->children.size() : 0;
|
||||
}
|
||||
}
|
||||
|
||||
static void checkSuspiciousIndices(const ASTFunction * index_function)
|
||||
{
|
||||
std::unordered_set<UInt64> unique_index_expression_hashes;
|
||||
@ -4899,7 +4854,7 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D
|
||||
}
|
||||
|
||||
void MergeTreeData::checkAlterPartitionIsPossible(
|
||||
const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr) const
|
||||
const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr local_context) const
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
@ -4927,15 +4882,7 @@ void MergeTreeData::checkAlterPartitionIsPossible(
|
||||
throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH PARTITION ALL currently");
|
||||
}
|
||||
else
|
||||
{
|
||||
// The below `getPartitionIDFromQuery` call will not work for attach / replace because it assumes the partition expressions
|
||||
// are the same and deliberately uses this storage. Later on, `MergeTreeData::replaceFrom` is called, and it makes the right
|
||||
// call to `getPartitionIDFromQuery` using source storage.
|
||||
// Note: `PartitionCommand::REPLACE_PARTITION` is used both for `REPLACE PARTITION` and `ATTACH PARTITION FROM` queries.
|
||||
// But not for `ATTACH PARTITION` queries.
|
||||
if (command.type != PartitionCommand::REPLACE_PARTITION)
|
||||
getPartitionIDFromQuery(command.partition, getContext());
|
||||
}
|
||||
getPartitionIDFromQuery(command.partition, local_context);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -5669,8 +5616,69 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
|
||||
MergeTreePartInfo::validatePartitionID(partition_ast.id->clone(), format_version);
|
||||
return partition_ast.id->as<ASTLiteral>()->value.safeGet<String>();
|
||||
}
|
||||
size_t partition_ast_fields_count = 0;
|
||||
ASTPtr partition_value_ast = partition_ast.value->clone();
|
||||
auto partition_ast_fields_count = getPartitionAstFieldsCount(partition_ast, partition_value_ast);
|
||||
if (!partition_ast.fields_count.has_value())
|
||||
{
|
||||
if (partition_value_ast->as<ASTLiteral>())
|
||||
{
|
||||
partition_ast_fields_count = 1;
|
||||
}
|
||||
else if (const auto * tuple_ast = partition_value_ast->as<ASTFunction>())
|
||||
{
|
||||
if (tuple_ast->name != "tuple")
|
||||
{
|
||||
if (isFunctionCast(tuple_ast))
|
||||
{
|
||||
if (tuple_ast->arguments->as<ASTExpressionList>()->children.empty())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name);
|
||||
}
|
||||
auto first_arg = tuple_ast->arguments->as<ASTExpressionList>()->children.at(0);
|
||||
if (const auto * inner_tuple = first_arg->as<ASTFunction>(); inner_tuple && inner_tuple->name == "tuple")
|
||||
{
|
||||
const auto * arguments_ast = tuple_ast->arguments->as<ASTExpressionList>();
|
||||
if (arguments_ast)
|
||||
partition_ast_fields_count = arguments_ast->children.size();
|
||||
else
|
||||
partition_ast_fields_count = 0;
|
||||
}
|
||||
else if (const auto * inner_literal_tuple = first_arg->as<ASTLiteral>(); inner_literal_tuple)
|
||||
{
|
||||
if (inner_literal_tuple->value.getType() == Field::Types::Tuple)
|
||||
partition_ast_fields_count = inner_literal_tuple->value.safeGet<Tuple>().size();
|
||||
else
|
||||
partition_ast_fields_count = 1;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name);
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto * arguments_ast = tuple_ast->arguments->as<ASTExpressionList>();
|
||||
if (arguments_ast)
|
||||
partition_ast_fields_count = arguments_ast->children.size();
|
||||
else
|
||||
partition_ast_fields_count = 0;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID());
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
partition_ast_fields_count = *partition_ast.fields_count;
|
||||
}
|
||||
|
||||
if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
@ -7006,35 +7014,23 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour
|
||||
if (my_snapshot->getColumns().getAllPhysical().sizeOfDifference(src_snapshot->getColumns().getAllPhysical()))
|
||||
throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure");
|
||||
|
||||
if (queryToStringNullable(my_snapshot->getSortingKeyAST()) != queryToStringNullable(src_snapshot->getSortingKeyAST()))
|
||||
auto query_to_string = [] (const ASTPtr & ast)
|
||||
{
|
||||
return ast ? queryToString(ast) : "";
|
||||
};
|
||||
|
||||
if (query_to_string(my_snapshot->getSortingKeyAST()) != query_to_string(src_snapshot->getSortingKeyAST()))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different ordering");
|
||||
|
||||
if (query_to_string(my_snapshot->getPartitionKeyAST()) != query_to_string(src_snapshot->getPartitionKeyAST()))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key");
|
||||
|
||||
if (format_version != src_data->format_version)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different format_version");
|
||||
|
||||
if (queryToStringNullable(my_snapshot->getPrimaryKeyAST()) != queryToStringNullable(src_snapshot->getPrimaryKeyAST()))
|
||||
if (query_to_string(my_snapshot->getPrimaryKeyAST()) != query_to_string(src_snapshot->getPrimaryKeyAST()))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different primary key");
|
||||
|
||||
const auto is_a_subset_of = [](const auto & lhs, const auto & rhs)
|
||||
{
|
||||
if (lhs.size() > rhs.size())
|
||||
return false;
|
||||
|
||||
const auto rhs_set = NameSet(rhs.begin(), rhs.end());
|
||||
for (const auto & lhs_element : lhs)
|
||||
if (!rhs_set.contains(lhs_element))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!is_a_subset_of(my_snapshot->getColumnsRequiredForPartitionKey(), src_snapshot->getColumnsRequiredForPartitionKey()))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Destination table partition expression columns must be a subset of source table partition expression columns");
|
||||
}
|
||||
|
||||
const auto check_definitions = [](const auto & my_descriptions, const auto & src_descriptions)
|
||||
{
|
||||
if (my_descriptions.size() != src_descriptions.size())
|
||||
@ -7075,56 +7071,128 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
|
||||
const ReadSettings & read_settings,
|
||||
const WriteSettings & write_settings)
|
||||
{
|
||||
return MergeTreeDataPartCloner::clone(
|
||||
this, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, require_part_metadata, params, read_settings, write_settings);
|
||||
}
|
||||
/// Check that the storage policy contains the disk where the src_part is located.
|
||||
bool does_storage_policy_allow_same_disk = false;
|
||||
for (const DiskPtr & disk : getStoragePolicy()->getDisks())
|
||||
{
|
||||
if (disk->getName() == src_part->getDataPartStorage().getDiskName())
|
||||
{
|
||||
does_storage_policy_allow_same_disk = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!does_storage_policy_allow_same_disk)
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Could not clone and load part {} because disk does not belong to storage policy",
|
||||
quoteString(src_part->getDataPartStorage().getFullPath()));
|
||||
|
||||
std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAndLoadPartOnSameDiskWithDifferentPartitionKey(
|
||||
const MergeTreeData::DataPartPtr & src_part,
|
||||
const MergeTreePartition & new_partition,
|
||||
const String & partition_id,
|
||||
const IMergeTreeDataPart::MinMaxIndex & min_max_index,
|
||||
const String & tmp_part_prefix,
|
||||
const StorageMetadataPtr & my_metadata_snapshot,
|
||||
const IDataPartStorage::ClonePartParams & clone_params,
|
||||
ContextPtr local_context,
|
||||
Int64 min_block,
|
||||
Int64 max_block
|
||||
)
|
||||
{
|
||||
MergeTreePartInfo dst_part_info(partition_id, min_block, max_block, src_part->info.level);
|
||||
String dst_part_name = src_part->getNewName(dst_part_info);
|
||||
String tmp_dst_part_name = tmp_part_prefix + dst_part_name;
|
||||
auto temporary_directory_lock = getTemporaryPartDirectoryHolder(tmp_dst_part_name);
|
||||
|
||||
return MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression(
|
||||
this,
|
||||
src_part,
|
||||
my_metadata_snapshot,
|
||||
dst_part_info,
|
||||
tmp_part_prefix,
|
||||
local_context->getReadSettings(),
|
||||
local_context->getWriteSettings(),
|
||||
new_partition,
|
||||
min_max_index,
|
||||
false,
|
||||
clone_params);
|
||||
}
|
||||
/// Why it is needed if we only hardlink files?
|
||||
auto reservation = src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk());
|
||||
auto src_part_storage = src_part->getDataPartStoragePtr();
|
||||
|
||||
std::pair<MergeTreePartition, IMergeTreeDataPart::MinMaxIndex> MergeTreeData::createPartitionAndMinMaxIndexFromSourcePart(
|
||||
const MergeTreeData::DataPartPtr & src_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ContextPtr local_context)
|
||||
{
|
||||
const auto & src_data = src_part->storage;
|
||||
scope_guard src_flushed_tmp_dir_lock;
|
||||
MergeTreeData::MutableDataPartPtr src_flushed_tmp_part;
|
||||
|
||||
auto metadata_manager = std::make_shared<PartMetadataManagerOrdinary>(src_part.get());
|
||||
IMergeTreeDataPart::MinMaxIndex min_max_index;
|
||||
/// If source part is in memory, flush it to disk and clone it already in on-disk format
|
||||
/// Protect tmp dir from removing by cleanup thread with src_flushed_tmp_dir_lock
|
||||
/// Construct src_flushed_tmp_part in order to delete part with its directory at destructor
|
||||
if (auto src_part_in_memory = asInMemoryPart(src_part))
|
||||
{
|
||||
auto flushed_part_path = *src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix);
|
||||
|
||||
min_max_index.load(src_data, metadata_manager);
|
||||
auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename();
|
||||
src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name);
|
||||
|
||||
MergeTreePartition new_partition;
|
||||
auto flushed_part_storage = src_part_in_memory->flushToDisk(flushed_part_path, metadata_snapshot);
|
||||
|
||||
new_partition.create(metadata_snapshot, min_max_index.getBlock(src_data), 0u, local_context);
|
||||
src_flushed_tmp_part = MergeTreeDataPartBuilder(*this, src_part->name, flushed_part_storage)
|
||||
.withPartInfo(src_part->info)
|
||||
.withPartFormatFromDisk()
|
||||
.build();
|
||||
|
||||
return {new_partition, min_max_index};
|
||||
src_flushed_tmp_part->is_temp = true;
|
||||
src_part_storage = flushed_part_storage;
|
||||
}
|
||||
|
||||
String with_copy;
|
||||
if (params.copy_instead_of_hardlink)
|
||||
with_copy = " (copying data)";
|
||||
|
||||
auto dst_part_storage = src_part_storage->freeze(
|
||||
relative_data_path,
|
||||
tmp_dst_part_name,
|
||||
read_settings,
|
||||
write_settings,
|
||||
/* save_metadata_callback= */ {},
|
||||
params);
|
||||
|
||||
if (params.metadata_version_to_write.has_value())
|
||||
{
|
||||
chassert(!params.keep_metadata_version);
|
||||
auto out_metadata = dst_part_storage->writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, getContext()->getWriteSettings());
|
||||
writeText(metadata_snapshot->getMetadataVersion(), *out_metadata);
|
||||
out_metadata->finalize();
|
||||
if (getSettings()->fsync_after_insert)
|
||||
out_metadata->sync();
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Clone{} part {} to {}{}",
|
||||
src_flushed_tmp_part ? " flushed" : "",
|
||||
src_part_storage->getFullPath(),
|
||||
std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name),
|
||||
with_copy);
|
||||
|
||||
auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage)
|
||||
.withPartFormatFromDisk()
|
||||
.build();
|
||||
|
||||
if (!params.copy_instead_of_hardlink && params.hardlinked_files)
|
||||
{
|
||||
params.hardlinked_files->source_part_name = src_part->name;
|
||||
params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID();
|
||||
|
||||
for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next())
|
||||
{
|
||||
if (!params.files_to_copy_instead_of_hardlinks.contains(it->name())
|
||||
&& it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED
|
||||
&& it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME)
|
||||
{
|
||||
params.hardlinked_files->hardlinks_from_source_part.insert(it->name());
|
||||
}
|
||||
}
|
||||
|
||||
auto projections = src_part->getProjectionParts();
|
||||
for (const auto & [name, projection_part] : projections)
|
||||
{
|
||||
const auto & projection_storage = projection_part->getDataPartStorage();
|
||||
for (auto it = projection_storage.iterate(); it->isValid(); it->next())
|
||||
{
|
||||
auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name();
|
||||
if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix)
|
||||
&& it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED
|
||||
&& it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME)
|
||||
{
|
||||
params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// We should write version metadata on part creation to distinguish it from parts that were created without transaction.
|
||||
TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID;
|
||||
dst_data_part->version.setCreationTID(tid, nullptr);
|
||||
dst_data_part->storeVersionMetadata();
|
||||
|
||||
dst_data_part->is_temp = true;
|
||||
|
||||
dst_data_part->loadColumnsChecksumsIndexes(require_part_metadata, true);
|
||||
dst_data_part->modification_time = dst_part_storage->getLastModified().epochTime();
|
||||
return std::make_pair(dst_data_part, std::move(temporary_directory_lock));
|
||||
}
|
||||
|
||||
String MergeTreeData::getFullPathOnDisk(const DiskPtr & disk) const
|
||||
|
@ -231,7 +231,6 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
using DataParts = std::set<DataPartPtr, LessDataPart>;
|
||||
using MutableDataParts = std::set<MutableDataPartPtr, LessDataPart>;
|
||||
using DataPartsVector = std::vector<DataPartPtr>;
|
||||
@ -849,23 +848,6 @@ public:
|
||||
const ReadSettings & read_settings,
|
||||
const WriteSettings & write_settings);
|
||||
|
||||
std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> cloneAndLoadPartOnSameDiskWithDifferentPartitionKey(
|
||||
const MergeTreeData::DataPartPtr & src_part,
|
||||
const MergeTreePartition & new_partition,
|
||||
const String & partition_id,
|
||||
const IMergeTreeDataPart::MinMaxIndex & min_max_index,
|
||||
const String & tmp_part_prefix,
|
||||
const StorageMetadataPtr & my_metadata_snapshot,
|
||||
const IDataPartStorage::ClonePartParams & clone_params,
|
||||
ContextPtr local_context,
|
||||
Int64 min_block,
|
||||
Int64 max_block);
|
||||
|
||||
static std::pair<MergeTreePartition, IMergeTreeDataPart::MinMaxIndex> createPartitionAndMinMaxIndexFromSourcePart(
|
||||
const MergeTreeData::DataPartPtr & src_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ContextPtr local_context);
|
||||
|
||||
virtual std::vector<MergeTreeMutationStatus> getMutationsStatus() const = 0;
|
||||
|
||||
/// Returns true if table can create new parts with adaptive granularity
|
||||
|
@ -1,320 +0,0 @@
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartBuilder.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartCloner.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
static Poco::Logger * log = &Poco::Logger::get("MergeTreeDataPartCloner");
|
||||
|
||||
namespace DistinctPartitionExpression
|
||||
{
|
||||
std::unique_ptr<WriteBufferFromFileBase> updatePartitionFile(
|
||||
const MergeTreeData & merge_tree_data,
|
||||
const MergeTreePartition & partition,
|
||||
const MergeTreeData::MutableDataPartPtr & dst_part,
|
||||
IDataPartStorage & storage)
|
||||
{
|
||||
storage.removeFile("partition.dat");
|
||||
// Leverage already implemented MergeTreePartition::store to create & store partition.dat.
|
||||
// Checksum is re-calculated later.
|
||||
return partition.store(merge_tree_data, storage, dst_part->checksums);
|
||||
}
|
||||
|
||||
IMergeTreeDataPart::MinMaxIndex::WrittenFiles updateMinMaxFiles(
|
||||
const MergeTreeData & merge_tree_data,
|
||||
const MergeTreeData::MutableDataPartPtr & dst_part,
|
||||
IDataPartStorage & storage,
|
||||
const StorageMetadataPtr & metadata_snapshot)
|
||||
{
|
||||
for (const auto & column_name : MergeTreeData::getMinMaxColumnsNames(metadata_snapshot->partition_key))
|
||||
{
|
||||
auto file = "minmax_" + escapeForFileName(column_name) + ".idx";
|
||||
storage.removeFile(file);
|
||||
}
|
||||
|
||||
return dst_part->minmax_idx->store(merge_tree_data, storage, dst_part->checksums);
|
||||
}
|
||||
|
||||
void finalizeNewFiles(const std::vector<std::unique_ptr<WriteBufferFromFileBase>> & files, bool sync_new_files)
|
||||
{
|
||||
for (const auto & file : files)
|
||||
{
|
||||
file->finalize();
|
||||
if (sync_new_files)
|
||||
file->sync();
|
||||
}
|
||||
}
|
||||
|
||||
void updateNewPartFiles(
|
||||
const MergeTreeData & merge_tree_data,
|
||||
const MergeTreeData::MutableDataPartPtr & dst_part,
|
||||
const MergeTreePartition & new_partition,
|
||||
const IMergeTreeDataPart::MinMaxIndex & new_min_max_index,
|
||||
const StorageMetadataPtr & src_metadata_snapshot,
|
||||
bool sync_new_files)
|
||||
{
|
||||
auto & storage = dst_part->getDataPartStorage();
|
||||
|
||||
*dst_part->minmax_idx = new_min_max_index;
|
||||
|
||||
auto partition_file = updatePartitionFile(merge_tree_data, new_partition, dst_part, storage);
|
||||
|
||||
auto min_max_files = updateMinMaxFiles(merge_tree_data, dst_part, storage, src_metadata_snapshot);
|
||||
|
||||
IMergeTreeDataPart::MinMaxIndex::WrittenFiles written_files;
|
||||
|
||||
if (partition_file)
|
||||
written_files.emplace_back(std::move(partition_file));
|
||||
|
||||
written_files.insert(written_files.end(), std::make_move_iterator(min_max_files.begin()), std::make_move_iterator(min_max_files.end()));
|
||||
|
||||
finalizeNewFiles(written_files, sync_new_files);
|
||||
|
||||
// MergeTreeDataPartCloner::finalize_part calls IMergeTreeDataPart::loadColumnsChecksumsIndexes, which will re-create
|
||||
// the checksum file if it doesn't exist. Relying on that is cumbersome, but this refactoring is simply a code extraction
|
||||
// with small improvements. It can be further improved in the future.
|
||||
storage.removeFile("checksums.txt");
|
||||
}
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
bool doesStoragePolicyAllowSameDisk(MergeTreeData * merge_tree_data, const MergeTreeData::DataPartPtr & src_part)
|
||||
{
|
||||
for (const DiskPtr & disk : merge_tree_data->getStoragePolicy()->getDisks())
|
||||
if (disk->getName() == src_part->getDataPartStorage().getDiskName())
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
DataPartStoragePtr flushPartStorageToDiskIfInMemory(
|
||||
MergeTreeData * merge_tree_data,
|
||||
const MergeTreeData::DataPartPtr & src_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const String & tmp_part_prefix,
|
||||
const String & tmp_dst_part_name,
|
||||
scope_guard & src_flushed_tmp_dir_lock,
|
||||
MergeTreeData::MutableDataPartPtr src_flushed_tmp_part)
|
||||
{
|
||||
if (auto src_part_in_memory = asInMemoryPart(src_part))
|
||||
{
|
||||
auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix);
|
||||
auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename();
|
||||
|
||||
src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name);
|
||||
|
||||
auto flushed_part_storage = src_part_in_memory->flushToDisk(*flushed_part_path, metadata_snapshot);
|
||||
|
||||
src_flushed_tmp_part = MergeTreeDataPartBuilder(*merge_tree_data, src_part->name, flushed_part_storage)
|
||||
.withPartInfo(src_part->info)
|
||||
.withPartFormatFromDisk()
|
||||
.build();
|
||||
|
||||
src_flushed_tmp_part->is_temp = true;
|
||||
|
||||
return flushed_part_storage;
|
||||
}
|
||||
|
||||
return src_part->getDataPartStoragePtr();
|
||||
}
|
||||
|
||||
std::shared_ptr<IDataPartStorage> hardlinkAllFiles(
|
||||
MergeTreeData * merge_tree_data,
|
||||
const DB::ReadSettings & read_settings,
|
||||
const DB::WriteSettings & write_settings,
|
||||
const DataPartStoragePtr & storage,
|
||||
const String & path,
|
||||
const DB::IDataPartStorage::ClonePartParams & params)
|
||||
{
|
||||
return storage->freeze(
|
||||
merge_tree_data->getRelativeDataPath(),
|
||||
path,
|
||||
read_settings,
|
||||
write_settings,
|
||||
/*save_metadata_callback=*/{},
|
||||
params);
|
||||
}
|
||||
|
||||
std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> cloneSourcePart(
|
||||
MergeTreeData * merge_tree_data,
|
||||
const MergeTreeData::DataPartPtr & src_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreePartInfo & dst_part_info,
|
||||
const String & tmp_part_prefix,
|
||||
const ReadSettings & read_settings,
|
||||
const WriteSettings & write_settings,
|
||||
const DB::IDataPartStorage::ClonePartParams & params)
|
||||
{
|
||||
const auto dst_part_name = src_part->getNewName(dst_part_info);
|
||||
|
||||
const auto tmp_dst_part_name = tmp_part_prefix + dst_part_name;
|
||||
|
||||
auto temporary_directory_lock = merge_tree_data->getTemporaryPartDirectoryHolder(tmp_dst_part_name);
|
||||
|
||||
src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk());
|
||||
|
||||
scope_guard src_flushed_tmp_dir_lock;
|
||||
MergeTreeData::MutableDataPartPtr src_flushed_tmp_part;
|
||||
|
||||
auto src_part_storage = flushPartStorageToDiskIfInMemory(
|
||||
merge_tree_data, src_part, metadata_snapshot, tmp_part_prefix, tmp_dst_part_name, src_flushed_tmp_dir_lock, src_flushed_tmp_part);
|
||||
|
||||
auto dst_part_storage = hardlinkAllFiles(merge_tree_data, read_settings, write_settings, src_part_storage, tmp_dst_part_name, params);
|
||||
|
||||
if (params.metadata_version_to_write.has_value())
|
||||
{
|
||||
chassert(!params.keep_metadata_version);
|
||||
auto out_metadata = dst_part_storage->writeFile(
|
||||
IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, merge_tree_data->getContext()->getWriteSettings());
|
||||
writeText(metadata_snapshot->getMetadataVersion(), *out_metadata);
|
||||
out_metadata->finalize();
|
||||
if (merge_tree_data->getSettings()->fsync_after_insert)
|
||||
out_metadata->sync();
|
||||
}
|
||||
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Clone {} part {} to {}{}",
|
||||
src_flushed_tmp_part ? "flushed" : "",
|
||||
src_part_storage->getFullPath(),
|
||||
std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name),
|
||||
false);
|
||||
|
||||
|
||||
auto part = MergeTreeDataPartBuilder(*merge_tree_data, dst_part_name, dst_part_storage).withPartFormatFromDisk().build();
|
||||
|
||||
return std::make_pair(part, std::move(temporary_directory_lock));
|
||||
}
|
||||
|
||||
void handleHardLinkedParameterFiles(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params)
|
||||
{
|
||||
const auto & hardlinked_files = params.hardlinked_files;
|
||||
|
||||
hardlinked_files->source_part_name = src_part->name;
|
||||
hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID();
|
||||
|
||||
for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next())
|
||||
{
|
||||
if (!params.files_to_copy_instead_of_hardlinks.contains(it->name())
|
||||
&& it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED
|
||||
&& it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME)
|
||||
{
|
||||
hardlinked_files->hardlinks_from_source_part.insert(it->name());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void handleProjections(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params)
|
||||
{
|
||||
auto projections = src_part->getProjectionParts();
|
||||
for (const auto & [name, projection_part] : projections)
|
||||
{
|
||||
const auto & projection_storage = projection_part->getDataPartStorage();
|
||||
for (auto it = projection_storage.iterate(); it->isValid(); it->next())
|
||||
{
|
||||
auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name();
|
||||
if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix)
|
||||
&& it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED
|
||||
&& it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME)
|
||||
{
|
||||
params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr finalizePart(
|
||||
const MergeTreeData::MutableDataPartPtr & dst_part, const DB::IDataPartStorage::ClonePartParams & params, bool require_part_metadata)
|
||||
{
|
||||
/// We should write version metadata on part creation to distinguish it from parts that were created without transaction.
|
||||
TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID;
|
||||
dst_part->version.setCreationTID(tid, nullptr);
|
||||
dst_part->storeVersionMetadata();
|
||||
|
||||
dst_part->is_temp = true;
|
||||
|
||||
dst_part->loadColumnsChecksumsIndexes(require_part_metadata, true);
|
||||
|
||||
dst_part->modification_time = dst_part->getDataPartStorage().getLastModified().epochTime();
|
||||
|
||||
return dst_part;
|
||||
}
|
||||
|
||||
std::pair<MergeTreeDataPartCloner::MutableDataPartPtr, scope_guard> cloneAndHandleHardlinksAndProjections(
|
||||
MergeTreeData * merge_tree_data,
|
||||
const DataPartPtr & src_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreePartInfo & dst_part_info,
|
||||
const String & tmp_part_prefix,
|
||||
const ReadSettings & read_settings,
|
||||
const WriteSettings & write_settings,
|
||||
const IDataPartStorage::ClonePartParams & params)
|
||||
{
|
||||
if (!doesStoragePolicyAllowSameDisk(merge_tree_data, src_part))
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Could not clone and load part {} because disk does not belong to storage policy",
|
||||
quoteString(src_part->getDataPartStorage().getFullPath()));
|
||||
|
||||
auto [destination_part, temporary_directory_lock] = cloneSourcePart(
|
||||
merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params);
|
||||
|
||||
if (!params.copy_instead_of_hardlink && params.hardlinked_files)
|
||||
{
|
||||
handleHardLinkedParameterFiles(src_part, params);
|
||||
handleProjections(src_part, params);
|
||||
}
|
||||
|
||||
return std::make_pair(destination_part, std::move(temporary_directory_lock));
|
||||
}
|
||||
}
|
||||
|
||||
std::pair<MergeTreeDataPartCloner::MutableDataPartPtr, scope_guard> MergeTreeDataPartCloner::clone(
|
||||
MergeTreeData * merge_tree_data,
|
||||
const DataPartPtr & src_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreePartInfo & dst_part_info,
|
||||
const String & tmp_part_prefix,
|
||||
bool require_part_metadata,
|
||||
const IDataPartStorage::ClonePartParams & params,
|
||||
const ReadSettings & read_settings,
|
||||
const WriteSettings & write_settings)
|
||||
{
|
||||
auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections(
|
||||
merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params);
|
||||
|
||||
return std::make_pair(finalizePart(destination_part, params, require_part_metadata), std::move(temporary_directory_lock));
|
||||
}
|
||||
|
||||
std::pair<MergeTreeDataPartCloner::MutableDataPartPtr, scope_guard> MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression(
|
||||
MergeTreeData * merge_tree_data,
|
||||
const DataPartPtr & src_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreePartInfo & dst_part_info,
|
||||
const String & tmp_part_prefix,
|
||||
const ReadSettings & read_settings,
|
||||
const WriteSettings & write_settings,
|
||||
const MergeTreePartition & new_partition,
|
||||
const IMergeTreeDataPart::MinMaxIndex & new_min_max_index,
|
||||
bool sync_new_files,
|
||||
const IDataPartStorage::ClonePartParams & params)
|
||||
{
|
||||
auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections(
|
||||
merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params);
|
||||
|
||||
DistinctPartitionExpression::updateNewPartFiles(
|
||||
*merge_tree_data, destination_part, new_partition, new_min_max_index, src_part->storage.getInMemoryMetadataPtr(), sync_new_files);
|
||||
|
||||
return std::make_pair(finalizePart(destination_part, params, false), std::move(temporary_directory_lock));
|
||||
}
|
||||
|
||||
}
|
@ -1,43 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
struct MergeTreePartition;
|
||||
class IMergeTreeDataPart;
|
||||
|
||||
class MergeTreeDataPartCloner
|
||||
{
|
||||
public:
|
||||
using DataPart = IMergeTreeDataPart;
|
||||
using MutableDataPartPtr = std::shared_ptr<DataPart>;
|
||||
using DataPartPtr = std::shared_ptr<const DataPart>;
|
||||
|
||||
static std::pair<MutableDataPartPtr, scope_guard> clone(
|
||||
MergeTreeData * merge_tree_data,
|
||||
const DataPartPtr & src_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreePartInfo & dst_part_info,
|
||||
const String & tmp_part_prefix,
|
||||
bool require_part_metadata,
|
||||
const IDataPartStorage::ClonePartParams & params,
|
||||
const ReadSettings & read_settings,
|
||||
const WriteSettings & write_settings);
|
||||
|
||||
static std::pair<MutableDataPartPtr, scope_guard> cloneWithDistinctPartitionExpression(
|
||||
MergeTreeData * merge_tree_data,
|
||||
const DataPartPtr & src_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreePartInfo & dst_part_info,
|
||||
const String & tmp_part_prefix,
|
||||
const ReadSettings & read_settings,
|
||||
const WriteSettings & write_settings,
|
||||
const MergeTreePartition & new_partition,
|
||||
const IMergeTreeDataPart::MinMaxIndex & new_min_max_index,
|
||||
bool sync_new_files,
|
||||
const IDataPartStorage::ClonePartParams & params);
|
||||
};
|
||||
|
||||
}
|
@ -7,6 +7,7 @@
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Storages/BlockNumberColumn.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -143,13 +144,22 @@ void MergeTreeDataPartWriterWide::addStreams(
|
||||
auto ast = parseQuery(codec_parser, "(" + Poco::toUpper(settings.marks_compression_codec) + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||
CompressionCodecPtr marks_compression_codec = CompressionCodecFactory::instance().get(ast, nullptr);
|
||||
|
||||
const auto column_desc = metadata_snapshot->columns.tryGetColumnDescription(GetColumnsOptions(GetColumnsOptions::AllPhysical), column.getNameInStorage());
|
||||
|
||||
UInt64 max_compress_block_size = 0;
|
||||
if (column_desc)
|
||||
if (const auto * value = column_desc->settings.tryGet("max_compress_block_size"))
|
||||
max_compress_block_size = value->safeGet<UInt64>();
|
||||
if (!max_compress_block_size)
|
||||
max_compress_block_size = settings.max_compress_block_size;
|
||||
|
||||
column_streams[stream_name] = std::make_unique<Stream<false>>(
|
||||
stream_name,
|
||||
data_part->getDataPartStoragePtr(),
|
||||
stream_name, DATA_FILE_EXTENSION,
|
||||
stream_name, marks_file_extension,
|
||||
compression_codec,
|
||||
settings.max_compress_block_size,
|
||||
max_compress_block_size,
|
||||
marks_compression_codec,
|
||||
settings.marks_compress_block_size,
|
||||
settings.query_write_settings);
|
||||
@ -323,6 +333,13 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn(
|
||||
WrittenOffsetColumns & offset_columns)
|
||||
{
|
||||
StreamsWithMarks result;
|
||||
const auto column_desc = metadata_snapshot->columns.tryGetColumnDescription(GetColumnsOptions(GetColumnsOptions::AllPhysical), column.getNameInStorage());
|
||||
UInt64 min_compress_block_size = 0;
|
||||
if (column_desc)
|
||||
if (const auto * value = column_desc->settings.tryGet("min_compress_block_size"))
|
||||
min_compress_block_size = value->safeGet<UInt64>();
|
||||
if (!min_compress_block_size)
|
||||
min_compress_block_size = settings.min_compress_block_size;
|
||||
data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path)
|
||||
{
|
||||
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
|
||||
@ -335,7 +352,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn(
|
||||
auto & stream = *column_streams[stream_name];
|
||||
|
||||
/// There could already be enough data to compress into the new block.
|
||||
if (stream.compressed_hashing.offset() >= settings.min_compress_block_size)
|
||||
if (stream.compressed_hashing.offset() >= min_compress_block_size)
|
||||
stream.compressed_hashing.next();
|
||||
|
||||
StreamNameAndMark stream_with_mark;
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Common/OpenTelemetryTraceContext.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Disks/createVolume.h>
|
||||
@ -314,8 +315,13 @@ Block MergeTreeDataWriter::mergeBlock(
|
||||
IColumn::Permutation *& permutation,
|
||||
const MergeTreeData::MergingParams & merging_params)
|
||||
{
|
||||
OpenTelemetry::SpanHolder span("MergeTreeDataWriter::mergeBlock");
|
||||
|
||||
size_t block_size = block.rows();
|
||||
|
||||
span.addAttribute("clickhouse.rows", block_size);
|
||||
span.addAttribute("clickhouse.columns", block.columns());
|
||||
|
||||
auto get_merging_algorithm = [&]() -> std::shared_ptr<IMergingAlgorithm>
|
||||
{
|
||||
switch (merging_params.mode)
|
||||
@ -351,6 +357,8 @@ Block MergeTreeDataWriter::mergeBlock(
|
||||
if (!merging_algorithm)
|
||||
return block;
|
||||
|
||||
span.addAttribute("clickhouse.merging_algorithm", merging_algorithm->getName());
|
||||
|
||||
Chunk chunk(block.getColumns(), block_size);
|
||||
|
||||
IMergingAlgorithm::Input input;
|
||||
|
@ -467,45 +467,6 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreePartition::createAndValidateMinMaxPartitionIds(
|
||||
const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context)
|
||||
{
|
||||
if (!metadata_snapshot->hasPartitionKey())
|
||||
return;
|
||||
|
||||
auto partition_key_names_and_types = executePartitionByExpression(metadata_snapshot, block_with_min_max_partition_ids, context);
|
||||
value.resize(partition_key_names_and_types.size());
|
||||
|
||||
/// Executing partition_by expression adds new columns to passed block according to partition functions.
|
||||
/// The block is passed by reference and is used afterwards. `moduloLegacy` needs to be substituted back
|
||||
/// with just `modulo`, because it was a temporary substitution.
|
||||
static constexpr std::string_view modulo_legacy_function_name = "moduloLegacy";
|
||||
|
||||
size_t i = 0;
|
||||
for (const auto & element : partition_key_names_and_types)
|
||||
{
|
||||
auto & partition_column = block_with_min_max_partition_ids.getByName(element.name);
|
||||
|
||||
if (element.name.starts_with(modulo_legacy_function_name))
|
||||
partition_column.name.replace(0, modulo_legacy_function_name.size(), "modulo");
|
||||
|
||||
Field extracted_min_partition_id_field;
|
||||
Field extracted_max_partition_id_field;
|
||||
|
||||
partition_column.column->get(0, extracted_min_partition_id_field);
|
||||
partition_column.column->get(1, extracted_max_partition_id_field);
|
||||
|
||||
if (extracted_min_partition_id_field != extracted_max_partition_id_field)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_PARTITION_VALUE,
|
||||
"Can not create the partition. A partition can not contain values that have different partition ids");
|
||||
}
|
||||
|
||||
partition_column.column->get(0u, value[i++]);
|
||||
}
|
||||
}
|
||||
|
||||
NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context)
|
||||
{
|
||||
auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context);
|
||||
|
@ -1,12 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <base/types.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <Storages/KeyDescription.h>
|
||||
#include <Storages/MergeTree/IPartMetadataManager.h>
|
||||
#include <Storages/MergeTree/PartMetadataManagerOrdinary.h>
|
||||
#include <base/types.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -52,11 +51,6 @@ public:
|
||||
|
||||
void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context);
|
||||
|
||||
/// Copy of MergeTreePartition::create, but also validates if min max partition keys are equal. If they are different,
|
||||
/// it means the partition can't be created because the data doesn't belong to the same partition.
|
||||
void createAndValidateMinMaxPartitionIds(
|
||||
const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context);
|
||||
|
||||
static void appendFiles(const MergeTreeData & storage, Strings & files);
|
||||
|
||||
/// Adjust partition key and execute its expression on block. Return sample block according to used expression.
|
||||
|
@ -1,91 +0,0 @@
|
||||
#include <Interpreters/MonotonicityCheckVisitor.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h>
|
||||
#include <Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
bool isDestinationPartitionExpressionMonotonicallyIncreasing(
|
||||
const std::vector<Range> & hyperrectangle, const MergeTreeData & destination_storage)
|
||||
{
|
||||
auto destination_table_metadata = destination_storage.getInMemoryMetadataPtr();
|
||||
|
||||
auto key_description = destination_table_metadata->getPartitionKey();
|
||||
auto definition_ast = key_description.definition_ast->clone();
|
||||
|
||||
auto table_identifier = std::make_shared<ASTIdentifier>(destination_storage.getStorageID().getTableName());
|
||||
auto table_with_columns
|
||||
= TableWithColumnNamesAndTypes{DatabaseAndTableWithAlias(table_identifier), destination_table_metadata->getColumns().getOrdinary()};
|
||||
|
||||
auto expression_list = extractKeyExpressionList(definition_ast);
|
||||
|
||||
MonotonicityCheckVisitor::Data data{{table_with_columns}, destination_storage.getContext(), /*group_by_function_hashes*/ {}};
|
||||
|
||||
for (auto i = 0u; i < expression_list->children.size(); i++)
|
||||
{
|
||||
data.range = hyperrectangle[i];
|
||||
|
||||
MonotonicityCheckVisitor(data).visit(expression_list->children[i]);
|
||||
|
||||
if (!data.monotonicity.is_monotonic || !data.monotonicity.is_positive)
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool isExpressionDirectSubsetOf(const ASTPtr source, const ASTPtr destination)
|
||||
{
|
||||
auto source_expression_list = extractKeyExpressionList(source);
|
||||
auto destination_expression_list = extractKeyExpressionList(destination);
|
||||
|
||||
std::unordered_set<std::string> source_columns;
|
||||
|
||||
for (auto i = 0u; i < source_expression_list->children.size(); ++i)
|
||||
source_columns.insert(source_expression_list->children[i]->getColumnName());
|
||||
|
||||
for (auto i = 0u; i < destination_expression_list->children.size(); ++i)
|
||||
if (!source_columns.contains(destination_expression_list->children[i]->getColumnName()))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreePartitionCompatibilityVerifier::verify(
|
||||
const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts)
|
||||
{
|
||||
const auto source_metadata = source_storage.getInMemoryMetadataPtr();
|
||||
const auto destination_metadata = destination_storage.getInMemoryMetadataPtr();
|
||||
|
||||
const auto source_partition_key_ast = source_metadata->getPartitionKeyAST();
|
||||
const auto destination_partition_key_ast = destination_metadata->getPartitionKeyAST();
|
||||
|
||||
// If destination partition expression columns are a subset of source partition expression columns,
|
||||
// there is no need to check for monotonicity.
|
||||
if (isExpressionDirectSubsetOf(source_partition_key_ast, destination_partition_key_ast))
|
||||
return;
|
||||
|
||||
const auto src_global_min_max_indexes = MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(source_parts, destination_storage);
|
||||
|
||||
assert(!src_global_min_max_indexes.hyperrectangle.empty());
|
||||
|
||||
if (!isDestinationPartitionExpressionMonotonicallyIncreasing(src_global_min_max_indexes.hyperrectangle, destination_storage))
|
||||
throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Destination table partition expression is not monotonically increasing");
|
||||
|
||||
MergeTreePartition().createAndValidateMinMaxPartitionIds(
|
||||
destination_storage.getInMemoryMetadataPtr(),
|
||||
src_global_min_max_indexes.getBlock(destination_storage),
|
||||
destination_storage.getContext());
|
||||
}
|
||||
|
||||
}
|
@ -1,30 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/*
|
||||
* Verifies that source and destination partitions are compatible.
|
||||
* To be compatible, one of the following criteria must be met:
|
||||
* 1. Destination partition expression columns are a subset of source partition columns; or
|
||||
* 2. Destination partition expression is monotonic on the source global min_max idx Range AND the computer partition id for
|
||||
* the source global min_max idx range is the same.
|
||||
*
|
||||
* If not, an exception is thrown.
|
||||
* */
|
||||
|
||||
class MergeTreePartitionCompatibilityVerifier
|
||||
{
|
||||
public:
|
||||
using DataPart = IMergeTreeDataPart;
|
||||
using DataPartPtr = std::shared_ptr<const DataPart>;
|
||||
using DataPartsVector = std::vector<DataPartPtr>;
|
||||
|
||||
static void
|
||||
verify(const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts);
|
||||
};
|
||||
|
||||
}
|
@ -1,25 +0,0 @@
|
||||
#include <Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
IMergeTreeDataPart::MinMaxIndex
|
||||
MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(const DataPartsVector & parts, const MergeTreeData & storage)
|
||||
{
|
||||
IMergeTreeDataPart::MinMaxIndex global_min_max_indexes;
|
||||
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
auto metadata_manager = std::make_shared<PartMetadataManagerOrdinary>(part.get());
|
||||
|
||||
auto local_min_max_index = MergeTreeData::DataPart::MinMaxIndex();
|
||||
|
||||
local_min_max_index.load(storage, metadata_manager);
|
||||
|
||||
global_min_max_indexes.merge(local_min_max_index);
|
||||
}
|
||||
|
||||
return global_min_max_indexes;
|
||||
}
|
||||
|
||||
}
|
@ -1,24 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <utility>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/*
|
||||
* Calculates global min max indexes for a given set of parts on given storage.
|
||||
* */
|
||||
class MergeTreePartitionGlobalMinMaxIdxCalculator
|
||||
{
|
||||
using DataPart = IMergeTreeDataPart;
|
||||
using DataPartPtr = std::shared_ptr<const DataPart>;
|
||||
using DataPartsVector = std::vector<DataPartPtr>;
|
||||
|
||||
public:
|
||||
static IMergeTreeDataPart::MinMaxIndex calculate(const DataPartsVector & parts, const MergeTreeData & storage);
|
||||
};
|
||||
|
||||
}
|
@ -213,6 +213,27 @@ void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeColumnSettings::validate(const SettingsChanges & changes)
|
||||
{
|
||||
static const MergeTreeSettings merge_tree_settings;
|
||||
static const std::set<String> allowed_column_level_settings =
|
||||
{
|
||||
"min_compress_block_size",
|
||||
"max_compress_block_size"
|
||||
};
|
||||
|
||||
for (const auto & change : changes)
|
||||
{
|
||||
if (!allowed_column_level_settings.contains(change.name))
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_SETTING,
|
||||
"Setting {} is unknown or not supported at column level, supported settings: {}",
|
||||
change.name,
|
||||
fmt::join(allowed_column_level_settings, ", "));
|
||||
merge_tree_settings.checkCanSet(change.name, change.value);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
std::vector<String> MergeTreeSettings::getAllRegisteredNames() const
|
||||
{
|
||||
|
@ -277,4 +277,11 @@ struct MergeTreeSettings : public BaseSettings<MergeTreeSettingsTraits>, public
|
||||
|
||||
using MergeTreeSettingsPtr = std::shared_ptr<const MergeTreeSettings>;
|
||||
|
||||
|
||||
/// Column-level Merge-Tree settings which overwrite MergeTree settings
|
||||
namespace MergeTreeColumnSettings
|
||||
{
|
||||
void validate(const SettingsChanges & changes);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,9 +5,9 @@
|
||||
#include <optional>
|
||||
#include <ranges>
|
||||
|
||||
#include <base/sort.h>
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <IO/copyData.h>
|
||||
#include "Common/Exception.h"
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
@ -20,30 +20,27 @@
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/ClusterProxy/executeQuery.h>
|
||||
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
|
||||
#include <Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <Parsers/ASTCheckQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTPartition.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Planner/Utils.h>
|
||||
#include <Storages/buildQueryTreeForShard.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/ActiveDataPartSet.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/MergeTree/MergeList.h>
|
||||
#include <Storages/MergeTree/MergePlainMergeTreeTask.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h>
|
||||
#include <Storages/MergeTree/MergeTreeSink.h>
|
||||
#include <Storages/MergeTree/PartMetadataManagerOrdinary.h>
|
||||
#include <Storages/MergeTree/PartitionPruner.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <Storages/PartitionCommands.h>
|
||||
#include <base/sort.h>
|
||||
#include <Storages/buildQueryTreeForShard.h>
|
||||
#include <Storages/MergeTree/MergeTreeSink.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/MergeTree/MergePlainMergeTreeTask.h>
|
||||
#include <Storages/MergeTree/PartitionPruner.h>
|
||||
#include <Storages/MergeTree/MergeList.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
@ -218,16 +215,25 @@ void StorageMergeTree::read(
|
||||
{
|
||||
if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree)
|
||||
{
|
||||
const auto table_id = getStorageID();
|
||||
const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery(
|
||||
local_context, query_info.query,
|
||||
table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr);
|
||||
|
||||
ASTPtr modified_query_ast;
|
||||
Block header;
|
||||
if (local_context->getSettingsRef().allow_experimental_analyzer)
|
||||
header = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze());
|
||||
{
|
||||
QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone();
|
||||
rewriteJoinToGlobalJoin(modified_query_tree);
|
||||
modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree);
|
||||
header = InterpreterSelectQueryAnalyzer::getSampleBlock(
|
||||
modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze());
|
||||
modified_query_ast = queryNodeToSelectQuery(modified_query_tree);
|
||||
}
|
||||
else
|
||||
header = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
|
||||
{
|
||||
const auto table_id = getStorageID();
|
||||
modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query,
|
||||
table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr);
|
||||
header
|
||||
= InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
|
||||
}
|
||||
|
||||
ClusterProxy::SelectStreamFactory select_stream_factory =
|
||||
ClusterProxy::SelectStreamFactory(
|
||||
@ -238,7 +244,6 @@ void StorageMergeTree::read(
|
||||
|
||||
ClusterProxy::executeQueryWithParallelReplicas(
|
||||
query_plan,
|
||||
getStorageID(),
|
||||
select_stream_factory,
|
||||
modified_query_ast,
|
||||
local_context,
|
||||
@ -2044,74 +2049,42 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con
|
||||
ProfileEventsScope profile_events_scope;
|
||||
|
||||
MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot);
|
||||
String partition_id = src_data.getPartitionIDFromQuery(partition, local_context);
|
||||
String partition_id = getPartitionIDFromQuery(partition, local_context);
|
||||
|
||||
DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id);
|
||||
|
||||
bool attach_empty_partition = !replace && src_parts.empty();
|
||||
if (attach_empty_partition)
|
||||
return;
|
||||
|
||||
MutableDataPartsVector dst_parts;
|
||||
std::vector<scope_guard> dst_parts_locks;
|
||||
|
||||
static const String TMP_PREFIX = "tmp_replace_from_";
|
||||
|
||||
const auto my_partition_expression = my_metadata_snapshot->getPartitionKeyAST();
|
||||
const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST();
|
||||
const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression);
|
||||
|
||||
if (is_partition_exp_different && !src_parts.empty())
|
||||
MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_parts);
|
||||
|
||||
for (DataPartPtr & src_part : src_parts)
|
||||
for (const DataPartPtr & src_part : src_parts)
|
||||
{
|
||||
if (!canReplacePartition(src_part))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Cannot replace partition '{}' because part '{}' has inconsistent granularity with table",
|
||||
partition_id, src_part->name);
|
||||
|
||||
IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()};
|
||||
/// This will generate unique name in scope of current server process.
|
||||
auto index = insert_increment.get();
|
||||
Int64 temp_index = insert_increment.get();
|
||||
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
|
||||
|
||||
if (is_partition_exp_different)
|
||||
{
|
||||
auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart(
|
||||
src_part, my_metadata_snapshot, local_context);
|
||||
|
||||
auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey(
|
||||
src_part,
|
||||
new_partition,
|
||||
new_partition.getID(*this),
|
||||
new_min_max_index,
|
||||
TMP_PREFIX,
|
||||
my_metadata_snapshot,
|
||||
clone_params,
|
||||
local_context,
|
||||
index,
|
||||
index);
|
||||
|
||||
dst_parts.emplace_back(std::move(dst_part));
|
||||
dst_parts_locks.emplace_back(std::move(part_lock));
|
||||
}
|
||||
else
|
||||
{
|
||||
MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level);
|
||||
|
||||
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(
|
||||
src_part,
|
||||
TMP_PREFIX,
|
||||
dst_part_info,
|
||||
my_metadata_snapshot,
|
||||
clone_params,
|
||||
local_context->getReadSettings(),
|
||||
local_context->getWriteSettings());
|
||||
dst_parts.emplace_back(std::move(dst_part));
|
||||
dst_parts_locks.emplace_back(std::move(part_lock));
|
||||
}
|
||||
IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()};
|
||||
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(
|
||||
src_part,
|
||||
TMP_PREFIX,
|
||||
dst_part_info,
|
||||
my_metadata_snapshot,
|
||||
clone_params,
|
||||
local_context->getReadSettings(),
|
||||
local_context->getWriteSettings());
|
||||
dst_parts.emplace_back(std::move(dst_part));
|
||||
dst_parts_locks.emplace_back(std::move(part_lock));
|
||||
}
|
||||
|
||||
/// ATTACH empty part set
|
||||
if (!replace && dst_parts.empty())
|
||||
return;
|
||||
|
||||
MergeTreePartInfo drop_range;
|
||||
if (replace)
|
||||
{
|
||||
|
@ -26,21 +26,22 @@
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <Storages/buildQueryTreeForShard.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/Freeze.h>
|
||||
#include <Storages/MergeTree/AsyncBlockIDsCache.h>
|
||||
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
|
||||
#include <Storages/MergeTree/extractZkPathFromCreateQuery.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/LeaderElection.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/MergeFromLogEntryTask.h>
|
||||
#include <Storages/MergeTree/MergeList.h>
|
||||
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataFormatVersion.h>
|
||||
#include <Storages/MergeTree/MergeTreePartInfo.h>
|
||||
#include <Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h>
|
||||
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/MutateFromLogEntryTask.h>
|
||||
#include <Storages/MergeTree/PinnedPartUUIDs.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
|
||||
@ -52,11 +53,9 @@
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeSink.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
|
||||
#include <Storages/MergeTree/ZeroCopyLock.h>
|
||||
#include <Storages/MergeTree/extractZkPathFromCreateQuery.h>
|
||||
#include <Storages/PartitionCommands.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Storages/buildQueryTreeForShard.h>
|
||||
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
@ -2714,48 +2713,16 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry)
|
||||
.copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || ((our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport()),
|
||||
.metadata_version_to_write = metadata_snapshot->getMetadataVersion()
|
||||
};
|
||||
|
||||
const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST();
|
||||
const auto src_partition_expression = source_table->getInMemoryMetadataPtr()->getPartitionKeyAST();
|
||||
|
||||
const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression);
|
||||
|
||||
if (is_partition_exp_different)
|
||||
{
|
||||
auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart(
|
||||
part_desc->src_table_part, metadata_snapshot, getContext());
|
||||
|
||||
auto partition_id = new_partition.getID(*this);
|
||||
|
||||
auto [res_part, temporary_part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey(
|
||||
part_desc->src_table_part,
|
||||
new_partition,
|
||||
partition_id,
|
||||
new_min_max_index,
|
||||
TMP_PREFIX + "clone_",
|
||||
metadata_snapshot,
|
||||
clone_params,
|
||||
getContext(),
|
||||
part_desc->new_part_info.min_block,
|
||||
part_desc->new_part_info.max_block);
|
||||
|
||||
part_desc->res_part = std::move(res_part);
|
||||
part_desc->temporary_part_lock = std::move(temporary_part_lock);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk(
|
||||
part_desc->src_table_part,
|
||||
TMP_PREFIX + "clone_",
|
||||
part_desc->new_part_info,
|
||||
metadata_snapshot,
|
||||
clone_params,
|
||||
getContext()->getReadSettings(),
|
||||
getContext()->getWriteSettings());
|
||||
|
||||
part_desc->res_part = std::move(res_part);
|
||||
part_desc->temporary_part_lock = std::move(temporary_part_lock);
|
||||
}
|
||||
auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk(
|
||||
part_desc->src_table_part,
|
||||
TMP_PREFIX + "clone_",
|
||||
part_desc->new_part_info,
|
||||
metadata_snapshot,
|
||||
clone_params,
|
||||
getContext()->getReadSettings(),
|
||||
getContext()->getWriteSettings());
|
||||
part_desc->res_part = std::move(res_part);
|
||||
part_desc->temporary_part_lock = std::move(temporary_part_lock);
|
||||
}
|
||||
else if (!part_desc->replica.empty())
|
||||
{
|
||||
@ -5418,7 +5385,9 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl(
|
||||
|
||||
if (local_context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree);
|
||||
QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone();
|
||||
rewriteJoinToGlobalJoin(modified_query_tree);
|
||||
modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree);
|
||||
|
||||
header = InterpreterSelectQueryAnalyzer::getSampleBlock(
|
||||
modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze());
|
||||
@ -5441,7 +5410,6 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl(
|
||||
|
||||
ClusterProxy::executeQueryWithParallelReplicas(
|
||||
query_plan,
|
||||
getStorageID(),
|
||||
select_stream_factory,
|
||||
modified_query_ast,
|
||||
local_context,
|
||||
@ -7885,22 +7853,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
|
||||
ProfileEventsScope profile_events_scope;
|
||||
|
||||
MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot);
|
||||
String partition_id = src_data.getPartitionIDFromQuery(partition, query_context);
|
||||
String partition_id = getPartitionIDFromQuery(partition, query_context);
|
||||
|
||||
/// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet.
|
||||
DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id);
|
||||
|
||||
bool attach_empty_partition = !replace && src_all_parts.empty();
|
||||
if (attach_empty_partition)
|
||||
return;
|
||||
|
||||
const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST();
|
||||
const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST();
|
||||
const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression);
|
||||
|
||||
if (is_partition_exp_different && !src_all_parts.empty())
|
||||
MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_all_parts);
|
||||
|
||||
LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size());
|
||||
|
||||
static const String TMP_PREFIX = "tmp_replace_from_";
|
||||
@ -7955,18 +7912,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
|
||||
"Cannot replace partition '{}' because part '{}"
|
||||
"' has inconsistent granularity with table", partition_id, src_part->name);
|
||||
|
||||
IMergeTreeDataPart::MinMaxIndex min_max_index = *src_part->minmax_idx;
|
||||
MergeTreePartition merge_tree_partition = src_part->partition;
|
||||
|
||||
if (is_partition_exp_different)
|
||||
{
|
||||
auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart(src_part, metadata_snapshot, query_context);
|
||||
|
||||
merge_tree_partition = new_partition;
|
||||
min_max_index = new_min_max_index;
|
||||
partition_id = merge_tree_partition.getID(*this);
|
||||
}
|
||||
|
||||
String hash_hex = src_part->checksums.getTotalChecksumHex();
|
||||
const bool is_duplicated_part = replaced_parts.contains(hash_hex);
|
||||
replaced_parts.insert(hash_hex);
|
||||
@ -7985,52 +7930,27 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
|
||||
continue;
|
||||
}
|
||||
|
||||
UInt64 index = lock->getNumber();
|
||||
MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level);
|
||||
|
||||
bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication
|
||||
|| dynamic_cast<const MergeTreeData *>(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication;
|
||||
|
||||
UInt64 index = lock->getNumber();
|
||||
|
||||
IDataPartStorage::ClonePartParams clone_params
|
||||
{
|
||||
.copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()),
|
||||
.metadata_version_to_write = metadata_snapshot->getMetadataVersion()
|
||||
};
|
||||
|
||||
if (is_partition_exp_different)
|
||||
{
|
||||
auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey(
|
||||
src_part,
|
||||
merge_tree_partition,
|
||||
partition_id,
|
||||
min_max_index,
|
||||
TMP_PREFIX,
|
||||
metadata_snapshot,
|
||||
clone_params,
|
||||
query_context,
|
||||
index,
|
||||
index);
|
||||
|
||||
dst_parts.emplace_back(dst_part);
|
||||
dst_parts_locks.emplace_back(std::move(part_lock));
|
||||
}
|
||||
else
|
||||
{
|
||||
MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level);
|
||||
|
||||
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(
|
||||
src_part,
|
||||
TMP_PREFIX,
|
||||
dst_part_info,
|
||||
metadata_snapshot,
|
||||
clone_params,
|
||||
query_context->getReadSettings(),
|
||||
query_context->getWriteSettings());
|
||||
|
||||
dst_parts.emplace_back(dst_part);
|
||||
dst_parts_locks.emplace_back(std::move(part_lock));
|
||||
}
|
||||
|
||||
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(
|
||||
src_part,
|
||||
TMP_PREFIX,
|
||||
dst_part_info,
|
||||
metadata_snapshot,
|
||||
clone_params,
|
||||
query_context->getReadSettings(),
|
||||
query_context->getWriteSettings());
|
||||
src_parts.emplace_back(src_part);
|
||||
dst_parts.emplace_back(dst_part);
|
||||
dst_parts_locks.emplace_back(std::move(part_lock));
|
||||
ephemeral_locks.emplace_back(std::move(*lock));
|
||||
block_id_paths.emplace_back(block_id_path);
|
||||
part_checksums.emplace_back(hash_hex);
|
||||
|
@ -373,11 +373,37 @@ QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeN
|
||||
|
||||
removeGroupingFunctionSpecializations(query_tree_to_modify);
|
||||
|
||||
// std::cerr << "====================== build 1 \n" << query_tree_to_modify->dumpTree() << std::endl;
|
||||
createUniqueTableAliases(query_tree_to_modify, nullptr, planner_context->getQueryContext());
|
||||
// std::cerr << "====================== build 2 \n" << query_tree_to_modify->dumpTree() << std::endl;
|
||||
|
||||
return query_tree_to_modify;
|
||||
}
|
||||
|
||||
class RewriteJoinToGlobalJoinVisitor : public InDepthQueryTreeVisitor<RewriteJoinToGlobalJoinVisitor>
|
||||
{
|
||||
public:
|
||||
using Base = InDepthQueryTreeVisitor<RewriteJoinToGlobalJoinVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (auto * join_node = node->as<JoinNode>())
|
||||
join_node->setLocality(JoinLocality::Global);
|
||||
}
|
||||
|
||||
static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child)
|
||||
{
|
||||
auto * join_node = parent->as<JoinNode>();
|
||||
if (join_node && join_node->getRightTableExpression() == child)
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify)
|
||||
{
|
||||
RewriteJoinToGlobalJoinVisitor visitor;
|
||||
visitor.visit(query_tree_to_modify);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,4 +12,6 @@ using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
|
||||
|
||||
QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify);
|
||||
|
||||
void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify);
|
||||
|
||||
}
|
||||
|
@ -27,8 +27,9 @@
|
||||
00917_multiple_joins_denny_crane
|
||||
02725_agg_projection_resprect_PK
|
||||
02763_row_policy_storage_merge_alias
|
||||
02784_parallel_replicas_automatic_decision_join
|
||||
02818_parameterized_view_with_cte_multiple_usage
|
||||
# Check after constants refactoring
|
||||
02901_parallel_replicas_rollup
|
||||
# Flaky. Please don't delete them without fixing them:
|
||||
01287_max_execution_speed
|
||||
02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET
|
||||
|
@ -1,17 +0,0 @@
|
||||
<clickhouse>
|
||||
<remote_servers>
|
||||
<test_cluster>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>replica1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>replica2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster>
|
||||
</remote_servers>
|
||||
</clickhouse>
|
@ -1,214 +0,0 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import assert_eq_with_retry
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
replica1 = cluster.add_instance(
|
||||
"replica1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"]
|
||||
)
|
||||
replica2 = cluster.add_instance(
|
||||
"replica2", with_zookeeper=True, main_configs=["configs/remote_servers.xml"]
|
||||
)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
except Exception as ex:
|
||||
print(ex)
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def cleanup(nodes):
|
||||
for node in nodes:
|
||||
node.query("DROP TABLE IF EXISTS source SYNC")
|
||||
node.query("DROP TABLE IF EXISTS destination SYNC")
|
||||
|
||||
|
||||
def create_table(node, table_name, replicated):
|
||||
replica = node.name
|
||||
engine = (
|
||||
f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')"
|
||||
if replicated
|
||||
else "MergeTree()"
|
||||
)
|
||||
partition_expression = (
|
||||
"toYYYYMMDD(timestamp)" if table_name == "source" else "toYYYYMM(timestamp)"
|
||||
)
|
||||
node.query_with_retry(
|
||||
"""
|
||||
CREATE TABLE {table_name}(timestamp DateTime)
|
||||
ENGINE = {engine}
|
||||
ORDER BY tuple() PARTITION BY {partition_expression}
|
||||
SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1;
|
||||
""".format(
|
||||
table_name=table_name,
|
||||
engine=engine,
|
||||
partition_expression=partition_expression,
|
||||
)
|
||||
)
|
||||
|
||||
|
||||
def test_both_replicated(start_cluster):
|
||||
for node in [replica1, replica2]:
|
||||
create_table(node, "source", True)
|
||||
create_table(node, "destination", True)
|
||||
|
||||
replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')")
|
||||
replica1.query("SYSTEM SYNC REPLICA source")
|
||||
replica1.query("SYSTEM SYNC REPLICA destination")
|
||||
replica1.query(
|
||||
f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source"
|
||||
)
|
||||
|
||||
assert_eq_with_retry(
|
||||
replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n"
|
||||
)
|
||||
assert_eq_with_retry(
|
||||
replica1,
|
||||
f"SELECT * FROM destination",
|
||||
replica2.query(f"SELECT * FROM destination"),
|
||||
)
|
||||
|
||||
cleanup([replica1, replica2])
|
||||
|
||||
|
||||
def test_only_destination_replicated(start_cluster):
|
||||
create_table(replica1, "source", False)
|
||||
create_table(replica1, "destination", True)
|
||||
create_table(replica2, "destination", True)
|
||||
|
||||
replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')")
|
||||
replica1.query("SYSTEM SYNC REPLICA destination")
|
||||
replica1.query(
|
||||
f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source"
|
||||
)
|
||||
|
||||
assert_eq_with_retry(
|
||||
replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n"
|
||||
)
|
||||
assert_eq_with_retry(
|
||||
replica1,
|
||||
f"SELECT * FROM destination",
|
||||
replica2.query(f"SELECT * FROM destination"),
|
||||
)
|
||||
|
||||
cleanup([replica1, replica2])
|
||||
|
||||
|
||||
def test_both_replicated_partitioned_to_unpartitioned(start_cluster):
|
||||
def create_tables(nodes):
|
||||
for node in nodes:
|
||||
source_engine = (
|
||||
f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')"
|
||||
)
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE source(timestamp DateTime)
|
||||
ENGINE = {engine}
|
||||
ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp)
|
||||
SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1;
|
||||
""".format(
|
||||
engine=source_engine,
|
||||
)
|
||||
)
|
||||
|
||||
destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')"
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE destination(timestamp DateTime)
|
||||
ENGINE = {engine}
|
||||
ORDER BY tuple() PARTITION BY tuple()
|
||||
SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1;
|
||||
""".format(
|
||||
engine=destination_engine,
|
||||
)
|
||||
)
|
||||
|
||||
create_tables([replica1, replica2])
|
||||
|
||||
replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')")
|
||||
replica1.query("INSERT INTO source VALUES ('2010-03-03 02:01:01')")
|
||||
replica1.query("SYSTEM SYNC REPLICA source")
|
||||
replica1.query("SYSTEM SYNC REPLICA destination")
|
||||
|
||||
replica1.query(
|
||||
f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source"
|
||||
)
|
||||
replica1.query(
|
||||
f"ALTER TABLE destination ATTACH PARTITION ID '20100303' FROM source"
|
||||
)
|
||||
|
||||
assert_eq_with_retry(
|
||||
replica1,
|
||||
f"SELECT * FROM destination ORDER BY timestamp",
|
||||
"2010-03-02 02:01:01\n2010-03-03 02:01:01\n",
|
||||
)
|
||||
assert_eq_with_retry(
|
||||
replica1,
|
||||
f"SELECT * FROM destination ORDER BY timestamp",
|
||||
replica2.query(f"SELECT * FROM destination ORDER BY timestamp"),
|
||||
)
|
||||
|
||||
cleanup([replica1, replica2])
|
||||
|
||||
|
||||
def test_both_replicated_different_exp_same_id(start_cluster):
|
||||
def create_tables(nodes):
|
||||
for node in nodes:
|
||||
source_engine = (
|
||||
f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')"
|
||||
)
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE source(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8)
|
||||
ENGINE = {engine}
|
||||
ORDER BY tuple() PARTITION BY a % 3
|
||||
SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1;
|
||||
""".format(
|
||||
engine=source_engine,
|
||||
)
|
||||
)
|
||||
|
||||
destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')"
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE destination(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8)
|
||||
ENGINE = {engine}
|
||||
ORDER BY tuple() PARTITION BY a
|
||||
SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1;
|
||||
""".format(
|
||||
engine=destination_engine,
|
||||
)
|
||||
)
|
||||
|
||||
create_tables([replica1, replica2])
|
||||
|
||||
replica1.query(
|
||||
"INSERT INTO source (a, b, c, extra, sign) VALUES (1, 5, 9, 1000, 1)"
|
||||
)
|
||||
replica1.query(
|
||||
"INSERT INTO source (a, b, c, extra, sign) VALUES (2, 6, 10, 1000, 1)"
|
||||
)
|
||||
replica1.query("SYSTEM SYNC REPLICA source")
|
||||
replica1.query("SYSTEM SYNC REPLICA destination")
|
||||
|
||||
replica1.query(f"ALTER TABLE destination ATTACH PARTITION 1 FROM source")
|
||||
replica1.query(f"ALTER TABLE destination ATTACH PARTITION 2 FROM source")
|
||||
|
||||
assert_eq_with_retry(
|
||||
replica1,
|
||||
f"SELECT * FROM destination ORDER BY a",
|
||||
"1\t5\t9\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n2\t6\t10\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n",
|
||||
)
|
||||
assert_eq_with_retry(
|
||||
replica1,
|
||||
f"SELECT * FROM destination ORDER BY a",
|
||||
replica2.query(f"SELECT * FROM destination ORDER BY a"),
|
||||
)
|
||||
|
||||
cleanup([replica1, replica2])
|
@ -0,0 +1,6 @@
|
||||
<clickhouse>
|
||||
|
||||
<asynchronous_metrics_update_period_s>60000</asynchronous_metrics_update_period_s>
|
||||
<asynchronous_heavy_metrics_update_period_s>60000</asynchronous_heavy_metrics_update_period_s>
|
||||
|
||||
</clickhouse>
|
47
tests/integration/test_system_reload_async_metrics/test.py
Normal file
47
tests/integration/test_system_reload_async_metrics/test.py
Normal file
@ -0,0 +1,47 @@
|
||||
import os
|
||||
import pytest
|
||||
import shutil
|
||||
import time
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
# Tests that SYSTEM RELOAD ASYNCHRONOUS METRICS works.
|
||||
|
||||
# Config default.xml sets a large refresh interval of asynchronous metrics, so that the periodic updates don't interfere with the manual
|
||||
# update below.
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node = cluster.add_instance(
|
||||
"node",
|
||||
main_configs=["configs/default.xml"],
|
||||
stay_alive=True,
|
||||
)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
CONFIG_DIR = os.path.join(SCRIPT_DIR, "configs")
|
||||
|
||||
|
||||
def test_system_reload_async_metrics(start_cluster):
|
||||
node.query("SYSTEM DROP QUERY CACHE")
|
||||
|
||||
res1 = node.query(
|
||||
"SELECT value FROM system.asynchronous_metrics WHERE metric = 'NumberOfTables'"
|
||||
)
|
||||
|
||||
# create table and test that the table creation is reflected in the asynchronous metrics
|
||||
node.query("CREATE TABLE tab (col UInt64) ENGINE MergeTree ORDER BY tuple()")
|
||||
|
||||
node.query("SYSTEM RELOAD ASYNCHRONOUS METRICS")
|
||||
|
||||
res2 = node.query(
|
||||
"SELECT value FROM system.asynchronous_metrics WHERE metric = 'NumberOfTables'"
|
||||
)
|
||||
assert int(res1.rstrip()) + 1 == int(res2.rstrip())
|
@ -120,6 +120,7 @@ SYSTEM RELOAD DICTIONARY ['SYSTEM RELOAD DICTIONARIES','RELOAD DICTIONARY','RELO
|
||||
SYSTEM RELOAD MODEL ['SYSTEM RELOAD MODELS','RELOAD MODEL','RELOAD MODELS'] GLOBAL SYSTEM RELOAD
|
||||
SYSTEM RELOAD FUNCTION ['SYSTEM RELOAD FUNCTIONS','RELOAD FUNCTION','RELOAD FUNCTIONS'] GLOBAL SYSTEM RELOAD
|
||||
SYSTEM RELOAD EMBEDDED DICTIONARIES ['RELOAD EMBEDDED DICTIONARIES'] GLOBAL SYSTEM RELOAD
|
||||
SYSTEM RELOAD ASYNCHRONOUS METRICS ['RELOAD ASYNCHRONOUS METRICS'] GLOBAL SYSTEM RELOAD
|
||||
SYSTEM RELOAD [] \N SYSTEM
|
||||
SYSTEM RESTART DISK ['SYSTEM RESTART DISK'] GLOBAL SYSTEM
|
||||
SYSTEM MERGES ['SYSTEM STOP MERGES','SYSTEM START MERGES','STOP MERGES','START MERGES'] TABLE SYSTEM
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user