Merge remote-tracking branch 'blessed/master' into 02294_decimal_second_errors

This commit is contained in:
Raúl Marín 2024-01-23 20:24:39 +01:00
commit a9f6dfa74d
111 changed files with 1202 additions and 2647 deletions

View File

@ -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

View File

@ -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)).

View File

@ -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 cant 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;
```

View File

@ -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**

View File

@ -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 columns type, default expression and TTL.
- [MODIFY COLUMN](#modify-column) — Changes columns 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 cant 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`).

View File

@ -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.

View File

@ -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();

View File

@ -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) \

View File

@ -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"

View File

@ -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:

View File

@ -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;
}

View File

@ -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);
};
}

View File

@ -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
{

View File

@ -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;
};

View File

@ -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();

View File

@ -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"},

View File

@ -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);

View File

@ -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>

View File

@ -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)
{

View File

@ -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,

View File

@ -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;
};

View File

@ -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())

View File

@ -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,

View File

@ -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(); }

View File

@ -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();

View File

@ -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_))
{
}

View File

@ -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;

View File

@ -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:

View File

@ -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())

View File

@ -36,6 +36,8 @@ public:
std::string getFileName() const override;
size_t getFileOffsetOfBufferEnd() const override;
int getFD() const;
size_t getFileSize() override;

View File

@ -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();
}
}

View File

@ -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:

View File

@ -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; }
};
}

View File

@ -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

View File

@ -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;

View File

@ -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;

View File

@ -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);
}
}

View File

@ -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;
};
}

View File

@ -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;
};

View File

@ -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; }

View File

@ -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());
}

View File

@ -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();

View File

@ -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();

View File

@ -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),

View File

@ -70,7 +70,6 @@ void executeQuery(
void executeQueryWithParallelReplicas(
QueryPlan & query_plan,
const StorageID & main_table,
SelectStreamFactory & stream_factory,
const ASTPtr & query_ast,
ContextPtr context,

View File

@ -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, [&] {

View File

@ -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

View File

@ -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));
}

View File

@ -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:
{

View File

@ -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>;

View File

@ -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();

View File

@ -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);
};
}

View File

@ -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);

View File

@ -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};
}
}

View File

@ -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);
}

View File

@ -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;

View File

@ -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 << ')';
}
}

View File

@ -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;
};
}

View File

@ -67,6 +67,7 @@ public:
RELOAD_EMBEDDED_DICTIONARIES,
RELOAD_CONFIG,
RELOAD_USERS,
RELOAD_ASYNCHRONOUS_METRICS,
RESTART_DISK,
STOP_MERGES,
START_MERGES,

View File

@ -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))

View File

@ -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;

View File

@ -3,11 +3,6 @@
namespace DB
{
String queryToStringNullable(const ASTPtr & query)
{
return query ? queryToString(query) : "";
}
String queryToString(const ASTPtr & query)
{
return queryToString(*query);

View File

@ -6,5 +6,4 @@ namespace DB
{
String queryToString(const ASTPtr & query);
String queryToString(const IAST & query);
String queryToStringNullable(const ASTPtr & query);
}

View File

@ -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)

View File

@ -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_)

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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");

View File

@ -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;

View File

@ -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_))

View File

@ -21,7 +21,7 @@ namespace DB
class IAsynchronousReader;
class AsynchronousReadBufferFromHDFS : public BufferWithOwnMemory<SeekableReadBuffer>, public WithFileName, public WithFileSize
class AsynchronousReadBufferFromHDFS : public ReadBufferFromFileBase
{
public:
AsynchronousReadBufferFromHDFS(

View File

@ -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
{

View File

@ -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>>;

View File

@ -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)

View File

@ -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

View File

@ -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

View File

@ -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));
}
}

View File

@ -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);
};
}

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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.

View File

@ -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());
}
}

View File

@ -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);
};
}

View File

@ -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;
}
}

View File

@ -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);
};
}

View File

@ -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
{

View File

@ -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);
}
}

View File

@ -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)
{

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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);
}

View File

@ -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

View File

@ -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>

View File

@ -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])

View File

@ -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>

View 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())

View File

@ -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