Merge branch 'master' into sqltest

This commit is contained in:
Alexey Milovidov 2023-08-03 15:20:22 +02:00
commit 2c085955d1
66 changed files with 706 additions and 535 deletions

View File

@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \
esac
ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release"
ARG VERSION="23.7.1.2470"
ARG VERSION="23.7.2.25"
ARG PACKAGES="clickhouse-keeper"
# user/group precreated explicitly with fixed uid/gid on purpose.

View File

@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="23.7.1.2470"
ARG VERSION="23.7.2.25"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# user/group precreated explicitly with fixed uid/gid on purpose.

View File

@ -23,7 +23,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
ARG VERSION="23.7.1.2470"
ARG VERSION="23.7.2.25"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# set non-empty deb_location_url url to create a docker image

View File

@ -0,0 +1,31 @@
---
sidebar_position: 1
sidebar_label: 2023
---
# 2023 Changelog
### ClickHouse release v23.7.2.25-stable (8dd1107b032) FIXME as compared to v23.7.1.2470-stable (a70127baecc)
#### Backward Incompatible Change
* Backported in [#52850](https://github.com/ClickHouse/ClickHouse/issues/52850): If a dynamic disk contains a name, it should be specified as `disk = disk(name = 'disk_name'`, ...) in disk function arguments. In previous version it could be specified as `disk = disk_<disk_name>(...)`, which is no longer supported. [#52820](https://github.com/ClickHouse/ClickHouse/pull/52820) ([Kseniia Sumarokova](https://github.com/kssenii)).
#### Build/Testing/Packaging Improvement
* Backported in [#52913](https://github.com/ClickHouse/ClickHouse/issues/52913): Add `clickhouse-keeper-client` symlink to the clickhouse-server package. [#51882](https://github.com/ClickHouse/ClickHouse/pull/51882) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Fix binary arithmetic for Nullable(IPv4) [#51642](https://github.com/ClickHouse/ClickHouse/pull/51642) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Support IPv4 and IPv6 as dictionary attributes [#51756](https://github.com/ClickHouse/ClickHouse/pull/51756) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* init and destroy ares channel on demand.. [#52634](https://github.com/ClickHouse/ClickHouse/pull/52634) ([Arthur Passos](https://github.com/arthurpassos)).
* Fix crash in function `tuple` with one sparse column argument [#52659](https://github.com/ClickHouse/ClickHouse/pull/52659) ([Anton Popov](https://github.com/CurtizJ)).
* Fix data race in Keeper reconfiguration [#52804](https://github.com/ClickHouse/ClickHouse/pull/52804) ([Antonio Andelic](https://github.com/antonio2368)).
* clickhouse-keeper: fix implementation of server with poll() [#52833](https://github.com/ClickHouse/ClickHouse/pull/52833) ([Andy Fiddaman](https://github.com/citrus-it)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Rename setting disable_url_encoding to enable_url_encoding and add a test [#52656](https://github.com/ClickHouse/ClickHouse/pull/52656) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix bugs and better test for SYSTEM STOP LISTEN [#52680](https://github.com/ClickHouse/ClickHouse/pull/52680) ([Nikolay Degterinsky](https://github.com/evillique)).
* Increase min protocol version for sparse serialization [#52835](https://github.com/ClickHouse/ClickHouse/pull/52835) ([Anton Popov](https://github.com/CurtizJ)).
* Docker improvements [#52869](https://github.com/ClickHouse/ClickHouse/pull/52869) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).

View File

@ -51,7 +51,3 @@ keeper foo bar
- `rmr <path>` -- Recursively deletes path. Confirmation required
- `flwc <command>` -- Executes four-letter-word command
- `help` -- Prints this message
- `get_stat [path]` -- Returns the node's stat (default `.`)
- `find_super_nodes <threshold> [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`)
- `delete_stable_backups` -- Deletes ClickHouse nodes used for backups that are now inactive
- `find_big_family [path] [n]` -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10)

View File

@ -314,6 +314,22 @@ Provides possibility to start background fetch tasks from replication queues whi
SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### STOP PULLING REPLICATION LOG
Stops loading new entries from replication log to replication queue in a `ReplicatedMergeTree` table.
``` sql
SYSTEM STOP PULLING REPLICATION LOG [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### START PULLING REPLICATION LOG
Cancels `SYSTEM STOP PULLING REPLICATION LOG`.
``` sql
SYSTEM START PULLING REPLICATION LOG [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
### SYNC REPLICA
Wait until a `ReplicatedMergeTree` table will be synced with other replicas in a cluster, but no more than `receive_timeout` seconds.

View File

@ -0,0 +1 @@
../../../en/operations/optimizing-performance/profile-guided-optimization.md

View File

@ -0,0 +1 @@
../../../en/operations/optimizing-performance/profile-guided-optimization.md

View File

@ -1,6 +1,5 @@
#include "Commands.h"
#include <queue>
#include "KeeperClient.h"
@ -25,18 +24,8 @@ void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con
else
path = client->cwd;
auto children = client->zookeeper->getChildren(path);
std::sort(children.begin(), children.end());
bool need_space = false;
for (const auto & child : children)
{
if (std::exchange(need_space, true))
std::cout << " ";
std::cout << child;
}
for (const auto & child : client->zookeeper->getChildren(path))
std::cout << child << " ";
std::cout << "\n";
}
@ -141,173 +130,6 @@ void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co
std::cout << client->zookeeper->get(client->getAbsolutePath(query->args[0].safeGet<String>())) << "\n";
}
bool GetStatCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
if (!parseKeeperPath(pos, expected, arg))
return true;
node->args.push_back(std::move(arg));
return true;
}
void GetStatCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
Coordination::Stat stat;
String path;
if (!query->args.empty())
path = client->getAbsolutePath(query->args[0].safeGet<String>());
else
path = client->cwd;
client->zookeeper->get(path, &stat);
std::cout << "cZxid = " << stat.czxid << "\n";
std::cout << "mZxid = " << stat.mzxid << "\n";
std::cout << "pZxid = " << stat.pzxid << "\n";
std::cout << "ctime = " << stat.ctime << "\n";
std::cout << "mtime = " << stat.mtime << "\n";
std::cout << "version = " << stat.version << "\n";
std::cout << "cversion = " << stat.cversion << "\n";
std::cout << "aversion = " << stat.aversion << "\n";
std::cout << "ephemeralOwner = " << stat.ephemeralOwner << "\n";
std::cout << "dataLength = " << stat.dataLength << "\n";
std::cout << "numChildren = " << stat.numChildren << "\n";
}
bool FindSuperNodes::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
ASTPtr threshold;
if (!ParserUnsignedInteger{}.parse(pos, threshold, expected))
return false;
node->args.push_back(threshold->as<ASTLiteral &>().value);
String path;
if (!parseKeeperPath(pos, expected, path))
path = ".";
node->args.push_back(std::move(path));
return true;
}
void FindSuperNodes::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
auto threshold = query->args[0].safeGet<UInt64>();
auto path = client->getAbsolutePath(query->args[1].safeGet<String>());
Coordination::Stat stat;
client->zookeeper->get(path, &stat);
if (stat.numChildren >= static_cast<Int32>(threshold))
{
std::cout << static_cast<String>(path) << "\t" << stat.numChildren << "\n";
return;
}
auto children = client->zookeeper->getChildren(path);
std::sort(children.begin(), children.end());
for (const auto & child : children)
{
auto next_query = *query;
next_query.args[1] = DB::Field(path / child);
execute(&next_query, client);
}
}
bool DeleteStableBackups::parse(IParser::Pos & /* pos */, std::shared_ptr<ASTKeeperQuery> & /* node */, Expected & /* expected */) const
{
return true;
}
void DeleteStableBackups::execute(const ASTKeeperQuery * /* query */, KeeperClient * client) const
{
client->askConfirmation(
"You are going to delete all inactive backups in /clickhouse/backups.",
[client]
{
fs::path backup_root = "/clickhouse/backups";
auto backups = client->zookeeper->getChildren(backup_root);
std::sort(backups.begin(), backups.end());
for (const auto & child : backups)
{
auto backup_path = backup_root / child;
std::cout << "Found backup " << backup_path << ", checking if it's active\n";
String stage_path = backup_path / "stage";
auto stages = client->zookeeper->getChildren(stage_path);
bool is_active = false;
for (const auto & stage : stages)
{
if (startsWith(stage, "alive"))
{
is_active = true;
break;
}
}
if (is_active)
{
std::cout << "Backup " << backup_path << " is active, not going to delete\n";
continue;
}
std::cout << "Backup " << backup_path << " is not active, deleting it\n";
client->zookeeper->removeRecursive(backup_path);
}
});
}
bool FindBigFamily::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String path;
if (!parseKeeperPath(pos, expected, path))
path = ".";
node->args.push_back(std::move(path));
ASTPtr count;
if (ParserUnsignedInteger{}.parse(pos, count, expected))
node->args.push_back(count->as<ASTLiteral &>().value);
else
node->args.push_back(UInt64(10));
return true;
}
void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
auto path = client->getAbsolutePath(query->args[0].safeGet<String>());
auto n = query->args[1].safeGet<UInt64>();
std::vector<std::tuple<Int32, String>> result;
std::queue<fs::path> queue;
queue.push(path);
while (!queue.empty())
{
auto next_path = queue.front();
queue.pop();
auto children = client->zookeeper->getChildren(next_path);
std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; });
auto response = client->zookeeper->get(children);
for (size_t i = 0; i < response.size(); ++i)
{
result.emplace_back(response[i].stat.numChildren, children[i]);
queue.push(children[i]);
}
}
std::sort(result.begin(), result.end(), std::greater());
for (UInt64 i = 0; i < std::min(result.size(), static_cast<size_t>(n)); ++i)
std::cout << std::get<1>(result[i]) << "\t" << std::get<0>(result[i]) << "\n";
}
bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String arg;
@ -348,7 +170,7 @@ bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptr<ASTKeeperQuery
void HelpCommand::execute(const ASTKeeperQuery * /* query */, KeeperClient * /* client */) const
{
for (const auto & pair : KeeperClient::commands)
std::cout << pair.second->generateHelpString() << "\n";
std::cout << pair.second->getHelpMessage() << "\n";
}
bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const

View File

@ -21,12 +21,6 @@ public:
virtual String getName() const = 0;
virtual ~IKeeperClientCommand() = default;
String generateHelpString() const
{
return fmt::vformat(getHelpMessage(), fmt::make_format_args(getName()));
}
};
using Command = std::shared_ptr<IKeeperClientCommand>;
@ -40,7 +34,7 @@ class LSCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} [path] -- Lists the nodes for the given path (default: cwd)"; }
String getHelpMessage() const override { return "ls [path] -- Lists the nodes for the given path (default: cwd)"; }
};
class CDCommand : public IKeeperClientCommand
@ -51,7 +45,7 @@ class CDCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} [path] -- Change the working path (default `.`)"; }
String getHelpMessage() const override { return "cd [path] -- Change the working path (default `.`)"; }
};
class SetCommand : public IKeeperClientCommand
@ -64,7 +58,7 @@ class SetCommand : public IKeeperClientCommand
String getHelpMessage() const override
{
return "{} <path> <value> [version] -- Updates the node's value. Only update if version matches (default: -1)";
return "set <path> <value> [version] -- Updates the node's value. Only update if version matches (default: -1)";
}
};
@ -76,7 +70,7 @@ class CreateCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} <path> <value> -- Creates new node"; }
String getHelpMessage() const override { return "create <path> <value> -- Creates new node"; }
};
class GetCommand : public IKeeperClientCommand
@ -87,63 +81,9 @@ class GetCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} <path> -- Returns the node's value"; }
String getHelpMessage() const override { return "get <path> -- Returns the node's value"; }
};
class GetStatCommand : public IKeeperClientCommand
{
String getName() const override { return "get_stat"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} [path] -- Returns the node's stat (default `.`)"; }
};
class FindSuperNodes : public IKeeperClientCommand
{
String getName() const override { return "find_super_nodes"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override
{
return "{} <threshold> [path] -- Finds nodes with number of children larger than some threshold for the given path (default `.`)";
}
};
class DeleteStableBackups : public IKeeperClientCommand
{
String getName() const override { return "delete_stable_backups"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override
{
return "{} -- Deletes ClickHouse nodes used for backups that are now inactive";
}
};
class FindBigFamily : public IKeeperClientCommand
{
String getName() const override { return "find_big_family"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override
{
return "{} [path] [n] -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10)";
}
};
class RMCommand : public IKeeperClientCommand
{
String getName() const override { return "rm"; }
@ -152,7 +92,7 @@ class RMCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} <path> -- Remove the node"; }
String getHelpMessage() const override { return "remove <path> -- Remove the node"; }
};
class RMRCommand : public IKeeperClientCommand
@ -163,7 +103,7 @@ class RMRCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} <path> -- Recursively deletes path. Confirmation required"; }
String getHelpMessage() const override { return "rmr <path> -- Recursively deletes path. Confirmation required"; }
};
class HelpCommand : public IKeeperClientCommand
@ -174,7 +114,7 @@ class HelpCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} -- Prints this message"; }
String getHelpMessage() const override { return "help -- Prints this message"; }
};
class FourLetterWordCommand : public IKeeperClientCommand
@ -185,7 +125,7 @@ class FourLetterWordCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} <command> -- Executes four-letter-word command"; }
String getHelpMessage() const override { return "flwc <command> -- Executes four-letter-word command"; }
};
}

View File

@ -177,10 +177,6 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */)
std::make_shared<SetCommand>(),
std::make_shared<CreateCommand>(),
std::make_shared<GetCommand>(),
std::make_shared<GetStatCommand>(),
std::make_shared<FindSuperNodes>(),
std::make_shared<DeleteStableBackups>(),
std::make_shared<FindBigFamily>(),
std::make_shared<RMCommand>(),
std::make_shared<RMRCommand>(),
std::make_shared<HelpCommand>(),

View File

@ -58,7 +58,6 @@ bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
String command_name(pos->begin, pos->end);
std::transform(command_name.begin(), command_name.end(), command_name.begin(), [](unsigned char c) { return std::tolower(c); });
Command command;
auto iter = KeeperClient::commands.find(command_name);

View File

@ -168,6 +168,7 @@ enum class AccessType
M(SYSTEM_TTL_MERGES, "SYSTEM STOP TTL MERGES, SYSTEM START TTL MERGES, STOP TTL MERGES, START TTL MERGES", TABLE, SYSTEM) \
M(SYSTEM_FETCHES, "SYSTEM STOP FETCHES, SYSTEM START FETCHES, STOP FETCHES, START FETCHES", TABLE, SYSTEM) \
M(SYSTEM_MOVES, "SYSTEM STOP MOVES, SYSTEM START MOVES, STOP MOVES, START MOVES", TABLE, SYSTEM) \
M(SYSTEM_PULLING_REPLICATION_LOG, "SYSTEM STOP PULLING REPLICATION LOG, SYSTEM START PULLING REPLICATION LOG", TABLE, SYSTEM) \
M(SYSTEM_DISTRIBUTED_SENDS, "SYSTEM STOP DISTRIBUTED SENDS, SYSTEM START DISTRIBUTED SENDS, STOP DISTRIBUTED SENDS, START DISTRIBUTED SENDS", TABLE, SYSTEM_SENDS) \
M(SYSTEM_REPLICATED_SENDS, "SYSTEM STOP REPLICATED SENDS, SYSTEM START REPLICATED SENDS, STOP REPLICATED SENDS, START REPLICATED SENDS", TABLE, SYSTEM_SENDS) \
M(SYSTEM_SENDS, "SYSTEM STOP SENDS, SYSTEM START SENDS, STOP SENDS, START SENDS", GROUP, SYSTEM) \

View File

@ -51,7 +51,7 @@ TEST(AccessRights, Union)
"CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, "
"TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, "
"SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, "
"SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, "
"SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, 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");
}

View File

@ -74,19 +74,22 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile(
}
void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog(
const FileSegment::Range & file_segment_range, CachedOnDiskReadBufferFromFile::ReadType type)
const FileSegment & file_segment, CachedOnDiskReadBufferFromFile::ReadType type)
{
if (!cache_log)
return;
const auto range = file_segment.range();
FilesystemCacheLogElement elem
{
.event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()),
.query_id = query_id,
.source_file_path = source_file_path,
.file_segment_range = { file_segment_range.left, file_segment_range.right },
.file_segment_range = { range.left, range.right },
.requested_range = { first_offset, read_until_position },
.file_segment_size = file_segment_range.size(),
.file_segment_key = file_segment.key().toString(),
.file_segment_offset = file_segment.offset(),
.file_segment_size = range.size(),
.read_from_cache_attempted = true,
.read_buffer_id = current_buffer_id,
.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(
@ -495,7 +498,7 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext()
auto completed_range = current_file_segment->range();
if (cache_log)
appendFilesystemCacheLog(completed_range, read_type);
appendFilesystemCacheLog(*current_file_segment, read_type);
chassert(file_offset_of_buffer_end > completed_range.right);
@ -518,7 +521,7 @@ CachedOnDiskReadBufferFromFile::~CachedOnDiskReadBufferFromFile()
{
if (cache_log && file_segments && !file_segments->empty())
{
appendFilesystemCacheLog(file_segments->front().range(), read_type);
appendFilesystemCacheLog(file_segments->front(), read_type);
}
}

View File

@ -90,7 +90,7 @@ private:
bool completeFileSegmentAndGetNext();
void appendFilesystemCacheLog(const FileSegment::Range & file_segment_range, ReadType read_type);
void appendFilesystemCacheLog(const FileSegment & file_segment, ReadType read_type);
bool writeCache(char * data, size_t size, size_t offset, FileSegment & file_segment);

View File

@ -109,6 +109,8 @@ void ReadBufferFromRemoteFSGather::appendUncachedReadInfo()
.source_file_path = current_object.remote_path,
.file_segment_range = { 0, current_object.bytes_size },
.cache_type = FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE,
.file_segment_key = {},
.file_segment_offset = {},
.file_segment_size = current_object.bytes_size,
.read_from_cache_attempted = false,
};

View File

@ -510,11 +510,12 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable
map.clear();
bool all_has_nullable = all_nullable;
bool current_has_nullable = false;
for (size_t arg_num = 0; arg_num < args; ++arg_num)
{
const auto & arg = arrays.args[arg_num];
bool current_has_nullable = false;
current_has_nullable = false;
size_t off;
// const array has only one row
@ -549,44 +550,93 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable
}
}
prev_off[arg_num] = off;
if (arg.is_const)
prev_off[arg_num] = 0;
// We update offsets for all the arrays except the first one. Offsets for the first array would be updated later.
// It is needed to iterate the first array again so that the elements in the result would have fixed order.
if (arg_num)
{
prev_off[arg_num] = off;
if (arg.is_const)
prev_off[arg_num] = 0;
}
if (!current_has_nullable)
all_has_nullable = false;
}
if (all_has_nullable)
{
++result_offset;
result_data.insertDefault();
null_map.push_back(1);
}
// We have NULL in output only once if it should be there
bool null_added = false;
const auto & arg = arrays.args[0];
size_t off;
// const array has only one row
if (arg.is_const)
off = (*arg.offsets)[0];
else
off = (*arg.offsets)[row];
for (const auto & pair : map)
for (auto i : collections::range(prev_off[0], off))
{
if (pair.getMapped() == args)
all_has_nullable = all_nullable;
typename Map::LookupResult pair = nullptr;
if (arg.null_map && (*arg.null_map)[i])
{
current_has_nullable = true;
if (all_has_nullable && !null_added)
{
++result_offset;
result_data.insertDefault();
null_map.push_back(1);
null_added = true;
}
if (null_added)
continue;
}
else if constexpr (is_numeric_column)
{
pair = map.find(columns[0]->getElement(i));
}
else if constexpr (std::is_same_v<ColumnType, ColumnString> || std::is_same_v<ColumnType, ColumnFixedString>)
pair = map.find(columns[0]->getDataAt(i));
else
{
const char * data = nullptr;
pair = map.find(columns[0]->serializeValueIntoArena(i, arena, data));
}
prev_off[0] = off;
if (arg.is_const)
prev_off[0] = 0;
if (!current_has_nullable)
all_has_nullable = false;
if (pair && pair->getMapped() == args)
{
// We increase pair->getMapped() here to not skip duplicate values from the first array.
++pair->getMapped();
++result_offset;
if constexpr (is_numeric_column)
result_data.insertValue(pair.getKey());
{
result_data.insertValue(pair->getKey());
}
else if constexpr (std::is_same_v<ColumnType, ColumnString> || std::is_same_v<ColumnType, ColumnFixedString>)
result_data.insertData(pair.getKey().data, pair.getKey().size);
{
result_data.insertData(pair->getKey().data, pair->getKey().size);
}
else
result_data.deserializeAndInsertFromArena(pair.getKey().data);
{
result_data.deserializeAndInsertFromArena(pair->getKey().data);
}
if (all_nullable)
null_map.push_back(0);
}
}
result_offsets.getElement(row) = result_offset;
}
}
ColumnPtr result_column = std::move(result_data_ptr);
if (all_nullable)
result_column = ColumnNullable::create(result_column, std::move(null_map_column));
return ColumnArray::create(result_column, std::move(result_offsets_ptr));
}

View File

@ -16,6 +16,7 @@ namespace ActionLocks
extern const StorageActionBlockType DistributedSend = 5;
extern const StorageActionBlockType PartsTTLMerge = 6;
extern const StorageActionBlockType PartsMove = 7;
extern const StorageActionBlockType PullReplicationLog = 8;
}

View File

@ -806,6 +806,13 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size)
return true;
}
void FileCache::removeKey(const Key & key)
{
assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW);
locked_key->removeAll();
}
void FileCache::removeKeyIfExists(const Key & key)
{
assertInitialized();
@ -818,7 +825,14 @@ void FileCache::removeKeyIfExists(const Key & key)
/// But if we have multiple replicated zero-copy tables on the same server
/// it became possible to start removing something from cache when it is used
/// by other "zero-copy" tables. That is why it's not an error.
locked_key->removeAllReleasable();
locked_key->removeAll(/* if_releasable */true);
}
void FileCache::removeFileSegment(const Key & key, size_t offset)
{
assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW);
locked_key->removeFileSegment(offset);
}
void FileCache::removePathIfExists(const String & path)
@ -830,22 +844,12 @@ void FileCache::removeAllReleasable()
{
assertInitialized();
auto lock = lockCache();
main_priority->iterate([&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata)
{
if (segment_metadata->releasable())
{
auto file_segment = segment_metadata->file_segment;
locked_key.removeFileSegment(file_segment->offset(), file_segment->lock());
return PriorityIterationResult::REMOVE_AND_CONTINUE;
}
return PriorityIterationResult::CONTINUE;
}, lock);
metadata.iterate([](LockedKey & locked_key) { locked_key.removeAll(/* if_releasable */true); });
if (stash)
{
/// Remove all access information.
auto lock = lockCache();
stash->records.clear();
stash->queue->removeAll(lock);
}
@ -914,7 +918,7 @@ void FileCache::loadMetadata()
continue;
}
const auto key = Key(unhexUInt<UInt128>(key_directory.filename().string().data()));
const auto key = Key::fromKeyString(key_directory.filename().string());
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, /* is_initial_load */true);
for (fs::directory_iterator offset_it{key_directory}; offset_it != fs::directory_iterator(); ++offset_it)
@ -1069,7 +1073,7 @@ FileSegmentsHolderPtr FileCache::getSnapshot()
FileSegmentsHolderPtr FileCache::getSnapshot(const Key & key)
{
FileSegments file_segments;
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW);
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW_LOGICAL);
for (const auto & [_, file_segment_metadata] : *locked_key->getKeyMetadata())
file_segments.push_back(FileSegment::getSnapshot(file_segment_metadata->file_segment));
return std::make_unique<FileSegmentsHolder>(std::move(file_segments));

View File

@ -83,13 +83,19 @@ public:
FileSegmentsHolderPtr set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
/// Remove files by `key`. Removes files which might be used at the moment.
/// Remove file segment by `key` and `offset`. Throws if file segment does not exist.
void removeFileSegment(const Key & key, size_t offset);
/// Remove files by `key`. Throws if key does not exist.
void removeKey(const Key & key);
/// Remove files by `key`.
void removeKeyIfExists(const Key & key);
/// Removes files by `path`. Removes files which might be used at the moment.
/// Removes files by `path`.
void removePathIfExists(const String & path);
/// Remove files by `key`. Will not remove files which are used at the moment.
/// Remove files by `key`.
void removeAllReleasable();
std::vector<String> tryGetCachePaths(const Key & key);

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
FileCacheKey::FileCacheKey(const std::string & path)
: key(sipHash128(path.data(), path.size()))
@ -28,4 +32,11 @@ FileCacheKey FileCacheKey::random()
return FileCacheKey(UUIDHelpers::generateV4().toUnderType());
}
FileCacheKey FileCacheKey::fromKeyString(const std::string & key_str)
{
if (key_str.size() != 32)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid cache key hex: {}", key_str);
return FileCacheKey(unhexUInt<UInt128>(key_str.data()));
}
}

View File

@ -21,6 +21,8 @@ struct FileCacheKey
static FileCacheKey random();
bool operator==(const FileCacheKey & other) const { return key == other.key; }
static FileCacheKey fromKeyString(const std::string & key_str);
};
using FileCacheKeyAndOffset = std::pair<FileCacheKey, size_t>;

View File

@ -25,6 +25,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
FileSegmentMetadata::FileSegmentMetadata(FileSegmentPtr && file_segment_)
@ -191,6 +192,8 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata(
if (it == end())
{
if (key_not_found_policy == KeyNotFoundPolicy::THROW)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}` in cache", key);
else if (key_not_found_policy == KeyNotFoundPolicy::THROW_LOGICAL)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}` in cache", key);
else if (key_not_found_policy == KeyNotFoundPolicy::RETURN_NULL)
return nullptr;
@ -215,6 +218,8 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata(
return locked_metadata;
if (key_not_found_policy == KeyNotFoundPolicy::THROW)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}` in cache", key);
else if (key_not_found_policy == KeyNotFoundPolicy::THROW_LOGICAL)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}` in cache", key);
if (key_not_found_policy == KeyNotFoundPolicy::RETURN_NULL)
@ -561,11 +566,11 @@ bool LockedKey::isLastOwnerOfFileSegment(size_t offset) const
return file_segment_metadata->file_segment.use_count() == 2;
}
void LockedKey::removeAllReleasable()
void LockedKey::removeAll(bool if_releasable)
{
for (auto it = key_metadata->begin(); it != key_metadata->end();)
{
if (!it->second->releasable())
if (if_releasable && !it->second->releasable())
{
++it;
continue;
@ -586,17 +591,32 @@ void LockedKey::removeAllReleasable()
}
}
KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset)
{
auto it = key_metadata->find(offset);
if (it == key_metadata->end())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no offset {}", offset);
auto file_segment = it->second->file_segment;
return removeFileSegmentImpl(it, file_segment->lock());
}
KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegmentGuard::Lock & segment_lock)
{
auto it = key_metadata->find(offset);
if (it == key_metadata->end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no offset {}", offset);
return removeFileSegmentImpl(it, segment_lock);
}
KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it, const FileSegmentGuard::Lock & segment_lock)
{
auto file_segment = it->second->file_segment;
LOG_DEBUG(
key_metadata->log, "Remove from cache. Key: {}, offset: {}, size: {}",
getKey(), offset, file_segment->reserved_size);
getKey(), file_segment->offset(), file_segment->reserved_size);
chassert(file_segment->assertCorrectnessUnlocked(segment_lock));

View File

@ -87,7 +87,7 @@ struct CacheMetadata : public std::unordered_map<FileCacheKey, KeyMetadataPtr>,
{
public:
using Key = FileCacheKey;
using IterateCacheMetadataFunc = std::function<void(const LockedKey &)>;
using IterateCacheMetadataFunc = std::function<void(LockedKey &)>;
explicit CacheMetadata(const std::string & path_);
@ -106,6 +106,7 @@ public:
enum class KeyNotFoundPolicy
{
THROW,
THROW_LOGICAL,
CREATE_EMPTY,
RETURN_NULL,
};
@ -169,9 +170,10 @@ struct LockedKey : private boost::noncopyable
std::shared_ptr<const KeyMetadata> getKeyMetadata() const { return key_metadata; }
std::shared_ptr<KeyMetadata> getKeyMetadata() { return key_metadata; }
void removeAllReleasable();
void removeAll(bool if_releasable = true);
KeyMetadata::iterator removeFileSegment(size_t offset, const FileSegmentGuard::Lock &);
KeyMetadata::iterator removeFileSegment(size_t offset);
void shrinkFileSegmentToDownloadedSize(size_t offset, const FileSegmentGuard::Lock &);
@ -188,6 +190,8 @@ struct LockedKey : private boost::noncopyable
std::string toString() const;
private:
KeyMetadata::iterator removeFileSegmentImpl(KeyMetadata::iterator it, const FileSegmentGuard::Lock &);
const std::shared_ptr<KeyMetadata> key_metadata;
KeyGuard::Lock lock; /// `lock` must be destructed before `key_metadata`.
};

View File

@ -40,6 +40,8 @@ NamesAndTypesList FilesystemCacheLogElement::getNamesAndTypes()
{"source_file_path", std::make_shared<DataTypeString>()},
{"file_segment_range", std::make_shared<DataTypeTuple>(types)},
{"total_requested_range", std::make_shared<DataTypeTuple>(types)},
{"key", std::make_shared<DataTypeString>()},
{"offset", std::make_shared<DataTypeUInt64>()},
{"size", std::make_shared<DataTypeUInt64>()},
{"read_type", std::make_shared<DataTypeString>()},
{"read_from_cache_attempted", std::make_shared<DataTypeUInt8>()},
@ -60,6 +62,8 @@ void FilesystemCacheLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(source_file_path);
columns[i++]->insert(Tuple{file_segment_range.first, file_segment_range.second});
columns[i++]->insert(Tuple{requested_range.first, requested_range.second});
columns[i++]->insert(file_segment_key);
columns[i++]->insert(file_segment_offset);
columns[i++]->insert(file_segment_size);
columns[i++]->insert(typeToString(cache_type));
columns[i++]->insert(read_from_cache_attempted);

View File

@ -11,16 +11,7 @@
namespace DB
{
///
/// -------- Column --------- Type ------
/// | event_date | DateTime |
/// | event_time | UInt64 |
/// | query_id | String |
/// | remote_file_path | String |
/// | segment_range | Tuple |
/// | read_type | String |
/// -------------------------------------
///
struct FilesystemCacheLogElement
{
enum class CacheType
@ -39,6 +30,8 @@ struct FilesystemCacheLogElement
std::pair<size_t, size_t> file_segment_range{};
std::pair<size_t, size_t> requested_range{};
CacheType cache_type{};
std::string file_segment_key;
size_t file_segment_offset;
size_t file_segment_size;
bool read_from_cache_attempted;
String read_buffer_id;

View File

@ -89,13 +89,14 @@ namespace ErrorCodes
namespace ActionLocks
{
extern StorageActionBlockType PartsMerge;
extern StorageActionBlockType PartsFetch;
extern StorageActionBlockType PartsSend;
extern StorageActionBlockType ReplicationQueue;
extern StorageActionBlockType DistributedSend;
extern StorageActionBlockType PartsTTLMerge;
extern StorageActionBlockType PartsMove;
extern const StorageActionBlockType PartsMerge;
extern const StorageActionBlockType PartsFetch;
extern const StorageActionBlockType PartsSend;
extern const StorageActionBlockType ReplicationQueue;
extern const StorageActionBlockType DistributedSend;
extern const StorageActionBlockType PartsTTLMerge;
extern const StorageActionBlockType PartsMove;
extern const StorageActionBlockType PullReplicationLog;
}
@ -155,6 +156,8 @@ AccessType getRequiredAccessType(StorageActionBlockType action_type)
return AccessType::SYSTEM_TTL_MERGES;
else if (action_type == ActionLocks::PartsMove)
return AccessType::SYSTEM_MOVES;
else if (action_type == ActionLocks::PullReplicationLog)
return AccessType::SYSTEM_PULLING_REPLICATION_LOG;
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown action type: {}", std::to_string(action_type));
}
@ -371,7 +374,18 @@ BlockIO InterpreterSystemQuery::execute()
else
{
auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name).cache;
cache->removeAllReleasable();
if (query.key_to_drop.empty())
{
cache->removeAllReleasable();
}
else
{
auto key = FileCacheKey::fromKeyString(query.key_to_drop);
if (query.offset_to_drop.has_value())
cache->removeFileSegment(key, query.offset_to_drop.value());
else
cache->removeKey(key);
}
}
break;
}
@ -502,6 +516,12 @@ BlockIO InterpreterSystemQuery::execute()
case Type::START_DISTRIBUTED_SENDS:
startStopAction(ActionLocks::DistributedSend, true);
break;
case Type::STOP_PULLING_REPLICATION_LOG:
startStopAction(ActionLocks::PullReplicationLog, false);
break;
case Type::START_PULLING_REPLICATION_LOG:
startStopAction(ActionLocks::PullReplicationLog, true);
break;
case Type::DROP_REPLICA:
dropReplica(query);
break;
@ -1079,6 +1099,15 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
required_access.emplace_back(AccessType::SYSTEM_MOVES, query.getDatabase(), query.getTable());
break;
}
case Type::STOP_PULLING_REPLICATION_LOG:
case Type::START_PULLING_REPLICATION_LOG:
{
if (!query.table)
required_access.emplace_back(AccessType::SYSTEM_PULLING_REPLICATION_LOG);
else
required_access.emplace_back(AccessType::SYSTEM_PULLING_REPLICATION_LOG, query.getDatabase(), query.getTable());
break;
}
case Type::STOP_FETCHES:
case Type::START_FETCHES:
{

View File

@ -91,34 +91,30 @@ void WindowFrame::toString(WriteBuffer & buf) const
void WindowFrame::checkValid() const
{
// Check the validity of offsets.
if (type == WindowFrame::FrameType::ROWS
|| type == WindowFrame::FrameType::GROUPS)
if (begin_type == BoundaryType::Offset
&& !((begin_offset.getType() == Field::Types::UInt64
|| begin_offset.getType() == Field::Types::Int64)
&& begin_offset.get<Int64>() >= 0
&& begin_offset.get<Int64>() < INT_MAX))
{
if (begin_type == BoundaryType::Offset
&& !((begin_offset.getType() == Field::Types::UInt64
|| begin_offset.getType() == Field::Types::Int64)
&& begin_offset.get<Int64>() >= 0
&& begin_offset.get<Int64>() < INT_MAX))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given",
type,
applyVisitor(FieldVisitorToString(), begin_offset),
begin_offset.getType());
}
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given",
type,
applyVisitor(FieldVisitorToString(), begin_offset),
begin_offset.getType());
}
if (end_type == BoundaryType::Offset
&& !((end_offset.getType() == Field::Types::UInt64
|| end_offset.getType() == Field::Types::Int64)
&& end_offset.get<Int64>() >= 0
&& end_offset.get<Int64>() < INT_MAX))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given",
type,
applyVisitor(FieldVisitorToString(), end_offset),
end_offset.getType());
}
if (end_type == BoundaryType::Offset
&& !((end_offset.getType() == Field::Types::UInt64
|| end_offset.getType() == Field::Types::Int64)
&& end_offset.get<Int64>() >= 0
&& end_offset.get<Int64>() < INT_MAX))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given",
type,
applyVisitor(FieldVisitorToString(), end_offset),
end_offset.getType());
}
// Check relative positioning of offsets.

View File

@ -45,10 +45,10 @@ namespace ErrorCodes
namespace ActionLocks
{
extern StorageActionBlockType PartsMerge;
extern StorageActionBlockType PartsFetch;
extern StorageActionBlockType PartsSend;
extern StorageActionBlockType DistributedSend;
extern const StorageActionBlockType PartsMerge;
extern const StorageActionBlockType PartsFetch;
extern const StorageActionBlockType PartsSend;
extern const StorageActionBlockType DistributedSend;
}
static void executeCreateQuery(

View File

@ -162,7 +162,9 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|| type == Type::STOP_REPLICATION_QUEUES
|| type == Type::START_REPLICATION_QUEUES
|| type == Type::STOP_DISTRIBUTED_SENDS
|| type == Type::START_DISTRIBUTED_SENDS)
|| type == Type::START_DISTRIBUTED_SENDS
|| type == Type::STOP_PULLING_REPLICATION_LOG
|| type == Type::START_PULLING_REPLICATION_LOG)
{
if (table)
print_database_table();
@ -210,7 +212,15 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
else if (type == Type::DROP_FILESYSTEM_CACHE)
{
if (!filesystem_cache_name.empty())
{
settings.ostr << (settings.hilite ? hilite_none : "") << " " << filesystem_cache_name;
if (!key_to_drop.empty())
{
settings.ostr << (settings.hilite ? hilite_none : "") << " KEY " << key_to_drop;
if (offset_to_drop.has_value())
settings.ostr << (settings.hilite ? hilite_none : "") << " OFFSET " << offset_to_drop.value();
}
}
}
else if (type == Type::UNFREEZE)
{

View File

@ -80,6 +80,8 @@ public:
UNFREEZE,
ENABLE_FAILPOINT,
DISABLE_FAILPOINT,
STOP_PULLING_REPLICATION_LOG,
START_PULLING_REPLICATION_LOG,
END
};
@ -108,6 +110,8 @@ public:
UInt64 seconds{};
String filesystem_cache_name;
std::string key_to_drop;
std::optional<size_t> offset_to_drop;
String backup_name;

View File

@ -379,6 +379,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::START_REPLICATED_SENDS:
case Type::STOP_REPLICATION_QUEUES:
case Type::START_REPLICATION_QUEUES:
case Type::STOP_PULLING_REPLICATION_LOG:
case Type::START_PULLING_REPLICATION_LOG:
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
parseDatabaseAndTableAsAST(pos, expected, res->database, res->table);
@ -405,7 +407,15 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
ParserLiteral path_parser;
ASTPtr ast;
if (path_parser.parse(pos, ast, expected))
{
res->filesystem_cache_name = ast->as<ASTLiteral>()->value.safeGet<String>();
if (ParserKeyword{"KEY"}.ignore(pos, expected) && ParserIdentifier().parse(pos, ast, expected))
{
res->key_to_drop = ast->as<ASTIdentifier>()->name();
if (ParserKeyword{"OFFSET"}.ignore(pos, expected) && ParserLiteral().parse(pos, ast, expected))
res->offset_to_drop = ast->as<ASTLiteral>()->value.safeGet<UInt64>();
}
}
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
break;

View File

@ -148,7 +148,7 @@ static int compareValuesWithOffsetFloat(const IColumn * _compared_column,
const auto * reference_column = assert_cast<const ColumnType *>(
_reference_column);
const auto offset = _offset.get<typename ColumnType::ValueType>();
assert(offset >= 0);
chassert(offset >= 0);
const auto compared_value_data = compared_column->getDataAt(compared_row);
assert(compared_value_data.size == sizeof(typename ColumnType::ValueType));

View File

@ -1738,15 +1738,24 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont
{
/// Asks to complete merges and does not allow them to start.
/// This protects against "revival" of data for a removed partition after completion of merge.
auto merge_blocker = stopMergesAndWait();
waitForOutdatedPartsToBeLoaded();
auto merge_blocker = stopMergesAndWait();
Stopwatch watch;
ProfileEventsScope profile_events_scope;
auto txn = query_context->getCurrentTransaction();
MergeTreeData::Transaction transaction(*this, txn.get());
if (txn)
{
auto data_parts_lock = lockParts();
auto parts_to_remove = getVisibleDataPartsVectorUnlocked(query_context, data_parts_lock);
removePartsFromWorkingSet(txn.get(), parts_to_remove, true, data_parts_lock);
LOG_INFO(log, "Removed {} parts: [{}]", parts_to_remove.size(), fmt::join(getPartsNames(parts_to_remove), ", "));
}
else
{
MergeTreeData::Transaction transaction(*this, txn.get());
auto operation_data_parts_lock = lockOperationsWithParts();
auto parts = getVisibleDataPartsVector(query_context);
@ -1790,8 +1799,15 @@ void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPt
/// It's important to create it outside of lock scope because
/// otherwise it can lock parts in destructor and deadlock is possible.
auto txn = query_context->getCurrentTransaction();
MergeTreeData::Transaction transaction(*this, txn.get());
if (txn)
{
if (auto part = outdatePart(txn.get(), part_name, /*force=*/ true))
dropPartsImpl({part}, detach);
}
else
{
MergeTreeData::Transaction transaction(*this, txn.get());
auto operation_data_parts_lock = lockOperationsWithParts();
auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active});
@ -1848,8 +1864,26 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont
/// It's important to create it outside of lock scope because
/// otherwise it can lock parts in destructor and deadlock is possible.
auto txn = query_context->getCurrentTransaction();
MergeTreeData::Transaction transaction(*this, txn.get());
if (txn)
{
DataPartsVector parts_to_remove;
{
auto data_parts_lock = lockParts();
if (partition_ast && partition_ast->all)
parts_to_remove = getVisibleDataPartsVectorUnlocked(query_context, data_parts_lock);
else
{
String partition_id = getPartitionIDFromQuery(partition, query_context, &data_parts_lock);
parts_to_remove = getVisibleDataPartsVectorInPartition(query_context, partition_id, data_parts_lock);
}
removePartsFromWorkingSet(txn.get(), parts_to_remove, true, data_parts_lock);
}
dropPartsImpl(std::move(parts_to_remove), detach);
}
else
{
MergeTreeData::Transaction transaction(*this, txn.get());
auto operation_data_parts_lock = lockOperationsWithParts();
DataPartsVector parts;
@ -1864,12 +1898,14 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont
}
if (detach)
{
for (const auto & part : parts)
{
auto metadata_snapshot = getInMemoryMetadataPtr();
LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory());
part->makeCloneInDetached("", metadata_snapshot);
}
}
auto future_parts = initCoverageWithNewEmptyParts(parts);
@ -1898,6 +1934,33 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont
clearEmptyParts();
}
void StorageMergeTree::dropPartsImpl(DataPartsVector && parts_to_remove, bool detach)
{
auto metadata_snapshot = getInMemoryMetadataPtr();
if (detach)
{
/// If DETACH clone parts to detached/ directory
/// NOTE: no race with background cleanup until we hold pointers to parts
for (const auto & part : parts_to_remove)
{
LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory());
part->makeCloneInDetached("", metadata_snapshot);
}
}
if (deduplication_log)
{
for (const auto & part : parts_to_remove)
deduplication_log->dropPart(part->info);
}
if (detach)
LOG_INFO(log, "Detached {} parts: [{}]", parts_to_remove.size(), fmt::join(getPartsNames(parts_to_remove), ", "));
else
LOG_INFO(log, "Removed {} parts: [{}]", parts_to_remove.size(), fmt::join(getPartsNames(parts_to_remove), ", "));
}
PartitionCommandsResultInfo StorageMergeTree::attachPartition(
const ASTPtr & partition, const StorageMetadataPtr & /* metadata_snapshot */,
bool attach_part, ContextPtr local_context)

View File

@ -237,6 +237,7 @@ private:
void dropPartNoWaitNoThrow(const String & part_name) override;
void dropPart(const String & part_name, bool detach, ContextPtr context) override;
void dropPartition(const ASTPtr & partition, bool detach, ContextPtr context) override;
void dropPartsImpl(DataPartsVector && parts_to_remove, bool detach);
PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr context) override;
void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) override;

View File

@ -197,6 +197,7 @@ namespace ActionLocks
extern const StorageActionBlockType ReplicationQueue;
extern const StorageActionBlockType PartsTTLMerge;
extern const StorageActionBlockType PartsMove;
extern const StorageActionBlockType PullReplicationLog;
}
@ -4340,7 +4341,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id)
{
auto zookeeper = getZooKeeper();
LOG_DEBUG(log, "Cleaning up last parent node for partition {}", partition_id);
LOG_DEBUG(log, "Cleaning up last part node for partition {}", partition_id);
/// The name of the previous part for which the quorum was reached.
const String quorum_last_part_path = fs::path(zookeeper_path) / "quorum" / "last_part";
@ -4361,6 +4362,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id)
if (!parts_with_quorum.added_parts.contains(partition_id))
{
/// There is no information about interested part.
LOG_TEST(log, "There is no information about the partition");
break;
}
@ -4378,6 +4380,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id)
else if (code == Coordination::Error::ZNONODE)
{
/// Node is deleted. It is impossible, but it is Ok.
LOG_WARNING(log, "The last part node {} was deleted", quorum_last_part_path);
break;
}
else if (code == Coordination::Error::ZBADVERSION)
@ -8169,6 +8172,9 @@ ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType acti
if (action_type == ActionLocks::PartsMove)
return parts_mover.moves_blocker.cancel();
if (action_type == ActionLocks::PullReplicationLog)
return queue.pull_log_blocker.cancel();
return {};
}

View File

@ -27,7 +27,6 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d
database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
if (database.getEngineName() != "Memory")
return;
bool is_uppercase = database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE;
String metadata_resource_name = view_name + ".sql";
if (query.empty())
@ -42,13 +41,18 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d
assert(view_name == ast_create.getTable());
ast_create.attach = false;
ast_create.setDatabase(database.getDatabaseName());
if (is_uppercase)
ast_create.setTable(Poco::toUpper(view_name));
StoragePtr view = createTableFromAST(ast_create, database.getDatabaseName(),
database.getTableDataPath(ast_create), context, true).second;
database.createTable(context, ast_create.getTable(), view, ast);
ASTPtr ast_upper = ast_create.clone();
auto & ast_create_upper = ast_upper->as<ASTCreateQuery &>();
ast_create_upper.setTable(Poco::toUpper(view_name));
StoragePtr view_upper = createTableFromAST(ast_create_upper, database.getDatabaseName(),
database.getTableDataPath(ast_create_upper), context, true).second;
database.createTable(context, ast_create_upper.getTable(), view_upper, ast_upper);
}
catch (...)
{

View File

@ -188,7 +188,7 @@ def test_grant_all_on_table():
instance.query("SHOW GRANTS FOR B")
== "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, "
"DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, "
"SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, "
"SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, "
"SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n"
)
instance.query("REVOKE ALL ON test.table FROM B", user="A")

View File

@ -1,7 +1,6 @@
import pytest
from helpers.client import CommandRequest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
@ -14,7 +13,7 @@ node = cluster.add_instance(
)
@pytest.fixture(scope="module", autouse=True)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
@ -24,122 +23,41 @@ def started_cluster():
cluster.shutdown()
def keeper_query(query: str):
return CommandRequest(
def test_base_commands(started_cluster):
_ = started_cluster
command = CommandRequest(
[
cluster.server_bin_path,
started_cluster.server_bin_path,
"keeper-client",
"--host",
str(cluster.get_instance_ip("zoo1")),
"--port",
str(cluster.zookeeper_port),
"-q",
query,
"create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;",
],
stdin="",
)
def test_big_family():
command = keeper_query(
"create test_big_family foo;"
"create test_big_family/1 foo;"
"create test_big_family/1/1 foo;"
"create test_big_family/1/2 foo;"
"create test_big_family/1/3 foo;"
"create test_big_family/1/4 foo;"
"create test_big_family/1/5 foo;"
"create test_big_family/2 foo;"
"create test_big_family/2/1 foo;"
"create test_big_family/2/2 foo;"
"create test_big_family/2/3 foo;"
"find_big_family test_big_family;"
)
assert command.get_answer() == TSV(
[
["/test_big_family/1", "5"],
["/test_big_family/2", "3"],
["/test_big_family/2/3", "0"],
["/test_big_family/2/2", "0"],
["/test_big_family/2/1", "0"],
["/test_big_family/1/5", "0"],
["/test_big_family/1/4", "0"],
["/test_big_family/1/3", "0"],
["/test_big_family/1/2", "0"],
["/test_big_family/1/1", "0"],
]
)
command = keeper_query("find_big_family test_big_family 1;")
assert command.get_answer() == TSV(
[
["/test_big_family/1", "5"],
]
)
def test_find_super_nodes():
command = keeper_query(
"create test_find_super_nodes foo;"
"create test_find_super_nodes/1 foo;"
"create test_find_super_nodes/1/1 foo;"
"create test_find_super_nodes/1/2 foo;"
"create test_find_super_nodes/1/3 foo;"
"create test_find_super_nodes/1/4 foo;"
"create test_find_super_nodes/1/5 foo;"
"create test_find_super_nodes/2 foo;"
"create test_find_super_nodes/2/1 foo;"
"create test_find_super_nodes/2/2 foo;"
"create test_find_super_nodes/2/3 foo;"
"create test_find_super_nodes/2/4 foo;"
"cd test_find_super_nodes;"
"find_super_nodes 4;"
)
assert command.get_answer() == TSV(
[
["/test_find_super_nodes/1", "5"],
["/test_find_super_nodes/2", "4"],
]
)
def test_delete_stable_backups():
command = keeper_query(
"create /clickhouse/backups foo;"
"create /clickhouse/backups/1 foo;"
"create /clickhouse/backups/1/stage foo;"
"create /clickhouse/backups/1/stage/alive123 foo;"
"create /clickhouse/backups/2 foo;"
"create /clickhouse/backups/2/stage foo;"
"create /clickhouse/backups/2/stage/dead123 foo;"
"delete_stable_backups;"
"y;"
"ls clickhouse/backups;"
)
assert command.get_answer() == (
"You are going to delete all inactive backups in /clickhouse/backups. Continue?\n"
'Found backup "/clickhouse/backups/1", checking if it\'s active\n'
'Backup "/clickhouse/backups/1" is active, not going to delete\n'
'Found backup "/clickhouse/backups/2", checking if it\'s active\n'
'Backup "/clickhouse/backups/2" is not active, deleting it\n'
"1\n"
)
def test_base_commands():
command = keeper_query(
"create test_create_zk_node1 testvalue1;"
"create test_create_zk_node_2 testvalue2;"
"get test_create_zk_node1;"
)
assert command.get_answer() == "testvalue1\n"
def test_four_letter_word_commands():
command = keeper_query("ruok")
def test_four_letter_word_commands(started_cluster):
_ = started_cluster
command = CommandRequest(
[
started_cluster.server_bin_path,
"keeper-client",
"--host",
str(cluster.get_instance_ip("zoo1")),
"--port",
str(cluster.zookeeper_port),
"-q",
"ruok",
],
stdin="",
)
assert command.get_answer() == "imok\n"

View File

@ -14,6 +14,7 @@ a UNSIGNED TINYINT
Result:
tables 1
tables 1
tables 1
Columns:
a
b

View File

@ -679,7 +679,7 @@ def test_php_client(started_cluster, php_container):
demux=True,
)
assert code == 0
assert stdout.decode() == "tables\ntables\n"
assert stdout.decode() == "tables\ntables\ntables\n"
code, (stdout, stderr) = php_container.exec_run(
"php -f test_ssl.php {host} {port} default 123".format(
@ -688,7 +688,7 @@ def test_php_client(started_cluster, php_container):
demux=True,
)
assert code == 0
assert stdout.decode() == "tables\ntables\n"
assert stdout.decode() == "tables\ntables\ntables\n"
code, (stdout, stderr) = php_container.exec_run(
"php -f test.php {host} {port} user_with_double_sha1 abacaba".format(
@ -697,7 +697,7 @@ def test_php_client(started_cluster, php_container):
demux=True,
)
assert code == 0
assert stdout.decode() == "tables\ntables\n"
assert stdout.decode() == "tables\ntables\ntables\n"
code, (stdout, stderr) = php_container.exec_run(
"php -f test_ssl.php {host} {port} user_with_double_sha1 abacaba".format(
@ -706,7 +706,7 @@ def test_php_client(started_cluster, php_container):
demux=True,
)
assert code == 0
assert stdout.decode() == "tables\ntables\n"
assert stdout.decode() == "tables\ntables\ntables\n"
def test_mysqljs_client(started_cluster, nodejs_container):

View File

@ -51,7 +51,12 @@ instance = cluster.add_instance(
"configs/server.key",
],
user_configs=["configs/users.xml"],
env_variables={"UBSAN_OPTIONS": "print_stacktrace=1"},
env_variables={
"UBSAN_OPTIONS": "print_stacktrace=1",
# Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387
"TSAN_OPTIONS": "report_atomic_races=0 "
+ os.getenv("TSAN_OPTIONS", default=""),
},
)

View File

@ -190,3 +190,15 @@ def test_information_schema():
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.TABLES WHERE table_name='TABLES'"
)
== "2\n"
)
assert (
node.query(
"SELECT count() FROM INFORMATION_SCHEMA.tables WHERE table_name='tables'"
)
== "3\n"
)

View File

@ -761,7 +761,7 @@ def test_multiple_tables_streaming_sync(started_cluster, mode):
@pytest.mark.parametrize("mode", AVAILABLE_MODES)
def test_multiple_tables_streaming_sync_distributed(started_cluster, mode):
files_to_generate = 100
poll_size = 10
poll_size = 2
prefix = f"test_multiple_{mode}"
bucket = started_cluster.minio_restricted_bucket
instance = started_cluster.instances["instance"]
@ -785,7 +785,12 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode):
CREATE TABLE test.s3_queue_persistent ({table_format})
ENGINE = MergeTree()
ORDER BY column1;
"""
)
for inst in [instance, instance_2]:
inst.query(
f"""
CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS
SELECT
*
@ -800,7 +805,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode):
def get_count(node, table_name):
return int(run_query(node, f"SELECT count() FROM {table_name}"))
for _ in range(100):
for _ in range(150):
if (
get_count(instance, "test.s3_queue_persistent")
+ get_count(instance_2, "test.s3_queue_persistent")
@ -816,11 +821,12 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode):
list(map(int, l.split())) for l in run_query(instance_2, get_query).splitlines()
]
assert len(res1) + len(res2) == files_to_generate
# Checking that all engines have made progress
assert len(res1) > 0
assert len(res2) > 0
assert len(res1) + len(res2) == files_to_generate
assert {tuple(v) for v in res1 + res2} == set([tuple(i) for i in total_values])
# Checking that all files were processed only once

View File

@ -105,8 +105,6 @@ def test_rollback_unfinished_on_restart1(start_cluster):
"0_4_4_0_7\t0\ttid3\tcsn18446744073709551615_\ttid0\tcsn0_\n"
"0_8_8_0\t0\ttid5\tcsn18446744073709551615_\ttid0\tcsn0_\n"
"1_1_1_0\t0\ttid0\tcsn1_\ttid1\tcsn_1\n"
"1_1_1_1\t1\ttid1\tcsn_1\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n"
"1_1_1_1_7\t0\ttid3\tcsn18446744073709551615_\ttid0\tcsn0_\n"
"1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n"
"1_3_3_0_7\t0\ttid3\tcsn18446744073709551615_\ttid0\tcsn0_\n"
"1_5_5_0\t1\ttid6\tcsn_6\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n"
@ -194,6 +192,5 @@ def test_rollback_unfinished_on_restart2(start_cluster):
"0_4_4_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n"
"0_5_5_0\t0\ttid5\tcsn18446744073709551615_\ttid0\tcsn0_\n"
"1_1_1_0\t0\ttid0\tcsn1_\ttid1\tcsn_1\n"
"1_1_1_1\t1\ttid1\tcsn_1\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n"
"1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n"
)

View File

@ -5,7 +5,7 @@
[1]
[1]
[1]
[NULL,1]
[1,NULL]
[1]
[1]
[[1,1]]

View File

@ -11,7 +11,7 @@ ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl"
function query()
{
for _ in {1..100}; do
for _ in {1..50}; do
${CLICKHOUSE_CLIENT} --query "CREATE DATABASE IF NOT EXISTS parallel_ddl"
${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl"
done

View File

@ -1,3 +1,7 @@
COLUMNS
SCHEMATA
TABLES
VIEWS
columns
schemata
tables
@ -6,6 +10,10 @@ COLUMNS
SCHEMATA
TABLES
VIEWS
columns
schemata
tables
views
INFORMATION_SCHEMA INFORMATION_SCHEMA default \N \N \N \N
information_schema information_schema default \N \N \N \N
default default mv VIEW

View File

@ -21,20 +21,20 @@ tx7 7 20 all_1_1_0_13
tx7 7 40 all_14_14_0
tx7 7 60 all_7_7_0_13
tx7 7 80 all_12_12_0_13
tx7 8 20 all_1_14_2_13
tx7 8 40 all_1_14_2_13
tx7 8 60 all_1_14_2_13
tx7 8 80 all_1_14_2_13
tx7 8 20 all_1_14_1_13
tx7 8 40 all_1_14_1_13
tx7 8 60 all_1_14_1_13
tx7 8 80 all_1_14_1_13
Serialization error
INVALID_TRANSACTION
tx11 9 21 all_1_14_2_17
tx11 9 41 all_1_14_2_17
tx11 9 61 all_1_14_2_17
tx11 9 81 all_1_14_2_17
tx11 9 21 all_1_14_1_17
tx11 9 41 all_1_14_1_17
tx11 9 61 all_1_14_1_17
tx11 9 81 all_1_14_1_17
1 1 RUNNING
tx14 10 22 all_1_14_2_18
tx14 10 42 all_1_14_2_18
tx14 10 62 all_1_14_2_18
tx14 10 82 all_1_14_2_18
tx14 10 22 all_1_14_1_18
tx14 10 42 all_1_14_1_18
tx14 10 62 all_1_14_1_18
tx14 10 82 all_1_14_1_18
11 2 all_2_2_0
11 10 all_1_1_0_3

View File

@ -53,9 +53,6 @@ tx 6 "alter table mt update n=n*10 wh
tx 6 "insert into mt values (40)"
tx 6 "commit"
function accept_both_parts() {
sed 's/all_1_14_1_1/all_1_14_2_1/g'
}
tx 7 "begin transaction"
tx 7 "select 7, n, _part from mt order by n"
@ -64,7 +61,7 @@ tx_async 8 "alter table mt update n = 0 whe
$CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_15.txt' format Null" 2>&1| grep -Fv "probably it finished"
tx_sync 8 "rollback"
tx 7 "optimize table mt final"
tx 7 "select 8, n, _part from mt order by n" | accept_both_parts
tx 7 "select 8, n, _part from mt order by n"
tx 10 "begin transaction"
tx 10 "alter table mt update n = 0 where 1" | grep -Eo "Serialization error" | uniq
tx 7 "alter table mt update n=n+1 where 1"
@ -74,7 +71,7 @@ tx 7 "commit"
tx_async 11 "begin transaction"
tx_async 11 "select 9, n, _part from mt order by n" | accept_both_parts
tx_async 11 "select 9, n, _part from mt order by n"
tx_async 12 "begin transaction"
tx_async 11 "alter table mt update n=n+1 where 1" >/dev/null
tx_async 12 "alter table mt update n=n+1 where 1" >/dev/null
@ -91,7 +88,7 @@ $CLICKHOUSE_CLIENT -q "kill transaction where tid=$tid_to_kill format Null"
tx_sync 13 "rollback"
tx 14 "begin transaction"
tx 14 "select 10, n, _part from mt order by n" | accept_both_parts
tx 14 "select 10, n, _part from mt order by n"
$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table mt"

View File

@ -0,0 +1,8 @@
1 1
2 1
3 1
4 1
1
10 100
1 1 1
2 1 1

View File

@ -0,0 +1,123 @@
#!/usr/bin/env bash
# Tags: long, no-replicated-database, no-ordinary-database
# shellcheck disable=SC2015
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
set -e
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src";
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS dst";
$CLICKHOUSE_CLIENT --query "CREATE TABLE src (n UInt64, type UInt8) ENGINE=MergeTree ORDER BY type SETTINGS old_parts_lifetime=0";
$CLICKHOUSE_CLIENT --query "CREATE TABLE dst (n UInt64, type UInt8) ENGINE=MergeTree ORDER BY type SETTINGS old_parts_lifetime=0";
function thread_insert()
{
set -e
val=1
while true; do
$CLICKHOUSE_CLIENT --multiquery --query "
BEGIN TRANSACTION;
INSERT INTO src VALUES /* ($val, 1) */ ($val, 1);
INSERT INTO src VALUES /* ($val, 2) */ ($val, 2);
COMMIT;"
val=$((val+1))
sleep 0.$RANDOM;
done
}
# NOTE
# ALTER PARTITION query stops merges,
# but serialization error is still possible if some merge was assigned (and committed) between BEGIN and ALTER.
function thread_partition_src_to_dst()
{
set -e
count=0
sum=0
for i in {1..20}; do
out=$(
$CLICKHOUSE_CLIENT --multiquery --query "
BEGIN TRANSACTION;
INSERT INTO src VALUES /* ($i, 3) */ ($i, 3);
INSERT INTO dst SELECT * FROM src;
ALTER TABLE src DROP PARTITION ID 'all';
SET throw_on_unsupported_query_inside_transaction=0;
SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=3) != ($count + 1, $sum + $i)) FORMAT Null;
COMMIT;" 2>&1) ||:
echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT --multiquery --query "
begin transaction;
set transaction snapshot 3;
select $i, 'src', type, n, _part from src order by type, n;
select $i, 'dst', type, n, _part from dst order by type, n;
rollback" ||:
echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || count=$((count+1))
echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || sum=$((sum+i))
done
}
function thread_partition_dst_to_src()
{
set -e
for i in {1..20}; do
action="ROLLBACK"
if (( i % 2 )); then
action="COMMIT"
fi
$CLICKHOUSE_CLIENT --multiquery --query "
SYSTEM STOP MERGES dst;
ALTER TABLE dst DROP PARTITION ID 'nonexistent'; -- STOP MERGES doesn't wait for started merges to finish, so we use this trick
SYSTEM SYNC TRANSACTION LOG;
BEGIN TRANSACTION;
INSERT INTO dst VALUES /* ($i, 4) */ ($i, 4);
INSERT INTO src SELECT * FROM dst;
ALTER TABLE dst DROP PARTITION ID 'all';
SET throw_on_unsupported_query_inside_transaction=0;
SYSTEM START MERGES dst;
SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=4) != (toUInt8($i/2 + 1), (select sum(number) from numbers(1, $i) where number % 2 or number=$i))) FORMAT Null;
$action;"
done
}
function thread_select()
{
set -e
while true; do
$CLICKHOUSE_CLIENT --multiquery --query "
BEGIN TRANSACTION;
-- no duplicates
SELECT type, throwIf(count(n) != countDistinct(n)) FROM src GROUP BY type FORMAT Null;
SELECT type, throwIf(count(n) != countDistinct(n)) FROM dst GROUP BY type FORMAT Null;
-- rows inserted by thread_insert moved together
SET throw_on_unsupported_query_inside_transaction=0;
SELECT _table, throwIf(arraySort(groupArrayIf(n, type=1)) != arraySort(groupArrayIf(n, type=2))) FROM merge(currentDatabase(), '') GROUP BY _table FORMAT Null;
-- all rows are inserted in insert_thread
SELECT type, throwIf(count(n) != max(n)), throwIf(sum(n) != max(n)*(max(n)+1)/2) FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type FORMAT Null;
COMMIT;"
done
}
thread_insert & PID_1=$!
thread_select & PID_2=$!
thread_partition_src_to_dst & PID_3=$!
thread_partition_dst_to_src & PID_4=$!
wait $PID_3 && wait $PID_4
kill -TERM $PID_1
kill -TERM $PID_2
wait
wait_for_queries_to_finish
$CLICKHOUSE_CLIENT -q "SELECT type, count(n) = countDistinct(n) FROM merge(currentDatabase(), '') GROUP BY type ORDER BY type"
$CLICKHOUSE_CLIENT -q "SELECT DISTINCT arraySort(groupArrayIf(n, type=1)) = arraySort(groupArrayIf(n, type=2)) FROM merge(currentDatabase(), '') GROUP BY _table ORDER BY _table"
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM merge(currentDatabase(), '') WHERE type=4"
$CLICKHOUSE_CLIENT -q "SELECT type, count(n) == max(n), sum(n) == max(n)*(max(n)+1)/2 FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type"
$CLICKHOUSE_CLIENT --query "DROP TABLE src";
$CLICKHOUSE_CLIENT --query "DROP TABLE dst";

View File

@ -29,13 +29,9 @@
4 1 Commit 1 1 1 0
5 1 Begin 1 1 1 1
5 1 AddPart 1 1 1 1 all_5_5_0
5 1 AddPart 1 1 1 1 all_1_1_1
5 1 LockPart 1 1 1 1 all_1_1_0
5 1 AddPart 1 1 1 1 all_3_3_1
5 1 LockPart 1 1 1 1 all_3_3_0
5 1 AddPart 1 1 1 1 all_4_4_1
5 1 LockPart 1 1 1 1 all_4_4_0
5 1 AddPart 1 1 1 1 all_5_5_1
5 1 LockPart 1 1 1 1 all_5_5_0
5 1 UnlockPart 1 1 1 1 all_1_1_0
5 1 UnlockPart 1 1 1 1 all_3_3_0

View File

@ -119,6 +119,7 @@ SYSTEM MERGES ['SYSTEM STOP MERGES','SYSTEM START MERGES','STOP MERGES','START M
SYSTEM TTL MERGES ['SYSTEM STOP TTL MERGES','SYSTEM START TTL MERGES','STOP TTL MERGES','START TTL MERGES'] TABLE SYSTEM
SYSTEM FETCHES ['SYSTEM STOP FETCHES','SYSTEM START FETCHES','STOP FETCHES','START FETCHES'] TABLE SYSTEM
SYSTEM MOVES ['SYSTEM STOP MOVES','SYSTEM START MOVES','STOP MOVES','START MOVES'] TABLE SYSTEM
SYSTEM PULLING REPLICATION LOG ['SYSTEM STOP PULLING REPLICATION LOG','SYSTEM START PULLING REPLICATION LOG'] TABLE SYSTEM
SYSTEM DISTRIBUTED SENDS ['SYSTEM STOP DISTRIBUTED SENDS','SYSTEM START DISTRIBUTED SENDS','STOP DISTRIBUTED SENDS','START DISTRIBUTED SENDS'] TABLE SYSTEM SENDS
SYSTEM REPLICATED SENDS ['SYSTEM STOP REPLICATED SENDS','SYSTEM START REPLICATED SENDS','STOP REPLICATED SENDS','START REPLICATED SENDS'] TABLE SYSTEM SENDS
SYSTEM SENDS ['SYSTEM STOP SENDS','SYSTEM START SENDS','STOP SENDS','START SENDS'] \N SYSTEM

View File

@ -1193,10 +1193,10 @@ select count() over () from numbers(4) where number < 2;
2
-- floating point RANGE frame
select
count(*) over (order by toFloat32(number) range 5. preceding),
count(*) over (order by toFloat64(number) range 5. preceding),
count(*) over (order by toFloat32(number) range between current row and 5. following),
count(*) over (order by toFloat64(number) range between current row and 5. following)
count(*) over (order by toFloat32(number) range 5 preceding),
count(*) over (order by toFloat64(number) range 5 preceding),
count(*) over (order by toFloat32(number) range between current row and 5 following),
count(*) over (order by toFloat64(number) range between current row and 5 following)
from numbers(7)
;
1 1 6 6

View File

@ -474,10 +474,10 @@ select count() over () from numbers(4) where number < 2;
-- floating point RANGE frame
select
count(*) over (order by toFloat32(number) range 5. preceding),
count(*) over (order by toFloat64(number) range 5. preceding),
count(*) over (order by toFloat32(number) range between current row and 5. following),
count(*) over (order by toFloat64(number) range between current row and 5. following)
count(*) over (order by toFloat32(number) range 5 preceding),
count(*) over (order by toFloat64(number) range 5 preceding),
count(*) over (order by toFloat32(number) range between current row and 5 following),
count(*) over (order by toFloat64(number) range between current row and 5 following)
from numbers(7)
;

View File

@ -298,7 +298,7 @@ CREATE TABLE system.grants
(
`user_name` Nullable(String),
`role_name` Nullable(String),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM PULLING REPLICATION LOG' = 121, 'SYSTEM DISTRIBUTED SENDS' = 122, 'SYSTEM REPLICATED SENDS' = 123, 'SYSTEM SENDS' = 124, 'SYSTEM REPLICATION QUEUES' = 125, 'SYSTEM DROP REPLICA' = 126, 'SYSTEM SYNC REPLICA' = 127, 'SYSTEM RESTART REPLICA' = 128, 'SYSTEM RESTORE REPLICA' = 129, 'SYSTEM WAIT LOADING PARTS' = 130, 'SYSTEM SYNC DATABASE REPLICA' = 131, 'SYSTEM SYNC TRANSACTION LOG' = 132, 'SYSTEM SYNC FILE CACHE' = 133, 'SYSTEM FLUSH DISTRIBUTED' = 134, 'SYSTEM FLUSH LOGS' = 135, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 136, 'SYSTEM FLUSH' = 137, 'SYSTEM THREAD FUZZER' = 138, 'SYSTEM UNFREEZE' = 139, 'SYSTEM FAILPOINT' = 140, 'SYSTEM LISTEN' = 141, 'SYSTEM' = 142, 'dictGet' = 143, 'displaySecretsInShowAndSelect' = 144, 'addressToLine' = 145, 'addressToLineWithInlines' = 146, 'addressToSymbol' = 147, 'demangle' = 148, 'INTROSPECTION' = 149, 'FILE' = 150, 'URL' = 151, 'REMOTE' = 152, 'MONGO' = 153, 'REDIS' = 154, 'MEILISEARCH' = 155, 'MYSQL' = 156, 'POSTGRES' = 157, 'SQLITE' = 158, 'ODBC' = 159, 'JDBC' = 160, 'HDFS' = 161, 'S3' = 162, 'HIVE' = 163, 'AZURE' = 164, 'SOURCES' = 165, 'CLUSTER' = 166, 'ALL' = 167, 'NONE' = 168),
`database` Nullable(String),
`table` Nullable(String),
`column` Nullable(String),
@ -586,10 +586,10 @@ ENGINE = SystemPartsColumns
COMMENT 'SYSTEM TABLE is built on the fly.'
CREATE TABLE system.privileges
(
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167),
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM PULLING REPLICATION LOG' = 121, 'SYSTEM DISTRIBUTED SENDS' = 122, 'SYSTEM REPLICATED SENDS' = 123, 'SYSTEM SENDS' = 124, 'SYSTEM REPLICATION QUEUES' = 125, 'SYSTEM DROP REPLICA' = 126, 'SYSTEM SYNC REPLICA' = 127, 'SYSTEM RESTART REPLICA' = 128, 'SYSTEM RESTORE REPLICA' = 129, 'SYSTEM WAIT LOADING PARTS' = 130, 'SYSTEM SYNC DATABASE REPLICA' = 131, 'SYSTEM SYNC TRANSACTION LOG' = 132, 'SYSTEM SYNC FILE CACHE' = 133, 'SYSTEM FLUSH DISTRIBUTED' = 134, 'SYSTEM FLUSH LOGS' = 135, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 136, 'SYSTEM FLUSH' = 137, 'SYSTEM THREAD FUZZER' = 138, 'SYSTEM UNFREEZE' = 139, 'SYSTEM FAILPOINT' = 140, 'SYSTEM LISTEN' = 141, 'SYSTEM' = 142, 'dictGet' = 143, 'displaySecretsInShowAndSelect' = 144, 'addressToLine' = 145, 'addressToLineWithInlines' = 146, 'addressToSymbol' = 147, 'demangle' = 148, 'INTROSPECTION' = 149, 'FILE' = 150, 'URL' = 151, 'REMOTE' = 152, 'MONGO' = 153, 'REDIS' = 154, 'MEILISEARCH' = 155, 'MYSQL' = 156, 'POSTGRES' = 157, 'SQLITE' = 158, 'ODBC' = 159, 'JDBC' = 160, 'HDFS' = 161, 'S3' = 162, 'HIVE' = 163, 'AZURE' = 164, 'SOURCES' = 165, 'CLUSTER' = 166, 'ALL' = 167, 'NONE' = 168),
`aliases` Array(String),
`level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)),
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167))
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM PULLING REPLICATION LOG' = 121, 'SYSTEM DISTRIBUTED SENDS' = 122, 'SYSTEM REPLICATED SENDS' = 123, 'SYSTEM SENDS' = 124, 'SYSTEM REPLICATION QUEUES' = 125, 'SYSTEM DROP REPLICA' = 126, 'SYSTEM SYNC REPLICA' = 127, 'SYSTEM RESTART REPLICA' = 128, 'SYSTEM RESTORE REPLICA' = 129, 'SYSTEM WAIT LOADING PARTS' = 130, 'SYSTEM SYNC DATABASE REPLICA' = 131, 'SYSTEM SYNC TRANSACTION LOG' = 132, 'SYSTEM SYNC FILE CACHE' = 133, 'SYSTEM FLUSH DISTRIBUTED' = 134, 'SYSTEM FLUSH LOGS' = 135, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 136, 'SYSTEM FLUSH' = 137, 'SYSTEM THREAD FUZZER' = 138, 'SYSTEM UNFREEZE' = 139, 'SYSTEM FAILPOINT' = 140, 'SYSTEM LISTEN' = 141, 'SYSTEM' = 142, 'dictGet' = 143, 'displaySecretsInShowAndSelect' = 144, 'addressToLine' = 145, 'addressToLineWithInlines' = 146, 'addressToSymbol' = 147, 'demangle' = 148, 'INTROSPECTION' = 149, 'FILE' = 150, 'URL' = 151, 'REMOTE' = 152, 'MONGO' = 153, 'REDIS' = 154, 'MEILISEARCH' = 155, 'MYSQL' = 156, 'POSTGRES' = 157, 'SQLITE' = 158, 'ODBC' = 159, 'JDBC' = 160, 'HDFS' = 161, 'S3' = 162, 'HIVE' = 163, 'AZURE' = 164, 'SOURCES' = 165, 'CLUSTER' = 166, 'ALL' = 167, 'NONE' = 168))
)
ENGINE = SystemPrivileges
COMMENT 'SYSTEM TABLE is built on the fly.'

View File

@ -1,4 +1,6 @@
CREATE DATABASE INFORMATION_SCHEMA\nENGINE = Memory
CREATE VIEW INFORMATION_SCHEMA.COLUMNS\n(\n `table_catalog` String,\n `table_schema` String,\n `table_name` String,\n `TABLE_SCHEMA` String,\n `TABLE_NAME` String,\n `column_name` String,\n `ordinal_position` UInt64,\n `column_default` String,\n `is_nullable` String,\n `data_type` String,\n `character_maximum_length` Nullable(UInt64),\n `character_octet_length` Nullable(UInt64),\n `numeric_precision` Nullable(UInt64),\n `numeric_precision_radix` Nullable(UInt64),\n `numeric_scale` Nullable(UInt64),\n `datetime_precision` Nullable(UInt64),\n `character_set_catalog` Nullable(String),\n `character_set_schema` Nullable(String),\n `character_set_name` Nullable(String),\n `collation_catalog` Nullable(String),\n `collation_schema` Nullable(String),\n `collation_name` Nullable(String),\n `domain_catalog` Nullable(String),\n `domain_schema` Nullable(String),\n `domain_name` Nullable(String),\n `column_comment` String,\n `column_type` String,\n `TABLE_CATALOG` String ALIAS table_catalog,\n `COLUMN_NAME` String ALIAS column_name,\n `ORDINAL_POSITION` UInt64 ALIAS ordinal_position,\n `COLUMN_DEFAULT` String ALIAS column_default,\n `IS_NULLABLE` String ALIAS is_nullable,\n `DATA_TYPE` String ALIAS data_type,\n `CHARACTER_MAXIMUM_LENGTH` Nullable(UInt64) ALIAS character_maximum_length,\n `CHARACTER_OCTET_LENGTH` Nullable(UInt64) ALIAS character_octet_length,\n `NUMERIC_PRECISION` Nullable(UInt64) ALIAS numeric_precision,\n `NUMERIC_PRECISION_RADIX` Nullable(UInt64) ALIAS numeric_precision_radix,\n `NUMERIC_SCALE` Nullable(UInt64) ALIAS numeric_scale,\n `DATETIME_PRECISION` Nullable(UInt64) ALIAS datetime_precision,\n `CHARACTER_SET_CATALOG` Nullable(String) ALIAS character_set_catalog,\n `CHARACTER_SET_SCHEMA` Nullable(String) ALIAS character_set_schema,\n `CHARACTER_SET_NAME` Nullable(String) ALIAS character_set_name,\n `COLLATION_CATALOG` Nullable(String) ALIAS collation_catalog,\n `COLLATION_SCHEMA` Nullable(String) ALIAS collation_schema,\n `COLLATION_NAME` Nullable(String) ALIAS collation_name,\n `DOMAIN_CATALOG` Nullable(String) ALIAS domain_catalog,\n `DOMAIN_SCHEMA` Nullable(String) ALIAS domain_schema,\n `DOMAIN_NAME` Nullable(String) ALIAS domain_name,\n `COLUMN_COMMENT` String ALIAS column_comment,\n `COLUMN_TYPE` String ALIAS column_type\n) AS\nSELECT\n database AS table_catalog,\n database AS table_schema,\n database AS TABLE_SCHEMA,\n table AS table_name,\n table AS TABLE_NAME,\n name AS column_name,\n position AS ordinal_position,\n default_expression AS column_default,\n type LIKE \'Nullable(%)\' AS is_nullable,\n type AS data_type,\n character_octet_length AS character_maximum_length,\n character_octet_length,\n numeric_precision,\n numeric_precision_radix,\n numeric_scale,\n datetime_precision,\n NULL AS character_set_catalog,\n NULL AS character_set_schema,\n NULL AS character_set_name,\n NULL AS collation_catalog,\n NULL AS collation_schema,\n NULL AS collation_name,\n NULL AS domain_catalog,\n NULL AS domain_schema,\n NULL AS domain_name,\n comment AS column_comment,\n type AS column_type\nFROM system.columns
CREATE VIEW INFORMATION_SCHEMA.TABLES (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables
CREATE VIEW INFORMATION_SCHEMA.tables (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables
CREATE VIEW information_schema.TABLES (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables
CREATE VIEW information_schema.tables (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables

View File

@ -5,19 +5,13 @@ tx21 3
UNKNOWN_TABLE
concurrent_insert
2
all_1_1_1 0
all_2_2_1 0
all_3_3_1 0
all_4_4_1 0
all_5_5_0 1
all_6_6_1 0
concurrent_drop_part_before
SERIALIZATION_ERROR
INVALID_TRANSACTION
1
3
all_1_1_0 1
all_2_2_1 0
all_3_3_0 1
read_from_snapshot
tx51 3
@ -28,15 +22,9 @@ tx51 3
concurrent_drop_part_after
NO_SUCH_DATA_PART
INVALID_TRANSACTION
all_1_1_1 0
all_2_2_1 0
all_3_3_1 0
NewPart all_1_1_0
NewPart all_1_1_1
NewPart all_2_2_0
NewPart all_2_2_1
NewPart all_3_3_0
NewPart all_3_3_1
concurrent_truncate_notx_after
tx71 3
tx71 0

View File

@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# Test that running distributed query and cancel it ASAP,
# this can trigger a hung/deadlock in ProcessorList.
for i in {1..100}; do
for i in {1..50}; do
query_id="$CLICKHOUSE_TEST_UNIQUE_NAME-$i"
$CLICKHOUSE_CLIENT --format Null --query_id "$query_id" -q "select * from remote('127.{1|2|3|4|5|6}', numbers(1e12))" 2>/dev/null &
while :; do

View File

@ -0,0 +1,5 @@
OK
1
0
1
0

View File

@ -0,0 +1,71 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings
# set -x
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}"
$CLICKHOUSE_CLIENT -nm --query """
DROP TABLE IF EXISTS test;
CREATE TABLE test (a Int32, b String)
ENGINE = MergeTree() ORDER BY tuple()
SETTINGS disk = disk(name = '$disk_name', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk);
INSERT INTO test SELECT 1, 'test';
"""
query_id=$RANDOM
$CLICKHOUSE_CLIENT --query_id "$query_id" --query "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1"
$CLICKHOUSE_CLIENT -nm --query """
SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY kek;
""" 2>&1 | grep -q "Invalid cache key hex: kek" && echo "OK" || echo "FAIL"
${CLICKHOUSE_CLIENT} -q " system flush logs"
key=$($CLICKHOUSE_CLIENT -nm --query """
SELECT key FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1;
""")
offset=$($CLICKHOUSE_CLIENT -nm --query """
SELECT offset FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1;
""")
$CLICKHOUSE_CLIENT -nm --query """
SELECT count() FROM system.filesystem_cache WHERE key = '$key' AND file_segment_range_begin = $offset;
"""
$CLICKHOUSE_CLIENT -nm --query """
SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY $key OFFSET $offset;
"""
$CLICKHOUSE_CLIENT -nm --query """
SELECT count() FROM system.filesystem_cache WHERE key = '$key' AND file_segment_range_begin = $offset;
"""
query_id=$RANDOM$RANDOM
$CLICKHOUSE_CLIENT --query_id "$query_id" --query "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1"
${CLICKHOUSE_CLIENT} -q " system flush logs"
key=$($CLICKHOUSE_CLIENT -nm --query """
SELECT key FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1;
""")
$CLICKHOUSE_CLIENT -nm --query """
SELECT count() FROM system.filesystem_cache WHERE key = '$key';
"""
$CLICKHOUSE_CLIENT -nm --query """
SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY $key
"""
$CLICKHOUSE_CLIENT -nm --query """
SELECT count() FROM system.filesystem_cache WHERE key = '$key';
"""

View File

@ -0,0 +1,6 @@
-- invalid start offset with RANGE
SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN 0.0 PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS }
SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN nan PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS }
-- invalid end offset with RANGE
SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.0 FOLLOWING); -- { serverError BAD_ARGUMENTS }
SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND nan FOLLOWING); -- { serverError BAD_ARGUMENTS }

View File

@ -1,3 +1,4 @@
v23.7.2.25-stable 2023-08-03
v23.7.1.2470-stable 2023-07-27
v23.6.2.18-stable 2023-07-09
v23.6.1.1524-stable 2023-06-30

1 v23.7.1.2470-stable v23.7.2.25-stable 2023-07-27 2023-08-03
1 v23.7.2.25-stable 2023-08-03
2 v23.7.1.2470-stable v23.7.1.2470-stable 2023-07-27 2023-07-27
3 v23.6.2.18-stable v23.6.2.18-stable 2023-07-09 2023-07-09
4 v23.6.1.1524-stable v23.6.1.1524-stable 2023-06-30 2023-06-30