Merge branch 'ClickHouse:master' into fix_dag_split

This commit is contained in:
李扬 2024-02-29 10:37:12 +08:00 committed by GitHub
commit f482f16114
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
787 changed files with 16267 additions and 6316 deletions

View File

@ -12,6 +12,7 @@ tests/ci/cancel_and_rerun_workflow_lambda/app.py
- Build/Testing/Packaging Improvement
- Documentation (changelog entry is not required)
- Bug Fix (user-visible misbehavior in an official stable release)
- CI Fix or Improvement (changelog entry is not required)
- Not for changelog (changelog entry is not required)

View File

@ -10,6 +10,7 @@ set (CMAKE_CXX_STANDARD 20)
set (SRCS
argsToConfig.cpp
cgroupsv2.cpp
coverage.cpp
demangle.cpp
getAvailableMemoryAmount.cpp

View File

@ -185,7 +185,8 @@ inline bool memequalWide(const char * p1, const char * p2, size_t size)
{
case 3: if (!compare8(p1 + 32, p2 + 32)) return false; [[fallthrough]];
case 2: if (!compare8(p1 + 16, p2 + 16)) return false; [[fallthrough]];
case 1: if (!compare8(p1, p2)) return false;
case 1: if (!compare8(p1, p2)) return false; [[fallthrough]];
default: ;
}
return compare8(p1 + size - 16, p2 + size - 16);

64
base/base/cgroupsv2.cpp Normal file
View File

@ -0,0 +1,64 @@
#include <base/cgroupsv2.h>
#include <base/defines.h>
#include <fstream>
#include <sstream>
bool cgroupsV2Enabled()
{
#if defined(OS_LINUX)
/// This file exists iff the host has cgroups v2 enabled.
auto controllers_file = default_cgroups_mount / "cgroup.controllers";
if (!std::filesystem::exists(controllers_file))
return false;
return true;
#else
return false;
#endif
}
bool cgroupsV2MemoryControllerEnabled()
{
#if defined(OS_LINUX)
chassert(cgroupsV2Enabled());
/// According to https://docs.kernel.org/admin-guide/cgroup-v2.html:
/// - file 'cgroup.controllers' defines which controllers *can* be enabled
/// - file 'cgroup.subtree_control' defines which controllers *are* enabled
/// Caveat: nested groups may disable controllers. For simplicity, check only the top-level group.
std::ifstream subtree_control_file(default_cgroups_mount / "cgroup.subtree_control");
if (!subtree_control_file.is_open())
return false;
std::string controllers;
std::getline(subtree_control_file, controllers);
if (controllers.find("memory") == std::string::npos)
return false;
return true;
#else
return false;
#endif
}
std::string cgroupV2OfProcess()
{
#if defined(OS_LINUX)
chassert(cgroupsV2Enabled());
/// All PIDs assigned to a cgroup are in /sys/fs/cgroups/{cgroup_name}/cgroup.procs
/// A simpler way to get the membership is:
std::ifstream cgroup_name_file("/proc/self/cgroup");
if (!cgroup_name_file.is_open())
return "";
/// With cgroups v2, there will be a *single* line with prefix "0::/"
/// (see https://docs.kernel.org/admin-guide/cgroup-v2.html)
std::string cgroup;
std::getline(cgroup_name_file, cgroup);
static const std::string v2_prefix = "0::/";
if (!cgroup.starts_with(v2_prefix))
return "";
cgroup = cgroup.substr(v2_prefix.length());
return cgroup;
#else
return "";
#endif
}

22
base/base/cgroupsv2.h Normal file
View File

@ -0,0 +1,22 @@
#pragma once
#include <filesystem>
#include <string>
#if defined(OS_LINUX)
/// I think it is possible to mount the cgroups hierarchy somewhere else (e.g. when in containers).
/// /sys/fs/cgroup was still symlinked to the actual mount in the cases that I have seen.
static inline const std::filesystem::path default_cgroups_mount = "/sys/fs/cgroup";
#endif
/// Is cgroups v2 enabled on the system?
bool cgroupsV2Enabled();
/// Is the memory controller of cgroups v2 enabled on the system?
/// Assumes that cgroupsV2Enabled() is enabled.
bool cgroupsV2MemoryControllerEnabled();
/// Which cgroup does the process belong to?
/// Returns an empty string if the cgroup cannot be determined.
/// Assumes that cgroupsV2Enabled() is enabled.
std::string cgroupV2OfProcess();

View File

@ -1,17 +1,14 @@
#include <base/getMemoryAmount.h>
#include <base/cgroupsv2.h>
#include <base/getPageSize.h>
#include <fstream>
#include <sstream>
#include <stdexcept>
#include <unistd.h>
#include <sys/types.h>
#include <sys/param.h>
#if defined(BSD)
#include <sys/sysctl.h>
#endif
namespace
@ -20,49 +17,14 @@ namespace
std::optional<uint64_t> getCgroupsV2MemoryLimit()
{
#if defined(OS_LINUX)
const std::filesystem::path default_cgroups_mount = "/sys/fs/cgroup";
/// This file exists iff the host has cgroups v2 enabled.
std::ifstream controllers_file(default_cgroups_mount / "cgroup.controllers");
if (!controllers_file.is_open())
if (!cgroupsV2Enabled())
return {};
/// Make sure that the memory controller is enabled.
/// - cgroup.controllers defines which controllers *can* be enabled.
/// - cgroup.subtree_control defines which controllers *are* enabled.
/// (see https://docs.kernel.org/admin-guide/cgroup-v2.html)
/// Caveat: nested groups may disable controllers. For simplicity, check only the top-level group.
/// ReadBufferFromFile subtree_control_file(default_cgroups_mount / "cgroup.subtree_control");
/// std::string subtree_control;
/// readString(subtree_control, subtree_control_file);
/// if (subtree_control.find("memory") == std::string::npos)
/// return {};
std::ifstream subtree_control_file(default_cgroups_mount / "cgroup.subtree_control");
std::stringstream subtree_control_buf;
subtree_control_buf << subtree_control_file.rdbuf();
std::string subtree_control = subtree_control_buf.str();
if (subtree_control.find("memory") == std::string::npos)
if (!cgroupsV2MemoryControllerEnabled())
return {};
/// Identify the cgroup the process belongs to
/// All PIDs assigned to a cgroup are in /sys/fs/cgroups/{cgroup_name}/cgroup.procs
/// A simpler way to get the membership is:
std::ifstream cgroup_name_file("/proc/self/cgroup");
if (!cgroup_name_file.is_open())
return {};
std::stringstream cgroup_name_buf;
cgroup_name_buf << cgroup_name_file.rdbuf();
std::string cgroup_name = cgroup_name_buf.str();
if (!cgroup_name.empty() && cgroup_name.back() == '\n')
cgroup_name.pop_back(); /// remove trailing newline, if any
/// With cgroups v2, there will be a *single* line with prefix "0::/"
const std::string v2_prefix = "0::/";
if (!cgroup_name.starts_with(v2_prefix))
return {};
cgroup_name = cgroup_name.substr(v2_prefix.length());
std::filesystem::path current_cgroup = cgroup_name.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup_name);
std::string cgroup = cgroupV2OfProcess();
auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup);
/// Open the bottom-most nested memory limit setting file. If there is no such file at the current
/// level, try again at the parent level as memory settings are inherited.

View File

