Merge branch 'master' of github.com:ClickHouse/ClickHouse into system-symbols

This commit is contained in:
Alexey Milovidov 2023-11-13 03:27:37 +01:00
commit a044c61c79
83 changed files with 655 additions and 215 deletions

2
contrib/grpc vendored

@ -1 +1 @@
Subproject commit 267af8c3a1ea4a5a4d9e5a070ad2d1ac7c701923 Subproject commit b723ecae0991bb873fe87a595dfb187178733fde

View File

@ -7,6 +7,7 @@ echo '/*' > $FILES_TO_CHECKOUT
echo '!/test/*' >> $FILES_TO_CHECKOUT echo '!/test/*' >> $FILES_TO_CHECKOUT
echo '/test/build/*' >> $FILES_TO_CHECKOUT echo '/test/build/*' >> $FILES_TO_CHECKOUT
echo '/test/core/tsi/alts/fake_handshaker/*' >> $FILES_TO_CHECKOUT echo '/test/core/tsi/alts/fake_handshaker/*' >> $FILES_TO_CHECKOUT
echo '/test/core/event_engine/fuzzing_event_engine/*' >> $FILES_TO_CHECKOUT
echo '!/tools/*' >> $FILES_TO_CHECKOUT echo '!/tools/*' >> $FILES_TO_CHECKOUT
echo '/tools/codegen/*' >> $FILES_TO_CHECKOUT echo '/tools/codegen/*' >> $FILES_TO_CHECKOUT
echo '!/examples/*' >> $FILES_TO_CHECKOUT echo '!/examples/*' >> $FILES_TO_CHECKOUT

View File