@ -68,7 +68,7 @@ public:
typedef typename Bucket::iterator BucketIterator;
typedef typename BucketVec::iterator BucketVecIterator;
class ConstIterator : public std::iterator<std::forward_iterator_tag, Value>
class ConstIterator
{
public:
ConstIterator() : _initialized(false) { }

View File

@ -46,5 +46,6 @@ if (COMPILER_CLANG)
no_warning(thread-safety-negative) # experimental flag, too many false positives
no_warning(enum-constexpr-conversion) # breaks magic-enum library in clang-16
no_warning(unsafe-buffer-usage) # too aggressive
no_warning(switch-default) # conflicts with "defaults in a switch covering all enum values"
# TODO Enable conversion, sign-conversion, double-promotion warnings.
endif ()

2
contrib/aws vendored

@ -1 +1 @@
Subproject commit 9eb5097a0abfa837722cca7a5114a25837817bf2
Subproject commit 5f0542b3ad7eef25b0540d37d778207e0345ea8f

2
contrib/liburing vendored

@ -1 +1 @@
Subproject commit f5a48392c4ea33f222cbebeb2e2fc31620162949
Subproject commit f4e42a515cd78c8c9cac2be14222834be5f8df2b

View File

@ -86,7 +86,7 @@ function download
chmod +x clickhouse
# clickhouse may be compressed - run once to decompress
./clickhouse ||:
./clickhouse --query "SELECT 1" ||:
ln -s ./clickhouse ./clickhouse-server
ln -s ./clickhouse ./clickhouse-client
ln -s ./clickhouse ./clickhouse-local
@ -387,6 +387,11 @@ if [ -f core.zst ]; then
fi
rg --text -F '<Fatal>' server.log > fatal.log ||:
FATAL_LINK=''
if [ -s fatal.log ]; then
FATAL_LINK='<a href="fatal.log">fatal.log</a>'
fi
dmesg -T > dmesg.log ||:
zstd --threads=0 --rm server.log
@ -419,6 +424,7 @@ p.links a { padding: 5px; margin: 3px; background: #FFF; line-height: 2; white-s
<a href="main.log">main.log</a>
<a href="dmesg.log">dmesg.log</a>
${CORE_LINK}
${FATAL_LINK}
</p>
<table>
<tr>

View File

@ -403,4 +403,3 @@ sidebar_label: 2023
* Do not remove part if `Too many open files` is thrown [#56238](https://github.com/ClickHouse/ClickHouse/pull/56238) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix ORC commit [#56261](https://github.com/ClickHouse/ClickHouse/pull/56261) ([Raúl Marín](https://github.com/Algunenano)).
* Fix typo in largestTriangleThreeBuckets.md [#56263](https://github.com/ClickHouse/ClickHouse/pull/56263) ([Nikita Taranov](https://github.com/nickitat)).

View File

@ -596,4 +596,3 @@ sidebar_label: 2023
* Fix assertion from stress test [#50718](https://github.com/ClickHouse/ClickHouse/pull/50718) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix flaky unit test [#50719](https://github.com/ClickHouse/ClickHouse/pull/50719) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Show correct sharing state in system.query_cache [#50728](https://github.com/ClickHouse/ClickHouse/pull/50728) ([Robert Schulze](https://github.com/rschu1ze)).

View File

@ -298,4 +298,3 @@ sidebar_label: 2023
* Update version_date.tsv and changelogs after v23.4.5.22-stable [#51638](https://github.com/ClickHouse/ClickHouse/pull/51638) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Update version_date.tsv and changelogs after v23.3.7.5-lts [#51639](https://github.com/ClickHouse/ClickHouse/pull/51639) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Update parts.md [#51643](https://github.com/ClickHouse/ClickHouse/pull/51643) ([Ramazan Polat](https://github.com/ramazanpolat)).

View File

@ -588,4 +588,3 @@ sidebar_label: 2023
* tests: mark 02152_http_external_tables_memory_tracking as no-parallel [#54155](https://github.com/ClickHouse/ClickHouse/pull/54155) ([Azat Khuzhin](https://github.com/azat)).
* The external logs have had colliding arguments [#54165](https://github.com/ClickHouse/ClickHouse/pull/54165) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Rename macro [#54169](https://github.com/ClickHouse/ClickHouse/pull/54169) ([Kseniia Sumarokova](https://github.com/kssenii)).

View File

@ -379,4 +379,3 @@ sidebar_label: 2023
* Fix typo in packager when ccache is used [#55104](https://github.com/ClickHouse/ClickHouse/pull/55104) ([Ilya Yatsishin](https://github.com/qoega)).
* Reduce flakiness of 01455_opentelemetry_distributed [#55111](https://github.com/ClickHouse/ClickHouse/pull/55111) ([Michael Kolupaev](https://github.com/al13n321)).
* Fix build [#55113](https://github.com/ClickHouse/ClickHouse/pull/55113) ([Alexey Milovidov](https://github.com/alexey-milovidov)).

View File

@ -37,7 +37,7 @@ sudo xcode-select --install
``` bash
brew update
brew install ccache cmake ninja libtool gettext llvm gcc binutils grep findutils
brew install ccache cmake ninja libtool gettext llvm gcc binutils grep findutils nasm
```
## Checkout ClickHouse Sources {#checkout-clickhouse-sources}

View File

@ -870,6 +870,11 @@ Tags:
- `load_balancing` - Policy for disk balancing, `round_robin` or `least_used`.
- `least_used_ttl_ms` - Configure timeout (in milliseconds) for the updating available space on all disks (`0` - update always, `-1` - never update, default is `60000`). Note, if the disk can be used by ClickHouse only and is not subject to a online filesystem resize/shrink you can use `-1`, in all other cases it is not recommended, since eventually it will lead to incorrect space distribution.
- `prefer_not_to_merge` — You should not use this setting. Disables merging of data parts on this volume (this is harmful and leads to performance degradation). When this setting is enabled (don't do it), merging data on this volume is not allowed (which is bad). This allows (but you don't need it) controlling (if you want to control something, you're making a mistake) how ClickHouse works with slow disks (but ClickHouse knows better, so please don't use this setting).
- `volume_priority` — Defines the priority (order) in which volumes are filled. Lower value means higher priority. The parameter values should be natural numbers and collectively cover the range from 1 to N (lowest priority given) without skipping any numbers.
* If _all_ volumes are tagged, they are prioritized in given order.
* If only _some_ volumes are tagged, those without the tag have the lowest priority, and they are prioritized in the order they are defined in config.
* If _no_ volumes are tagged, their priority is set correspondingly to their order they are declared in configuration.
* Two volumes cannot have the same priority value.
Configuration examples:
@ -919,7 +924,8 @@ In given example, the `hdd_in_order` policy implements the [round-robin](https:/
If there are different kinds of disks available in the system, `moving_from_ssd_to_hdd` policy can be used instead. The volume `hot` consists of an SSD disk (`fast_ssd`), and the maximum size of a part that can be stored on this volume is 1GB. All the parts with the size larger than 1GB will be stored directly on the `cold` volume, which contains an HDD disk `disk1`.
Also, once the disk `fast_ssd` gets filled by more than 80%, data will be transferred to the `disk1` by a background process.
The order of volume enumeration within a storage policy is important. Once a volume is overfilled, data are moved to the next one. The order of disk enumeration is important as well because data are stored on them in turns.
The order of volume enumeration within a storage policy is important in case at least one of the volumes listed has no explicit `volume_priority` parameter.
Once a volume is overfilled, data are moved to the next one. The order of disk enumeration is important as well because data are stored on them in turns.
When creating a table, one can apply one of the configured storage policies to it:

View File

@ -304,6 +304,24 @@ We use the term `MergeTree` to refer to all table engines in the `MergeTree fami
If you had a `MergeTree` table that was manually replicated, you can convert it to a replicated table. You might need to do this if you have already collected a large amount of data in a `MergeTree` table and now you want to enable replication.
`MergeTree` table can be automatically converted on server restart if `convert_to_replicated` flag is set at the table's data directory (`/var/lib/clickhouse/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/` for `Atomic` database).
Create empty `convert_to_replicated` file and the table will be loaded as replicated on next server restart.
This query can be used to get the table's data path.
```sql
SELECT data_paths FROM system.tables WHERE table = 'table_name' AND database = 'database_name';
```
Note that ReplicatedMergeTree table will be created with values of `default_replica_path` and `default_replica_name` settings.
To create a converted table on other replicas, you will need to explicitly specify its path in the first argument of the `ReplicatedMergeTree` engine. The following query can be used to get its path.
```sql
SELECT zookeeper_path FROM system.replicas WHERE table = 'table_name';
```
There is also a manual way to do this without server restart.
If the data differs on various replicas, first sync it, or delete this data on all the replicas except one.
Rename the existing MergeTree table, then create a `ReplicatedMergeTree` table with the old name.

View File

@ -74,6 +74,10 @@ Specifying the `sharding_key` is necessary for the following:
`fsync_directories` - do the `fsync` for directories. Guarantees that the OS refreshed directory metadata after operations related to background inserts on Distributed table (after insert, after sending the data to shard, etc.).
#### skip_unavailable_shards
`skip_unavailable_shards` - If true, ClickHouse silently skips unavailable shards. Shard is marked as unavailable when: 1) The shard cannot be reached due to a connection failure. 2) Shard is unresolvable through DNS. 3) Table does not exist on the shard. Default false.
#### bytes_to_throw_insert
`bytes_to_throw_insert` - if more than this number of compressed bytes will be pending for background INSERT, an exception will be thrown. 0 - do not throw. Default 0.
@ -102,6 +106,10 @@ Specifying the `sharding_key` is necessary for the following:
`background_insert_max_sleep_time_ms` - same as [distributed_background_insert_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_max_sleep_time_ms)
#### flush_on_detach
`flush_on_detach` - Flush data to remote nodes on DETACH/DROP/server shutdown. Default true.
:::note
**Durability settings** (`fsync_...`):

View File

@ -79,10 +79,7 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun
#### Setup the Debian repository
``` bash
sudo apt-get install -y apt-transport-https ca-certificates dirmngr
GNUPGHOME=$(mktemp -d)
sudo GNUPGHOME="$GNUPGHOME" gpg --no-default-keyring --keyring /usr/share/keyrings/clickhouse-keyring.gpg --keyserver hkp://keyserver.ubuntu.com:80 --recv-keys 8919F6BD2B48D754
sudo rm -rf "$GNUPGHOME"
sudo chmod +r /usr/share/keyrings/clickhouse-keyring.gpg
sudo gpg --no-default-keyring --keyring /usr/share/keyrings/clickhouse-keyring.gpg --keyserver hkp://keyserver.ubuntu.com:80 --recv-keys 8919F6BD2B48D754
echo "deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb stable main" | sudo tee \
/etc/apt/sources.list.d/clickhouse.list

View File

@ -306,3 +306,18 @@ License: [commercial](https://tablum.io/pricing) product with 3-month free perio
Try it out for free [in the cloud](https://tablum.io/try).
Learn more about the product at [TABLUM.IO](https://tablum.io/)
### CKMAN {#ckman}
[CKMAN] (https://www.github.com/housepower/ckman) is a tool for managing and monitoring ClickHouse clusters!
Features:
- Rapid and convenient automated deployment of clusters through a browser interface
- Clusters can be scaled or scaled
- Load balance the data of the cluster
- Upgrade the cluster online
- Modify the cluster configuration on the page
- Provides cluster node monitoring and zookeeper monitoring
- Monitor the status of tables and partitions, and monitor slow SQL statements
- Provides an easy-to-use SQL execution page

View File

@ -199,6 +199,20 @@ Type: Bool
Default: 0
## dns_cache_max_size
Internal DNS cache max size in bytes.
:::note
ClickHouse also has a reverse cache, so the actual memory usage could be twice as much.
:::
Type: UInt64
Default: 1024
## dns_cache_update_period
Internal DNS cache update period in seconds.
@ -458,6 +472,38 @@ Type: Double
Default: 0.9
## cgroups_memory_usage_observer_wait_time
Interval in seconds during which the server's maximum allowed memory consumption is adjusted by the corresponding threshold in cgroups. (see
settings `cgroup_memory_watcher_hard_limit_ratio` and `cgroup_memory_watcher_soft_limit_ratio`).
Type: UInt64
Default: 15
## cgroup_memory_watcher_hard_limit_ratio
Specifies the "hard" threshold with regards to the memory consumption of the server process according to cgroups after which the server's
maximum memory consumption is adjusted to the threshold value.
See settings `cgroups_memory_usage_observer_wait_time` and `cgroup_memory_watcher_soft_limit_ratio`
Type: Double
Default: 0.95
## cgroup_memory_watcher_soft_limit_ratio
Specifies the "soft" threshold with regards to the memory consumption of the server process according to cgroups after which arenas in
jemalloc are purged.
See settings `cgroups_memory_usage_observer_wait_time` and `cgroup_memory_watcher_hard_limit_ratio`
Type: Double
Default: 0.95
## max_table_size_to_drop
Restriction on deleting tables.
@ -2881,3 +2927,15 @@ If set to true, then alter operations will be surrounded by parentheses in forma
Type: Bool
Default: 0
## ignore_empty_sql_security_in_create_view_query {#ignore_empty_sql_security_in_create_view_query}
If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries.
:::note
This setting is only necessary for the migration period and will become obsolete in 24.4
:::
Type: Bool
Default: 1

View File

@ -1656,6 +1656,33 @@ Result:
└─────────────────────────┴─────────┘
```
### output_format_pretty_single_large_number_tip_threshold {#output_format_pretty_single_large_number_tip_threshold}
Print a readable number tip on the right side of the table if the block consists of a single number which exceeds
this value (except 0).
Possible values:
- 0 — The readable number tip will not be printed.
- Positive integer — The readable number tip will be printed if the single number exceeds this value.
Default value: `1000000`.
**Example**
Query:
```sql
SELECT 1000000000 as a;
```
Result:
```text
┌──────────a─┐
│ 1000000000 │ -- 1.00 billion
└────────────┘
```
## Template format settings {#template-format-settings}
### format_template_resultset {#format_template_resultset}

View File

@ -755,7 +755,7 @@ By default: 1,000,000. It only works when reading from MergeTree engines.
## max_concurrent_queries_for_user {#max-concurrent-queries-for-user}
The maximum number of simultaneously processed queries related to MergeTree table per user.
The maximum number of simultaneously processed queries per user.
Possible values:
@ -1776,7 +1776,7 @@ Default value: 0 (no restriction).
## insert_quorum {#insert_quorum}
:::note
`insert_quorum` does not apply when using the [`SharedMergeTree` table engine](/en/cloud/reference/shared-merge-tree) in ClickHouse Cloud as all inserts are quorum inserted.
This setting is not applicable to SharedMergeTree, see [SharedMergeTree consistency](/docs/en/cloud/reference/shared-merge-tree/#consistency) for more information.
:::
Enables the quorum writes.
@ -1819,7 +1819,7 @@ See also:
## insert_quorum_parallel {#insert_quorum_parallel}
:::note
`insert_quorum_parallel` does not apply when using the [`SharedMergeTree` table engine](/en/cloud/reference/shared-merge-tree) in ClickHouse Cloud as all inserts are quorum inserted.
This setting is not applicable to SharedMergeTree, see [SharedMergeTree consistency](/docs/en/cloud/reference/shared-merge-tree/#consistency) for more information.
:::
Enables or disables parallelism for quorum `INSERT` queries. If enabled, additional `INSERT` queries can be sent while previous queries have not yet finished. If disabled, additional writes to the same table will be rejected.
@ -1839,6 +1839,10 @@ See also:
## select_sequential_consistency {#select_sequential_consistency}
:::note
This setting differ in behavior between SharedMergeTree and ReplicatedMergeTree, see [SharedMergeTree consistency](/docs/en/cloud/reference/shared-merge-tree/#consistency) for more information about the behavior of `select_sequential_consistency` in SharedMergeTree.
:::
Enables or disables sequential consistency for `SELECT` queries. Requires `insert_quorum_parallel` to be disabled (enabled by default).
Possible values:
@ -2037,7 +2041,7 @@ Possible values:
- 0 — Disabled.
- 1 — Enabled.
Default value: 1.
Default value: 0.
By default, async inserts are inserted into replicated tables by the `INSERT` statement enabling [async_insert](#async-insert) are deduplicated (see [Data Replication](../../engines/table-engines/mergetree-family/replication.md)).
For the replicated tables, by default, only 10000 of the most recent inserts for each partition are deduplicated (see [replicated_deduplication_window_for_async_inserts](merge-tree-settings.md/#replicated-deduplication-window-async-inserts), [replicated_deduplication_window_seconds_for_async_inserts](merge-tree-settings.md/#replicated-deduplication-window-seconds-async-inserts)).
@ -3445,7 +3449,7 @@ Has an effect only when the connection is made through the MySQL wire protocol.
- 0 - Use `BLOB`.
- 1 - Use `TEXT`.
Default value: `0`.
Default value: `1`.
## mysql_map_fixed_string_to_text_in_show_columns {#mysql_map_fixed_string_to_text_in_show_columns}
@ -3456,7 +3460,7 @@ Has an effect only when the connection is made through the MySQL wire protocol.
- 0 - Use `BLOB`.
- 1 - Use `TEXT`.
Default value: `0`.
Default value: `1`.
## execute_merges_on_single_replica_time_threshold {#execute-merges-on-single-replica-time-threshold}
@ -3706,7 +3710,7 @@ Default value: `0`.
## allow_experimental_live_view {#allow-experimental-live-view}
Allows creation of experimental [live views](../../sql-reference/statements/create/view.md/#live-view).
Allows creation of a deprecated LIVE VIEW.
Possible values:
@ -3717,21 +3721,15 @@ Default value: `0`.
## live_view_heartbeat_interval {#live-view-heartbeat-interval}
Sets the heartbeat interval in seconds to indicate [live view](../../sql-reference/statements/create/view.md/#live-view) is alive .
Default value: `15`.
Deprecated.
## max_live_view_insert_blocks_before_refresh {#max-live-view-insert-blocks-before-refresh}
Sets the maximum number of inserted blocks after which mergeable blocks are dropped and query for [live view](../../sql-reference/statements/create/view.md/#live-view) is re-executed.
Default value: `64`.
Deprecated.
## periodic_live_view_refresh {#periodic-live-view-refresh}
Sets the interval in seconds after which periodically refreshed [live view](../../sql-reference/statements/create/view.md/#live-view) is forced to refresh.
Default value: `60`.
Deprecated.
## http_connection_timeout {#http_connection_timeout}
@ -4279,6 +4277,41 @@ Result:
└─────┴─────┴───────┘
```
## enable_order_by_all {#enable-order-by-all}
Enables or disables sorting with `ORDER BY ALL` syntax, see [ORDER BY](../../sql-reference/statements/select/order-by.md).
Possible values:
- 0 — Disable ORDER BY ALL.
- 1 — Enable ORDER BY ALL.
Default value: `1`.
**Example**
Query:
```sql
CREATE TABLE TAB(C1 Int, C2 Int, ALL Int) ENGINE=Memory();
INSERT INTO TAB VALUES (10, 20, 30), (20, 20, 10), (30, 10, 20);
SELECT * FROM TAB ORDER BY ALL; -- returns an error that ALL is ambiguous
SELECT * FROM TAB ORDER BY ALL SETTINGS enable_order_by_all = 0;
```
Result:
```text
┌─C1─┬─C2─┬─ALL─┐
│ 20 │ 20 │ 10 │
│ 30 │ 10 │ 20 │
│ 10 │ 20 │ 30 │
└────┴────┴─────┘
```
## splitby_max_substrings_includes_remaining_string {#splitby_max_substrings_includes_remaining_string}
Controls whether function [splitBy*()](../../sql-reference/functions/splitting-merging-functions.md) with argument `max_substrings` > 0 will include the remaining string in the last element of the result array.
@ -5345,6 +5378,24 @@ SELECT map('a', range(number), 'b', number, 'c', 'str_' || toString(number)) as
Default value: `false`.
## default_normal_view_sql_security {#default_normal_view_sql_security}
Allows to set default `SQL SECURITY` option while creating a normal view. [More about SQL security](../../sql-reference/statements/create/view.md#sql_security).
The default value is `INVOKER`.
## default_materialized_view_sql_security {#default_materialized_view_sql_security}
Allows to set a default value for SQL SECURITY option when creating a materialized view. [More about SQL security](../../sql-reference/statements/create/view.md#sql_security).
The default value is `DEFINER`.
## default_view_definer {#default_view_definer}
Allows to set default `DEFINER` option while creating a view. [More about SQL security](../../sql-reference/statements/create/view.md#sql_security).
The default value is `CURRENT_USER`.
## max_partition_size_to_drop
Restriction on dropping partitions in query time. The value 0 means that you can drop partitions without any restrictions.

View File

@ -0,0 +1,38 @@
---
slug: /en/operations/system-tables/dns_cache
---
# dns_cache
Contains information about cached DNS records.
Columns:
- `hostname` ([String](../../sql-reference/data-types/string.md)) — cached hostname
- `ip_address` ([String](../../sql-reference/data-types/string.md)) — ip address for the hostname
- `ip_family` ([Enum](../../sql-reference/data-types/enum.md)) — family of the ip address, possible values:
- 'IPv4'
- 'IPv6'
- 'UNIX_LOCAL'
- `cached_at` ([DateTime](../../sql-reference/data-types/datetime.md)) - when the record was cached
**Example**
Query:
```sql
SELECT * FROM system.dns_cache;
```
Result:
| hostname | ip\_address | ip\_family | cached\_at |
| :--- | :--- | :--- | :--- |
| localhost | ::1 | IPv6 | 2024-02-11 17:04:40 |
| localhost | 127.0.0.1 | IPv4 | 2024-02-11 17:04:40 |
**See also**
- [disable_internal_dns_cache setting](../../operations/server-configuration-parameters/settings.md#disable_internal_dns_cache)
- [dns_cache_max_size setting](../../operations/server-configuration-parameters/settings.md#dns_cache_max_size)
- [dns_cache_update_period setting](../../operations/server-configuration-parameters/settings.md#dns_cache_update_period)
- [dns_max_consecutive_failures setting](../../operations/server-configuration-parameters/settings.md#dns_max_consecutive_failures)

View File

@ -0,0 +1,32 @@
---
slug: /en/operations/system-tables/settings_changes
---
# settings_changes
Contains information about setting changes in previous ClickHouse versions.
Columns:
- `version` ([String](../../sql-reference/data-types/string.md)) — The ClickHouse version in which settings were changed
- `changes` ([Array](../../sql-reference/data-types/array.md) of [Tuple](../../sql-reference/data-types/tuple.md)) — A description of the setting changes: (setting name, previous value, new value, reason for the change)
**Example**
``` sql
SELECT *
FROM system.settings_changes
WHERE version = '23.5'
FORMAT Vertical
```
``` text
Row 1:
──────
version: 23.5
changes: [('input_format_parquet_preserve_order','1','0','Allow Parquet reader to reorder rows for better parallelism.'),('parallelize_output_from_storages','0','1','Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows.'),('use_with_fill_by_sorting_prefix','0','1','Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently'),('output_format_parquet_compliant_nested_types','0','1','Change an internal field name in output Parquet file schema.')]
```
**See also**
- [Settings](../../operations/settings/index.md#session-settings-intro)
- [system.settings](settings.md)

View File

@ -26,6 +26,6 @@ Columns:
- `max` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — The maximum value of the setting. NULL if not set.
- `readonly` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges))) — Profile that allows only read queries.
- `writability` ([Nullable](../../sql-reference/data-types/nullable.md)([Enum8](../../sql-reference/data-types/enum.md)('WRITABLE' = 0, 'CONST' = 1, 'CHANGEABLE_IN_READONLY' = 2))) — Sets the settings constraint writability kind.
- `inherit_profile` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — A parent profile for this setting profile. `NULL` if not set. Setting profile will inherit all the settings' values and constraints (`min`, `max`, `readonly`) from its parent profiles.

View File

@ -111,6 +111,14 @@ On newer Linux kernels transparent huge pages are alright.
$ echo 'madvise' | sudo tee /sys/kernel/mm/transparent_hugepage/enabled
```
If you want to modify the transparent huge pages setting permanently, editing the `/etc/default/grub` to add the `transparent_hugepage=never` to the `GRUB_CMDLINE_LINUX_DEFAULT` option:
```bash
$ GRUB_CMDLINE_LINUX_DEFAULT="transparent_hugepage=madvise ..."
```
After that, run the `sudo update-grub` command then reboot to take effect.
## Hypervisor configuration
If you are using OpenStack, set

View File

@ -0,0 +1,50 @@
---
slug: /en/sql-reference/aggregate-functions/reference/grouparrayintersect
sidebar_position: 115
---
# groupArrayIntersect
Return an intersection of given arrays (Return all items of arrays, that are in all given arrays).
**Syntax**
``` sql
groupArrayIntersect(x)
```
**Arguments**
- `x` — Argument (column name or expression).
**Returned values**
- Array that contains elements that are in all arrays.
Type: [Array](../../data-types/array.md).
**Examples**
Consider table `numbers`:
``` text
┌─a──────────────┐
│ [1,2,4] │
│ [1,5,2,8,-1,0] │
│ [1,5,7,5,8,2] │
└────────────────┘
```
Query with column name as argument:
``` sql
SELECT groupArrayIntersect(a) as intersection FROM numbers;
```
Result:
```text
┌─intersection──────┐
│ [1, 2] │
└───────────────────┘
```

View File

@ -55,6 +55,7 @@ ClickHouse-specific aggregate functions:
- [groupArrayMovingSum](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingsum.md)
- [groupArraySample](./grouparraysample.md)
- [groupArraySorted](/docs/en/sql-reference/aggregate-functions/reference/grouparraysorted.md)
- [groupArrayIntersect](./grouparrayintersect.md)
- [groupBitAnd](/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md)
- [groupBitOr](/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md)
- [groupBitXor](/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md)

View File

@ -5,25 +5,25 @@ sidebar_position: 221
# stochasticLinearRegression
This function implements stochastic linear regression. It supports custom parameters for learning rate, L2 regularization coefficient, mini-batch size and has few methods for updating weights ([Adam](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Adam) (used by default), [simple SGD](https://en.wikipedia.org/wiki/Stochastic_gradient_descent), [Momentum](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Momentum), [Nesterov](https://mipt.ru/upload/medialibrary/d7e/41-91.pdf)).
This function implements stochastic linear regression. It supports custom parameters for learning rate, L2 regularization coefficient, mini-batch size, and has a few methods for updating weights ([Adam](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Adam) (used by default), [simple SGD](https://en.wikipedia.org/wiki/Stochastic_gradient_descent), [Momentum](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Momentum), and [Nesterov](https://mipt.ru/upload/medialibrary/d7e/41-91.pdf)).
### Parameters
There are 4 customizable parameters. They are passed to the function sequentially, but there is no need to pass all four - default values will be used, however good model required some parameter tuning.
``` text
stochasticLinearRegression(1.0, 1.0, 10, 'SGD')
stochasticLinearRegression(0.00001, 0.1, 15, 'Adam')
```
1. `learning rate` is the coefficient on step length, when gradient descent step is performed. Too big learning rate may cause infinite weights of the model. Default is `0.00001`.
1. `learning rate` is the coefficient on step length, when the gradient descent step is performed. A learning rate that is too big may cause infinite weights of the model. Default is `0.00001`.
2. `l2 regularization coefficient` which may help to prevent overfitting. Default is `0.1`.
3. `mini-batch size` sets the number of elements, which gradients will be computed and summed to perform one step of gradient descent. Pure stochastic descent uses one element, however having small batches(about 10 elements) make gradient steps more stable. Default is `15`.
4. `method for updating weights`, they are: `Adam` (by default), `SGD`, `Momentum`, `Nesterov`. `Momentum` and `Nesterov` require little bit more computations and memory, however they happen to be useful in terms of speed of convergence and stability of stochastic gradient methods.
3. `mini-batch size` sets the number of elements, which gradients will be computed and summed to perform one step of gradient descent. Pure stochastic descent uses one element, however, having small batches (about 10 elements) makes gradient steps more stable. Default is `15`.
4. `method for updating weights`, they are: `Adam` (by default), `SGD`, `Momentum`, and `Nesterov`. `Momentum` and `Nesterov` require a little bit more computations and memory, however, they happen to be useful in terms of speed of convergence and stability of stochastic gradient methods.
### Usage
`stochasticLinearRegression` is used in two steps: fitting the model and predicting on new data. In order to fit the model and save its state for later usage we use `-State` combinator, which basically saves the state (model weights, etc).
To predict we use function [evalMLMethod](../../../sql-reference/functions/machine-learning-functions.md#machine_learning_methods-evalmlmethod), which takes a state as an argument as well as features to predict on.
`stochasticLinearRegression` is used in two steps: fitting the model and predicting on new data. In order to fit the model and save its state for later usage, we use the `-State` combinator, which saves the state (e.g. model weights).
To predict, we use the function [evalMLMethod](../../../sql-reference/functions/machine-learning-functions.md#machine_learning_methods-evalmlmethod), which takes a state as an argument as well as features to predict on.
<a name="stochasticlinearregression-usage-fitting"></a>
@ -44,12 +44,12 @@ stochasticLinearRegressionState(0.1, 0.0, 5, 'SGD')(target, param1, param2)
AS state FROM train_data;
```
Here we also need to insert data into `train_data` table. The number of parameters is not fixed, it depends only on number of arguments, passed into `linearRegressionState`. They all must be numeric values.
Note that the column with target value(which we would like to learn to predict) is inserted as the first argument.
Here, we also need to insert data into the `train_data` table. The number of parameters is not fixed, it depends only on the number of arguments passed into `linearRegressionState`. They all must be numeric values.
Note that the column with target value (which we would like to learn to predict) is inserted as the first argument.
**2.** Predicting
After saving a state into the table, we may use it multiple times for prediction, or even merge with other states and create new even better models.
After saving a state into the table, we may use it multiple times for prediction or even merge with other states and create new, even better models.
``` sql
WITH (SELECT state FROM your_model) AS model SELECT

View File

@ -12,6 +12,11 @@ has a value of either type `T1` or `T2` or ... or `TN` or none of them (`NULL` v
The order of nested types doesn't matter: Variant(T1, T2) = Variant(T2, T1).
Nested types can be arbitrary types except Nullable(...), LowCardinality(Nullable(...)) and Variant(...) types.
:::note
It's not recommended to use similar types as variants (for example different numeric types like `Variant(UInt32, Int64)` or different date types like `Variant(Date, DateTime)`),
because working with values of such types can lead to ambiguity. By default, creating such `Variant` type will lead to an exception, but can be enabled using setting `allow_suspicious_variant_types`
:::
:::note
The Variant data type is an experimental feature. To use it, set `allow_experimental_variant_type = 1`.
:::
@ -272,3 +277,121 @@ $$)
│ [1,2,3] │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ [1,2,3] │
└─────────────────────┴───────────────┴──────┴───────┴─────────────────────┴─────────┘
```
## Comparing values of Variant type
Values of a `Variant` type can be compared only with values with the same `Variant` type.
The result of operator `<` for values `v1` with underlying type `T1` and `v2` with underlying type `T2` of a type `Variant(..., T1, ... T2, ...)` is defined as follows:
- If `T1 = T2 = T`, the result will be `v1.T < v2.T` (underlying values will be compared).
- If `T1 != T2`, the result will be `T1 < T2` (type names will be compared).
Examples:
```sql
CREATE TABLE test (v1 Variant(String, UInt64, Array(UInt32)), v2 Variant(String, UInt64, Array(UInt32))) ENGINE=Memory;
INSERT INTO test VALUES (42, 42), (42, 43), (42, 'abc'), (42, [1, 2, 3]), (42, []), (42, NULL);
```
```sql
SELECT v2, variantType(v2) as v2_type from test order by v2;
```
```text
┌─v2──────┬─v2_type───────┐
│ [] │ Array(UInt32) │
│ [1,2,3] │ Array(UInt32) │
│ abc │ String │
│ 42 │ UInt64 │
│ 43 │ UInt64 │
│ ᴺᵁᴸᴸ │ None │
└─────────┴───────────────┘
```
```sql
SELECT v1, variantType(v1) as v1_type, v2, variantType(v2) as v2_type, v1 = v2, v1 < v2, v1 > v2 from test;
```
```text
┌─v1─┬─v1_type─┬─v2──────┬─v2_type───────┬─equals(v1, v2)─┬─less(v1, v2)─┬─greater(v1, v2)─┐
│ 42 │ UInt64 │ 42 │ UInt64 │ 1 │ 0 │ 0 │
│ 42 │ UInt64 │ 43 │ UInt64 │ 0 │ 1 │ 0 │
│ 42 │ UInt64 │ abc │ String │ 0 │ 0 │ 1 │
│ 42 │ UInt64 │ [1,2,3] │ Array(UInt32) │ 0 │ 0 │ 1 │
│ 42 │ UInt64 │ [] │ Array(UInt32) │ 0 │ 0 │ 1 │
│ 42 │ UInt64 │ ᴺᵁᴸᴸ │ None │ 0 │ 1 │ 0 │
└────┴─────────┴─────────┴───────────────┴────────────────┴──────────────┴─────────────────┘
```
If you need to find the row with specific `Variant` value, you can do one of the following:
- Cast value to the corresponding `Variant` type:
```sql
SELECT * FROM test WHERE v2 == [1,2,3]::Array(UInt32)::Variant(String, UInt64, Array(UInt32));
```
```text
┌─v1─┬─v2──────┐
│ 42 │ [1,2,3] │
└────┴─────────┘
```
- Compare `Variant` subcolumn with required type:
```sql
SELECT * FROM test WHERE v2.`Array(UInt32)` == [1,2,3] -- or using variantElement(v2, 'Array(UInt32)')
```
```text
┌─v1─┬─v2──────┐
│ 42 │ [1,2,3] │
└────┴─────────┘
```
Sometimes it can be useful to make additional check on variant type as subcolumns with complex types like `Array/Map/Tuple` cannot be inside `Nullable` and will have default values instead of `NULL` on rows with different types:
```sql
SELECT v2, v2.`Array(UInt32)`, variantType(v2) FROM test WHERE v2.`Array(UInt32)` == [];
```
```text
┌─v2───┬─v2.Array(UInt32)─┬─variantType(v2)─┐
│ 42 │ [] │ UInt64 │
│ 43 │ [] │ UInt64 │
│ abc │ [] │ String │
│ [] │ [] │ Array(UInt32) │
│ ᴺᵁᴸᴸ │ [] │ None │
└──────┴──────────────────┴─────────────────┘
```
```sql
SELECT v2, v2.`Array(UInt32)`, variantType(v2) FROM test WHERE variantType(v2) == 'Array(UInt32)' AND v2.`Array(UInt32)` == [];
```
```text
┌─v2─┬─v2.Array(UInt32)─┬─variantType(v2)─┐
│ [] │ [] │ Array(UInt32) │
└────┴──────────────────┴─────────────────┘
```
**Note:** values of variants with different numeric types are considered as different variants and not compared between each other, their type names are compared instead.
Example:
```sql
SET allow_suspicious_variant_types = 1;
CREATE TABLE test (v Variant(UInt32, Int64)) ENGINE=Memory;
INSERT INTO test VALUES (1::UInt32), (1::Int64), (100::UInt32), (100::Int64);
SELECT v, variantType(v) FROM test ORDER by v;
```
```text
┌─v───┬─variantType(v)─┐
│ 1 │ Int64 │
│ 100 │ Int64 │
│ 1 │ UInt32 │
│ 100 │ UInt32 │
└─────┴────────────────┘
```

View File

@ -780,8 +780,52 @@ If executed in the context of a distributed table, this function generates a nor
## version()
Returns the server version as a string.
If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value.
Returns the current version of ClickHouse as a string in the form of:
- Major version
- Minor version
- Patch version
- Number of commits since the previous stable release.
```plaintext
major_version.minor_version.patch_version.number_of_commits_since_the_previous_stable_release
```
If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise, it produces a constant value.
**Syntax**
```sql
version()
```
**Arguments**
None.
**Returned value**
Type: [String](../data-types/string)
**Implementation details**
None.
**Example**
Query:
```sql
SELECT version()
```
**Result**:
```response
┌─version()─┐
│ 24.2.1.1 │
└───────────┘
```
## buildId()

View File

@ -13,7 +13,9 @@ Creates a new view. Views can be [normal](#normal-view), [materialized](#materia
Syntax:
``` sql
CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] AS SELECT ...
CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name]
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
AS SELECT ...
```
Normal views do not store any data. They just perform a read from another table on each access. In other words, a normal view is nothing more than a saved query. When reading from a view, this saved query is used as a subquery in the [FROM](../../../sql-reference/statements/select/from.md) clause.
@ -52,7 +54,9 @@ SELECT * FROM view(column1=value1, column2=value2 ...)
## Materialized View
``` sql
CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE]
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
AS SELECT ...
```
:::tip
@ -91,6 +95,49 @@ Views look the same as normal tables. For example, they are listed in the result
To delete a view, use [DROP VIEW](../../../sql-reference/statements/drop.md#drop-view). Although `DROP TABLE` works for VIEWs as well.
## SQL security {#sql_security}
`DEFINER` and `SQL SECURITY` allow you to specify which ClickHouse user to use when executing the view's underlying query.
`SQL SECURITY` has three legal values: `DEFINER`, `INVOKER`, or `NONE`. You can specify any existing user or `CURRENT_USER` in the `DEFINER` clause.
The following table will explain which rights are required for which user in order to select from view.
Note that regardless of the SQL security option, in every case it is still required to have `GRANT SELECT ON <view>` in order to read from it.
| SQL security option | View | Materialized View |
|---------------------|-----------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------|
| `DEFINER alice` | `alice` must have a `SELECT` grant for the view's source table. | `alice` must have a `SELECT` grant for the view's source table and an `INSERT` grant for the view's target table. |
| `INVOKER` | User must have a `SELECT` grant for the view's source table. | `SQL SECURITY INVOKER` can't be specified for materialized views. |
| `NONE` | - | - |
:::note
`SQL SECURITY NONE` is a deprecated option. Any user with the rights to create views with `SQL SECURITY NONE` will be able to execute any arbitrary query.
Thus, it is required to have `GRANT ALLOW SQL SECURITY NONE TO <user>` in order to create a view with this option.
:::
If `DEFINER`/`SQL SECURITY` aren't specified, the default values are used:
- `SQL SECURITY`: `INVOKER` for normal views and `DEFINER` for materialized views ([configurable by settings](../../../operations/settings/settings.md#default_normal_view_sql_security))
- `DEFINER`: `CURRENT_USER` ([configurable by settings](../../../operations/settings/settings.md#default_view_definer))
If a view is attached without `DEFINER`/`SQL SECURITY` specified, the default value is `SQL SECURITY NONE` for the materialized view and `SQL SECURITY INVOKER` for the normal view.
To change SQL security for an existing view, use
```sql
ALTER TABLE MODIFY SQL SECURITY { DEFINER | INVOKER | NONE } [DEFINER = { user | CURRENT_USER }]
```
### Examples sql security
```sql
CREATE test_view
DEFINER = alice SQL SECURITY DEFINER
AS SELECT ...
```
```sql
CREATE test_view
SQL SECURITY INVOKER
AS SELECT ...
```
## Live View [Deprecated]
This feature is deprecated and will be removed in the future.

View File

@ -114,6 +114,7 @@ Hierarchy of privileges:
- `ALTER VIEW`
- `ALTER VIEW REFRESH`
- `ALTER VIEW MODIFY QUERY`
- `ALTER VIEW MODIFY SQL SECURITY`
- [CREATE](#grant-create)
- `CREATE DATABASE`
- `CREATE TABLE`
@ -307,6 +308,7 @@ Allows executing [ALTER](../../sql-reference/statements/alter/index.md) queries
- `ALTER VIEW` Level: `GROUP`
- `ALTER VIEW REFRESH`. Level: `VIEW`. Aliases: `ALTER LIVE VIEW REFRESH`, `REFRESH VIEW`
- `ALTER VIEW MODIFY QUERY`. Level: `VIEW`. Aliases: `ALTER TABLE MODIFY QUERY`
- `ALTER VIEW MODIFY SQL SECURITY`. Level: `VIEW`. Aliases: `ALTER TABLE MODIFY SQL SECURITY`
Examples of how this hierarchy is treated:
@ -409,6 +411,7 @@ Allows a user to execute queries that manage users, roles and row policies.
- `SHOW_ROW_POLICIES`. Level: `GLOBAL`. Aliases: `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, `SHOW CREATE POLICY`
- `SHOW_QUOTAS`. Level: `GLOBAL`. Aliases: `SHOW CREATE QUOTA`
- `SHOW_SETTINGS_PROFILES`. Level: `GLOBAL`. Aliases: `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, `SHOW CREATE PROFILE`
- `ALLOW SQL SECURITY NONE`. Level: `GLOBAL`. Aliases: `CREATE SQL SECURITY NONE`, `SQL SECURITY NONE`, `SECURITY NONE`
The `ROLE ADMIN` privilege allows a user to assign and revoke any roles including those which are not assigned to the user with the admin option.

View File

@ -176,7 +176,7 @@ INSERT INTO infile_globs FROM INFILE 'input_?.csv' FORMAT CSV;
```
:::
## Inserting into Table Function
## Inserting using a Table Function
Data can be inserted into tables referenced by [table functions](../../sql-reference/table-functions/index.md).
@ -204,7 +204,7 @@ Result:
└─────┴───────────────────────┘
```
## Inserts into ClickHouse Cloud
## Inserting into ClickHouse Cloud
By default, services on ClickHouse Cloud provide multiple replicas for high availability. When you connect to a service, a connection is established to one of these replicas.
@ -218,6 +218,12 @@ SELECT .... SETTINGS select_sequential_consistency = 1;
Note that using `select_sequential_consistency` will increase the load on ClickHouse Keeper (used by ClickHouse Cloud internally) and may result in slower performance depending on the load on the service. We recommend against enabling this setting unless necessary. The recommended approach is to execute read/writes in the same session or to use a client driver that uses the native protocol (and thus supports sticky connections).
## Inserting into a replicated setup
In a replicated setup, data will be visible on other replicas after it has been replicated. Data begins being replicated (downloaded on other replicas) immediately after an `INSERT`. This differs from ClickHouse Cloud, where data is immediately written to shared storage and replicas subscribe to metadata changes.
Note that for replicated setups, `INSERTs` can sometimes take a considerable amount of time (in the order of one second) as it requires committing to ClickHouse Keeper for distributed consensus. Using S3 for storage also adds additional latency.
## Performance Considerations
`INSERT` sorts the input data by primary key and splits them into partitions by a partition key. If you insert data into several partitions at once, it can significantly reduce the performance of the `INSERT` query. To avoid this:
@ -230,7 +236,15 @@ Performance will not decrease if:
- Data is added in real time.
- You upload data that is usually sorted by time.
It's also possible to asynchronously insert data in small but frequent inserts. The data from such insertions is combined into batches and then safely inserted into a table. To enable the asynchronous mode, switch on the [async_insert](../../operations/settings/settings.md#async-insert) setting. Note that asynchronous insertions are supported only over HTTP protocol, and deduplication is not supported for them.
### Asynchronous inserts
It is possible to asynchronously insert data in small but frequent inserts. The data from such insertions is combined into batches and then safely inserted into a table. To use asynchronous inserts, enable the [`async_insert`](../../operations/settings/settings.md#async-insert) setting.
Using `async_insert` or the [`Buffer` table engine](/en/engines/table-engines/special/buffer) results in additional buffering.
### Large or long-running inserts
When you are inserting large amounts of data, ClickHouse will optimize write performance through a process called "squashing". Small blocks of inserted data in memory are merged and squashed into larger blocks before being written to disk. Squashing reduces the overhead associated with each write operation. In this process, inserted data will be available to query after ClickHouse completes writing each [`max_insert_block_size`](/en/operations/settings/settings#max_insert_block_size) rows.
**See Also**

View File

@ -9,9 +9,10 @@ The `ORDER BY` clause contains
- a list of expressions, e.g. `ORDER BY visits, search_phrase`,
- a list of numbers referring to columns in the `SELECT` clause, e.g. `ORDER BY 2, 1`, or
- `*` (without other expressions or numbers) which means all columns of the `SELECT` clause: `ORDER BY *`.
- `ALL` which means all columns of the `SELECT` clause, e.g. `ORDER BY ALL`.
To disable sorting by column numbers, set setting [enable_positional_arguments](../../../operations/settings/settings.md#enable-positional-arguments) = 0.
To disable sorting by `ALL`, set setting [enable_order_by_all](../../../operations/settings/settings.md#enable-order-by-all) = 0.
The `ORDER BY` clause can be attributed by a `DESC` (descending) or `ASC` (ascending) modifier which determines the sorting direction.
Unless an explicit sort order is specified, `ASC` is used by default.

View File

@ -68,7 +68,7 @@ RELOAD FUNCTION [ON CLUSTER cluster_name] function_name
Clears ClickHouses internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries).
For more convenient (automatic) cache management, see disable_internal_dns_cache, dns_cache_update_period parameters.
For more convenient (automatic) cache management, see disable_internal_dns_cache, dns_cache_max_size, dns_cache_update_period parameters.
## DROP MARK CACHE

View File

@ -59,9 +59,7 @@ INSERT INTO TABLE FUNCTION file('file2.csv', 'CSV', 'i UInt32, s String') VALUES
Now, read data contents of `test1.csv` and `test2.csv` via `fileCluster` table function:
```sql
SELECT * from fileCluster(
'my_cluster', 'file{1,2}.csv', 'CSV', 'i UInt32, s String') ORDER BY (i, s)"""
)
SELECT * FROM fileCluster('my_cluster', 'file{1,2}.csv', 'CSV', 'i UInt32, s String') ORDER BY i, s
```
```

View File

@ -11,11 +11,11 @@ Creates a temporary [Merge](../../engines/table-engines/special/merge.md) table.
**Syntax**
```sql
merge('db_name', 'tables_regexp')
merge(['db_name',] 'tables_regexp')
```
**Arguments**
- `db_name` — Possible values:
- `db_name` — Possible values (optional, default is `currentDatabase()`):
- database name,
- constant expression that returns a string with a database name, for example, `currentDatabase()`,
- `REGEXP(expression)`, where `expression` is a regular expression to match the DB names.

View File

@ -0,0 +1,83 @@
---
slug: /en/sql-reference/table-functions/mergeTreeIndex
sidebar_position: 77
sidebar_label: mergeTreeIndex
---
# mergeTreeIndex
Represents the contents of index and marks files of MergeTree tables. It can be used for introspection
``` sql
mergeTreeIndex(database, table, [with_marks = true])
```
**Arguments**
- `database`- The database name to read index and marks from.
- `table`- The table name to read index and marks from.
- `with_marks` - Whether include columns with marks to the result.
**Returned Value**
A table object with columns with values of primary index of source table, columns with values of marks (if enabled) for all possible files in data parts of source table and virtual columns:
- `part_name` - The name of data part.
- `mark_number` - The number of current mark in data part.
- `rows_in_granule` - The number of rows in current granule.
Marks column may contain `(NULL, NULL)` value in case when column is absent in data part or marks for one of its substreams are not written (e.g. in compact parts).
## Usage Example
```sql
CREATE TABLE test_table
(
`id` UInt64,
`n` UInt64,
`arr` Array(UInt64)
)
ENGINE = MergeTree
ORDER BY id
SETTINGS index_granularity = 3, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 8;
INSERT INTO test_table SELECT number, number, range(number % 5) FROM numbers(5);
INSERT INTO test_table SELECT number, number, range(number % 5) FROM numbers(10, 10);
```
```sql
SELECT * FROM mergeTreeIndex(currentDatabase(), test_table, with_marks = true);
```
```text
┌─part_name─┬─mark_number─┬─rows_in_granule─┬─id─┬─id.mark─┬─n.mark──┬─arr.size0.mark─┬─arr.mark─┐
│ all_1_1_0 │ 0 │ 3 │ 0 │ (0,0) │ (42,0) │ (NULL,NULL) │ (84,0) │
│ all_1_1_0 │ 1 │ 2 │ 3 │ (133,0) │ (172,0) │ (NULL,NULL) │ (211,0) │
│ all_1_1_0 │ 2 │ 0 │ 4 │ (271,0) │ (271,0) │ (NULL,NULL) │ (271,0) │
└───────────┴─────────────┴─────────────────┴────┴─────────┴─────────┴────────────────┴──────────┘
┌─part_name─┬─mark_number─┬─rows_in_granule─┬─id─┬─id.mark─┬─n.mark─┬─arr.size0.mark─┬─arr.mark─┐
│ all_2_2_0 │ 0 │ 3 │ 10 │ (0,0) │ (0,0) │ (0,0) │ (0,0) │
│ all_2_2_0 │ 1 │ 3 │ 13 │ (0,24) │ (0,24) │ (0,24) │ (0,24) │
│ all_2_2_0 │ 2 │ 3 │ 16 │ (0,48) │ (0,48) │ (0,48) │ (0,80) │
│ all_2_2_0 │ 3 │ 1 │ 19 │ (0,72) │ (0,72) │ (0,72) │ (0,128) │
│ all_2_2_0 │ 4 │ 0 │ 19 │ (0,80) │ (0,80) │ (0,80) │ (0,160) │
└───────────┴─────────────┴─────────────────┴────┴─────────┴────────┴────────────────┴──────────┘
```
```sql
DESCRIBE mergeTreeIndex(currentDatabase(), test_table, with_marks = true) SETTINGS describe_compact_output = 1;
```
```text
┌─name────────────┬─type─────────────────────────────────────────────────────────────────────────────────────────────┐
│ part_name │ String │
│ mark_number │ UInt64 │
│ rows_in_granule │ UInt64 │
│ id │ UInt64 │
│ id.mark │ Tuple(offset_in_compressed_file Nullable(UInt64), offset_in_decompressed_block Nullable(UInt64)) │
│ n.mark │ Tuple(offset_in_compressed_file Nullable(UInt64), offset_in_decompressed_block Nullable(UInt64)) │
│ arr.size0.mark │ Tuple(offset_in_compressed_file Nullable(UInt64), offset_in_decompressed_block Nullable(UInt64)) │
│ arr.mark │ Tuple(offset_in_compressed_file Nullable(UInt64), offset_in_decompressed_block Nullable(UInt64)) │
└─────────────────┴──────────────────────────────────────────────────────────────────────────────────────────────────┘
```

View File

@ -679,11 +679,20 @@ TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y);
Тэги:
- `policy_name_N` — название политики. Названия политик должны быть уникальны.
- `volume_name_N` — название тома. Названия томов должны быть уникальны.
- `disk` — диск, находящийся внутри тома.
- `max_data_part_size_bytes` — максимальный размер куска данных, который может находиться на любом из дисков этого тома. Если в результате слияния размер куска ожидается больше, чем max_data_part_size_bytes, то этот кусок будет записан в следующий том. В основном эта функция позволяет хранить новые / мелкие куски на горячем (SSD) томе и перемещать их на холодный (HDD) том, когда они достигают большого размера. Не используйте этот параметр, если политика имеет только один том.
- `move_factor` — доля доступного свободного места на томе, если места становится меньше, то данные начнут перемещение на следующий том, если он есть (по умолчанию 0.1). Для перемещения куски сортируются по размеру от большего к меньшему (по убыванию) и выбираются куски, совокупный размер которых достаточен для соблюдения условия `move_factor`, если совокупный размер всех партов недостаточен, будут перемещены все парты.
- `policy_name_N` — название политики. Названия политик должны быть уникальны.
- `volume_name_N` — название тома. Названия томов должны быть уникальны.
- `disk` — диск, находящийся внутри тома.
- `max_data_part_size_bytes` — максимальный размер куска данных, который может находиться на любом из дисков этого тома. Если в результате слияния размер куска ожидается больше, чем max_data_part_size_bytes, то этот кусок будет записан в следующий том. В основном эта функция позволяет хранить новые / мелкие куски на горячем (SSD) томе и перемещать их на холодный (HDD) том, когда они достигают большого размера. Не используйте этот параметр, если политика имеет только один том.
- `move_factor` — доля доступного свободного места на томе, если места становится меньше, то данные начнут перемещение на следующий том, если он есть (по умолчанию 0.1). Для перемещения куски сортируются по размеру от большего к меньшему (по убыванию) и выбираются куски, совокупный размер которых достаточен для соблюдения условия `move_factor`, если совокупный размер всех партов недостаточен, будут перемещены все парты.
- `perform_ttl_move_on_insert` — отключает перемещение данных с истекшим TTL при вставке. По умолчанию (если включено), если мы вставляем часть данных, которая уже просрочилась по правилу перемещения по сроку жизни, она немедленно перемещается на том / диск, указанный в правиле перемещения. Это может значительно замедлить вставку в случае, если целевой том / диск медленный (например, S3). Если отключено, то просроченная часть данных записывается на том по умолчанию, а затем сразу перемещается на том, указанный в правиле для истёкшего TTL.
- `load_balancing` - политика балансировки дисков, `round_robin` или `least_used`.
- `least_used_ttl_ms` - устанавливает таймаут (в миллисекундах) для обновления доступного пространства на всех дисках (`0` - обновлять всегда, `-1` - никогда не обновлять, значение по умолчанию - `60000`). Обратите внимание, если диск используется только ClickHouse и не будет подвергаться изменению размеров файловой системы на лету, можно использовать значение `-1`. Во всех остальных случаях это не рекомендуется, так как в конечном итоге это приведет к неправильному распределению пространства.
- `prefer_not_to_merge` — эту настройку лучше не использовать. Она отключает слияние частей данных на этом томе (что потенциально вредно и может привести к замедлению). Когда эта настройка включена (не делайте этого), объединение данных на этом томе запрещено (что плохо). Это позволяет (но вам это не нужно) контролировать (если вы хотите что-то контролировать, вы делаете ошибку), как ClickHouse взаимодействует с медленными дисками (но ClickHouse лучше знает, поэтому, пожалуйста, не используйте эту настройку).
- `volume_priority` — Определяет приоритет (порядок), в котором заполняются тома. Чем меньше значение -- тем выше приоритет. Значения параметра должны быть натуральными числами и охватывать диапазон от 1 до N (N - наибольшее значение параметра из указанных) без пропусков.
* Если се_ тома имеют этот параметр, они приоритизируются в указанном порядке.
* Если его имеют лишь екоторые_, то не имеющие этого параметра тома имеют самый низкий приоритет. Те, у которых он указан, приоритизируются в соответствии со значением тега, приоритет остальных определяется порядком описания в конфигурационном файле относительно друг друга.
* Если _ни одному_ тому не присвоен этот параметр, их порядок определяется порядком описания в конфигурационном файле.
* Приоритет нескольких томов не может быть одинаковым.
Примеры конфигураций:
@ -733,7 +742,7 @@ TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y);
Если система содержит диски различных типов, то может пригодиться политика `moving_from_ssd_to_hdd`. В томе `hot` находится один SSD-диск (`fast_ssd`), а также задается ограничение на максимальный размер куска, который может храниться на этом томе (1GB). Все куски такой таблицы больше 1GB будут записываться сразу на том `cold`, в котором содержится один HDD-диск `disk1`. Также при заполнении диска `fast_ssd` более чем на 80% данные будут переноситься на диск `disk1` фоновым процессом.
Порядок томов в политиках хранения важен, при достижении условий на переполнение тома данные переносятся на следующий. Порядок дисков в томах так же важен, данные пишутся по очереди на каждый из них.
Порядок томов в политиках хранения важен в случае, если приоритеты томов (`volume_priority`) не указаны явно: при достижении условий на переполнение тома данные переносятся на следующий. Порядок дисков в томах так же важен, данные пишутся по очереди на каждый из них.
После задания конфигурации политик хранения их можно использовать, как настройку при создании таблиц:

View File

@ -260,3 +260,19 @@ SeekTable [бесплатен](https://www.seektable.com/help/cloud-pricing) д
Протестировать TABLUM.IO без разворачивания на собственном сервере можно [здесь](https://tablum.io/try).
Подробно о продукте смотрите на [TABLUM.IO](https://tablum.io/)
### CKMAN {#ckman}
[CKMAN] (https://www.github.com/housepower/ckman) — инструмент управления и мониторинга кластеров ClickHouse!
Основные возможности:
- Быстрое и простое развертывание кластеров через веб-интерфейс
- Кластеры можно масштабировать или масштабировать
- Балансировка нагрузки данных кластера
- Обновление кластера в режиме онлайн
- Измените конфигурацию кластера на странице
- Обеспечивает мониторинг узлов кластера и zookeeper
- Мониторинг состояния таблиц и секций, а также медленные SQL-операторы
- Предоставляет простую в использовании страницу выполнения SQL

View File

@ -3258,7 +3258,7 @@ SELECT * FROM test2;
## allow_experimental_live_view {#allow-experimental-live-view}
Включает экспериментальную возможность использования [LIVE-представлений](../../sql-reference/statements/create/view.md#live-view).
Включает устаревшую возможность использования [LIVE-представлений](../../sql-reference/statements/create/view.md#live-view).
Возможные значения:
- 0 — живые представления не поддерживаются.
@ -3268,21 +3268,15 @@ SELECT * FROM test2;
## live_view_heartbeat_interval {#live-view-heartbeat-interval}
Задает интервал в секундах для периодической проверки существования [LIVE VIEW](../../sql-reference/statements/create/view.md#live-view).
Значение по умолчанию: `15`.
Устарело.
## max_live_view_insert_blocks_before_refresh {#max-live-view-insert-blocks-before-refresh}
Задает наибольшее число вставок, после которых запрос на формирование [LIVE VIEW](../../sql-reference/statements/create/view.md#live-view) исполняется снова.
Значение по умолчанию: `64`.
Устарело.
## periodic_live_view_refresh {#periodic-live-view-refresh}
Задает время в секундах, по истечении которого [LIVE VIEW](../../sql-reference/statements/create/view.md#live-view) с установленным автообновлением обновляется.
Значение по умолчанию: `60`.
Устарело.
## check_query_single_value_result {#check_query_single_value_result}

View File

@ -11,7 +11,9 @@ sidebar_label: "Представление"
## Обычные представления {#normal}
``` sql
CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] AS SELECT ...
CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name]
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
AS SELECT ...
```
Обычные представления не хранят никаких данных, они выполняют чтение данных из другой таблицы при каждом доступе. Другими словами, обычное представление — это не что иное, как сохраненный запрос. При чтении данных из представления этот сохраненный запрос используется как подзапрос в секции [FROM](../../../sql-reference/statements/select/from.md).
@ -37,7 +39,9 @@ SELECT a, b, c FROM (SELECT ...)
## Материализованные представления {#materialized}
``` sql
CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE]
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
AS SELECT ...
```
Материализованные (MATERIALIZED) представления хранят данные, преобразованные соответствующим запросом [SELECT](../../../sql-reference/statements/select/index.md).
@ -66,6 +70,52 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na
Чтобы удалить представление, следует использовать [DROP VIEW](../../../sql-reference/statements/drop.md#drop-view). Впрочем, `DROP TABLE` тоже работает для представлений.
## SQL безопасность {#sql_security}
Параметры `DEFINER` и `SQL SECURITY` позволяют задать правило от имени какого пользователя будут выполняться запросы к таблицам, на которые ссылается представление.
Для `SQL SECURITY` допустимо три значения: `DEFINER`, `INVOKER`, или `NONE`.
Для `DEFINER` можно указать имя любого существующего пользователя или же `CURRENT_USER`.
Далее приведена таблица, объясняющая какие права необходимы каким пользователям при заданных параметрах SQL безопасности.
Обратите внимание, что, в независимости от заданных параметров SQL безопасности,
у пользователя должно быть право `GRANT SELECT ON <view>` для чтения из представления.
| SQL security option | View | Materialized View |
|---------------------|----------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------|
| `DEFINER alice` | У `alice` должно быть право `SELECT` на таблицу-источник. | У `alice` должны быть права `SELECT` на таблицу-источник и `INSERT` на таблицу-назначение. |
| `INVOKER` | У пользователя выполняющего запрос к представлению должно быть право `SELECT` на таблицу-источник. | Тип `SQL SECURITY INVOKER` не может быть указан для материализованных представлений. |
| `NONE` | - | - |
:::note
Тип `SQL SECURITY NONE` не безопасен для использования. Любой пользователь с правом создавать представления с `SQL SECURITY NONE` сможет исполнять любые запросы без проверки прав.
По умолчанию, у пользователей нет прав указывать `SQL SECURITY NONE`, однако, при необходимости, это право можно выдать с помощью `GRANT ALLOW SQL SECURITY NONE TO <user>`.
:::
Если `DEFINER`/`SQL SECURITY` не указан, будут использованы значения по умолчанию:
- `SQL SECURITY`: `INVOKER` для обычных представлений и `DEFINER` для материализованных ([изменяется в настройках](../../../operations/settings/settings.md#default_normal_view_sql_security))
- `DEFINER`: `CURRENT_USER` ([изменяется в настройках](../../../operations/settings/settings.md#default_view_definer))
Если представление подключается с помощью ключевого слова `ATTACH` и настройки SQL безопасности не были заданы,
то по умолчанию будет использоваться `SQL SECURITY NONE` для материализованных представлений и `SQL SECURITY INVOKER` для обычных.
Изменить параметры SQL безопасности возможно с помощью следующего запроса:
```sql
ALTER TABLE MODIFY SQL SECURITY { DEFINER | INVOKER | NONE } [DEFINER = { user | CURRENT_USER }]
```
### Примеры представлений с SQL безопасностью
```sql
CREATE test_view
DEFINER = alice SQL SECURITY DEFINER
AS SELECT ...
```
```sql
CREATE test_view
SQL SECURITY INVOKER
AS SELECT ...
```
## LIVE-представления [экспериментальный функционал] {#live-view}
:::note Важно

View File

@ -129,3 +129,18 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix).
- 数据编辑器。
- 重构。
- 搜索和导航。
### CKMAN {#ckman}
[CKMAN](https://www.github.com/housepower/ckman) 是一个用于管理和监控ClickHouse集群的可视化工具
特征:
- 非常快速便捷的通过浏览器界面自动化部署集群
- 支持对集群进行扩缩容操作
- 对集群的数据进行负载均衡
- 对集群进行在线升级
- 通过界面修改集群配置
- 提供集群节点监控zookeeper监控
- 监控表、分区状态慢SQL监控
- 提供简单易操作的SQL执行页面

View File

@ -280,9 +280,6 @@ GRANT INSERT(x,y) ON db.table TO john
- `ALTER MOVE PARTITION`. 级别: `TABLE`. 别名: `ALTER MOVE PART`, `MOVE PARTITION`, `MOVE PART`
- `ALTER FETCH PARTITION`. 级别: `TABLE`. 别名: `FETCH PARTITION`
- `ALTER FREEZE PARTITION`. 级别: `TABLE`. 别名: `FREEZE PARTITION`
- `ALTER VIEW` 级别: `GROUP`
- `ALTER VIEW REFRESH`. 级别: `VIEW`. 别名: `ALTER LIVE VIEW REFRESH`, `REFRESH VIEW`
- `ALTER VIEW MODIFY QUERY`. 级别: `VIEW`. 别名: `ALTER TABLE MODIFY QUERY`
如何对待该层级的示例:
- `ALTER` 权限包含所有其它 `ALTER *` 的权限

View File

@ -61,14 +61,14 @@ sidebar_label: ORDER BY
我们只建议使用 `COLLATE` 对于少量行的最终排序,因为排序与 `COLLATE` 比正常的按字节排序效率低。
## ORDER BY *
## ORDER BY ALL
`ORDER BY *` 对所有选定的列进行升序排序。
`ORDER BY ALL` 对所有选定的列进行升序排序。
示例:
``` sql
SELECT a, b, c FROM t ORDER BY *
SELECT a, b, c FROM t ORDER BY ALL
```
等同于:

View File

@ -24,6 +24,7 @@
#include <Common/MemoryTracker.h>
#include <Common/ClickHouseRevision.h>
#include <Common/DNSResolver.h>
#include <Common/CgroupsMemoryUsageObserver.h>
#include <Common/CurrentMetrics.h>
#include <Common/ConcurrencyControl.h>
#include <Common/Macros.h>
@ -1282,6 +1283,18 @@ try
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
std::optional<CgroupsMemoryUsageObserver> cgroups_memory_usage_observer;
try
{
UInt64 wait_time = server_settings.cgroups_memory_usage_observer_wait_time;
if (wait_time != 0)
cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time));
}
catch (Exception &)
{
tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization");
}
const std::string cert_path = config().getString("openSSL.server.certificateFile", "");
const std::string key_path = config().getString("openSSL.server.privateKeyFile", "");
@ -1335,6 +1348,15 @@ try
total_memory_tracker.setDescription("(total)");
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
if (cgroups_memory_usage_observer)
{
double hard_limit_ratio = new_server_settings.cgroup_memory_watcher_hard_limit_ratio;
double soft_limit_ratio = new_server_settings.cgroup_memory_watcher_soft_limit_ratio;
cgroups_memory_usage_observer->setLimits(
static_cast<uint64_t>(max_server_memory_usage * hard_limit_ratio),
static_cast<uint64_t>(max_server_memory_usage * soft_limit_ratio));
}
size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit;
size_t default_merges_mutations_server_memory_usage = static_cast<size_t>(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
@ -1752,6 +1774,8 @@ try
}
else
{
DNSResolver::instance().setCacheMaxSize(server_settings.dns_cache_max_size);
/// Initialize a watcher periodically updating DNS cache
dns_cache_updater = std::make_unique<DNSCacheUpdater>(
global_context, server_settings.dns_cache_update_period, server_settings.dns_max_consecutive_failures);

View File

@ -0,0 +1 @@
../../../tests/config/config.d/filesystem_cache_log.xml

View File

@ -0,0 +1,4 @@
<clickhouse>
<filesystem_caches_path>/tmp/filesystem_caches/</filesystem_caches_path>
<custom_cached_disks_base_directory replace="replace">/tmp/filesystem_caches/</custom_cached_disks_base_directory>
</clickhouse>

View File

@ -1392,13 +1392,27 @@
<!-- <host_name>replica</host_name> -->
</distributed_ddl>
<!-- Settings to fine tune MergeTree tables. See documentation in source code, in MergeTreeSettings.h -->
<!-- Settings to fine-tune MergeTree tables. See documentation in source code, in MergeTreeSettings.h -->
<!--
<merge_tree>
<max_suspicious_broken_parts>5</max_suspicious_broken_parts>
</merge_tree>
-->
<!-- Settings to fine-tune ReplicatedMergeTree tables. See documentation in source code, in MergeTreeSettings.h -->
<!--
<replicated_merge_tree>
<max_replicated_fetches_network_bandwidth>1000000000</max_replicated_fetches_network_bandwidth>
</replicated_merge_tree>
-->
<!-- Settings to fine-tune Distributed tables. See documentation in source code, in DistributedSettings.h -->
<!--
<distributed>
<flush_on_detach>false</flush_on_detach>
</distributed>
-->
<!-- Protection from accidental DROP.
If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query.
If you want do delete one table and don't want to change clickhouse-server config, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.

View File

@ -85,11 +85,10 @@
gap: 1rem;
}
.chart {
flex: 1 40%;
min-width: 20rem;
flex: 1 1 40rem;
min-height: 16rem;
background: var(--chart-background);
box-shadow: 0 0 1rem var(--shadow-color);
box-shadow: 1px 1px 0 var(--shadow-color);
overflow: hidden;
position: relative;
}
@ -195,7 +194,7 @@
}
.inputs input {
box-shadow: 0 0 1rem var(--shadow-color);
box-shadow: 1px 1px 0 var(--shadow-color);
padding: 0.25rem;
}
@ -255,8 +254,6 @@
font-weight: bold;
user-select: none;
cursor: pointer;
padding-left: 0.5rem;
padding-right: 0.5rem;
background: var(--new-chart-background-color);
color: var(--new-chart-text-color);
float: right;
@ -275,7 +272,6 @@
width: 36%;
}
#global-error {
align-self: center;
width: 60%;
@ -298,7 +294,7 @@
background: var(--param-background-color);
color: var(--param-text-color);
display: inline-block;
box-shadow: 0 0 1rem var(--shadow-color);
box-shadow: 1px 1px 0 var(--shadow-color);
margin-bottom: 1rem;
}
@ -491,17 +487,10 @@
* - if a query returned something unusual, display the table;
*/
let host = 'https://play.clickhouse.com/';
let user = 'explorer';
let host = location.protocol != 'file:' ? location.origin : 'http://localhost:8123/';
let user = 'default';
let password = '';
let add_http_cors_header = true;
/// If it is hosted on server, assume that it is the address of ClickHouse.
if (location.protocol != 'file:') {
host = location.origin;
user = 'default';
add_http_cors_header = false;
}
let add_http_cors_header = (location.protocol != 'file:');
const errorCodeMessageMap = {
516: 'Error authenticating with database. Please check your connection params and try again.'
@ -1273,8 +1262,11 @@ function hideError() {
}
let firstLoad = true;
let is_drawing = false; // Prevent race condition leading to duplicate/dangling charts.
async function drawAll() {
if (is_drawing) return;
is_drawing = true;
let params = getParamsForURL();
const chartsArray = document.getElementsByClassName('chart');
@ -1301,12 +1293,12 @@ async function drawAll() {
document.getElementById('edit').style.display = 'inline-block';
document.getElementById('search-span').style.display = '';
hideError();
}
else {
const charts = document.getElementById('charts')
charts.style.height = '0px';
} else {
document.getElementById('charts').style.height = '0px';
}
});
is_drawing = false;
}
function resize() {

View File

@ -164,7 +164,7 @@ public:
int getBcryptWorkfactor() const;
/// Enables logic that users without permissive row policies can still read rows using a SELECT query.
/// For example, if there two users A, B and a row policy is defined only for A, then
/// For example, if there are two users A, B and a row policy is defined only for A, then
/// if this setting is true the user B will see all rows, and if this setting is false the user B will see no rows.
void setEnabledUsersWithoutRowPoliciesCanReadRows(bool enable) { users_without_row_policies_can_read_rows = enable; }
bool isEnabledUsersWithoutRowPoliciesCanReadRows() const { return users_without_row_policies_can_read_rows; }

View File

@ -1,7 +1,6 @@
#include <Access/Common/AccessFlags.h>
#include <Access/Common/AccessType.h>
#include <Common/Exception.h>
#include <base/types.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
@ -103,6 +102,7 @@ namespace
const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; }
const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; }
const Flags & getNamedCollectionFlags() const { return all_flags_for_target[NAMED_COLLECTION]; }
const Flags & getUserNameFlags() const { return all_flags_for_target[USER_NAME]; }
const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); }
const Flags & getAllFlagsGrantableOnGlobalWithParameterLevel() const { return getGlobalWithParameterFlags(); }
const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; }
@ -121,6 +121,7 @@ namespace
COLUMN,
DICTIONARY,
NAMED_COLLECTION,
USER_NAME,
};
struct Node;
@ -300,7 +301,7 @@ namespace
collectAllFlags(child.get());
all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN];
all_flags_grantable_on_global_with_parameter_level = all_flags_for_target[NAMED_COLLECTION];
all_flags_grantable_on_global_with_parameter_level = all_flags_for_target[NAMED_COLLECTION] | all_flags_for_target[USER_NAME];
all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level;
}
@ -351,7 +352,7 @@ namespace
std::unordered_map<std::string_view, Flags> keyword_to_flags_map;
std::vector<Flags> access_type_to_flags_mapping;
Flags all_flags;
Flags all_flags_for_target[static_cast<size_t>(NAMED_COLLECTION) + 1];
Flags all_flags_for_target[static_cast<size_t>(USER_NAME) + 1];
Flags all_flags_grantable_on_database_level;
Flags all_flags_grantable_on_table_level;
Flags all_flags_grantable_on_global_with_parameter_level;
@ -371,7 +372,11 @@ std::unordered_map<AccessFlags::ParameterType, AccessFlags> AccessFlags::splitIn
if (named_collection_flags)
result.emplace(ParameterType::NAMED_COLLECTION, named_collection_flags);
auto other_flags = (~AccessFlags::allNamedCollectionFlags()) & *this;
auto user_flags = AccessFlags::allUserNameFlags() & *this;
if (user_flags)
result.emplace(ParameterType::USER_NAME, user_flags);
auto other_flags = (~named_collection_flags & ~user_flags) & *this;
if (other_flags)
result.emplace(ParameterType::NONE, other_flags);
@ -387,6 +392,9 @@ AccessFlags::ParameterType AccessFlags::getParameterType() const
if (AccessFlags::allNamedCollectionFlags().contains(*this))
return AccessFlags::NAMED_COLLECTION;
if (AccessFlags::allUserNameFlags().contains(*this))
return AccessFlags::USER_NAME;
throw Exception(ErrorCodes::MIXED_ACCESS_PARAMETER_TYPES, "Having mixed parameter types: {}", toString());
}
@ -405,6 +413,7 @@ AccessFlags AccessFlags::allTableFlags() { return Helper::instance().getTableFla
AccessFlags AccessFlags::allColumnFlags() { return Helper::instance().getColumnFlags(); }
AccessFlags AccessFlags::allDictionaryFlags() { return Helper::instance().getDictionaryFlags(); }
AccessFlags AccessFlags::allNamedCollectionFlags() { return Helper::instance().getNamedCollectionFlags(); }
AccessFlags AccessFlags::allUserNameFlags() { return Helper::instance().getUserNameFlags(); }
AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalLevel(); }
AccessFlags AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalWithParameterLevel(); }
AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Helper::instance().getAllFlagsGrantableOnDatabaseLevel(); }

View File

@ -57,6 +57,7 @@ public:
{
NONE,
NAMED_COLLECTION,
USER_NAME,
};
ParameterType getParameterType() const;
std::unordered_map<ParameterType, AccessFlags> splitIntoParameterTypes() const;
@ -103,6 +104,9 @@ public:
/// Returns all the flags related to a named collection.
static AccessFlags allNamedCollectionFlags();
/// Returns all the flags related to a user.
static AccessFlags allUserNameFlags();
/// Returns all the flags which could be granted on the global level.
/// The same as allFlags().
static AccessFlags allFlagsGrantableOnGlobalLevel();

View File

@ -12,7 +12,7 @@ enum class AccessType
/// Macro M should be defined as M(name, aliases, node_type, parent_group_name)
/// where name is identifier with underscores (instead of spaces);
/// aliases is a string containing comma-separated list;
/// node_type either specifies access type's level (GLOBAL/NAMED_COLLECTION/DATABASE/TABLE/DICTIONARY/VIEW/COLUMNS),
/// node_type either specifies access type's level (GLOBAL/NAMED_COLLECTION/USER_NAME/DATABASE/TABLE/DICTIONARY/VIEW/COLUMNS),
/// or specifies that the access type is a GROUP of other access types;
/// parent_group_name is the name of the group containing this access type (or NONE if there is no such group).
/// NOTE A parent group must be declared AFTER all its children.
@ -80,13 +80,13 @@ enum class AccessType
M(ALTER_TABLE, "", GROUP, ALTER) \
M(ALTER_DATABASE, "", GROUP, ALTER) \
\
M(ALTER_VIEW_REFRESH, "ALTER LIVE VIEW REFRESH, REFRESH VIEW", VIEW, ALTER_VIEW) \
M(ALTER_VIEW_MODIFY_QUERY, "ALTER TABLE MODIFY QUERY", VIEW, ALTER_VIEW) \
M(ALTER_VIEW_MODIFY_REFRESH, "ALTER TABLE MODIFY QUERY", VIEW, ALTER_VIEW) \
M(ALTER_VIEW_MODIFY_SQL_SECURITY, "ALTER TABLE MODIFY SQL SECURITY", VIEW, ALTER_VIEW) \
M(ALTER_VIEW, "", GROUP, ALTER) /* allows to execute ALTER VIEW REFRESH, ALTER VIEW MODIFY QUERY, ALTER VIEW MODIFY REFRESH;
implicitly enabled by the grant ALTER_TABLE */\
\
M(ALTER, "", GROUP, ALL) /* allows to execute ALTER {TABLE|LIVE VIEW} */\
M(ALTER, "", GROUP, ALL) /* allows to execute ALTER TABLE */\
\
M(CREATE_DATABASE, "", DATABASE, CREATE) /* allows to execute {CREATE|ATTACH} DATABASE */\
M(CREATE_TABLE, "", TABLE, CREATE) /* allows to execute {CREATE|ATTACH} {TABLE|VIEW} */\
@ -139,6 +139,7 @@ enum class AccessType
M(CREATE_SETTINGS_PROFILE, "CREATE PROFILE", GLOBAL, ACCESS_MANAGEMENT) \
M(ALTER_SETTINGS_PROFILE, "ALTER PROFILE", GLOBAL, ACCESS_MANAGEMENT) \
M(DROP_SETTINGS_PROFILE, "DROP PROFILE", GLOBAL, ACCESS_MANAGEMENT) \
M(ALLOW_SQL_SECURITY_NONE, "CREATE SQL SECURITY NONE, ALLOW SQL SECURITY NONE, SQL SECURITY NONE, SECURITY NONE", GLOBAL, ACCESS_MANAGEMENT) \
M(SHOW_USERS, "SHOW CREATE USER", GLOBAL, SHOW_ACCESS) \
M(SHOW_ROLES, "SHOW CREATE ROLE", GLOBAL, SHOW_ACCESS) \
M(SHOW_ROW_POLICIES, "SHOW POLICIES, SHOW CREATE ROW POLICY, SHOW CREATE POLICY", TABLE, SHOW_ACCESS) \
@ -150,6 +151,7 @@ enum class AccessType
M(SHOW_NAMED_COLLECTIONS_SECRETS, "SHOW NAMED COLLECTIONS SECRETS", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) \
M(NAMED_COLLECTION, "NAMED COLLECTION USAGE, USE NAMED COLLECTION", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) \
M(NAMED_COLLECTION_ADMIN, "NAMED COLLECTION CONTROL", NAMED_COLLECTION, ALL) \
M(SET_DEFINER, "", USER_NAME, ALL) \
\
M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \
M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \

View File

@ -0,0 +1,11 @@
#pragma once
#include <Core/Types.h>
/// SQL security enum. Used in ASTSQLSecurity::type. For more info, please refer to the docs/sql-reference/statements/create/view.md#sql_security
enum class SQLSecurityType : uint8_t
{
INVOKER, /// All queries will be executed with the current user's context.
DEFINER, /// All queries will be executed with the specified user's context.
NONE, /// All queries will be executed with the global context.
};

View File

@ -31,7 +31,7 @@ struct AuthResult
{
UUID user_id;
/// Session settings received from authentication server (if any)
SettingsChanges settings;
SettingsChanges settings{};
};
/// Contains entities, i.e. instances of classes derived from IAccessEntity.

View File

@ -53,7 +53,8 @@ TEST(AccessRights, Union)
"SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, "
"SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, "
"SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, "
"SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*, GRANT NAMED COLLECTION ADMIN ON db1");
"SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*, "
"GRANT SET DEFINER ON db1, GRANT NAMED COLLECTION ADMIN ON db1");
}

View File

@ -219,7 +219,7 @@ public:
: IAggregateFunctionDataHelper<AggregateFunctionCountData, AggregateFunctionCountNotNullUnary>({argument}, params, createResultType())
{
if (!argument->isNullable())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: not Nullable data type passed to AggregateFunctionCountNotNullUnary");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not Nullable data type passed to AggregateFunctionCountNotNullUnary");
}
String getName() const override { return "count"; }

View File

@ -100,7 +100,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
{
AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null");
if (!combinator)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: cannot find aggregate function combinator "
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find aggregate function combinator "
"to apply a function to Nullable arguments.");
DataTypes nested_types = combinator->transformArguments(types_without_low_cardinality);
@ -123,7 +123,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
auto with_original_arguments = getImpl(name, action, types_without_low_cardinality, parameters, out_properties, false);
if (!with_original_arguments)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: AggregateFunctionFactory returned nullptr");
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory returned nullptr");
return with_original_arguments;
}

View File

@ -0,0 +1,439 @@
#include <cassert>
#include <memory>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadHelpersArena.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnArray.h>
#include <Common/HashTable/HashSet.h>
#include <Common/HashTable/HashTableKeyHolder.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/KeyHolderHelpers.h>
#include <Core/Field.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
struct Settings;
template <typename T>
struct AggregateFunctionGroupArrayIntersectData
{
using Set = HashSet<T>;
Set value;
UInt64 version = 0;
};
/// Puts all values to the hash set. Returns an array of unique values. Implemented for numeric types.
template <typename T>
class AggregateFunctionGroupArrayIntersect
: public IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectData<T>, AggregateFunctionGroupArrayIntersect<T>>
{
private:
using State = AggregateFunctionGroupArrayIntersectData<T>;
public:
AggregateFunctionGroupArrayIntersect(const DataTypePtr & argument_type, const Array & parameters_)
: IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectData<T>,
AggregateFunctionGroupArrayIntersect<T>>({argument_type}, parameters_, argument_type) {}
AggregateFunctionGroupArrayIntersect(const DataTypePtr & argument_type, const Array & parameters_, const DataTypePtr & result_type_)
: IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectData<T>,
AggregateFunctionGroupArrayIntersect<T>>({argument_type}, parameters_, result_type_) {}
String getName() const override { return "GroupArrayIntersect"; }
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto & version = this->data(place).version;
auto & set = this->data(place).value;
const auto data_column = assert_cast<const ColumnArray &>(*columns[0]).getDataPtr();
const auto & offsets = assert_cast<const ColumnArray &>(*columns[0]).getOffsets();
const size_t offset = offsets[static_cast<ssize_t>(row_num) - 1];
const auto arr_size = offsets[row_num] - offset;
++version;
if (version == 1)
{
for (size_t i = 0; i < arr_size; ++i)
set.insert(static_cast<T>((*data_column)[offset + i].get<T>()));
}
else if (!set.empty())
{
typename State::Set new_set;
for (size_t i = 0; i < arr_size; ++i)
{
typename State::Set::LookupResult set_value = set.find(static_cast<T>((*data_column)[offset + i].get<T>()));
if (set_value != nullptr)
new_set.insert(static_cast<T>((*data_column)[offset + i].get<T>()));
}
set = std::move(new_set);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto & set = this->data(place).value;
const auto & rhs_set = this->data(rhs).value;
if (this->data(rhs).version == 0)
return;
UInt64 version = this->data(place).version++;
if (version == 0)
{
for (auto & rhs_elem : rhs_set)
set.insert(rhs_elem.getValue());
return;
}
if (!set.empty())
{
auto create_new_set = [](auto & lhs_val, auto & rhs_val)
{
typename State::Set new_set;
for (auto & lhs_elem : lhs_val)
{
auto res = rhs_val.find(lhs_elem.getValue());
if (res != nullptr)
new_set.insert(lhs_elem.getValue());
}
return new_set;
};
auto new_set = rhs_set.size() < set.size() ? create_new_set(rhs_set, set) : create_new_set(set, rhs_set);
set = std::move(new_set);
}
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
auto & set = this->data(place).value;
auto version = this->data(place).version;
writeVarUInt(version, buf);
writeVarUInt(set.size(), buf);
for (const auto & elem : set)
writeIntBinary(elem.getValue(), buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
readVarUInt(this->data(place).version, buf);
this->data(place).value.read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
ColumnArray & arr_to = assert_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = arr_to.getOffsets();
const auto & set = this->data(place).value;
offsets_to.push_back(offsets_to.back() + set.size());
typename ColumnVector<T>::Container & data_to = assert_cast<ColumnVector<T> &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(old_size + set.size());
size_t i = 0;
for (auto it = set.begin(); it != set.end(); ++it, ++i)
data_to[old_size + i] = it->getValue();
}
};
/// Generic implementation, it uses serialized representation as object descriptor.
struct AggregateFunctionGroupArrayIntersectGenericData
{
using Set = HashSet<StringRef>;
Set value;
UInt64 version = 0;
};
/** Template parameter with true value should be used for columns that store their elements in memory continuously.
* For such columns GroupArrayIntersect() can be implemented more efficiently (especially for small numeric arrays).
*/
template <bool is_plain_column = false>
class AggregateFunctionGroupArrayIntersectGeneric
: public IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectGenericData,
AggregateFunctionGroupArrayIntersectGeneric<is_plain_column>>
{
const DataTypePtr & input_data_type;
using State = AggregateFunctionGroupArrayIntersectGenericData;
public:
AggregateFunctionGroupArrayIntersectGeneric(const DataTypePtr & input_data_type_, const Array & parameters_)
: IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectGenericData, AggregateFunctionGroupArrayIntersectGeneric<is_plain_column>>({input_data_type_}, parameters_, input_data_type_)
, input_data_type(this->argument_types[0]) {}
AggregateFunctionGroupArrayIntersectGeneric(const DataTypePtr & input_data_type_, const Array & parameters_, const DataTypePtr & result_type_)
: IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectGenericData, AggregateFunctionGroupArrayIntersectGeneric<is_plain_column>>({input_data_type_}, parameters_, result_type_)
, input_data_type(result_type_) {}
String getName() const override { return "GroupArrayIntersect"; }
bool allocatesMemoryInArena() const override { return true; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
auto & set = this->data(place).value;
auto & version = this->data(place).version;
bool inserted;
State::Set::LookupResult it;
const auto data_column = assert_cast<const ColumnArray &>(*columns[0]).getDataPtr();
const auto & offsets = assert_cast<const ColumnArray &>(*columns[0]).getOffsets();
const size_t offset = offsets[static_cast<ssize_t>(row_num) - 1];
const auto arr_size = offsets[row_num] - offset;
++version;
if (version == 1)
{
for (size_t i = 0; i < arr_size; ++i)
{
if constexpr (is_plain_column)
set.emplace(ArenaKeyHolder{data_column->getDataAt(offset + i), *arena}, it, inserted);
else
{
const char * begin = nullptr;
StringRef serialized = data_column->serializeValueIntoArena(offset + i, *arena, begin);
assert(serialized.data != nullptr);
set.emplace(SerializedKeyHolder{serialized, *arena}, it, inserted);
}
}
}
else if (!set.empty())
{
typename State::Set new_set;
for (size_t i = 0; i < arr_size; ++i)
{
if constexpr (is_plain_column)
{
it = set.find(data_column->getDataAt(offset + i));
if (it != nullptr)
new_set.emplace(ArenaKeyHolder{data_column->getDataAt(offset + i), *arena}, it, inserted);
}
else
{
const char * begin = nullptr;
StringRef serialized = data_column->serializeValueIntoArena(offset + i, *arena, begin);
assert(serialized.data != nullptr);
it = set.find(serialized);
if (it != nullptr)
new_set.emplace(SerializedKeyHolder{serialized, *arena}, it, inserted);
}
}
set = std::move(new_set);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & set = this->data(place).value;
const auto & rhs_value = this->data(rhs).value;
if (this->data(rhs).version == 0)
return;
UInt64 version = this->data(place).version++;
if (version == 0)
{
bool inserted;
State::Set::LookupResult it;
for (auto & rhs_elem : rhs_value)
{
set.emplace(ArenaKeyHolder{rhs_elem.getValue(), *arena}, it, inserted);
}
}
else if (!set.empty())
{
auto create_new_map = [](auto & lhs_val, auto & rhs_val)
{
typename State::Set new_map;
for (auto & lhs_elem : lhs_val)
{
auto val = rhs_val.find(lhs_elem.getValue());
if (val != nullptr)
new_map.insert(lhs_elem.getValue());
}
return new_map;
};
auto new_map = rhs_value.size() < set.size() ? create_new_map(rhs_value, set) : create_new_map(set, rhs_value);
set = std::move(new_map);
}
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
auto & set = this->data(place).value;
auto & version = this->data(place).version;
writeVarUInt(version, buf);
writeVarUInt(set.size(), buf);
for (const auto & elem : set)
writeStringBinary(elem.getValue(), buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
auto & set = this->data(place).value;
auto & version = this->data(place).version;
size_t size;
readVarUInt(version, buf);
readVarUInt(size, buf);
set.reserve(size);
UInt64 elem_version;
for (size_t i = 0; i < size; ++i)
{
auto key = readStringBinaryInto(*arena, buf);
readVarUInt(elem_version, buf);
set.insert(key);
}
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
ColumnArray & arr_to = assert_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = arr_to.getOffsets();
IColumn & data_to = arr_to.getData();
auto & set = this->data(place).value;
offsets_to.push_back(offsets_to.back() + set.size());
for (auto & elem : set)
{
if constexpr (is_plain_column)
data_to.insertData(elem.getValue().data, elem.getValue().size);
else
std::ignore = data_to.deserializeAndInsertFromArena(elem.getValue().data);
}
}
};
namespace
{
/// Substitute return type for Date and DateTime
class AggregateFunctionGroupArrayIntersectDate : public AggregateFunctionGroupArrayIntersect<DataTypeDate::FieldType>
{
public:
explicit AggregateFunctionGroupArrayIntersectDate(const DataTypePtr & argument_type, const Array & parameters_)
: AggregateFunctionGroupArrayIntersect<DataTypeDate::FieldType>(argument_type, parameters_, createResultType()) {}
static DataTypePtr createResultType() { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDate>()); }
};
class AggregateFunctionGroupArrayIntersectDateTime : public AggregateFunctionGroupArrayIntersect<DataTypeDateTime::FieldType>
{
public:
explicit AggregateFunctionGroupArrayIntersectDateTime(const DataTypePtr & argument_type, const Array & parameters_)
: AggregateFunctionGroupArrayIntersect<DataTypeDateTime::FieldType>(argument_type, parameters_, createResultType()) {}
static DataTypePtr createResultType() { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>()); }
};
class AggregateFunctionGroupArrayIntersectDate32 : public AggregateFunctionGroupArrayIntersect<DataTypeDate32::FieldType>
{
public:
explicit AggregateFunctionGroupArrayIntersectDate32(const DataTypePtr & argument_type, const Array & parameters_)
: AggregateFunctionGroupArrayIntersect<DataTypeDate32::FieldType>(argument_type, parameters_, createResultType()) {}
static DataTypePtr createResultType() { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDate32>()); }
};
IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, const Array & parameters)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupArrayIntersectDate(argument_type, parameters);
else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupArrayIntersectDateTime(argument_type, parameters);
else if (which.idx == TypeIndex::Date32) return new AggregateFunctionGroupArrayIntersectDate32(argument_type, parameters);
else if (which.idx == TypeIndex::DateTime64)
{
const auto * datetime64_type = dynamic_cast<const DataTypeDateTime64 *>(argument_type.get());
const auto return_type = std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime64>(datetime64_type->getScale()));
return new AggregateFunctionGroupArrayIntersectGeneric<true>(argument_type, parameters, return_type);
}
else
{
/// Check that we can use plain version of AggregateFunctionGroupArrayIntersectGeneric
if (argument_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
return new AggregateFunctionGroupArrayIntersectGeneric<true>(argument_type, parameters);
else
return new AggregateFunctionGroupArrayIntersectGeneric<false>(argument_type, parameters);
}
}
inline AggregateFunctionPtr createAggregateFunctionGroupArrayIntersectImpl(const std::string & name, const DataTypePtr & argument_type, const Array & parameters)
{
const auto & nested_type = dynamic_cast<const DataTypeArray &>(*argument_type).getNestedType();
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionGroupArrayIntersect, const DataTypePtr &>(*nested_type, argument_type, parameters));
if (!res)
{
res = AggregateFunctionPtr(createWithExtraTypes(argument_type, parameters));
}
if (!res)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_type->getName(), name);
return res;
}
AggregateFunctionPtr createAggregateFunctionGroupArrayIntersect(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);
if (!WhichDataType(argument_types.at(0)).isArray())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function groupArrayIntersect accepts only array type argument.");
if (!parameters.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Incorrect number of parameters for aggregate function {}, should be 0", name);
return createAggregateFunctionGroupArrayIntersectImpl(name, argument_types[0], parameters);
}
}
void registerAggregateFunctionGroupArrayIntersect(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true };
factory.registerFunction("groupArrayIntersect", { createAggregateFunctionGroupArrayIntersect, properties });
}
}

View File

@ -249,7 +249,7 @@ public:
: Base(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size())
{
if (number_of_arguments == 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: single argument is passed to AggregateFunctionIfNullVariadic");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Single argument is passed to AggregateFunctionIfNullVariadic");
if (number_of_arguments > MAX_ARGS)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -429,7 +429,7 @@ public:
, number_of_arguments(arguments.size())
{
if (number_of_arguments == 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: single argument is passed to AggregateFunctionNullVariadic");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Single argument is passed to AggregateFunctionNullVariadic");
if (number_of_arguments > MAX_ARGS)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -18,6 +18,7 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArraySorted(AggregateFunctionFactory & factory);
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArrayIntersect(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory &);
@ -116,6 +117,7 @@ void registerAggregateFunctions()
registerAggregateFunctionGroupArraySorted(factory);
registerAggregateFunctionGroupUniqArray(factory);
registerAggregateFunctionGroupArrayInsertAt(factory);
registerAggregateFunctionGroupArrayIntersect(factory);
registerAggregateFunctionsQuantile(factory);
registerAggregateFunctionsQuantileDeterministic(factory);
registerAggregateFunctionsQuantileExact(factory);

View File

@ -2,10 +2,10 @@
#include <Functions/FunctionFactory.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Common/DateLUT.h>
#include <Common/DateLUTImpl.h>
@ -14,20 +14,19 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int LOGICAL_ERROR;
}
namespace
{
class OptimizeDateOrDateTimeConverterWithPreimageVisitor : public InDepthQueryTreeVisitorWithContext<OptimizeDateOrDateTimeConverterWithPreimageVisitor>
class OptimizeDateOrDateTimeConverterWithPreimageVisitor
: public InDepthQueryTreeVisitorWithContext<OptimizeDateOrDateTimeConverterWithPreimageVisitor>
{
public:
using Base = InDepthQueryTreeVisitorWithContext<OptimizeDateOrDateTimeConverterWithPreimageVisitor>;
explicit OptimizeDateOrDateTimeConverterWithPreimageVisitor(ContextPtr context)
: Base(std::move(context))
{}
explicit OptimizeDateOrDateTimeConverterWithPreimageVisitor(ContextPtr context) : Base(std::move(context)) { }
static bool needChildVisit(QueryTreeNodePtr & node, QueryTreeNodePtr & /*child*/)
{
@ -41,9 +40,7 @@ public:
};
if (const auto * function = node->as<FunctionNode>())
{
return !relations.contains(function->getFunctionName());
}
return true;
}
@ -59,11 +56,16 @@ public:
{"greaterOrEquals", "lessOrEquals"},
};
if (!getSettings().optimize_time_filter_with_preimage)
return;
const auto * function = node->as<FunctionNode>();
if (!function || !swap_relations.contains(function->getFunctionName())) return;
if (!function || !swap_relations.contains(function->getFunctionName()))
return;
if (function->getArguments().getNodes().size() != 2) return;
if (function->getArguments().getNodes().size() != 2)
return;
size_t func_id = function->getArguments().getNodes().size();
@ -72,51 +74,62 @@ public:
if (const auto * func = function->getArguments().getNodes()[i]->as<FunctionNode>())
{
func_id = i;
break;
}
}
if (func_id == function->getArguments().getNodes().size()) return;
if (func_id == function->getArguments().getNodes().size())
return;
size_t literal_id = 1 - func_id;
const auto * literal = function->getArguments().getNodes()[literal_id]->as<ConstantNode>();
if (!literal || literal->getValue().getType() != Field::Types::UInt64) return;
if (!literal || !literal->getResultType()->isValueRepresentedByUnsignedInteger())
return;
String comparator = literal_id > func_id ? function->getFunctionName(): swap_relations.at(function->getFunctionName());
String comparator = literal_id > func_id ? function->getFunctionName() : swap_relations.at(function->getFunctionName());
const auto * func_node = function->getArguments().getNodes()[func_id]->as<FunctionNode>();
/// Currently we only handle single-argument functions.
if (!func_node || func_node->getArguments().getNodes().size() != 1) return;
if (!func_node || func_node->getArguments().getNodes().size() != 1)
return;
const auto * column_id = func_node->getArguments().getNodes()[0]->as<ColumnNode>();
if (!column_id) return;
if (!column_id)
return;
if (column_id->getColumnName() == "__grouping_set")
return;
const auto * column_type = column_id->getColumnType().get();
if (!isDateOrDate32(column_type) && !isDateTime(column_type) && !isDateTime64(column_type)) return;
if (!isDateOrDate32(column_type) && !isDateTime(column_type) && !isDateTime64(column_type))
return;
const auto & converter = FunctionFactory::instance().tryGet(func_node->getFunctionName(), getContext());
if (!converter) return;
if (!converter)
return;
ColumnsWithTypeAndName args;
args.emplace_back(column_id->getColumnType(), "tmp");
auto converter_base = converter->build(args);
if (!converter_base || !converter_base->hasInformationAboutPreimage()) return;
if (!converter_base || !converter_base->hasInformationAboutPreimage())
return;
auto preimage_range = converter_base->getPreimage(*(column_id->getColumnType()), literal->getValue());
if (!preimage_range) return;
if (!preimage_range)
return;
const auto new_node = generateOptimizedDateFilter(comparator, *column_id, *preimage_range);
if (!new_node) return;
if (!new_node)
return;
node = new_node;
}
private:
QueryTreeNodePtr generateOptimizedDateFilter(const String & comparator, const ColumnNode & column_node, const std::pair<Field, Field>& range) const
QueryTreeNodePtr
generateOptimizedDateFilter(const String & comparator, const ColumnNode & column_node, const std::pair<Field, Field> & range) const
{
const DateLUTImpl & date_lut = DateLUT::instance("UTC");
@ -133,7 +146,8 @@ private:
start_date_or_date_time = date_lut.timeToString(range.first.get<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.timeToString(range.second.get<DateLUTImpl::Time>());
}
else [[unlikely]] return {};
else [[unlikely]]
return {};
if (comparator == "equals")
{
@ -174,7 +188,8 @@ private:
else if (comparator == "greater")
{
const auto new_date_filter = std::make_shared<FunctionNode>("greaterOrEquals");
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
new_date_filter->getArguments().getNodes().push_back(
std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(end_date_or_date_time));
resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName());
@ -183,7 +198,8 @@ private:
else if (comparator == "lessOrEquals")
{
const auto new_date_filter = std::make_shared<FunctionNode>("less");
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
new_date_filter->getArguments().getNodes().push_back(
std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(end_date_or_date_time));
resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName());
@ -192,7 +208,8 @@ private:
else if (comparator == "less" || comparator == "greaterOrEquals")
{
const auto new_date_filter = std::make_shared<FunctionNode>(comparator);
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
new_date_filter->getArguments().getNodes().push_back(
std::make_shared<ColumnNode>(column_node.getColumn(), column_node.getColumnSource()));
new_date_filter->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(start_date_or_date_time));
resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName());
@ -200,7 +217,8 @@ private:
}
else [[unlikely]]
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Expected equals, notEquals, less, lessOrEquals, greater, greaterOrEquals. Actual {}",
comparator);
}

View File

@ -80,6 +80,8 @@
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/Identifier.h>
#include <boost/algorithm/string.hpp>
namespace ProfileEvents
{
extern const Event ScalarSubqueriesGlobalCacheHit;
@ -120,6 +122,7 @@ namespace ErrorCodes
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
extern const int FUNCTION_CANNOT_HAVE_PARAMETERS;
extern const int SYNTAX_ERROR;
extern const int UNEXPECTED_EXPRESSION;
extern const int INVALID_IDENTIFIER;
}
@ -1214,7 +1217,7 @@ private:
static void expandGroupByAll(QueryNode & query_tree_node_typed);
void expandOrderByAll(QueryNode & query_tree_node_typed);
void expandOrderByAll(QueryNode & query_tree_node_typed, const Settings & settings);
static std::string
rewriteAggregateFunctionNameIfNeeded(const std::string & aggregate_function_name, NullsAction action, const ContextPtr & context);
@ -2366,9 +2369,9 @@ void QueryAnalyzer::expandGroupByAll(QueryNode & query_tree_node_typed)
query_tree_node_typed.setIsGroupByAll(false);
}
void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed)
void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed, const Settings & settings)
{
if (!query_tree_node_typed.isOrderByAll())
if (!settings.enable_order_by_all || !query_tree_node_typed.isOrderByAll())
return;
auto * all_node = query_tree_node_typed.getOrderBy().getNodes()[0]->as<SortNode>();
@ -2381,6 +2384,13 @@ void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed)
for (auto & node : projection_nodes)
{
/// Detect and reject ambiguous statements:
/// E.g. for a table with columns "all", "a", "b":
/// - SELECT all, a, b ORDER BY all; -- should we sort by all columns in SELECT or by column "all"?
/// - SELECT a, b AS all ORDER BY all; -- like before but "all" as alias
/// - SELECT func(...) AS all ORDER BY all; -- like before but "all" as function
/// - SELECT a, b ORDER BY all; -- tricky in other way: does the user want to sort by columns in SELECT clause or by not SELECTed column "all"?
auto resolved_expression_it = resolved_expressions.find(node);
if (resolved_expression_it != resolved_expressions.end())
{
@ -2389,6 +2399,9 @@ void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expression nodes list expected 1 projection names. Actual {}",
projection_names.size());
if (boost::iequals(projection_names[0], "all"))
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION,
"Cannot use ORDER BY ALL to sort a column with name 'all', please disable setting `enable_order_by_all` and try again");
}
auto sort_node = std::make_shared<SortNode>(node, all_node->getSortDirection(), all_node->getNullsSortDirection());
@ -7564,7 +7577,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
if (settings.enable_positional_arguments)
replaceNodesWithPositionalArguments(query_node_typed.getOrderByNode(), query_node_typed.getProjection().getNodes(), scope);
expandOrderByAll(query_node_typed);
expandOrderByAll(query_node_typed, settings);
resolveSortNodeList(query_node_typed.getOrderByNode(), scope);
}

View File

@ -219,13 +219,13 @@ public:
is_group_by_all = is_group_by_all_value;
}
/// Returns true, if query node has ORDER BY * modifier, false otherwise
/// Returns true, if query node has ORDER BY ALL modifier, false otherwise
bool isOrderByAll() const
{
return is_order_by_all;
}
/// Set query node ORDER BY * modifier value
/// Set query node ORDER BY ALL modifier value
void setIsOrderByAll(bool is_order_by_all_value)
{
is_order_by_all = is_order_by_all_value;

View File

@ -5,6 +5,7 @@
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
@ -557,7 +558,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
}
else if (const auto * ast_literal = expression->as<ASTLiteral>())
{
result = std::make_shared<ConstantNode>(ast_literal->value);
if (context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_as_common_type)
result = std::make_shared<ConstantNode>(ast_literal->value, applyVisitor(FieldToDataType<LeastSupertypeOnError::Variant>(), ast_literal->value));
else
result = std::make_shared<ConstantNode>(ast_literal->value);
}
else if (const auto * function = expression->as<ASTFunction>())
{

View File

@ -388,7 +388,7 @@ QueryTreeNodes extractAllTableReferences(const QueryTreeNodePtr & tree)
return result;
}
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join)
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join, bool recursive)
{
QueryTreeNodes result;
@ -406,15 +406,28 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node,
{
case QueryTreeNodeType::TABLE:
[[fallthrough]];
case QueryTreeNodeType::QUERY:
[[fallthrough]];
case QueryTreeNodeType::UNION:
[[fallthrough]];
case QueryTreeNodeType::TABLE_FUNCTION:
{
result.push_back(std::move(node_to_process));
break;
}
case QueryTreeNodeType::QUERY:
{
if (recursive)
nodes_to_process.push_back(node_to_process->as<QueryNode>()->getJoinTree());
result.push_back(std::move(node_to_process));
break;
}
case QueryTreeNodeType::UNION:
{
if (recursive)
{
for (const auto & union_node : node_to_process->as<UnionNode>()->getQueries().getNodes())
nodes_to_process.push_back(union_node);
}
result.push_back(std::move(node_to_process));
break;
}
case QueryTreeNodeType::ARRAY_JOIN:
{
auto & array_join_node = node_to_process->as<ArrayJoinNode &>();

View File

@ -54,7 +54,7 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q
QueryTreeNodes extractAllTableReferences(const QueryTreeNodePtr & tree);
/// Extract table, table function, query, union from join tree.
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false);
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false, bool recursive = false);
/// Extract left table expression from join tree.
QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node);

View File

@ -32,8 +32,6 @@ namespace ErrorCodes
M(UInt64, shard_num) \
M(UInt64, replica_num) \
M(Bool, check_parts) \
M(Bool, check_projection_parts) \
M(Bool, allow_backup_broken_projections) \
M(Bool, internal) \
M(String, host_id) \
M(OptionalUUID, backup_uuid)

View File

@ -62,12 +62,6 @@ struct BackupSettings
/// Check checksums of the data parts before writing them to a backup.
bool check_parts = true;
/// Check checksums of the projection data parts before writing them to a backup.
bool check_projection_parts = true;
/// Allow to create backup with broken projections.
bool allow_backup_broken_projections = false;
/// Internal, should not be specified by user.
/// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER.
bool internal = false;

View File

@ -271,8 +271,8 @@ enum class BackupsWorker::ThreadPoolId
/// "RESTORE ASYNC" waits in background while the data of all tables are restored.
RESTORE_ASYNC,
/// Restores the data of tables.
RESTORE_TABLES_DATA,
/// Restores from backups.
RESTORE,
};
@ -320,13 +320,13 @@ public:
case ThreadPoolId::RESTORE_ASYNC:
case ThreadPoolId::RESTORE_ASYNC_ON_CLUSTER:
case ThreadPoolId::RESTORE_TABLES_DATA:
case ThreadPoolId::RESTORE:
{
metric_threads = CurrentMetrics::RestoreThreads;
metric_active_threads = CurrentMetrics::RestoreThreadsActive;
metric_active_threads = CurrentMetrics::RestoreThreadsScheduled;
max_threads = num_restore_threads;
use_queue = (thread_pool_id != ThreadPoolId::RESTORE_TABLES_DATA);
use_queue = (thread_pool_id != ThreadPoolId::RESTORE);
break;
}
}
@ -347,7 +347,7 @@ public:
auto wait_sequence = {
ThreadPoolId::RESTORE_ASYNC_ON_CLUSTER,
ThreadPoolId::RESTORE_ASYNC,
ThreadPoolId::RESTORE_TABLES_DATA,
ThreadPoolId::RESTORE,
ThreadPoolId::BACKUP_ASYNC_ON_CLUSTER,
ThreadPoolId::BACKUP_ASYNC,
ThreadPoolId::BACKUP_COPY_FILES,
@ -975,7 +975,7 @@ void BackupsWorker::doRestore(
String addr_database = address->default_database.empty() ? current_database : address->default_database;
for (auto & element : restore_elements)
element.setCurrentDatabase(addr_database);
RestorerFromBackup dummy_restorer{restore_elements, restore_settings, nullptr, backup, context};
RestorerFromBackup dummy_restorer{restore_elements, restore_settings, nullptr, backup, context, getThreadPool(ThreadPoolId::RESTORE), {}};
dummy_restorer.run(RestorerFromBackup::CHECK_ACCESS_ONLY);
}
}
@ -1004,19 +1004,17 @@ void BackupsWorker::doRestore(
{
restore_query->setCurrentDatabase(current_database);
/// Restore metadata and prepare data restoring tasks.
DataRestoreTasks data_restore_tasks;
auto after_task_callback = [&]
{
RestorerFromBackup restorer{restore_query->elements, restore_settings, restore_coordination,
backup, context};
data_restore_tasks = restorer.run(RestorerFromBackup::RESTORE);
}
maybeSleepForTesting();
setNumFilesAndSize(restore_id, backup->getNumFiles(), backup->getTotalSize(), backup->getNumEntries(),
backup->getUncompressedSize(), backup->getCompressedSize(), backup->getNumReadFiles(), backup->getNumReadBytes());
};
/// Execute the data restoring tasks.
restoreTablesData(restore_id, backup, std::move(data_restore_tasks), getThreadPool(ThreadPoolId::RESTORE_TABLES_DATA), context->getProcessListElement());
/// We have restored everything, we need to tell other hosts (they could be waiting for it).
restore_coordination->setStage(Stage::COMPLETED, "");
/// Restore from the backup.
RestorerFromBackup restorer{restore_query->elements, restore_settings, restore_coordination,
backup, context, getThreadPool(ThreadPoolId::RESTORE), after_task_callback};
restorer.run(RestorerFromBackup::RESTORE);
}
LOG_INFO(log, "Restored from {} {} successfully", (restore_settings.internal ? "internal backup" : "backup"), backup_name_for_logging);
@ -1024,83 +1022,6 @@ void BackupsWorker::doRestore(
}
void BackupsWorker::restoreTablesData(const OperationID & restore_id, BackupPtr backup, DataRestoreTasks && tasks, ThreadPool & thread_pool, QueryStatusPtr process_list_element)
{
size_t num_active_jobs = 0;
std::mutex mutex;
std::condition_variable event;
std::exception_ptr exception;
auto thread_group = CurrentThread::getGroup();
for (auto & task : tasks)
{
{
std::unique_lock lock{mutex};
if (exception)
break;
++num_active_jobs;
}
auto job = [&]()
{
SCOPE_EXIT_SAFE(
std::lock_guard lock{mutex};
if (!--num_active_jobs)
event.notify_all();
CurrentThread::detachFromGroupIfNotDetached();
);
try
{
if (thread_group)
CurrentThread::attachToGroup(thread_group);
setThreadName("RestoreWorker");
{
std::lock_guard lock{mutex};
if (exception)
return;
}
if (process_list_element)
process_list_element->checkTimeLimit();
std::move(task)();
maybeSleepForTesting();
setNumFilesAndSize(
restore_id,
backup->getNumFiles(),
backup->getTotalSize(),
backup->getNumEntries(),
backup->getUncompressedSize(),
backup->getCompressedSize(),
backup->getNumReadFiles(),
backup->getNumReadBytes());
}
catch (...)
{
std::lock_guard lock{mutex};
if (!exception)
exception = std::current_exception();
}
};
thread_pool.scheduleOrThrowOnError(job);
}
{
std::unique_lock lock{mutex};
event.wait(lock, [&] { return !num_active_jobs; });
if (exception)
std::rethrow_exception(exception);
}
}
void BackupsWorker::addInfo(const OperationID & id, const String & name, const String & base_backup_name, const String & query_id,
bool internal, QueryStatusPtr process_list_element, BackupStatus status)
{

View File

@ -25,6 +25,7 @@
#include <base/insertAtEnd.h>
#include <boost/algorithm/string/join.hpp>
#include <filesystem>
#include <ranges>
namespace fs = std::filesystem;
@ -80,23 +81,34 @@ RestorerFromBackup::RestorerFromBackup(
const RestoreSettings & restore_settings_,
std::shared_ptr<IRestoreCoordination> restore_coordination_,
const BackupPtr & backup_,
const ContextMutablePtr & context_)
const ContextMutablePtr & context_,
ThreadPool & thread_pool_,
const std::function<void()> & after_task_callback_)
: restore_query_elements(restore_query_elements_)
, restore_settings(restore_settings_)
, restore_coordination(restore_coordination_)
, backup(backup_)
, context(context_)
, process_list_element(context->getProcessListElement())
, after_task_callback(after_task_callback_)
, on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000))
, create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000))
, log(getLogger("RestorerFromBackup"))
, tables_dependencies("RestorerFromBackup")
, thread_pool(thread_pool_)
{
}
RestorerFromBackup::~RestorerFromBackup() = default;
RestorerFromBackup::~RestorerFromBackup()
{
if (!futures.empty())
{
LOG_ERROR(log, "RestorerFromBackup must not be destroyed while {} tasks are still running", futures.size());
chassert(false && "RestorerFromBackup must not be destroyed while some tasks are still running");
}
}
RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode)
void RestorerFromBackup::run(Mode mode)
{
/// run() can be called onle once.
if (!current_stage.empty())
@ -115,31 +127,87 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode)
/// Find all the databases and tables which we will read from the backup.
setStage(Stage::FINDING_TABLES_IN_BACKUP);
findDatabasesAndTablesInBackup();
waitFutures();
/// Check access rights.
checkAccessForObjectsFoundInBackup();
if (mode == Mode::CHECK_ACCESS_ONLY)
return {};
return;
/// Create databases using the create queries read from the backup.
setStage(Stage::CREATING_DATABASES);
createDatabases();
waitFutures();
/// Create tables using the create queries read from the backup.
setStage(Stage::CREATING_TABLES);
removeUnresolvedDependencies();
createTables();
waitFutures();
/// All what's left is to insert data to tables.
/// No more data restoring tasks are allowed after this point.
setStage(Stage::INSERTING_DATA_TO_TABLES);
return getDataRestoreTasks();
insertDataToTables();
waitFutures();
runDataRestoreTasks();
/// Restored successfully!
setStage(Stage::COMPLETED);
}
void RestorerFromBackup::waitFutures()
{
std::exception_ptr error;
for (;;)
{
std::vector<std::future<void>> futures_to_wait;
{
std::lock_guard lock{mutex};
std::swap(futures_to_wait, futures);
}
if (futures_to_wait.empty())
break;
/// Wait for all tasks.
for (auto & future : futures_to_wait)
future.wait();
/// Check if there is an exception.
for (auto & future : futures_to_wait)
{
try
{
future.get();
}
catch (...)
{
if (!error)
error = std::current_exception();
exception_caught = true;
}
}
}
if (error)
std::rethrow_exception(error);
}
size_t RestorerFromBackup::getNumFutures() const
{
std::lock_guard lock{mutex};
return futures.size();
}
void RestorerFromBackup::setStage(const String & new_stage, const String & message)
{
LOG_TRACE(log, "Setting stage: {}", new_stage);
if (getNumFutures() != 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot change the stage while some tasks ({}) are still running", getNumFutures());
checkIsQueryCancelled();
current_stage = new_stage;
@ -154,6 +222,32 @@ void RestorerFromBackup::setStage(const String & new_stage, const String & messa
}
}
void RestorerFromBackup::schedule(std::function<void()> && task_, const char * thread_name_)
{
if (exception_caught)
return;
checkIsQueryCancelled();
auto future = scheduleFromThreadPool<void>(
[this, task = std::move(task_)]() mutable
{
if (exception_caught)
return;
checkIsQueryCancelled();
std::move(task)();
if (after_task_callback)
after_task_callback();
},
thread_pool,
thread_name_);
std::lock_guard lock{mutex};
futures.push_back(std::move(future));
}
void RestorerFromBackup::checkIsQueryCancelled() const
{
if (process_list_element)
@ -241,8 +335,6 @@ void RestorerFromBackup::findRootPathsInBackup()
void RestorerFromBackup::findDatabasesAndTablesInBackup()
{
database_infos.clear();
table_infos.clear();
for (const auto & element : restore_query_elements)
{
switch (element.type)
@ -270,10 +362,17 @@ void RestorerFromBackup::findDatabasesAndTablesInBackup()
}
}
LOG_INFO(log, "Will restore {} databases and {} tables", database_infos.size(), table_infos.size());
LOG_INFO(log, "Will restore {} databases and {} tables", getNumDatabases(), getNumTables());
}
void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & partitions)
{
schedule(
[this, table_name_in_backup, partitions]() { findTableInBackupImpl(table_name_in_backup, partitions); },
"Restore_FindTbl");
}
void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & partitions)
{
bool is_temporary_table = (table_name_in_backup.database == DatabaseCatalog::TEMPORARY_DATABASE);
@ -326,30 +425,38 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name
ASTPtr create_table_query = parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
applyCustomStoragePolicy(create_table_query);
renameDatabaseAndTableNameInCreateQuery(create_table_query, renaming_map, context->getGlobalContext());
String create_table_query_str = serializeAST(*create_table_query);
QualifiedTableName table_name = renaming_map.getNewTableName(table_name_in_backup);
bool is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table});
auto table_dependencies = getDependenciesFromCreateQuery(context, table_name, create_table_query);
bool table_has_data = backup->hasFiles(data_path_in_backup);
std::lock_guard lock{mutex};
if (auto it = table_infos.find(table_name); it != table_infos.end())
{
const TableInfo & table_info = it->second;
if (table_info.create_table_query && (serializeAST(*table_info.create_table_query) != serializeAST(*create_table_query)))
if (table_info.create_table_query && (table_info.create_table_query_str != create_table_query_str))
{
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,
"Extracted two different create queries for the same {}: {} and {}",
tableNameWithTypeToString(table_name.database, table_name.table, false),
serializeAST(*table_info.create_table_query),
serializeAST(*create_table_query));
table_info.create_table_query_str,
create_table_query_str);
}
}
TableInfo & res_table_info = table_infos[table_name];
res_table_info.create_table_query = create_table_query;
res_table_info.is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table});
res_table_info.has_data = backup->hasFiles(data_path_in_backup);
res_table_info.create_table_query_str = create_table_query_str;
res_table_info.is_predefined_table = is_predefined_table;
res_table_info.has_data = table_has_data;
res_table_info.data_path_in_backup = data_path_in_backup;
tables_dependencies.addDependencies(table_name, getDependenciesFromCreateQuery(context, table_name, create_table_query));
tables_dependencies.addDependencies(table_name, table_dependencies);
if (partitions)
{
@ -379,6 +486,13 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name
}
void RestorerFromBackup::findDatabaseInBackup(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names)
{
schedule(
[this, database_name_in_backup, except_table_names]() { findDatabaseInBackupImpl(database_name_in_backup, except_table_names); },
"Restore_FindDB");
}
void RestorerFromBackup::findDatabaseInBackupImpl(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names)
{
std::optional<fs::path> metadata_path;
std::unordered_set<String> table_names_in_backup;
@ -420,22 +534,28 @@ void RestorerFromBackup::findDatabaseInBackup(const String & database_name_in_ba
ParserCreateQuery create_parser;
ASTPtr create_database_query = parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
renameDatabaseAndTableNameInCreateQuery(create_database_query, renaming_map, context->getGlobalContext());
String create_database_query_str = serializeAST(*create_database_query);
String database_name = renaming_map.getNewDatabaseName(database_name_in_backup);
bool is_predefined_database = DatabaseCatalog::isPredefinedDatabase(database_name);
std::lock_guard lock{mutex};
DatabaseInfo & database_info = database_infos[database_name];
if (database_info.create_database_query && (serializeAST(*database_info.create_database_query) != serializeAST(*create_database_query)))
if (database_info.create_database_query && (database_info.create_database_query_str != create_database_query_str))
{
throw Exception(
ErrorCodes::CANNOT_RESTORE_DATABASE,
"Extracted two different create queries for the same database {}: {} and {}",
backQuoteIfNeed(database_name),
serializeAST(*database_info.create_database_query),
serializeAST(*create_database_query));
database_info.create_database_query_str,
create_database_query_str);
}
database_info.create_database_query = create_database_query;
database_info.is_predefined_database = DatabaseCatalog::isPredefinedDatabase(database_name);
database_info.create_database_query_str = create_database_query_str;
database_info.is_predefined_database = is_predefined_database;
}
for (const String & table_name_in_backup : table_names_in_backup)
@ -474,79 +594,95 @@ void RestorerFromBackup::findEverythingInBackup(const std::set<String> & except_
}
}
size_t RestorerFromBackup::getNumDatabases() const
{
std::lock_guard lock{mutex};
return database_infos.size();
}
size_t RestorerFromBackup::getNumTables() const
{
std::lock_guard lock{mutex};
return table_infos.size();
}
void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
{
AccessRightsElements required_access;
for (const auto & [database_name, database_info] : database_infos)
{
if (database_info.is_predefined_database)
continue;
AccessFlags flags;
if (restore_settings.create_database != RestoreDatabaseCreationMode::kMustExist)
flags |= AccessType::CREATE_DATABASE;
if (!flags)
flags = AccessType::SHOW_DATABASES;
required_access.emplace_back(flags, database_name);
}
for (const auto & [table_name, table_info] : table_infos)
{
if (table_info.is_predefined_table)
std::lock_guard lock{mutex};
for (const auto & [database_name, database_info] : database_infos)
{
if (isSystemFunctionsTableName(table_name))
if (database_info.is_predefined_database)
continue;
AccessFlags flags;
if (restore_settings.create_database != RestoreDatabaseCreationMode::kMustExist)
flags |= AccessType::CREATE_DATABASE;
if (!flags)
flags = AccessType::SHOW_DATABASES;
required_access.emplace_back(flags, database_name);
}
for (const auto & [table_name, table_info] : table_infos)
{
if (table_info.is_predefined_table)
{
/// CREATE_FUNCTION privilege is required to restore the "system.functions" table.
if (!restore_settings.structure_only && table_info.has_data)
required_access.emplace_back(AccessType::CREATE_FUNCTION);
if (isSystemFunctionsTableName(table_name))
{
/// CREATE_FUNCTION privilege is required to restore the "system.functions" table.
if (!restore_settings.structure_only && table_info.has_data)
required_access.emplace_back(AccessType::CREATE_FUNCTION);
}
/// Privileges required to restore ACL system tables are checked separately
/// (see access_restore_task->getRequiredAccess() below).
continue;
}
/// Privileges required to restore ACL system tables are checked separately
/// (see access_restore_task->getRequiredAccess() below).
continue;
}
if (table_name.database == DatabaseCatalog::TEMPORARY_DATABASE)
{
if (table_name.database == DatabaseCatalog::TEMPORARY_DATABASE)
{
if (restore_settings.create_table != RestoreTableCreationMode::kMustExist)
required_access.emplace_back(AccessType::CREATE_TEMPORARY_TABLE);
continue;
}
AccessFlags flags;
const ASTCreateQuery & create = table_info.create_table_query->as<const ASTCreateQuery &>();
if (restore_settings.create_table != RestoreTableCreationMode::kMustExist)
required_access.emplace_back(AccessType::CREATE_TEMPORARY_TABLE);
continue;
{
if (create.is_dictionary)
flags |= AccessType::CREATE_DICTIONARY;
else if (create.is_ordinary_view || create.is_materialized_view || create.is_live_view)
flags |= AccessType::CREATE_VIEW;
else
flags |= AccessType::CREATE_TABLE;
}
if (!restore_settings.structure_only && table_info.has_data)
{
flags |= AccessType::INSERT;
}
if (!flags)
{
if (create.is_dictionary)
flags = AccessType::SHOW_DICTIONARIES;
else
flags = AccessType::SHOW_TABLES;
}
required_access.emplace_back(flags, table_name.database, table_name.table);
}
AccessFlags flags;
const ASTCreateQuery & create = table_info.create_table_query->as<const ASTCreateQuery &>();
if (restore_settings.create_table != RestoreTableCreationMode::kMustExist)
{
if (create.is_dictionary)
flags |= AccessType::CREATE_DICTIONARY;
else if (create.is_ordinary_view || create.is_materialized_view || create.is_live_view)
flags |= AccessType::CREATE_VIEW;
else
flags |= AccessType::CREATE_TABLE;
}
if (!restore_settings.structure_only && table_info.has_data)
{
flags |= AccessType::INSERT;
}
if (!flags)
{
if (create.is_dictionary)
flags = AccessType::SHOW_DICTIONARIES;
else
flags = AccessType::SHOW_TABLES;
}
required_access.emplace_back(flags, table_name.database, table_name.table);
if (access_restorer)
insertAtEnd(required_access, access_restorer->getRequiredAccess());
}
if (access_restorer)
insertAtEnd(required_access, access_restorer->getRequiredAccess());
/// We convert to AccessRights and back to check access rights in a predictable way
/// (some elements could be duplicated or not sorted).
required_access = AccessRights{required_access}.getElements();
@ -556,7 +692,14 @@ void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
void RestorerFromBackup::createDatabases()
{
for (const auto & database_name : database_infos | boost::adaptors::map_keys)
Strings database_names;
{
std::lock_guard lock{mutex};
database_names.reserve(database_infos.size());
std::ranges::copy(database_infos | boost::adaptors::map_keys, std::back_inserter(database_names));
}
for (const auto & database_name : database_names)
{
createDatabase(database_name);
checkDatabase(database_name);
@ -568,6 +711,8 @@ void RestorerFromBackup::createDatabase(const String & database_name) const
if (restore_settings.create_database == RestoreDatabaseCreationMode::kMustExist)
return;
std::lock_guard lock{mutex};
/// Predefined databases always exist.
const auto & database_info = database_infos.at(database_name);
if (database_info.is_predefined_database)
@ -603,7 +748,9 @@ void RestorerFromBackup::createDatabase(const String & database_name) const
void RestorerFromBackup::checkDatabase(const String & database_name)
{
std::lock_guard lock{mutex};
auto & database_info = database_infos.at(database_name);
try
{
DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name);
@ -653,8 +800,9 @@ void RestorerFromBackup::applyCustomStoragePolicy(ASTPtr query_ptr)
void RestorerFromBackup::removeUnresolvedDependencies()
{
auto need_exclude_dependency = [this](const StorageID & table_id)
{
std::lock_guard lock{mutex};
auto need_exclude_dependency = [&](const StorageID & table_id) TSA_REQUIRES(mutex) -> bool {
/// Table will be restored.
if (table_infos.contains(table_id.getQualifiedName()))
return false;
@ -700,14 +848,18 @@ void RestorerFromBackup::removeUnresolvedDependencies()
void RestorerFromBackup::createTables()
{
/// We need to create tables considering their dependencies.
tables_dependencies.log();
auto tables_to_create = tables_dependencies.getTablesSortedByDependency();
std::vector<StorageID> tables_to_create;
{
std::lock_guard lock{mutex};
tables_dependencies.log();
tables_to_create = tables_dependencies.getTablesSortedByDependency();
}
for (const auto & table_id : tables_to_create)
{
auto table_name = table_id.getQualifiedName();
createTable(table_name);
checkTable(table_name);
insertDataToTable(table_name);
}
}
@ -716,6 +868,8 @@ void RestorerFromBackup::createTable(const QualifiedTableName & table_name)
if (restore_settings.create_table == RestoreTableCreationMode::kMustExist)
return;
std::lock_guard lock{mutex};
/// Predefined tables always exist.
auto & table_info = table_infos.at(table_name);
if (table_info.is_predefined_table)
@ -758,6 +912,7 @@ void RestorerFromBackup::createTable(const QualifiedTableName & table_name)
void RestorerFromBackup::checkTable(const QualifiedTableName & table_name)
{
std::lock_guard lock{mutex};
auto & table_info = table_infos.at(table_name);
try
@ -796,20 +951,44 @@ void RestorerFromBackup::checkTable(const QualifiedTableName & table_name)
}
}
void RestorerFromBackup::insertDataToTables()
{
std::vector<QualifiedTableName> table_names;
{
std::lock_guard lock{mutex};
table_names.reserve(table_infos.size());
std::ranges::copy(table_infos | boost::adaptors::map_keys, std::back_inserter(table_names));
}
for (const auto & table_name : table_names)
insertDataToTable(table_name);
}
void RestorerFromBackup::insertDataToTable(const QualifiedTableName & table_name)
{
if (restore_settings.structure_only)
return;
auto & table_info = table_infos.at(table_name);
auto storage = table_info.storage;
StoragePtr storage;
String data_path_in_backup;
std::optional<ASTs> partitions;
{
std::lock_guard lock{mutex};
auto & table_info = table_infos.at(table_name);
storage = table_info.storage;
data_path_in_backup = table_info.data_path_in_backup;
partitions = table_info.partitions;
}
checkIsQueryCancelled();
schedule(
[this, table_name, storage, data_path_in_backup, partitions]() { insertDataToTableImpl(table_name, storage, data_path_in_backup, partitions); },
"Restore_TblData");
}
void RestorerFromBackup::insertDataToTableImpl(const QualifiedTableName & table_name, StoragePtr storage, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
{
try
{
const auto & data_path_in_backup = table_info.data_path_in_backup;
const auto & partitions = table_info.partitions;
if (partitions && !storage->supportsBackupPartition())
{
throw Exception(
@ -828,45 +1007,47 @@ void RestorerFromBackup::insertDataToTable(const QualifiedTableName & table_name
void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task)
{
if (current_stage == Stage::INSERTING_DATA_TO_TABLES)
if (current_stage != Stage::INSERTING_DATA_TO_TABLES)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of data-restoring tasks is not allowed");
std::lock_guard lock{mutex};
data_restore_tasks.push_back(std::move(new_task));
}
void RestorerFromBackup::addDataRestoreTasks(DataRestoreTasks && new_tasks)
{
if (current_stage == Stage::INSERTING_DATA_TO_TABLES)
if (current_stage != Stage::INSERTING_DATA_TO_TABLES)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding of data-restoring tasks is not allowed");
std::lock_guard lock{mutex};
insertAtEnd(data_restore_tasks, std::move(new_tasks));
}
RestorerFromBackup::DataRestoreTasks RestorerFromBackup::getDataRestoreTasks()
void RestorerFromBackup::runDataRestoreTasks()
{
if (data_restore_tasks.empty())
return {};
LOG_TRACE(log, "Will insert data to tables");
/// Storages and table locks must exist while we're executing data restoring tasks.
auto storages = std::make_shared<std::vector<StoragePtr>>();
auto table_locks = std::make_shared<std::vector<TableLockHolder>>();
storages->reserve(table_infos.size());
table_locks->reserve(table_infos.size());
for (const auto & table_info : table_infos | boost::adaptors::map_values)
/// Iterations are required here because data restore tasks are allowed to call addDataRestoreTask() and add other data restore tasks.
for (;;)
{
storages->push_back(table_info.storage);
table_locks->push_back(table_info.table_lock);
std::vector<DataRestoreTask> tasks_to_run;
{
std::lock_guard lock{mutex};
std::swap(tasks_to_run, data_restore_tasks);
}
if (tasks_to_run.empty())
break;
for (auto & task : tasks_to_run)
schedule(std::move(task), "Restore_TblTask");
waitFutures();
}
DataRestoreTasks res_tasks;
for (const auto & task : data_restore_tasks)
res_tasks.push_back([task, storages, table_locks] { task(); });
return res_tasks;
}
std::vector<std::pair<UUID, AccessEntityPtr>> RestorerFromBackup::getAccessEntitiesToRestore()
{
std::lock_guard lock{mutex};
if (!access_restorer || access_restored)
return {};

View File

@ -7,6 +7,7 @@
#include <Storages/TableLockHolder.h>
#include <Storages/IStorage_fwd.h>
#include <Interpreters/Context_fwd.h>
#include <Common/ThreadPool_fwd.h>
#include <filesystem>
@ -34,7 +35,9 @@ public:
const RestoreSettings & restore_settings_,
std::shared_ptr<IRestoreCoordination> restore_coordination_,
const BackupPtr & backup_,
const ContextMutablePtr & context_);
const ContextMutablePtr & context_,
ThreadPool & thread_pool_,
const std::function<void()> & after_task_callback_);
~RestorerFromBackup();
@ -51,7 +54,7 @@ public:
using DataRestoreTasks = std::vector<DataRestoreTask>;
/// Restores the metadata of databases and tables and returns tasks to restore the data of tables.
DataRestoreTasks run(Mode mode);
void run(Mode mode);
BackupPtr getBackup() const { return backup; }
const RestoreSettings & getRestoreSettings() const { return restore_settings; }
@ -77,6 +80,7 @@ private:
BackupPtr backup;
ContextMutablePtr context;
QueryStatusPtr process_list_element;
std::function<void()> after_task_callback;
std::chrono::milliseconds on_cluster_first_sync_timeout;
std::chrono::milliseconds create_table_timeout;
LoggerPtr log;
@ -89,9 +93,14 @@ private:
void findDatabasesAndTablesInBackup();
void findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & partitions);
void findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & partitions);
void findDatabaseInBackup(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names);
void findDatabaseInBackupImpl(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names);
void findEverythingInBackup(const std::set<String> & except_database_names, const std::set<DatabaseAndTableName> & except_table_names);
size_t getNumDatabases() const;
size_t getNumTables() const;
void checkAccessForObjectsFoundInBackup() const;
void createDatabases();
@ -104,18 +113,32 @@ private:
void createTables();
void createTable(const QualifiedTableName & table_name);
void checkTable(const QualifiedTableName & table_name);
void insertDataToTable(const QualifiedTableName & table_name);
DataRestoreTasks getDataRestoreTasks();
void insertDataToTables();
void insertDataToTable(const QualifiedTableName & table_name);
void insertDataToTableImpl(const QualifiedTableName & table_name, StoragePtr storage, const String & data_path_in_backup, const std::optional<ASTs> & partitions);
void runDataRestoreTasks();
void setStage(const String & new_stage, const String & message = "");
/// Schedule a task from the thread pool and start executing it.
void schedule(std::function<void()> && task_, const char * thread_name_);
/// Returns the number of currently scheduled or executing tasks.
size_t getNumFutures() const;
/// Waits until all tasks are processed (including the tasks scheduled while we're waiting).
/// Throws an exception if any of the tasks throws an exception.
void waitFutures();
/// Throws an exception if the RESTORE query was cancelled.
void checkIsQueryCancelled() const;
struct DatabaseInfo
{
ASTPtr create_database_query;
String create_database_query_str;
bool is_predefined_database = false;
DatabasePtr database;
};
@ -123,6 +146,7 @@ private:
struct TableInfo
{
ASTPtr create_table_query;
String create_table_query_str;
bool is_predefined_table = false;
bool has_data = false;
std::filesystem::path data_path_in_backup;
@ -133,12 +157,18 @@ private:
};
String current_stage;
std::unordered_map<String, DatabaseInfo> database_infos;
std::map<QualifiedTableName, TableInfo> table_infos;
TablesDependencyGraph tables_dependencies;
std::vector<DataRestoreTask> data_restore_tasks;
std::unique_ptr<AccessRestorerFromBackup> access_restorer;
bool access_restored = false;
std::unordered_map<String, DatabaseInfo> database_infos TSA_GUARDED_BY(mutex);
std::map<QualifiedTableName, TableInfo> table_infos TSA_GUARDED_BY(mutex);
TablesDependencyGraph tables_dependencies TSA_GUARDED_BY(mutex);
std::vector<DataRestoreTask> data_restore_tasks TSA_GUARDED_BY(mutex);
std::unique_ptr<AccessRestorerFromBackup> access_restorer TSA_GUARDED_BY(mutex);
bool access_restored TSA_GUARDED_BY(mutex) = false;
std::vector<std::future<void>> futures TSA_GUARDED_BY(mutex);
std::atomic<bool> exception_caught = false;
ThreadPool & thread_pool;
mutable std::mutex mutex;
};
}

View File

@ -1915,7 +1915,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
for (const auto & [name, value] : set_query->query_parameters)
query_parameters.insert_or_assign(name, value);
global_context->addQueryParameters(set_query->query_parameters);
global_context->addQueryParameters(NameToNameMap{set_query->query_parameters.begin(), set_query->query_parameters.end()});
}
if (const auto * use_query = parsed_query->as<ASTUseQuery>())
{

View File

@ -3,7 +3,6 @@
#include <Common/AsyncTaskExecutor.h>
#include <Common/Epoll.h>
#include <Common/Fiber.h>
#include <Common/FiberStack.h>
#include <Common/TimerDescriptor.h>
#include <Common/PoolWithFailoverBase.h>
#include <Client/ConnectionPool.h>

View File

@ -177,6 +177,10 @@ void HedgedConnections::sendQuery(
{
Settings modified_settings = settings;
/// Queries in foreign languages are transformed to ClickHouse-SQL. Ensure the setting before sending.
modified_settings.dialect = Dialect::clickhouse;
modified_settings.dialect.changed = false;
if (disable_two_level_aggregation)
{
/// Disable two-level aggregation due to version incompatibility.

View File

@ -126,6 +126,10 @@ void MultiplexedConnections::sendQuery(
Settings modified_settings = settings;
/// Queries in foreign languages are transformed to ClickHouse-SQL. Ensure the setting before sending.
modified_settings.dialect = Dialect::clickhouse;
modified_settings.dialect.changed = false;
for (auto & replica : replica_states)
{
if (!replica.connection)
@ -320,7 +324,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac
ReplicaState & state = getReplicaForReading();
current_connection = state.connection;
if (current_connection == nullptr)
throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "Logical error: no available replica");
throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "No available replica");
Packet packet;
try

View File

@ -5,7 +5,6 @@
#include <variant>
#include <Client/IConnections.h>
#include <Common/FiberStack.h>
#include <Common/Fiber.h>
#include <Common/Epoll.h>
#include <Common/TimerDescriptor.h>

View File

@ -303,7 +303,7 @@ ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_
auto & res_data = res->data;
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
res_data.reserve_exact(result_size_hint > 0 ? result_size_hint : size);
for (size_t i = 0; i < size; ++i)
if (filter[i])
@ -518,6 +518,23 @@ void ColumnAggregateFunction::insert(const Field & x)
func->deserialize(data.back(), read_buffer, version, &arena);
}
bool ColumnAggregateFunction::tryInsert(const DB::Field & x)
{
if (x.getType() != Field::Types::AggregateFunctionState)
return false;
const auto & field_name = x.get<const AggregateFunctionStateData &>().name;
if (type_string != field_name)
return false;
ensureOwnership();
Arena & arena = createOrGetArena();
pushBackAndCreateState(data, arena, func.get());
ReadBufferFromString read_buffer(x.get<const AggregateFunctionStateData &>().data);
func->deserialize(data.back(), read_buffer, version, &arena);
return true;
}
void ColumnAggregateFunction::insertDefault()
{
ensureOwnership();
@ -584,7 +601,7 @@ ColumnPtr ColumnAggregateFunction::replicate(const IColumn::Offsets & offsets) c
auto res = createView();
auto & res_data = res->data;
res_data.reserve(offsets.back());
res_data.reserve_exact(offsets.back());
IColumn::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)

View File

@ -160,6 +160,8 @@ public:
void insert(const Field & x) override;
bool tryInsert(const Field & x) override;
void insertDefault() override;
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const UInt8 *) const override;

View File

@ -305,6 +305,25 @@ void ColumnArray::insert(const Field & x)
getOffsets().push_back(getOffsets().back() + size);
}
bool ColumnArray::tryInsert(const Field & x)
{
if (x.getType() != Field::Types::Which::Array)
return false;
const Array & array = x.get<const Array &>();
size_t size = array.size();
for (size_t i = 0; i < size; ++i)
{
if (!getData().tryInsert(array[i]))
{
getData().popBack(i);
return false;
}
}
getOffsets().push_back(getOffsets().back() + size);
return true;
}
void ColumnArray::insertFrom(const IColumn & src_, size_t n)
{
@ -423,7 +442,7 @@ struct ColumnArray::ComparatorCollationBase
void ColumnArray::reserve(size_t n)
{
getOffsets().reserve(n);
getOffsets().reserve_exact(n);
getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1.
}
@ -684,9 +703,9 @@ ColumnPtr ColumnArray::filterString(const Filter & filt, ssize_t result_size_hin
if (result_size_hint < 0) /// Other cases are not considered.
{
res_chars.reserve(src_chars.size());
res_string_offsets.reserve(src_string_offsets.size());
res_offsets.reserve(col_size);
res_chars.reserve_exact(src_chars.size());
res_string_offsets.reserve_exact(src_string_offsets.size());
res_offsets.reserve_exact(col_size);
}
Offset prev_src_offset = 0;
@ -760,7 +779,7 @@ ColumnPtr ColumnArray::filterGeneric(const Filter & filt, ssize_t result_size_hi
Offsets & res_offsets = res->getOffsets();
if (result_size_hint)
res_offsets.reserve(result_size_hint > 0 ? result_size_hint : size);
res_offsets.reserve_exact(result_size_hint > 0 ? result_size_hint : size);
size_t current_offset = 0;
for (size_t i = 0; i < size; ++i)
@ -810,7 +829,7 @@ ColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_size_hint
size_t tuple_size = tuple.tupleSize();
if (tuple_size == 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty tuple");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple");
Columns temporary_arrays(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)
@ -847,7 +866,7 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray<T> & indexes, size_t limit
/// Convert indexes to UInt64 in case of overflow.
auto nested_indexes_column = ColumnUInt64::create();
PaddedPODArray<UInt64> & nested_indexes = nested_indexes_column->getData();
nested_indexes.reserve(getOffsets().back());
nested_indexes.reserve_exact(getOffsets().back());
auto res = ColumnArray::create(data->cloneEmpty());
@ -1060,8 +1079,8 @@ ColumnPtr ColumnArray::replicateNumber(const Offsets & replicate_offsets) const
typename ColVecType::Container & res_data = typeid_cast<ColVecType &>(res_arr.getData()).getData();
Offsets & res_offsets = res_arr.getOffsets();
res_data.reserve(data->size() / col_size * replicate_offsets.back());
res_offsets.reserve(replicate_offsets.back());
res_data.reserve_exact(data->size() / col_size * replicate_offsets.back());
res_offsets.reserve_exact(replicate_offsets.back());
Offset prev_replicate_offset = 0;
Offset prev_data_offset = 0;
@ -1114,9 +1133,9 @@ ColumnPtr ColumnArray::replicateString(const Offsets & replicate_offsets) const
Offsets & res_string_offsets = typeid_cast<ColumnString &>(res_arr.getData()).getOffsets();
Offsets & res_offsets = res_arr.getOffsets();
res_chars.reserve(src_chars.size() / col_size * replicate_offsets.back());
res_string_offsets.reserve(src_string_offsets.size() / col_size * replicate_offsets.back());
res_offsets.reserve(replicate_offsets.back());
res_chars.reserve_exact(src_chars.size() / col_size * replicate_offsets.back());
res_string_offsets.reserve_exact(src_string_offsets.size() / col_size * replicate_offsets.back());
res_offsets.reserve_exact(replicate_offsets.back());
Offset prev_replicate_offset = 0;
@ -1183,7 +1202,7 @@ ColumnPtr ColumnArray::replicateConst(const Offsets & replicate_offsets) const
auto res_column_offsets = ColumnOffsets::create();
Offsets & res_offsets = res_column_offsets->getData();
res_offsets.reserve(replicate_offsets.back());
res_offsets.reserve_exact(replicate_offsets.back());
Offset prev_replicate_offset = 0;
Offset prev_data_offset = 0;
@ -1263,7 +1282,7 @@ ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const
size_t tuple_size = tuple.tupleSize();
if (tuple_size == 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty tuple");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple");
Columns temporary_arrays(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)

View File

@ -85,6 +85,7 @@ public:
void updateHashFast(SipHash & hash) const override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
void insert(const Field & x) override;
bool tryInsert(const Field & x) override;
void insertFrom(const IColumn & src_, size_t n) override;
void insertDefault() override;
void popBack(size_t n) override;

View File

@ -84,6 +84,7 @@ public:
StringRef getDataAt(size_t) const override { throwMustBeDecompressed(); }
bool isDefaultAt(size_t) const override { throwMustBeDecompressed(); }
void insert(const Field &) override { throwMustBeDecompressed(); }
bool tryInsert(const Field &) override { throwMustBeDecompressed(); }
void insertRangeFrom(const IColumn &, size_t, size_t) override { throwMustBeDecompressed(); }
void insertData(const char *, size_t) override { throwMustBeDecompressed(); }
void insertDefault() override { throwMustBeDecompressed(); }

View File

@ -131,6 +131,15 @@ public:
++s;
}
bool tryInsert(const Field & field) override
{
auto tmp = data->cloneEmpty();
if (!tmp->tryInsert(field))
return false;
++s;
return true;
}
void insertData(const char *, size_t) override
{
++s;
@ -263,7 +272,7 @@ public:
if (!data->isDefaultAt(0))
{
size_t to = limit && from + limit < size() ? from + limit : size();
indices.reserve(indices.size() + to - from);
indices.reserve_exact(indices.size() + to - from);
for (size_t i = from; i < to; ++i)
indices.push_back(i);
}

View File

@ -334,6 +334,16 @@ MutableColumnPtr ColumnDecimal<T>::cloneResized(size_t size) const
return res;
}
template <is_decimal T>
bool ColumnDecimal<T>::tryInsert(const Field & x)
{
DecimalField<T> value;
if (!x.tryGet<DecimalField<T>>(value))
return false;
data.push_back(value);
return true;
}
template <is_decimal T>
void ColumnDecimal<T>::insertData(const char * src, size_t /*length*/)
{
@ -369,7 +379,7 @@ ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter & filt, ssize_t result_
Container & res_data = res->getData();
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
res_data.reserve_exact(result_size_hint > 0 ? result_size_hint : size);
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + size;
@ -445,7 +455,7 @@ ColumnPtr ColumnDecimal<T>::replicate(const IColumn::Offsets & offsets) const
return res;
typename Self::Container & res_data = res->getData();
res_data.reserve(offsets.back());
res_data.reserve_exact(offsets.back());
IColumn::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)

View File

@ -54,7 +54,7 @@ public:
size_t byteSizeAt(size_t) const override { return sizeof(data[0]); }
size_t allocatedBytes() const override { return data.allocated_bytes(); }
void protect() override { data.protect(); }
void reserve(size_t n) override { data.reserve(n); }
void reserve(size_t n) override { data.reserve_exact(n); }
void shrinkToFit() override { data.shrink_to_fit(); }
void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast<const Self &>(src).getData()[n]); }
@ -62,6 +62,7 @@ public:
void insertDefault() override { data.push_back(T()); }
void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); }
void insert(const Field & x) override { data.push_back(x.get<T>()); }
bool tryInsert(const Field & x) override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
void popBack(size_t n) override

View File

@ -63,6 +63,17 @@ void ColumnFixedString::insert(const Field & x)
insertData(s.data(), s.size());
}
bool ColumnFixedString::tryInsert(const Field & x)
{
if (x.getType() != Field::Types::Which::String)
return false;
const String & s = x.get<const String &>();
if (s.size() > n)
return false;
insertData(s.data(), s.size());
return true;
}
void ColumnFixedString::insertFrom(const IColumn & src_, size_t index)
{
const ColumnFixedString & src = assert_cast<const ColumnFixedString &>(src_);
@ -224,7 +235,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result
auto res = ColumnFixedString::create(n);
if (result_size_hint)
res->chars.reserve(result_size_hint > 0 ? result_size_hint * n : chars.size());
res->chars.reserve_exact(result_size_hint > 0 ? result_size_hint * n : chars.size());
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + col_size;

View File

@ -96,6 +96,8 @@ public:
void insert(const Field & x) override;
bool tryInsert(const Field & x) override;
void insertFrom(const IColumn & src_, size_t index) override;
void insertData(const char * pos, size_t length) override;
@ -184,7 +186,7 @@ public:
void reserve(size_t size) override
{
chars.reserve(n * size);
chars.reserve_exact(n * size);
}
void shrinkToFit() override

View File

@ -84,6 +84,11 @@ public:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert into {}", getName());
}
bool tryInsert(const Field &) override
{
return false;
}
void insertDefault() override
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert into {}", getName());

View File

@ -140,6 +140,18 @@ void ColumnLowCardinality::insert(const Field & x)
idx.insertPosition(dictionary.getColumnUnique().uniqueInsert(x));
}
bool ColumnLowCardinality::tryInsert(const Field & x)
{
compactIfSharedDictionary();
size_t index;
if (!dictionary.getColumnUnique().tryUniqueInsert(x, index))
return false;
idx.insertPosition(index);
return true;
}
void ColumnLowCardinality::insertDefault()
{
idx.insertPosition(getDictionary().getDefaultValueIndex());

View File

@ -74,6 +74,7 @@ public:
}
void insert(const Field & x) override;
bool tryInsert(const Field & x) override;
void insertDefault() override;
void insertFrom(const IColumn & src, size_t n) override;

View File

@ -102,6 +102,15 @@ void ColumnMap::insert(const Field & x)
nested->insert(Array(map.begin(), map.end()));
}
bool ColumnMap::tryInsert(const Field & x)
{
if (x.getType() != Field::Types::Which::Map)
return false;
const auto & map = x.get<const Map &>();
return nested->tryInsert(Array(map.begin(), map.end()));
}
void ColumnMap::insertDefault()
{
nested->insertDefault();

View File

@ -56,6 +56,7 @@ public:
StringRef getDataAt(size_t n) const override;
void insertData(const char * pos, size_t length) override;
void insert(const Field & x) override;
bool tryInsert(const Field & x) override;
void insertDefault() override;
void popBack(size_t n) override;
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const UInt8 *) const override;

View File

@ -1,7 +1,5 @@
#include <Common/Arena.h>
#include <Common/SipHash.h>
#include <Common/NaNUtils.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Columns/ColumnDecimal.h>
@ -28,7 +26,6 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_COLUMN;
extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT;
extern const int NOT_IMPLEMENTED;
}
@ -259,6 +256,22 @@ void ColumnNullable::insert(const Field & x)
}
}
bool ColumnNullable::tryInsert(const Field & x)
{
if (x.isNull())
{
getNestedColumn().insertDefault();
getNullMapData().push_back(1);
return true;
}
if (!getNestedColumn().tryInsert(x))
return false;
getNullMapData().push_back(0);
return true;
}
void ColumnNullable::insertFrom(const IColumn & src, size_t n)
{
const ColumnNullable & src_concrete = assert_cast<const ColumnNullable &>(src);
@ -829,8 +842,7 @@ void ColumnNullable::applyNullMap(const ColumnNullable & other)
void ColumnNullable::checkConsistency() const
{
if (null_map->size() != getNestedColumn().size())
throw Exception(ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT,
"Logical error: Sizes of nested column and null map of Nullable column are not equal");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Sizes of nested column and null map of Nullable column are not equal");
}
ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const

View File

@ -68,6 +68,7 @@ public:
const char * skipSerializedInArena(const char * pos) const override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
void insert(const Field & x) override;
bool tryInsert(const Field & x) override;
void insertFrom(const IColumn & src, size_t n) override;
void insertFromNotNullable(const IColumn & src, size_t n);

Some files were not shown because too many files have changed in this diff Show More