@ -19,6 +19,11 @@ EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, c
EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, toLowCardinality('') AS check_name, toLowCardinality('') AS instance_type, '' AS instance_id"} EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, toLowCardinality('') AS check_name, toLowCardinality('') AS instance_type, '' AS instance_id"}
EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "}
# trace_log needs more columns for symbolization
EXTRA_COLUMNS_TRACE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), lines Array(LowCardinality(String)), "
EXTRA_COLUMNS_EXPRESSION_TRACE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> toLowCardinality(demangle(addressToSymbol(x))), trace) AS symbols, arrayMap(x -> toLowCardinality(addressToLine(x)), trace) AS lines"
function __set_connection_args function __set_connection_args
{ {
# It's impossible to use generous $CONNECTION_ARGS string, it's unsafe from word splitting perspective. # It's impossible to use generous $CONNECTION_ARGS string, it's unsafe from word splitting perspective.
@ -125,9 +130,18 @@ function setup_logs_replication
echo 'Create %_log tables' echo 'Create %_log tables'
clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table
do do
if [[ "$table" = "trace_log" ]]
then
EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS_TRACE_LOG}"
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_TRACE_LOG}"
else
EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS}"
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}"
fi
# Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed: # Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed:
hash=$(clickhouse-client --query " hash=$(clickhouse-client --query "
SELECT sipHash64(7, groupArray((name, type))) SELECT sipHash64(9, groupArray((name, type)))
FROM (SELECT name, type FROM system.columns FROM (SELECT name, type FROM system.columns
WHERE database = 'system' AND table = '$table' WHERE database = 'system' AND table = '$table'
ORDER BY position) ORDER BY position)
@ -135,7 +149,7 @@ function setup_logs_replication
# Create the destination table with adapted name and structure: # Create the destination table with adapted name and structure:
statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e '
s/^\($/('"$EXTRA_COLUMNS"'/; s/^\($/('"$EXTRA_COLUMNS_FOR_TABLE"'/;
s/ORDER BY \(/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"'/; s/ORDER BY \(/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"'/;
s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/;
/^TTL /d /^TTL /d
@ -155,7 +169,7 @@ function setup_logs_replication
ENGINE = Distributed(${CLICKHOUSE_CI_LOGS_CLUSTER}, default, ${table}_${hash}) ENGINE = Distributed(${CLICKHOUSE_CI_LOGS_CLUSTER}, default, ${table}_${hash})
SETTINGS flush_on_detach=0 SETTINGS flush_on_detach=0
EMPTY AS EMPTY AS
SELECT ${EXTRA_COLUMNS_EXPRESSION}, * SELECT ${EXTRA_COLUMNS_EXPRESSION_FOR_TABLE}, *
FROM system.${table} FROM system.${table}
" || continue " || continue
@ -163,7 +177,7 @@ function setup_logs_replication
clickhouse-client --query " clickhouse-client --query "
CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS
SELECT ${EXTRA_COLUMNS_EXPRESSION}, * SELECT ${EXTRA_COLUMNS_EXPRESSION_FOR_TABLE}, *
FROM system.${table} FROM system.${table}
" || continue " || continue
done done

View File

@ -19,6 +19,11 @@ dpkg -i package_folder/clickhouse-common-static-dbg_*.deb
dpkg -i package_folder/clickhouse-server_*.deb dpkg -i package_folder/clickhouse-server_*.deb
dpkg -i package_folder/clickhouse-client_*.deb dpkg -i package_folder/clickhouse-client_*.deb
# Check that the tools are available under short names
ch --query "SELECT 1" || exit 1
chl --query "SELECT 1" || exit 1
chc --version || exit 1
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
# shellcheck disable=SC1091 # shellcheck disable=SC1091
@ -62,7 +67,7 @@ if [ "$NUM_TRIES" -gt "1" ]; then
export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000 export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000
mkdir -p /var/run/clickhouse-server mkdir -p /var/run/clickhouse-server
# simpliest way to forward env variables to server # simplest way to forward env variables to server
sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server/config.xml --daemon --pid-file /var/run/clickhouse-server/clickhouse-server.pid sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server/config.xml --daemon --pid-file /var/run/clickhouse-server/clickhouse-server.pid
else else
sudo clickhouse start sudo clickhouse start

View File

@ -12,6 +12,7 @@ A client application to interact with clickhouse-keeper by its native protocol.
- `-q QUERY`, `--query=QUERY` — Query to execute. If this parameter is not passed, `clickhouse-keeper-client` will start in interactive mode. - `-q QUERY`, `--query=QUERY` — Query to execute. If this parameter is not passed, `clickhouse-keeper-client` will start in interactive mode.
- `-h HOST`, `--host=HOST` — Server host. Default value: `localhost`. - `-h HOST`, `--host=HOST` — Server host. Default value: `localhost`.
- `-p N`, `--port=N` — Server port. Default value: 9181 - `-p N`, `--port=N` — Server port. Default value: 9181
- `-c FILE_PATH`, `--config-file=FILE_PATH` — Set path of config file to get the connection string. Default value: `config.xml`.
- `--connection-timeout=TIMEOUT` — Set connection timeout in seconds. Default value: 10s. - `--connection-timeout=TIMEOUT` — Set connection timeout in seconds. Default value: 10s.
- `--session-timeout=TIMEOUT` — Set session timeout in seconds. Default value: 10s. - `--session-timeout=TIMEOUT` — Set session timeout in seconds. Default value: 10s.
- `--operation-timeout=TIMEOUT` — Set operation timeout in seconds. Default value: 10s. - `--operation-timeout=TIMEOUT` — Set operation timeout in seconds. Default value: 10s.

View File

@ -107,11 +107,7 @@ round(3.65, 1) = 3.6
Rounds a number to a specified decimal position. Rounds a number to a specified decimal position.
- If the rounding number is halfway between two numbers, the function uses bankers rounding. - If the rounding number is halfway between two numbers, the function uses bankers rounding. Banker's rounding is a method of rounding fractional numbers. When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. For example: 3.5 rounds up to 4, 2.5 rounds down to 2. It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`.
Banker's rounding is a method of rounding fractional numbers. When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. For example: 3.5 rounds up to 4, 2.5 rounds down to 2.
It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`.
- In other cases, the function rounds numbers to the nearest integer. - In other cases, the function rounds numbers to the nearest integer.

View File

@ -171,7 +171,8 @@ Result:
Can be used with [MinHash](../../sql-reference/functions/hash-functions.md#ngramminhash) functions for detection of semi-duplicate strings: Can be used with [MinHash](../../sql-reference/functions/hash-functions.md#ngramminhash) functions for detection of semi-duplicate strings:
``` sql ``` sql
SELECT tupleHammingDistance(wordShingleMinHash(string), wordShingleMinHashCaseInsensitive(string)) as HammingDistance FROM (SELECT 'ClickHouse is a column-oriented database management system for online analytical processing of queries.' AS string); SELECT tupleHammingDistance(wordShingleMinHash(string), wordShingleMinHashCaseInsensitive(string)) AS HammingDistance
FROM (SELECT 'ClickHouse is a column-oriented database management system for online analytical processing of queries.' AS string);
``` ```
Result: Result:

View File

@ -5,6 +5,7 @@
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h> #include <Common/ZooKeeper/KeeperException.h>
#include <Common/randomSeed.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Interpreters/InterpreterInsertQuery.h> #include <Interpreters/InterpreterInsertQuery.h>
@ -59,7 +60,7 @@ void ClusterCopier::init()
getContext()->setClustersConfig(task_cluster_current_config, false, task_cluster->clusters_prefix); getContext()->setClustersConfig(task_cluster_current_config, false, task_cluster->clusters_prefix);
/// Set up shards and their priority /// Set up shards and their priority
task_cluster->random_engine.seed(task_cluster->random_device()); task_cluster->random_engine.seed(randomSeed());
for (auto & task_table : task_cluster->table_tasks) for (auto & task_table : task_cluster->table_tasks)
{ {
task_table.cluster_pull = getContext()->getCluster(task_table.cluster_pull_name); task_table.cluster_pull = getContext()->getCluster(task_table.cluster_pull_name);

View File

@ -7,7 +7,7 @@
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <random> #include <pcg_random.hpp>
namespace DB namespace DB
{ {
@ -45,7 +45,6 @@ struct TaskCluster
/// Subtasks /// Subtasks
TasksTable table_tasks; TasksTable table_tasks;
std::random_device random_device;
pcg64 random_engine; pcg64 random_engine;
}; };

View File

@ -420,7 +420,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
/// Create symlinks. /// Create symlinks.
std::initializer_list<const char *> tools std::initializer_list<std::string_view> tools
{ {
"clickhouse-server", "clickhouse-server",
"clickhouse-client", "clickhouse-client",
@ -435,6 +435,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
"clickhouse-keeper", "clickhouse-keeper",
"clickhouse-keeper-converter", "clickhouse-keeper-converter",
"clickhouse-disks", "clickhouse-disks",
"ch",
"chl",
"chc",
}; };
for (const auto & tool : tools) for (const auto & tool : tools)
@ -444,29 +447,39 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
if (fs::exists(symlink_path)) if (fs::exists(symlink_path))
{ {
bool is_symlink = FS::isSymlink(symlink_path); /// Do not replace short named symlinks if they are already present in the system
fs::path points_to; /// to avoid collision with other tools.
if (is_symlink) if (!tool.starts_with("clickhouse"))
points_to = fs::weakly_canonical(FS::readSymlink(symlink_path));
if (is_symlink && (points_to == main_bin_path || (options.count("link") && points_to == binary_self_canonical_path)))
{ {
fmt::print("Symlink {} already exists. Will keep it.\n", symlink_path.string());
need_to_create = false; need_to_create = false;
} }
else else
{ {
if (!is_symlink) bool is_symlink = FS::isSymlink(symlink_path);
fs::path points_to;
if (is_symlink)
points_to = fs::weakly_canonical(FS::readSymlink(symlink_path));
if (is_symlink && (points_to == main_bin_path || (options.count("link") && points_to == binary_self_canonical_path)))
{ {
fs::path rename_path = symlink_path.replace_extension(".old"); need_to_create = false;
fmt::print("File {} already exists but it's not a symlink. Will rename to {}.\n",
symlink_path.string(), rename_path.string());
fs::rename(symlink_path, rename_path);
} }
else if (points_to != main_bin_path) else
{ {
fmt::print("Symlink {} already exists but it points to {}. Will replace the old symlink to {}.\n", if (!is_symlink)
symlink_path.string(), points_to.string(), main_bin_path.string()); {
fs::remove(symlink_path); fs::path rename_path = symlink_path.replace_extension(".old");
fmt::print("File {} already exists but it's not a symlink. Will rename to {}.\n",
symlink_path.string(), rename_path.string());
fs::rename(symlink_path, rename_path);
}
else if (points_to != main_bin_path)
{
fmt::print("Symlink {} already exists but it points to {}. Will replace the old symlink to {}.\n",
symlink_path.string(), points_to.string(), main_bin_path.string());
fs::remove(symlink_path);
}
} }
} }
} }

View File

@ -2,15 +2,12 @@
#include <csetjmp> #include <csetjmp>
#include <unistd.h> #include <unistd.h>
#ifdef OS_LINUX
#include <sys/mman.h>
#endif
#include <new> #include <new>
#include <iostream> #include <iostream>
#include <vector> #include <vector>
#include <string> #include <string>
#include <tuple> #include <tuple>
#include <string_view>
#include <utility> /// pair #include <utility> /// pair
#include <fmt/format.h> #include <fmt/format.h>
@ -22,7 +19,6 @@
#include <Common/IO.h> #include <Common/IO.h>
#include <base/phdr_cache.h> #include <base/phdr_cache.h>
#include <base/scope_guard.h>
/// Universal executable for various clickhouse applications /// Universal executable for various clickhouse applications
@ -98,7 +94,7 @@ using MainFunc = int (*)(int, char**);
#if !defined(FUZZING_MODE) #if !defined(FUZZING_MODE)
/// Add an item here to register new application /// Add an item here to register new application
std::pair<const char *, MainFunc> clickhouse_applications[] = std::pair<std::string_view, MainFunc> clickhouse_applications[] =
{ {
#if ENABLE_CLICKHOUSE_LOCAL #if ENABLE_CLICKHOUSE_LOCAL
{"local", mainEntryClickHouseLocal}, {"local", mainEntryClickHouseLocal},
@ -158,6 +154,18 @@ std::pair<const char *, MainFunc> clickhouse_applications[] =
#endif #endif
}; };
/// Add an item here to register a new short name
std::pair<std::string_view, std::string_view> clickhouse_short_names[] =
{
#if ENABLE_CLICKHOUSE_LOCAL
{"ch", "local"},
{"chl", "local"},
#endif
#if ENABLE_CLICKHOUSE_CLIENT
{"chc", "client"},
#endif
};
int printHelp(int, char **) int printHelp(int, char **)
{ {
std::cerr << "Use one of the following commands:" << std::endl; std::cerr << "Use one of the following commands:" << std::endl;
@ -387,15 +395,21 @@ void checkHarmfulEnvironmentVariables(char ** argv)
} }
bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv) bool isClickhouseApp(std::string_view app_suffix, std::vector<char *> & argv)
{ {
for (const auto & [alias, name] : clickhouse_short_names)
if (app_suffix == name
&& !argv.empty() && (alias == argv[0] || endsWith(argv[0], "/" + std::string(alias))))
return true;
/// Use app if the first arg 'app' is passed (the arg should be quietly removed) /// Use app if the first arg 'app' is passed (the arg should be quietly removed)
if (argv.size() >= 2) if (argv.size() >= 2)
{ {
auto first_arg = argv.begin() + 1; auto first_arg = argv.begin() + 1;
/// 'clickhouse --client ...' and 'clickhouse client ...' are Ok /// 'clickhouse --client ...' and 'clickhouse client ...' are Ok
if (*first_arg == "--" + app_suffix || *first_arg == app_suffix) if (*first_arg == app_suffix
|| (std::string_view(*first_arg).starts_with("--") && std::string_view(*first_arg).substr(2) == app_suffix))
{ {
argv.erase(first_arg); argv.erase(first_arg);
return true; return true;
@ -403,7 +417,7 @@ bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
} }
/// Use app if clickhouse binary is run through symbolic link with name clickhouse-app /// Use app if clickhouse binary is run through symbolic link with name clickhouse-app
std::string app_name = "clickhouse-" + app_suffix; std::string app_name = "clickhouse-" + std::string(app_suffix);
return !argv.empty() && (app_name == argv[0] || endsWith(argv[0], "/" + app_name)); return !argv.empty() && (app_name == argv[0] || endsWith(argv[0], "/" + app_name));
} }

View File

@ -1,7 +0,0 @@
<clickhouse>
<profiles>
<default>
<allow_introspection_functions>1</allow_introspection_functions>
</default>
</profiles>
</clickhouse>

View File

@ -0,0 +1 @@
../../../tests/config/users.d/allow_introspection_functions.yaml

View File

@ -3,6 +3,7 @@
#include <Backups/BackupFileInfo.h> #include <Backups/BackupFileInfo.h>
#include <Backups/BackupIO.h> #include <Backups/BackupIO.h>
#include <Backups/IBackupEntry.h> #include <Backups/IBackupEntry.h>
#include <Common/ProfileEvents.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <base/hex.h> #include <base/hex.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
@ -24,6 +25,14 @@
#include <Poco/DOM/DOMParser.h> #include <Poco/DOM/DOMParser.h>
namespace ProfileEvents
{
extern const Event BackupsOpenedForRead;
extern const Event BackupsOpenedForWrite;
extern const Event BackupReadMetadataMicroseconds;
extern const Event BackupWriteMetadataMicroseconds;
}
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
@ -89,12 +98,14 @@ BackupImpl::BackupImpl(
, archive_params(archive_params_) , archive_params(archive_params_)
, open_mode(OpenMode::READ) , open_mode(OpenMode::READ)
, reader(std::move(reader_)) , reader(std::move(reader_))
, context(context_)
, is_internal_backup(false) , is_internal_backup(false)
, version(INITIAL_BACKUP_VERSION) , version(INITIAL_BACKUP_VERSION)
, base_backup_info(base_backup_info_) , base_backup_info(base_backup_info_)
, use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_) , use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_)
, log(&Poco::Logger::get("BackupImpl"))
{ {
open(context_); open();
} }
@ -115,6 +126,7 @@ BackupImpl::BackupImpl(
, archive_params(archive_params_) , archive_params(archive_params_)
, open_mode(OpenMode::WRITE) , open_mode(OpenMode::WRITE)
, writer(std::move(writer_)) , writer(std::move(writer_))
, context(context_)
, is_internal_backup(is_internal_backup_) , is_internal_backup(is_internal_backup_)
, coordination(coordination_) , coordination(coordination_)
, uuid(backup_uuid_) , uuid(backup_uuid_)
@ -124,7 +136,7 @@ BackupImpl::BackupImpl(
, use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_) , use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_)
, log(&Poco::Logger::get("BackupImpl")) , log(&Poco::Logger::get("BackupImpl"))
{ {
open(context_); open();
} }
@ -140,9 +152,11 @@ BackupImpl::~BackupImpl()
} }
} }
void BackupImpl::open(const ContextPtr & context) void BackupImpl::open()
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
LOG_INFO(log, "{} backup: {}", ((open_mode == OpenMode::WRITE) ? "Writing" : "Reading"), backup_name_for_logging);
ProfileEvents::increment((open_mode == OpenMode::WRITE) ? ProfileEvents::BackupsOpenedForWrite : ProfileEvents::BackupsOpenedForRead);
if (open_mode == OpenMode::WRITE) if (open_mode == OpenMode::WRITE)
{ {
@ -166,35 +180,8 @@ void BackupImpl::open(const ContextPtr & context)
if (open_mode == OpenMode::READ) if (open_mode == OpenMode::READ)
readBackupMetadata(); readBackupMetadata();
if (base_backup_info) if ((open_mode == OpenMode::WRITE) && base_backup_info)
{ base_backup_uuid = getBaseBackupUnlocked()->getUUID();
if (use_same_s3_credentials_for_base_backup)
backup_info.copyS3CredentialsTo(*base_backup_info);
BackupFactory::CreateParams params;
params.backup_info = *base_backup_info;
params.open_mode = OpenMode::READ;
params.context = context;
/// use_same_s3_credentials_for_base_backup should be inherited for base backups
params.use_same_s3_credentials_for_base_backup = use_same_s3_credentials_for_base_backup;
base_backup = BackupFactory::instance().createBackup(params);
if (open_mode == OpenMode::WRITE)
{
base_backup_uuid = base_backup->getUUID();
}
else if (base_backup_uuid != base_backup->getUUID())
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: The base backup {} has different UUID ({} != {})",
backup_name_for_logging,
base_backup->getNameForLogging(),
toString(base_backup->getUUID()),
(base_backup_uuid ? toString(*base_backup_uuid) : ""));
}
}
} }
void BackupImpl::close() void BackupImpl::close()
@ -239,6 +226,42 @@ void BackupImpl::closeArchive()
archive_writer.reset(); archive_writer.reset();
} }
std::shared_ptr<const IBackup> BackupImpl::getBaseBackup() const
{
std::lock_guard lock{mutex};
return getBaseBackupUnlocked();
}
std::shared_ptr<const IBackup> BackupImpl::getBaseBackupUnlocked() const
{
if (!base_backup && base_backup_info)
{
if (use_same_s3_credentials_for_base_backup)
backup_info.copyS3CredentialsTo(*base_backup_info);
BackupFactory::CreateParams params;
params.backup_info = *base_backup_info;
params.open_mode = OpenMode::READ;
params.context = context;
/// use_same_s3_credentials_for_base_backup should be inherited for base backups
params.use_same_s3_credentials_for_base_backup = use_same_s3_credentials_for_base_backup;
base_backup = BackupFactory::instance().createBackup(params);
if ((open_mode == OpenMode::READ) && (base_backup_uuid != base_backup->getUUID()))
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: The base backup {} has different UUID ({} != {})",
backup_name_for_logging,
base_backup->getNameForLogging(),
toString(base_backup->getUUID()),
(base_backup_uuid ? toString(*base_backup_uuid) : ""));
}
}
return base_backup;
}
size_t BackupImpl::getNumFiles() const size_t BackupImpl::getNumFiles() const
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
@ -289,8 +312,10 @@ UInt64 BackupImpl::getNumReadBytes() const
void BackupImpl::writeBackupMetadata() void BackupImpl::writeBackupMetadata()
{ {
assert(!is_internal_backup); LOG_TRACE(log, "Backup {}: Writing metadata", backup_name_for_logging);
auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::BackupWriteMetadataMicroseconds);
assert(!is_internal_backup);
checkLockFile(true); checkLockFile(true);
std::unique_ptr<WriteBuffer> out; std::unique_ptr<WriteBuffer> out;
@ -374,11 +399,16 @@ void BackupImpl::writeBackupMetadata()
out->finalize(); out->finalize();
uncompressed_size = size_of_entries + out->count(); uncompressed_size = size_of_entries + out->count();
LOG_TRACE(log, "Backup {}: Metadata was written", backup_name_for_logging);
} }
void BackupImpl::readBackupMetadata() void BackupImpl::readBackupMetadata()
{ {
LOG_TRACE(log, "Backup {}: Reading metadata", backup_name_for_logging);
auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::BackupReadMetadataMicroseconds);
using namespace XMLUtils; using namespace XMLUtils;
std::unique_ptr<ReadBuffer> in; std::unique_ptr<ReadBuffer> in;
@ -482,6 +512,8 @@ void BackupImpl::readBackupMetadata()
compressed_size = uncompressed_size; compressed_size = uncompressed_size;
if (!use_archive) if (!use_archive)
setCompressedSize(); setCompressedSize();
LOG_TRACE(log, "Backup {}: Metadata was read", backup_name_for_logging);
} }
void BackupImpl::checkBackupDoesntExist() const void BackupImpl::checkBackupDoesntExist() const
@ -705,7 +737,8 @@ std::unique_ptr<SeekableReadBuffer> BackupImpl::readFileImpl(const SizeAndChecks
if (info.base_size) if (info.base_size)
{ {
/// Make `base_read_buffer` if there is data for this backup entry in the base backup. /// Make `base_read_buffer` if there is data for this backup entry in the base backup.
if (!base_backup) auto base = getBaseBackup();
if (!base)
{ {
throw Exception( throw Exception(
ErrorCodes::NO_BASE_BACKUP, ErrorCodes::NO_BASE_BACKUP,
@ -713,7 +746,7 @@ std::unique_ptr<SeekableReadBuffer> BackupImpl::readFileImpl(const SizeAndChecks
backup_name_for_logging, formatSizeAndChecksum(size_and_checksum)); backup_name_for_logging, formatSizeAndChecksum(size_and_checksum));
} }
if (!base_backup->fileExists(std::pair(info.base_size, info.base_checksum))) if (!base->fileExists(std::pair(info.base_size, info.base_checksum)))
{ {
throw Exception( throw Exception(
ErrorCodes::WRONG_BASE_BACKUP, ErrorCodes::WRONG_BASE_BACKUP,
@ -721,7 +754,7 @@ std::unique_ptr<SeekableReadBuffer> BackupImpl::readFileImpl(const SizeAndChecks
backup_name_for_logging, formatSizeAndChecksum(size_and_checksum)); backup_name_for_logging, formatSizeAndChecksum(size_and_checksum));
} }
base_read_buffer = base_backup->readFile(std::pair{info.base_size, info.base_checksum}); base_read_buffer = base->readFile(std::pair{info.base_size, info.base_checksum});
} }
{ {
@ -809,7 +842,7 @@ size_t BackupImpl::copyFileToDisk(const SizeAndChecksum & size_and_checksum,
else if (info.size && (info.size == info.base_size)) else if (info.size && (info.size == info.base_size))
{ {
/// Data comes completely from the base backup (nothing comes from this backup). /// Data comes completely from the base backup (nothing comes from this backup).
base_backup->copyFileToDisk(std::pair{info.base_size, info.base_checksum}, destination_disk, destination_path, write_mode); getBaseBackup()->copyFileToDisk(std::pair{info.base_size, info.base_checksum}, destination_disk, destination_path, write_mode);
file_copied = true; file_copied = true;
} }

View File

@ -60,7 +60,7 @@ public:
OpenMode getOpenMode() const override { return open_mode; } OpenMode getOpenMode() const override { return open_mode; }
time_t getTimestamp() const override { return timestamp; } time_t getTimestamp() const override { return timestamp; }
UUID getUUID() const override { return *uuid; } UUID getUUID() const override { return *uuid; }
BackupPtr getBaseBackup() const override { return base_backup; } BackupPtr getBaseBackup() const override;
size_t getNumFiles() const override; size_t getNumFiles() const override;
UInt64 getTotalSize() const override; UInt64 getTotalSize() const override;
size_t getNumEntries() const override; size_t getNumEntries() const override;
@ -85,7 +85,7 @@ public:
bool supportsWritingInMultipleThreads() const override { return !use_archive; } bool supportsWritingInMultipleThreads() const override { return !use_archive; }
private: private:
void open(const ContextPtr & context); void open();
void close(); void close();
void openArchive(); void openArchive();
@ -95,6 +95,9 @@ private:
void writeBackupMetadata() TSA_REQUIRES(mutex); void writeBackupMetadata() TSA_REQUIRES(mutex);
void readBackupMetadata() TSA_REQUIRES(mutex); void readBackupMetadata() TSA_REQUIRES(mutex);
/// Returns the base backup or null if there is no base backup.
std::shared_ptr<const IBackup> getBaseBackupUnlocked() const TSA_REQUIRES(mutex);
/// Checks that a new backup doesn't exist yet. /// Checks that a new backup doesn't exist yet.
void checkBackupDoesntExist() const; void checkBackupDoesntExist() const;
@ -118,6 +121,7 @@ private:
const OpenMode open_mode; const OpenMode open_mode;
std::shared_ptr<IBackupWriter> writer; std::shared_ptr<IBackupWriter> writer;
std::shared_ptr<IBackupReader> reader; std::shared_ptr<IBackupReader> reader;
const ContextPtr context;
const bool is_internal_backup; const bool is_internal_backup;
std::shared_ptr<IBackupCoordination> coordination; std::shared_ptr<IBackupCoordination> coordination;
@ -138,8 +142,8 @@ private:
mutable size_t num_read_files = 0; mutable size_t num_read_files = 0;
mutable UInt64 num_read_bytes = 0; mutable UInt64 num_read_bytes = 0;
int version; int version;
std::optional<BackupInfo> base_backup_info; mutable std::optional<BackupInfo> base_backup_info;
std::shared_ptr<const IBackup> base_backup; mutable std::shared_ptr<const IBackup> base_backup;
std::optional<UUID> base_backup_uuid; std::optional<UUID> base_backup_uuid;
std::shared_ptr<IArchiveReader> archive_reader; std::shared_ptr<IArchiveReader> archive_reader;
std::shared_ptr<IArchiveWriter> archive_writer; std::shared_ptr<IArchiveWriter> archive_writer;

View File

@ -43,7 +43,7 @@ public:
/// Returns UUID of the backup. /// Returns UUID of the backup.
virtual UUID getUUID() const = 0; virtual UUID getUUID() const = 0;
/// Returns the base backup (can be null). /// Returns the base backup or null if there is no base backup.
virtual std::shared_ptr<const IBackup> getBaseBackup() const = 0; virtual std::shared_ptr<const IBackup> getBaseBackup() const = 0;
/// Returns the number of files stored in the backup. Compare with getNumEntries(). /// Returns the number of files stored in the backup. Compare with getNumEntries().

View File

@ -30,7 +30,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati
, port(connection_port.value_or(getPortFromConfig(config))) , port(connection_port.value_or(getPortFromConfig(config)))
{ {
bool is_secure = config.getBool("secure", false); bool is_secure = config.getBool("secure", false);
security = is_secure ? Protocol::Secure::Enable : Protocol::Secure::Disable; bool is_clickhouse_cloud = connection_host.ends_with(".clickhouse.cloud") || connection_host.ends_with(".clickhouse-staging.com");
security = (is_secure || is_clickhouse_cloud) ? Protocol::Secure::Enable : Protocol::Secure::Disable;
default_database = config.getString("database", ""); default_database = config.getString("database", "");

View File

@ -546,6 +546,10 @@ The server successfully detected this situation and will download merged part fr
M(IOUringCQEsCompleted, "Total number of successfully completed io_uring CQEs") \ M(IOUringCQEsCompleted, "Total number of successfully completed io_uring CQEs") \
M(IOUringCQEsFailed, "Total number of completed io_uring CQEs with failures") \ M(IOUringCQEsFailed, "Total number of completed io_uring CQEs with failures") \
\ \
M(BackupsOpenedForRead, "Number of backups opened for reading") \
M(BackupsOpenedForWrite, "Number of backups opened for writing") \
M(BackupReadMetadataMicroseconds, "Time spent reading backup metadata from .backup file") \
M(BackupWriteMetadataMicroseconds, "Time spent writing backup metadata to .backup file") \
M(BackupEntriesCollectorMicroseconds, "Time spent making backup entries") \ M(BackupEntriesCollectorMicroseconds, "Time spent making backup entries") \
M(BackupEntriesCollectorForTablesDataMicroseconds, "Time spent making backup entries for tables data") \ M(BackupEntriesCollectorForTablesDataMicroseconds, "Time spent making backup entries for tables data") \
M(BackupEntriesCollectorRunPostTasksMicroseconds, "Time spent running post tasks after making backup entries") \ M(BackupEntriesCollectorRunPostTasksMicroseconds, "Time spent running post tasks after making backup entries") \

View File

@ -7,6 +7,7 @@
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Common/randomSeed.h> #include <Common/randomSeed.h>
#include "Coordination/KeeperConstants.h" #include "Coordination/KeeperConstants.h"
#include <pcg_random.hpp>
namespace DB namespace DB
{ {
@ -42,7 +43,7 @@ public:
} }
private: private:
std::mt19937_64 rndgen; pcg64_fast rndgen;
std::bernoulli_distribution distribution; std::bernoulli_distribution distribution;
}; };

View File

@ -2,7 +2,9 @@
#include <ctime> #include <ctime>
#include <random> #include <random>
#include <thread> #include <thread>
#include <pcg_random.hpp>
#include <mysqlxx/PoolWithFailover.h> #include <mysqlxx/PoolWithFailover.h>
#include <Common/randomSeed.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <IO/Operators.h> #include <IO/Operators.h>
@ -44,10 +46,7 @@ PoolWithFailover::PoolWithFailover(
/// PoolWithFailover objects are stored in a cache inside PoolFactory. /// PoolWithFailover objects are stored in a cache inside PoolFactory.
/// This cache is reset by ExternalDictionariesLoader after every SYSTEM RELOAD DICTIONAR{Y|IES} /// This cache is reset by ExternalDictionariesLoader after every SYSTEM RELOAD DICTIONAR{Y|IES}
/// which triggers massive re-constructing of connection pools. /// which triggers massive re-constructing of connection pools.
/// The state of PRNGs like std::mt19937 is considered to be quite heavy static thread_local pcg64_fast rnd_generator(randomSeed());
/// thus here we attempt to optimize its construction.
static thread_local std::mt19937 rnd_generator(static_cast<uint_fast32_t>(
std::hash<std::thread::id>{}(std::this_thread::get_id()) + std::clock()));
for (auto & [_, replicas] : replicas_by_priority) for (auto & [_, replicas] : replicas_by_priority)
{ {
if (replicas.size() > 1) if (replicas.size() > 1)

View File

@ -13,10 +13,10 @@
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int CANNOT_CLOCK_GETTIME; extern const int CANNOT_CLOCK_GETTIME;
} }
} }

View File

@ -6,6 +6,7 @@
#include <Compression/CompressionFactory.h> #include <Compression/CompressionFactory.h>
#include <Compression/CompressionInfo.h> #include <Compression/CompressionInfo.h>
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <Common/randomSeed.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include "libaccel_config.h" #include "libaccel_config.h"
#include <Common/MemorySanitizer.h> #include <Common/MemorySanitizer.h>
@ -29,7 +30,7 @@ DeflateQplJobHWPool & DeflateQplJobHWPool::instance()
DeflateQplJobHWPool::DeflateQplJobHWPool() DeflateQplJobHWPool::DeflateQplJobHWPool()
: max_hw_jobs(0) : max_hw_jobs(0)
, random_engine(std::random_device()()) , random_engine(randomSeed())
{ {
Poco::Logger * log = &Poco::Logger::get("DeflateQplJobHWPool"); Poco::Logger * log = &Poco::Logger::get("DeflateQplJobHWPool");
const char * qpl_version = qpl_get_library_version(); const char * qpl_version = qpl_get_library_version();

View File

@ -3,6 +3,7 @@
#include <Compression/ICompressionCodec.h> #include <Compression/ICompressionCodec.h>
#include <map> #include <map>
#include <random> #include <random>
#include <pcg_random.hpp>
#include <qpl/qpl.h> #include <qpl/qpl.h>
namespace Poco namespace Poco
@ -41,7 +42,7 @@ private:
std::unique_ptr<std::atomic_bool[]> hw_job_ptr_locks; std::unique_ptr<std::atomic_bool[]> hw_job_ptr_locks;
bool job_pool_ready; bool job_pool_ready;
std::mt19937 random_engine; pcg64_fast random_engine;
std::uniform_int_distribution<int> distribution; std::uniform_int_distribution<int> distribution;
}; };

View File

@ -36,7 +36,7 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco
} }
const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc,clrs,ftfl"; const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc,clrs,ftfl,ydld";
KeeperConfigurationAndSettings::KeeperConfigurationAndSettings() KeeperConfigurationAndSettings::KeeperConfigurationAndSettings()
: server_id(NOT_EXIST) : server_id(NOT_EXIST)

View File

@ -172,6 +172,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat
FourLetterCommandPtr feature_flags_command = std::make_shared<FeatureFlagsCommand>(keeper_dispatcher); FourLetterCommandPtr feature_flags_command = std::make_shared<FeatureFlagsCommand>(keeper_dispatcher);
factory.registerCommand(feature_flags_command); factory.registerCommand(feature_flags_command);
FourLetterCommandPtr yield_leadership_command = std::make_shared<YieldLeadershipCommand>(keeper_dispatcher);
factory.registerCommand(yield_leadership_command);
factory.initializeAllowList(keeper_dispatcher); factory.initializeAllowList(keeper_dispatcher);
factory.setInitialize(true); factory.setInitialize(true);
} }
@ -579,4 +582,10 @@ String FeatureFlagsCommand::run()
return ret.str(); return ret.str();
} }
String YieldLeadershipCommand::run()
{
keeper_dispatcher.yieldLeadership();
return "Sent yield leadership request to leader.";
}
} }

View File

@ -415,4 +415,17 @@ struct FeatureFlagsCommand : public IFourLetterCommand
~FeatureFlagsCommand() override = default; ~FeatureFlagsCommand() override = default;
}; };
/// Yield leadership and become follower.
struct YieldLeadershipCommand : public IFourLetterCommand
{
explicit YieldLeadershipCommand(KeeperDispatcher & keeper_dispatcher_)
: IFourLetterCommand(keeper_dispatcher_)
{
}
String name() override { return "ydld"; }
String run() override;
~YieldLeadershipCommand() override = default;
};
} }

View File

@ -237,6 +237,12 @@ public:
return server->requestLeader(); return server->requestLeader();
} }
/// Yield leadership and become follower.
void yieldLeadership()
{
return server->yieldLeadership();
}
void recalculateStorageStats() void recalculateStorageStats()
{ {
return server->recalculateStorageStats(); return server->recalculateStorageStats();

View File

@ -1101,6 +1101,12 @@ bool KeeperServer::requestLeader()
return isLeader() || raft_instance->request_leadership(); return isLeader() || raft_instance->request_leadership();
} }
void KeeperServer::yieldLeadership()
{
if (isLeader())
raft_instance->yield_leadership();
}
void KeeperServer::recalculateStorageStats() void KeeperServer::recalculateStorageStats()
{ {
state_machine->recalculateStorageStats(); state_machine->recalculateStorageStats();

View File

@ -144,6 +144,8 @@ public:
bool requestLeader(); bool requestLeader();
void yieldLeadership();
void recalculateStorageStats(); void recalculateStorageStats();
}; };

View File

@ -26,12 +26,14 @@
#include <Interpreters/executeQuery.h> #include <Interpreters/executeQuery.h>
#include <Storages/StorageMergeTree.h> #include <Storages/StorageMergeTree.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Common/randomSeed.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <base/sleep.h> #include <base/sleep.h>
#include <boost/algorithm/string/split.hpp> #include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp> #include <boost/algorithm/string/trim.hpp>
#include <Parsers/CommonParsers.h> #include <Parsers/CommonParsers.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <pcg_random.hpp>
namespace DB namespace DB
{ {
@ -428,9 +430,8 @@ static inline void dumpDataForTables(
static inline UInt32 randomNumber() static inline UInt32 randomNumber()
{ {
std::mt19937 rng; pcg64_fast rng{randomSeed()};
rng.seed(std::random_device()()); std::uniform_int_distribution<pcg64_fast::result_type> dist6(
std::uniform_int_distribution<std::mt19937::result_type> dist6(
std::numeric_limits<UInt32>::min(), std::numeric_limits<UInt32>::max()); std::numeric_limits<UInt32>::min(), std::numeric_limits<UInt32>::max());
return static_cast<UInt32>(dist6(rng)); return static_cast<UInt32>(dist6(rng));
} }

View File

@ -4159,6 +4159,61 @@ arguments, result_type, input_rows_count); \
}; };
} }
template <typename EnumType>
WrapperType createEnumToStringWrapper() const
{
const char * function_name = cast_name;
return [function_name] (
ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/)
{
using ColumnEnumType = EnumType::ColumnType;
const auto & first_col = arguments.front().column.get();
const auto & first_type = arguments.front().type.get();
const ColumnEnumType * enum_col = typeid_cast<const ColumnEnumType *>(first_col);
const EnumType * enum_type = typeid_cast<const EnumType *>(first_type);
if (enum_col && nullable_col && nullable_col->size() != enum_col->size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "ColumnNullable is not compatible with original");
if (enum_col && enum_type)
{
const auto size = enum_col->size();
const auto & enum_data = enum_col->getData();
auto res = res_type->createColumn();
if (nullable_col)
{
for (size_t i = 0; i < size; ++i)
{
if (!nullable_col->isNullAt(i))
{
const auto & value = enum_type->getNameForValue(enum_data[i]);
res->insertData(value.data, value.size);
}
else
res->insertDefault();
}
}
else
{
for (size_t i = 0; i < size; ++i)
{
const auto & value = enum_type->getNameForValue(enum_data[i]);
res->insertData(value.data, value.size);
}
}
return res;
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected column {} as first argument of function {}",
first_col->getName(), function_name);
};
}
static WrapperType createIdentityWrapper(const DataTypePtr &) static WrapperType createIdentityWrapper(const DataTypePtr &)
{ {
return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/) return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/)
@ -4546,7 +4601,12 @@ arguments, result_type, input_rows_count); \
if constexpr (WhichDataType(ToDataType::type_id).isStringOrFixedString()) if constexpr (WhichDataType(ToDataType::type_id).isStringOrFixedString())
{ {
if (from_type->getCustomSerialization()) if constexpr (WhichDataType(FromDataType::type_id).isEnum())
{
ret = createEnumToStringWrapper<FromDataType>();
return true;
}
else if (from_type->getCustomSerialization())
{ {
ret = [](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr ret = [](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr
{ {

View File

@ -147,17 +147,18 @@ QueryCache::Key::Key(ASTPtr ast_, const String & user_name_)
{ {
} }
/// Hashing of ASTs must consider aliases (issue #56258)
constexpr bool ignore_aliases = false;
bool QueryCache::Key::operator==(const Key & other) const bool QueryCache::Key::operator==(const Key & other) const
{ {
return ast->getTreeHash() == other.ast->getTreeHash(); return ast->getTreeHash(ignore_aliases) == other.ast->getTreeHash(ignore_aliases);
} }
size_t QueryCache::KeyHasher::operator()(const Key & key) const size_t QueryCache::KeyHasher::operator()(const Key & key) const
{ {
SipHash hash; IAST::Hash hash = key.ast->getTreeHash(ignore_aliases);
hash.update(key.ast->getTreeHash()); return hash.low64;
auto res = hash.get64();
return res;
} }
size_t QueryCache::QueryCacheEntryWeight::operator()(const Entry & entry) const size_t QueryCache::QueryCacheEntryWeight::operator()(const Entry & entry) const

View File

@ -4,6 +4,7 @@
#include <Common/isLocalAddress.h> #include <Common/isLocalAddress.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/parseAddress.h> #include <Common/parseAddress.h>
#include <Common/randomSeed.h>
#include <Common/Config/AbstractConfigurationComparison.h> #include <Common/Config/AbstractConfigurationComparison.h>
#include <Common/Config/ConfigHelper.h> #include <Common/Config/ConfigHelper.h>
#include <Core/Settings.h> #include <Core/Settings.h>
@ -16,6 +17,7 @@
#include <boost/range/algorithm_ext/erase.hpp> #include <boost/range/algorithm_ext/erase.hpp>
#include <span> #include <span>
#include <pcg_random.hpp>
namespace DB namespace DB
{ {
@ -660,8 +662,7 @@ namespace
void shuffleReplicas(std::vector<Cluster::Address> & replicas, const Settings & settings, size_t replicas_needed) void shuffleReplicas(std::vector<Cluster::Address> & replicas, const Settings & settings, size_t replicas_needed)
{ {
std::random_device rd; pcg64_fast gen{randomSeed()};
std::mt19937 gen{rd()};
if (settings.prefer_localhost_replica) if (settings.prefer_localhost_replica)
{ {

View File

@ -14,24 +14,25 @@
/** /**
#include <Common/randomSeed.h>
#include <fstream> #include <fstream>
#include <random> #include <random>
#include <pcg_random.hpp>
using namespace std; using namespace std;
int main() int main()
{ {
std::string s; std::string s;
std::random_device dev; pcg64_fast rng{randomSeed()};
std::mt19937 rng(dev()); std::uniform_int_distribution<pcg64_fast::result_type> dist(0, 25);
std::uniform_int_distribution<std::mt19937::result_type> dist(0, 25); std::binomial_distribution<pcg64_fast::result_type> binomial1(100, 0.01);
std::binomial_distribution<std::mt19937::result_type> binomial1(100, 0.01); std::binomial_distribution<pcg64_fast::result_type> binomial2(100, 0.02);
std::binomial_distribution<std::mt19937::result_type> binomial2(100, 0.02); std::binomial_distribution<pcg64_fast::result_type> binomial4(100, 0.04);
std::binomial_distribution<std::mt19937::result_type> binomial4(100, 0.04); std::binomial_distribution<pcg64_fast::result_type> binomial8(100, 0.08);
std::binomial_distribution<std::mt19937::result_type> binomial8(100, 0.08); std::binomial_distribution<pcg64_fast::result_type> binomial16(100, 0.16);
std::binomial_distribution<std::mt19937::result_type> binomial16(100, 0.16); std::binomial_distribution<pcg64_fast::result_type> binomial24(100, 0.24);
std::binomial_distribution<std::mt19937::result_type> binomial24(100, 0.24); std::binomial_distribution<pcg64_fast::result_type> binomial48(100, 0.48);
std::binomial_distribution<std::mt19937::result_type> binomial48(100, 0.48);
// 11GB // 11GB
std::ofstream f("/tmp/terms.csv"); std::ofstream f("/tmp/terms.csv");
size_t l1, l2, l4, l8, l16, l24, l48; size_t l1, l2, l4, l8, l16, l24, l48;

View File

@ -46,11 +46,11 @@ void ASTColumnsRegexpMatcher::appendColumnName(WriteBuffer & ostr) const
writeChar(')', ostr); writeChar(')', ostr);
} }
void ASTColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state) const void ASTColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(original_pattern.size()); hash_state.update(original_pattern.size());
hash_state.update(original_pattern); hash_state.update(original_pattern);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
void ASTColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const void ASTColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
@ -201,11 +201,11 @@ const std::shared_ptr<re2::RE2> & ASTQualifiedColumnsRegexpMatcher::getMatcher()
return column_matcher; return column_matcher;
} }
void ASTQualifiedColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state) const void ASTQualifiedColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(original_pattern.size()); hash_state.update(original_pattern.size());
hash_state.update(original_pattern); hash_state.update(original_pattern);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const

View File

@ -27,7 +27,7 @@ public:
const String & getPattern() const; const String & getPattern() const;
const std::shared_ptr<re2::RE2> & getMatcher() const; const std::shared_ptr<re2::RE2> & getMatcher() const;
bool isColumnMatching(const String & column_name) const; bool isColumnMatching(const String & column_name) const;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
ASTPtr expression; ASTPtr expression;
ASTPtr transformers; ASTPtr transformers;
@ -65,7 +65,7 @@ public:
const std::shared_ptr<re2::RE2> & getMatcher() const; const std::shared_ptr<re2::RE2> & getMatcher() const;
void setPattern(String pattern, bool set_matcher = true); void setPattern(String pattern, bool set_matcher = true);
void setMatcher(std::shared_ptr<re2::RE2> matcher); void setMatcher(std::shared_ptr<re2::RE2> matcher);
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
ASTPtr qualifier; ASTPtr qualifier;
ASTPtr transformers; ASTPtr transformers;

View File

@ -151,15 +151,15 @@ void ASTColumnsApplyTransformer::appendColumnName(WriteBuffer & ostr) const
} }
} }
void ASTColumnsApplyTransformer::updateTreeHashImpl(SipHash & hash_state) const void ASTColumnsApplyTransformer::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(func_name.size()); hash_state.update(func_name.size());
hash_state.update(func_name); hash_state.update(func_name);
if (parameters) if (parameters)
parameters->updateTreeHashImpl(hash_state); parameters->updateTreeHashImpl(hash_state, ignore_aliases);
if (lambda) if (lambda)
lambda->updateTreeHashImpl(hash_state); lambda->updateTreeHashImpl(hash_state, ignore_aliases);
hash_state.update(lambda_arg.size()); hash_state.update(lambda_arg.size());
hash_state.update(lambda_arg); hash_state.update(lambda_arg);
@ -167,7 +167,7 @@ void ASTColumnsApplyTransformer::updateTreeHashImpl(SipHash & hash_state) const
hash_state.update(column_name_prefix.size()); hash_state.update(column_name_prefix.size());
hash_state.update(column_name_prefix); hash_state.update(column_name_prefix);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
@ -216,13 +216,13 @@ void ASTColumnsExceptTransformer::appendColumnName(WriteBuffer & ostr) const
writeChar(')', ostr); writeChar(')', ostr);
} }
void ASTColumnsExceptTransformer::updateTreeHashImpl(SipHash & hash_state) const void ASTColumnsExceptTransformer::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(is_strict); hash_state.update(is_strict);
hash_state.update(original_pattern.size()); hash_state.update(original_pattern.size());
hash_state.update(original_pattern); hash_state.update(original_pattern);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
void ASTColumnsExceptTransformer::transform(ASTs & nodes) const void ASTColumnsExceptTransformer::transform(ASTs & nodes) const
@ -312,14 +312,14 @@ void ASTColumnsReplaceTransformer::Replacement::appendColumnName(WriteBuffer & o
writeProbablyBackQuotedString(name, ostr); writeProbablyBackQuotedString(name, ostr);
} }
void ASTColumnsReplaceTransformer::Replacement::updateTreeHashImpl(SipHash & hash_state) const void ASTColumnsReplaceTransformer::Replacement::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
assert(children.size() == 1); assert(children.size() == 1);
hash_state.update(name.size()); hash_state.update(name.size());
hash_state.update(name); hash_state.update(name);
children[0]->updateTreeHashImpl(hash_state); children[0]->updateTreeHashImpl(hash_state, ignore_aliases);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
@ -361,10 +361,10 @@ void ASTColumnsReplaceTransformer::appendColumnName(WriteBuffer & ostr) const
writeChar(')', ostr); writeChar(')', ostr);
} }
void ASTColumnsReplaceTransformer::updateTreeHashImpl(SipHash & hash_state) const void ASTColumnsReplaceTransformer::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(is_strict); hash_state.update(is_strict);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
void ASTColumnsReplaceTransformer::replaceChildren(ASTPtr & node, const ASTPtr & replacement, const String & name) void ASTColumnsReplaceTransformer::replaceChildren(ASTPtr & node, const ASTPtr & replacement, const String & name)

View File

@ -48,7 +48,7 @@ public:
} }
void transform(ASTs & nodes) const override; void transform(ASTs & nodes) const override;
void appendColumnName(WriteBuffer & ostr) const override; void appendColumnName(WriteBuffer & ostr) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
// Case 1 APPLY (quantile(0.9)) // Case 1 APPLY (quantile(0.9))
String func_name; String func_name;
@ -80,7 +80,7 @@ public:
const std::shared_ptr<re2::RE2> & getMatcher() const; const std::shared_ptr<re2::RE2> & getMatcher() const;
bool isColumnMatching(const String & column_name) const; bool isColumnMatching(const String & column_name) const;
void appendColumnName(WriteBuffer & ostr) const override; void appendColumnName(WriteBuffer & ostr) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected: protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
@ -103,7 +103,7 @@ public:
} }
void appendColumnName(WriteBuffer & ostr) const override; void appendColumnName(WriteBuffer & ostr) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
String name; String name;
@ -121,7 +121,7 @@ public:
} }
void transform(ASTs & nodes) const override; void transform(ASTs & nodes) const override;
void appendColumnName(WriteBuffer & ostr) const override; void appendColumnName(WriteBuffer & ostr) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected: protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;

View File

@ -599,11 +599,11 @@ ASTPtr ASTFunction::clone() const
} }
void ASTFunction::updateTreeHashImpl(SipHash & hash_state) const void ASTFunction::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(name.size()); hash_state.update(name.size());
hash_state.update(name); hash_state.update(name);
IAST::updateTreeHashImpl(hash_state); ASTWithAlias::updateTreeHashImpl(hash_state, ignore_aliases);
} }
template <typename Container> template <typename Container>

View File

@ -63,7 +63,7 @@ public:
ASTPtr clone() const override; ASTPtr clone() const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
ASTSelectWithUnionQuery * tryGetQueryArgument() const; ASTSelectWithUnionQuery * tryGetQueryArgument() const;

View File

@ -53,12 +53,12 @@ bool ASTPair::hasSecretParts() const
} }
void ASTPair::updateTreeHashImpl(SipHash & hash_state) const void ASTPair::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(first.size()); hash_state.update(first.size());
hash_state.update(first); hash_state.update(first);
hash_state.update(second_with_brackets); hash_state.update(second_with_brackets);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
@ -92,12 +92,12 @@ void ASTFunctionWithKeyValueArguments::formatImpl(const FormatSettings & setting
} }
void ASTFunctionWithKeyValueArguments::updateTreeHashImpl(SipHash & hash_state) const void ASTFunctionWithKeyValueArguments::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(name.size()); hash_state.update(name.size());
hash_state.update(name); hash_state.update(name);
hash_state.update(has_brackets); hash_state.update(has_brackets);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
} }

View File

@ -32,7 +32,7 @@ public:
bool hasSecretParts() const override; bool hasSecretParts() const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
void forEachPointerToChild(std::function<void(void**)> f) override void forEachPointerToChild(std::function<void(void**)> f) override
{ {
@ -66,7 +66,7 @@ public:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
}; };
} }

View File

@ -87,6 +87,11 @@ void ASTIdentifier::setShortName(const String & new_name)
semantic->table = table; semantic->table = table;
} }
void ASTIdentifier::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{
ASTWithAlias::updateTreeHashImpl(hash_state, ignore_aliases);
}
const String & ASTIdentifier::name() const const String & ASTIdentifier::name() const
{ {
if (children.empty()) if (children.empty())
@ -244,10 +249,10 @@ void ASTTableIdentifier::resetTable(const String & database_name, const String &
uuid = identifier->uuid; uuid = identifier->uuid;
} }
void ASTTableIdentifier::updateTreeHashImpl(SipHash & hash_state) const void ASTTableIdentifier::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(uuid); hash_state.update(uuid);
IAST::updateTreeHashImpl(hash_state); ASTIdentifier::updateTreeHashImpl(hash_state, ignore_aliases);
} }
String getIdentifierName(const IAST * ast) String getIdentifierName(const IAST * ast)

View File

@ -47,6 +47,8 @@ public:
const String & shortName() const { return name_parts.back(); } const String & shortName() const { return name_parts.back(); }
const String & name() const; const String & name() const;
void updateTreeHashImpl(SipHash & hash_state, bool ignore_alias) const override;
void restoreTable(); // TODO(ilezhankin): get rid of this void restoreTable(); // TODO(ilezhankin): get rid of this
std::shared_ptr<ASTTableIdentifier> createTable() const; // returns |nullptr| if identifier is not table. std::shared_ptr<ASTTableIdentifier> createTable() const; // returns |nullptr| if identifier is not table.
@ -91,7 +93,7 @@ public:
// FIXME: used only when it's needed to rewrite distributed table name to real remote table name. // FIXME: used only when it's needed to rewrite distributed table name to real remote table name.
void resetTable(const String & database_name, const String & table_name); // TODO(ilezhankin): get rid of this void resetTable(const String & database_name, const String & table_name); // TODO(ilezhankin): get rid of this
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
}; };
} }

View File

@ -138,13 +138,13 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s
} }
} }
void ASTInsertQuery::updateTreeHashImpl(SipHash & hash_state) const void ASTInsertQuery::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(table_id.database_name); hash_state.update(table_id.database_name);
hash_state.update(table_id.table_name); hash_state.update(table_id.table_name);
hash_state.update(table_id.uuid); hash_state.update(table_id.uuid);
hash_state.update(format); hash_state.update(format);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }

View File

@ -72,7 +72,7 @@ public:
protected: protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
}; };
} }

View File

@ -10,11 +10,13 @@
namespace DB namespace DB
{ {
void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const void ASTLiteral::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
const char * prefix = "Literal_"; const char * prefix = "Literal_";
hash_state.update(prefix, strlen(prefix)); hash_state.update(prefix, strlen(prefix));
applyVisitor(FieldVisitorHash(hash_state), value); applyVisitor(FieldVisitorHash(hash_state), value);
if (!ignore_aliases)
ASTWithAlias::updateTreeHashImpl(hash_state, ignore_aliases);
} }
ASTPtr ASTLiteral::clone() const ASTPtr ASTLiteral::clone() const

View File

@ -41,7 +41,7 @@ public:
ASTPtr clone() const override; ASTPtr clone() const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected: protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;

View File

@ -7,13 +7,13 @@
namespace DB namespace DB
{ {
void ASTOrderByElement::updateTreeHashImpl(SipHash & hash_state) const void ASTOrderByElement::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(direction); hash_state.update(direction);
hash_state.update(nulls_direction); hash_state.update(nulls_direction);
hash_state.update(nulls_direction_was_explicitly_specified); hash_state.update(nulls_direction_was_explicitly_specified);
hash_state.update(with_fill); hash_state.update(with_fill);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const

View File

@ -32,7 +32,7 @@ public:
return clone; return clone;
} }
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected: protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;

View File

@ -23,4 +23,9 @@ void ASTQueryParameter::appendColumnNameImpl(WriteBuffer & ostr) const
writeString(name, ostr); writeString(name, ostr);
} }
void ASTQueryParameter::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{
ASTWithAlias::updateTreeHashImpl(hash_state, ignore_aliases);
}
} }

View File

@ -21,6 +21,8 @@ public:
ASTPtr clone() const override { return std::make_shared<ASTQueryParameter>(*this); } ASTPtr clone() const override { return std::make_shared<ASTQueryParameter>(*this); }
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected: protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override;

View File

@ -42,14 +42,14 @@ ASTPtr ASTSelectQuery::clone() const
} }
void ASTSelectQuery::updateTreeHashImpl(SipHash & hash_state) const void ASTSelectQuery::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(distinct); hash_state.update(distinct);
hash_state.update(group_by_with_totals); hash_state.update(group_by_with_totals);
hash_state.update(group_by_with_rollup); hash_state.update(group_by_with_rollup);
hash_state.update(group_by_with_cube); hash_state.update(group_by_with_cube);
hash_state.update(limit_with_ties); hash_state.update(limit_with_ties);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }

View File

@ -137,7 +137,7 @@ public:
void replaceDatabaseAndTable(const String & database_name, const String & table_name); void replaceDatabaseAndTable(const String & database_name, const String & table_name);
void replaceDatabaseAndTable(const StorageID & table_id); void replaceDatabaseAndTable(const StorageID & table_id);
void addTableFunction(ASTPtr & table_function_ptr); void addTableFunction(ASTPtr & table_function_ptr);
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
void setFinal(); void setFinal();

View File

@ -9,7 +9,7 @@
namespace DB namespace DB
{ {
void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state) const void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state, bool /*ignore_aliases*/) const
{ {
for (const auto & change : changes) for (const auto & change : changes)
{ {

View File

@ -34,7 +34,7 @@ public:
void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
QueryKind getQueryKind() const override { return QueryKind::Set; } QueryKind getQueryKind() const override { return QueryKind::Set; }

View File

@ -51,11 +51,11 @@ void ASTSubquery::formatImplWithoutAlias(const FormatSettings & settings, Format
settings.ostr << nl_or_nothing << indent_str << ")"; settings.ostr << nl_or_nothing << indent_str << ")";
} }
void ASTSubquery::updateTreeHashImpl(SipHash & hash_state) const void ASTSubquery::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
if (!cte_name.empty()) if (!cte_name.empty())
hash_state.update(cte_name); hash_state.update(cte_name);
IAST::updateTreeHashImpl(hash_state); ASTWithAlias::updateTreeHashImpl(hash_state, ignore_aliases);
} }
String ASTSubquery::getAliasOrColumnName() const String ASTSubquery::getAliasOrColumnName() const

View File

@ -14,7 +14,7 @@ class ASTSubquery : public ASTWithAlias
public: public:
// Stored the name when the subquery is defined in WITH clause. For example: // Stored the name when the subquery is defined in WITH clause. For example:
// WITH (SELECT 1) AS a SELECT * FROM a AS b; cte_name will be `a`. // WITH (SELECT 1) AS a SELECT * FROM a AS b; cte_name will be `a`.
std::string cte_name; String cte_name;
/** Get the text that identifies this element. */ /** Get the text that identifies this element. */
String getID(char) const override { return "Subquery"; } String getID(char) const override { return "Subquery"; }
@ -26,7 +26,7 @@ public:
return clone; return clone;
} }
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
String getAliasOrColumnName() const override; String getAliasOrColumnName() const override;
String tryGetAlias() const override; String tryGetAlias() const override;

View File

@ -21,10 +21,10 @@ do \
while (false) while (false)
void ASTTableExpression::updateTreeHashImpl(SipHash & hash_state) const void ASTTableExpression::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(final); hash_state.update(final);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
@ -42,12 +42,12 @@ ASTPtr ASTTableExpression::clone() const
return res; return res;
} }
void ASTTableJoin::updateTreeHashImpl(SipHash & hash_state) const void ASTTableJoin::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(locality); hash_state.update(locality);
hash_state.update(strictness); hash_state.update(strictness);
hash_state.update(kind); hash_state.update(kind);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
ASTPtr ASTTableJoin::clone() const ASTPtr ASTTableJoin::clone() const
@ -61,10 +61,10 @@ ASTPtr ASTTableJoin::clone() const
return res; return res;
} }
void ASTArrayJoin::updateTreeHashImpl(SipHash & hash_state) const void ASTArrayJoin::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(kind); hash_state.update(kind);
IAST::updateTreeHashImpl(hash_state); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
ASTPtr ASTArrayJoin::clone() const ASTPtr ASTArrayJoin::clone() const

View File

@ -59,7 +59,7 @@ struct ASTTableExpression : public IAST
String getID(char) const override { return "TableExpression"; } String getID(char) const override { return "TableExpression"; }
ASTPtr clone() const override; ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
}; };
@ -81,7 +81,7 @@ struct ASTTableJoin : public IAST
void formatImplBeforeTable(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const; void formatImplBeforeTable(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const;
void formatImplAfterTable(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const; void formatImplAfterTable(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
}; };
/// Specification of ARRAY JOIN. /// Specification of ARRAY JOIN.
@ -102,7 +102,7 @@ struct ASTArrayJoin : public IAST
String getID(char) const override { return "ArrayJoin"; } String getID(char) const override { return "ArrayJoin"; }
ASTPtr clone() const override; ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
}; };

View File

@ -39,7 +39,7 @@ IAST::QueryKind ASTTransactionControl::getQueryKind() const
} }
} }
void ASTTransactionControl::updateTreeHashImpl(SipHash & hash_state) const void ASTTransactionControl::updateTreeHashImpl(SipHash & hash_state, bool /*ignore_aliases*/) const
{ {
hash_state.update(action); hash_state.update(action);
} }

View File

@ -20,13 +20,13 @@ public:
UInt64 snapshot; /// For SET TRANSACTION SNAPSHOT ... UInt64 snapshot; /// For SET TRANSACTION SNAPSHOT ...
ASTTransactionControl(QueryType action_) : action(action_) {} explicit ASTTransactionControl(QueryType action_) : action(action_) {}
String getID(char /*delimiter*/) const override { return "ASTTransactionControl"; } String getID(char /*delimiter*/) const override { return "ASTTransactionControl"; }
ASTPtr clone() const override { return std::make_shared<ASTTransactionControl>(*this); } ASTPtr clone() const override { return std::make_shared<ASTTransactionControl>(*this); }
void formatImpl(const FormatSettings & format, FormatState & /*state*/, FormatStateStacked /*frame*/) const override; void formatImpl(const FormatSettings & format, FormatState & /*state*/, FormatStateStacked /*frame*/) const override;
void updateTreeHashImpl(SipHash & hash_state) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
QueryKind getQueryKind() const override; QueryKind getQueryKind() const override;
}; };

View File

@ -1,3 +1,4 @@
#include <Common/SipHash.h>
#include <Parsers/ASTWithAlias.h> #include <Parsers/ASTWithAlias.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/Operators.h> #include <IO/Operators.h>
@ -42,6 +43,13 @@ void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & sta
} }
} }
void ASTWithAlias::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{
if (!alias.empty() && !ignore_aliases)
hash_state.update(alias);
IAST::updateTreeHashImpl(hash_state, ignore_aliases);
}
void ASTWithAlias::appendColumnName(WriteBuffer & ostr) const void ASTWithAlias::appendColumnName(WriteBuffer & ostr) const
{ {
if (prefer_alias_to_column_name && !alias.empty()) if (prefer_alias_to_column_name && !alias.empty())

View File

@ -27,7 +27,9 @@ public:
void setAlias(const String & to) override { alias = to; } void setAlias(const String & to) override { alias = to; }
/// Calls formatImplWithoutAlias, and also outputs an alias. If necessary, encloses the entire expression in brackets. /// Calls formatImplWithoutAlias, and also outputs an alias. If necessary, encloses the entire expression in brackets.
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override final; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const final;
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
virtual void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const = 0; virtual void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const = 0;

View File

@ -114,24 +114,24 @@ size_t IAST::checkSize(size_t max_size) const
} }
IAST::Hash IAST::getTreeHash() const IAST::Hash IAST::getTreeHash(bool ignore_aliases) const
{ {
SipHash hash_state; SipHash hash_state;
updateTreeHash(hash_state); updateTreeHash(hash_state, ignore_aliases);
return getSipHash128AsPair(hash_state); return getSipHash128AsPair(hash_state);
} }
void IAST::updateTreeHash(SipHash & hash_state) const void IAST::updateTreeHash(SipHash & hash_state, bool ignore_aliases) const
{ {
updateTreeHashImpl(hash_state); updateTreeHashImpl(hash_state, ignore_aliases);
hash_state.update(children.size()); hash_state.update(children.size());
for (const auto & child : children) for (const auto & child : children)
child->updateTreeHash(hash_state); child->updateTreeHash(hash_state, ignore_aliases);
} }
void IAST::updateTreeHashImpl(SipHash & hash_state) const void IAST::updateTreeHashImpl(SipHash & hash_state, bool /*ignore_aliases*/) const
{ {
auto id = getID(); auto id = getID();
hash_state.update(id.data(), id.size()); hash_state.update(id.data(), id.size());

View File

@ -78,11 +78,13 @@ public:
virtual ASTPtr clone() const = 0; virtual ASTPtr clone() const = 0;
/** Get hash code, identifying this element and its subtree. /** Get hash code, identifying this element and its subtree.
* Hashing by default ignores aliases (e.g. identifier aliases, function aliases, literal aliases) which is
* useful for common subexpression elimination. Set 'ignore_aliases = false' if you don't want that behavior.
*/ */
using Hash = CityHash_v1_0_2::uint128; using Hash = CityHash_v1_0_2::uint128;
Hash getTreeHash() const; Hash getTreeHash(bool ignore_aliases = true) const;
void updateTreeHash(SipHash & hash_state) const; void updateTreeHash(SipHash & hash_state, bool ignore_aliases = true) const;
virtual void updateTreeHashImpl(SipHash & hash_state) const; virtual void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const;
void dumpTree(WriteBuffer & ostr, size_t indent = 0) const; void dumpTree(WriteBuffer & ostr, size_t indent = 0) const;
std::string dumpTree(size_t indent = 0) const; std::string dumpTree(size_t indent = 0) const;

View File

@ -7,9 +7,10 @@
#include "PostgreSQLHandler.h" #include "PostgreSQLHandler.h"
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Server/TCPServer.h> #include <Server/TCPServer.h>
#include <Common/randomSeed.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <base/scope_guard.h> #include <base/scope_guard.h>
#include <random> #include <pcg_random.hpp>
#include "config_version.h" #include "config_version.h"
@ -284,8 +285,7 @@ void PostgreSQLHandler::processQuery()
if (!parse_res.second) if (!parse_res.second)
throw Exception(ErrorCodes::SYNTAX_ERROR, "Cannot parse and execute the following part of query: {}", String(parse_res.first)); throw Exception(ErrorCodes::SYNTAX_ERROR, "Cannot parse and execute the following part of query: {}", String(parse_res.first));
std::random_device rd; pcg64_fast gen{randomSeed()};
std::mt19937 gen(rd());
std::uniform_int_distribution<Int32> dis(0, INT32_MAX); std::uniform_int_distribution<Int32> dis(0, INT32_MAX);
for (const auto & spl_query : queries) for (const auto & spl_query : queries)

View File

@ -82,6 +82,7 @@ ln -sf $SRC_PATH/users.d/marks.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/insert_keeper_retries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/insert_keeper_retries.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/prefetch_settings.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/prefetch_settings.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/nonconst_timezone.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/nonconst_timezone.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/allow_introspection_functions.yaml $DEST_SERVER_PATH/users.d/
if [[ -n "$USE_NEW_ANALYZER" ]] && [[ "$USE_NEW_ANALYZER" -eq 1 ]]; then if [[ -n "$USE_NEW_ANALYZER" ]] && [[ "$USE_NEW_ANALYZER" -eq 1 ]]; then
ln -sf $SRC_PATH/users.d/analyzer.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/analyzer.xml $DEST_SERVER_PATH/users.d/

View File

@ -0,0 +1,3 @@
profiles:
default:
allow_introspection_functions: 1

View File

@ -248,6 +248,11 @@ def is_leader(cluster, node, port=9181):
return "Mode: leader" in stat return "Mode: leader" in stat
def is_follower(cluster, node, port=9181):
stat = send_4lw_cmd(cluster, node, "stat", port)
return "Mode: follower" in stat
def get_leader(cluster, nodes): def get_leader(cluster, nodes):
for node in nodes: for node in nodes:
if is_leader(cluster, node): if is_leader(cluster, node):

View File

@ -725,3 +725,30 @@ def test_cmd_clrs(started_cluster):
finally: finally:
destroy_zk_client(zk) destroy_zk_client(zk)
def test_cmd_ydld(started_cluster):
wait_nodes()
for node in [node1, node3]:
data = keeper_utils.send_4lw_cmd(cluster, node, cmd="ydld")
assert data == "Sent yield leadership request to leader."
print("ydld output -------------------------------------")
print(data)
# Whenever there is a leader switch, there is a brief amount of time when any
# of the 4 letter commands will return empty result. Thus, we need to test for
# negative condition. So we can't use keeper_utils.is_leader() here and likewise
# in the while loop below.
if not keeper_utils.is_follower(cluster, node):
# wait for it to yield leadership
retry = 0
while not keeper_utils.is_follower(cluster, node) and retry < 30:
time.sleep(1)
retry += 1
if retry == 30:
print(
node.name
+ " did not become follower after 30s of yielding leadership, maybe there is something wrong."
)
assert keeper_utils.is_follower(cluster, node)

View File

@ -18,7 +18,7 @@ system stop distributed sends dist_01555;
insert into dist_01555 values (1)(2); insert into dist_01555 values (1)(2);
-- since test_cluster_with_incorrect_pw contains incorrect password ignore error -- since test_cluster_with_incorrect_pw contains incorrect password ignore error
system flush distributed dist_01555; -- { serverError 516 } system flush distributed dist_01555; -- { serverError 516 }
select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 5 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV;
drop table dist_01555; drop table dist_01555;
@ -31,7 +31,7 @@ create table dist_01555 (key Int) Engine=Distributed(test_cluster_with_incorrect
insert into dist_01555 values (1)(2); insert into dist_01555 values (1)(2);
-- since test_cluster_with_incorrect_pw contains incorrect password ignore error -- since test_cluster_with_incorrect_pw contains incorrect password ignore error
system flush distributed dist_01555; -- { serverError 516 } system flush distributed dist_01555; -- { serverError 516 }
select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 5 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 3600 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV;
drop table dist_01555; drop table dist_01555;

View File

@ -0,0 +1,24 @@
-- Bug 56258: Check literals (ASTLiteral)
Row 1:
──────
10: 10
Row 1:
──────
x: 10
2
-- Bug 56258: Check functions (ASTFunction)
Row 1:
──────
toUInt64(42): 42
Row 1:
──────
x: 42
2
-- Bug 56258: Check identifiers (ASTIdentifier)
Row 1:
──────
c: 1
Row 1:
──────
x: 1
2

View File

@ -0,0 +1,39 @@
-- Tags: no-parallel
-- Tag no-parallel: Messes with internal cache
-- Test for Bug 56258
SYSTEM DROP QUERY CACHE;
SELECT '-- Bug 56258: Check literals (ASTLiteral)';
SELECT 10 FORMAT Vertical SETTINGS use_query_cache = 1;
SELECT 10 AS x FORMAT Vertical SETTINGS use_query_cache = 1;
SELECT count(*) FROM system.query_cache;
SYSTEM DROP QUERY CACHE;
SELECT '-- Bug 56258: Check functions (ASTFunction)';
SELECT toUInt64(42) FORMAT Vertical SETTINGS use_query_cache = 1;
SELECT toUInt64(42) AS x FORMAT Vertical SETTINGS use_query_cache = 1;
SELECT count(*) FROM system.query_cache;
SYSTEM DROP QUERY CACHE;
SELECT '-- Bug 56258: Check identifiers (ASTIdentifier)';
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(c UInt64) ENGINE = Memory AS SELECT 1;
SELECT c FROM tab FORMAT Vertical SETTINGS use_query_cache = 1;
SELECT c AS x FROM tab FORMAT Vertical SETTINGS use_query_cache = 1;
SELECT count(*) FROM system.query_cache;
DROP TABLE tab;
SYSTEM DROP QUERY CACHE;

View File

@ -1,10 +0,0 @@
DROP TABLE IF EXISTS keeper_fault_inject_sequential_cleanup;
CREATE TABLE keeper_fault_inject_sequential_cleanup (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_02725/tables/keeper_fault_inject_sequential_cleanup', '1') ORDER BY d;
INSERT INTO keeper_fault_inject_sequential_cleanup VALUES (1);
INSERT INTO keeper_fault_inject_sequential_cleanup SETTINGS insert_deduplicate = 0 VALUES (1);
INSERT INTO keeper_fault_inject_sequential_cleanup SETTINGS insert_deduplicate = 0, insert_keeper_fault_injection_probability = 0.4, insert_keeper_fault_injection_seed = 5619964844601345291 VALUES (1);
-- with database ordinary it produced a warning
DROP TABLE keeper_fault_inject_sequential_cleanup;

View File

@ -1,14 +1,27 @@
use_same_s3_credentials_for_base_backup for S3 use_same_s3_credentials_for_base_backup for S3
base
BACKUP_CREATED BACKUP_CREATED
inc_1
BACKUP_CREATED BACKUP_CREATED
inc_2
BACKUP_CREATED
inc_3_bad
The request signature we calculated does not match the signature you provided. Check your key and signing method. (S3_ERROR) The request signature we calculated does not match the signature you provided. Check your key and signing method. (S3_ERROR)
inc_4
BACKUP_CREATED BACKUP_CREATED
restore inc_1
The request signature we calculated does not match the signature you provided. Check your key and signing method. (S3_ERROR) The request signature we calculated does not match the signature you provided. Check your key and signing method. (S3_ERROR)
restore inc_1
RESTORED RESTORED
restore inc_2
RESTORED RESTORED
use_same_s3_credentials_for_base_backup for S3 (invalid arguments) use_same_s3_credentials_for_base_backup for S3 (invalid arguments)
inc_5_bad
BACKUP_CREATED BACKUP_CREATED
inc_6_bad
NUMBER_OF_ARGUMENTS_DOESNT_MATCH NUMBER_OF_ARGUMENTS_DOESNT_MATCH
use_same_s3_credentials_for_base_backup for Disk use_same_s3_credentials_for_base_backup for Disk
backup_1
BACKUP_CREATED BACKUP_CREATED
backup_2
BAD_ARGUMENTS BAD_ARGUMENTS

View File

@ -22,24 +22,41 @@ function write_invalid_password_to_base_backup()
# Returns the arguments for the BACKUP TO S3() function, i.e. (url, access_key_id, secret_access_key) # Returns the arguments for the BACKUP TO S3() function, i.e. (url, access_key_id, secret_access_key)
function s3_location() { echo "'http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE/use_same_s3_credentials_for_base_backup_base_$*', 'test', 'testtest'"; } function s3_location() { echo "'http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE/use_same_s3_credentials_for_base_backup_base_$*', 'test', 'testtest'"; }
function s3_location_with_invalid_password() { echo "'http://localhost:11111/test/backups/$CLICKHOUSE_DATABASE/use_same_s3_credentials_for_base_backup_base_$*', 'test', 'INVALID_PASSWORD'"; }
echo 'use_same_s3_credentials_for_base_backup for S3' echo 'use_same_s3_credentials_for_base_backup for S3'
echo "base"
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location base))" | cut -f2 $CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location base))" | cut -f2
echo "inc_1"
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location inc_1)) SETTINGS base_backup=S3($(s3_location base))" | cut -f2 $CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location inc_1)) SETTINGS base_backup=S3($(s3_location base))" | cut -f2
write_invalid_password_to_base_backup inc_1 write_invalid_password_to_base_backup inc_1
$CLICKHOUSE_CLIENT --format Null -q "BACKUP TABLE data TO S3($(s3_location inc_2)) SETTINGS base_backup=S3($(s3_location inc_1))" |& grep -m1 -o 'The request signature we calculated does not match the signature you provided. Check your key and signing method. (S3_ERROR)'
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location inc_3)) SETTINGS base_backup=S3($(s3_location inc_1)), use_same_s3_credentials_for_base_backup=1" | cut -f2
echo "inc_2"
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location inc_2)) SETTINGS base_backup=S3($(s3_location inc_1))" | cut -f2
echo "inc_3_bad"
$CLICKHOUSE_CLIENT --format Null -q "BACKUP TABLE data TO S3($(s3_location inc_3_bad)) SETTINGS base_backup=S3($(s3_location_with_invalid_password inc_1))" |& grep -m1 -o 'The request signature we calculated does not match the signature you provided. Check your key and signing method. (S3_ERROR)'
echo "inc_4"
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location inc_4)) SETTINGS base_backup=S3($(s3_location_with_invalid_password inc_1)), use_same_s3_credentials_for_base_backup=1" | cut -f2
echo "restore inc_1"
$CLICKHOUSE_CLIENT --format Null -q "RESTORE TABLE data AS data FROM S3($(s3_location inc_1))" |& grep -m1 -o 'The request signature we calculated does not match the signature you provided. Check your key and signing method. (S3_ERROR)' $CLICKHOUSE_CLIENT --format Null -q "RESTORE TABLE data AS data FROM S3($(s3_location inc_1))" |& grep -m1 -o 'The request signature we calculated does not match the signature you provided. Check your key and signing method. (S3_ERROR)'
echo "restore inc_1"
$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM S3($(s3_location inc_1)) SETTINGS use_same_s3_credentials_for_base_backup=1" | cut -f2 $CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM S3($(s3_location inc_1)) SETTINGS use_same_s3_credentials_for_base_backup=1" | cut -f2
$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM S3($(s3_location inc_3)) SETTINGS use_same_s3_credentials_for_base_backup=1" | cut -f2 echo "restore inc_2"
$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM S3($(s3_location inc_2)) SETTINGS use_same_s3_credentials_for_base_backup=1" | cut -f2
echo 'use_same_s3_credentials_for_base_backup for S3 (invalid arguments)' echo 'use_same_s3_credentials_for_base_backup for S3 (invalid arguments)'
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location inc_4_bad)) SETTINGS base_backup=S3($(s3_location inc_1), 'foo'), use_same_s3_credentials_for_base_backup=1" |& cut -f2 echo "inc_5_bad"
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location inc_5_bad), 'foo') SETTINGS base_backup=S3($(s3_location inc_1)), use_same_s3_credentials_for_base_backup=1" |& grep -o -m1 NUMBER_OF_ARGUMENTS_DOESNT_MATCH $CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location inc_5_bad)) SETTINGS base_backup=S3($(s3_location inc_1), 'foo'), use_same_s3_credentials_for_base_backup=1" |& cut -f2
echo "inc_6_bad"
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO S3($(s3_location inc_6_bad), 'foo') SETTINGS base_backup=S3($(s3_location inc_1)), use_same_s3_credentials_for_base_backup=1" |& grep -o -m1 NUMBER_OF_ARGUMENTS_DOESNT_MATCH
echo 'use_same_s3_credentials_for_base_backup for Disk' echo 'use_same_s3_credentials_for_base_backup for Disk'
echo "backup_1"
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '$CLICKHOUSE_DATABASE/backup_1') SETTINGS use_same_s3_credentials_for_base_backup=1" | cut -f2 $CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '$CLICKHOUSE_DATABASE/backup_1') SETTINGS use_same_s3_credentials_for_base_backup=1" | cut -f2
echo "backup_2"
$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '$CLICKHOUSE_DATABASE/backup_2') SETTINGS use_same_s3_credentials_for_base_backup=1, base_backup=Disk('backups', '$CLICKHOUSE_DATABASE/backup_1')" |& grep -o -m1 BAD_ARGUMENTS $CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '$CLICKHOUSE_DATABASE/backup_2') SETTINGS use_same_s3_credentials_for_base_backup=1, base_backup=Disk('backups', '$CLICKHOUSE_DATABASE/backup_1')" |& grep -o -m1 BAD_ARGUMENTS
exit 0 exit 0

View File

@ -0,0 +1,4 @@
\N
\N
A
A

View File

@ -0,0 +1,4 @@
SELECT CAST(materialize(CAST(NULL, 'Nullable(Enum(\'A\' = 1, \'B\' = 2))')), 'Nullable(String)');
SELECT CAST(CAST(NULL, 'Nullable(Enum(\'A\' = 1, \'B\' = 2))'), 'Nullable(String)');
SELECT CAST(materialize(CAST(1, 'Nullable(Enum(\'A\' = 1, \'B\' = 2))')), 'Nullable(String)');
SELECT CAST(CAST(1, 'Nullable(Enum(\'A\' = 1, \'B\' = 2))'), 'Nullable(String)');

View File

@ -0,0 +1,12 @@
BACKUP_CREATED
BACKUP_CREATED
BACKUP_CREATED
RESTORED
RESTORED
RESTORED
a 0 1
b 1 1
c 1 1
r1 3 0
r2 2 0
r3 1 0

View File

@ -0,0 +1,85 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
a_backup_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_a
a_backup="Disk('backups', '$a_backup_id')"
b_backup_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_b
b_backup="Disk('backups', '$b_backup_id')"
c_backup_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_c
c_backup="Disk('backups', '$c_backup_id')"
${CLICKHOUSE_CLIENT} -nm --query "
DROP TABLE IF EXISTS tbl1;
DROP TABLE IF EXISTS tbl2;
DROP TABLE IF EXISTS tbl3;
"
${CLICKHOUSE_CLIENT} -nm --query "
CREATE TABLE tbl1 (a Int32) ENGINE = MergeTree() ORDER BY tuple();
"
# The following BACKUP command must write backup 'a'.
${CLICKHOUSE_CLIENT} -nm --query "
BACKUP DATABASE ${CLICKHOUSE_DATABASE} TO $a_backup SETTINGS id='$a_backup_id';
" | grep -o "BACKUP_CREATED"
${CLICKHOUSE_CLIENT} -nm --query "
CREATE TABLE tbl2 (a Int32) ENGINE = MergeTree() ORDER BY tuple();
"
# The following BACKUP command must read backup 'a' and write backup 'b'.
${CLICKHOUSE_CLIENT} -nm --query "
BACKUP DATABASE ${CLICKHOUSE_DATABASE} TO $b_backup SETTINGS id='$b_backup_id', base_backup=$a_backup;
" | grep -o "BACKUP_CREATED"
${CLICKHOUSE_CLIENT} -nm --query "
CREATE TABLE tbl3 (a Int32) ENGINE = MergeTree() ORDER BY tuple();
"
# The following BACKUP command must read only backup 'b' (and not 'a') and write backup 'c'.
${CLICKHOUSE_CLIENT} -nm --query "
BACKUP DATABASE ${CLICKHOUSE_DATABASE} TO $c_backup SETTINGS id='$c_backup_id', base_backup=$b_backup;
" | grep -o "BACKUP_CREATED"
${CLICKHOUSE_CLIENT} -nm --query "
DROP TABLE tbl1;
DROP TABLE tbl2;
DROP TABLE tbl3;
"
r1_restore_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_r1
r2_restore_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_r2
r3_restore_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_r3
# The following RESTORE command must read all 3 backups 'a', 'b', c' because the table 'tbl1' was in the first backup.
${CLICKHOUSE_CLIENT} -nm --query "
RESTORE TABLE ${CLICKHOUSE_DATABASE}.tbl1 FROM $c_backup SETTINGS id='$r1_restore_id';
" | grep -o "RESTORED"
# The following RESTORE command must read only 2 backups 'b', c' (and not 'a') because the table 'tbl2' was in the second backup.
${CLICKHOUSE_CLIENT} -nm --query "
RESTORE TABLE ${CLICKHOUSE_DATABASE}.tbl2 FROM $c_backup SETTINGS id='$r2_restore_id';
" | grep -o "RESTORED"
# The following RESTORE command must read only 1 backup 'c' (and not 'a' or 'b') because the table 'tbl3' was in the third backup.
${CLICKHOUSE_CLIENT} -nm --query "
RESTORE TABLE ${CLICKHOUSE_DATABASE}.tbl3 FROM $c_backup SETTINGS id='$r3_restore_id';
" | grep -o "RESTORED"
all_ids="['$a_backup_id', '$b_backup_id', '$c_backup_id', '$r1_restore_id', '$r2_restore_id', '$r3_restore_id']"
id_prefix_len=`expr "${CLICKHOUSE_TEST_UNIQUE_NAME}_" : '.*'`
${CLICKHOUSE_CLIENT} -nm --query "
SELECT substr(id, 1 + $id_prefix_len) as short_id, ProfileEvents['BackupsOpenedForRead'], ProfileEvents['BackupsOpenedForWrite'] FROM system.backups WHERE id IN ${all_ids} ORDER BY short_id
"
${CLICKHOUSE_CLIENT} -nm --query "
DROP TABLE tbl1;
DROP TABLE tbl2;
DROP TABLE tbl3;
"

View File

@ -394,6 +394,11 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS | grep -vP $EXCLUDE_DIRS |
xargs grep -P '__builtin_unreachable' && echo "Use UNREACHABLE() from defines.h instead" xargs grep -P '__builtin_unreachable' && echo "Use UNREACHABLE() from defines.h instead"
# Forbid mt19937() and random_device() which are outdated and slow
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -P '(std::mt19937|std::mersenne_twister_engine|std::random_device)' && echo "Use pcg64_fast (from pcg_random.h) and randomSeed (from Common/randomSeed.h) instead"
# Require checking return value of close(), # Require checking return value of close(),
# since it can hide fd misuse and break other places. # since it can hide fd misuse and break other places.
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |