Merge branch 'master' into case-insensitive-column-matching

This commit is contained in:
Antonio Andelic 2022-03-22 07:36:35 +00:00
commit 6cebb6bc88
141 changed files with 3949 additions and 2053 deletions

View File

@ -7,6 +7,6 @@ assignees: ''
---
> Make sure to check documentation https://clickhouse.yandex/docs/en/ first. If the question is concise and probably has a short answer, asking it in Telegram chat https://telegram.me/clickhouse_en is probably the fastest way to find the answer. For more complicated questions, consider asking them on StackOverflow with "clickhouse" tag https://stackoverflow.com/questions/tagged/clickhouse
> Make sure to check documentation https://clickhouse.com/docs/en/ first. If the question is concise and probably has a short answer, asking it in Telegram chat https://telegram.me/clickhouse_en is probably the fastest way to find the answer. For more complicated questions, consider asking them on StackOverflow with "clickhouse" tag https://stackoverflow.com/questions/tagged/clickhouse
> If you still prefer GitHub issues, remove all this text and ask your question here.

View File

@ -7,7 +7,7 @@ assignees: ''
---
> Make sure that `git diff` result is empty and you've just pulled fresh master. Try cleaning up cmake cache. Just in case, official build instructions are published here: https://clickhouse.yandex/docs/en/development/build/
> Make sure that `git diff` result is empty and you've just pulled fresh master. Try cleaning up cmake cache. Just in case, official build instructions are published here: https://clickhouse.com/docs/en/development/build/
**Operating system**

View File

@ -25,6 +25,7 @@ RUN apt-get update -y \
brotli
COPY ./stress /stress
COPY ./download_previous_release /download_previous_release
COPY run.sh /
ENV DATASETS="hits visits"

View File

@ -0,0 +1,97 @@
#!/usr/bin/env python3
import requests
import re
import os
CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags"
CLICKHOUSE_COMMON_STATIC_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-common-static_{version}_amd64.deb"
CLICKHOUSE_COMMON_STATIC_DBG_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-common-static-dbg_{version}_amd64.deb"
CLICKHOUSE_SERVER_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-server_{version}_all.deb"
CLICKHOUSE_CLIENT_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-client_{version}_amd64.deb"
CLICKHOUSE_COMMON_STATIC_PACKET_NAME = "clickhouse-common-static_{version}_amd64.deb"
CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME = "clickhouse-common-static-dbg_{version}_amd64.deb"
CLICKHOUSE_SERVER_PACKET_NAME = "clickhouse-server_{version}_all.deb"
CLICKHOUSE_CLIENT_PACKET_NAME = "clickhouse-client_{version}_all.deb"
PACKETS_DIR = "previous_release_package_folder/"
VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)"
class Version:
def __init__(self, version):
self.version = version
def __lt__(self, other):
return list(map(int, self.version.split('.'))) < list(map(int, other.version.split('.')))
def __str__(self):
return self.version
class ReleaseInfo:
def __init__(self, version, release_type):
self.version = version
self.type = release_type
def find_previous_release(server_version, releases):
releases.sort(key=lambda x: x.version, reverse=True)
for release in releases:
if release.version < server_version:
return True, release
return False, None
def get_previous_release(server_version):
page = 1
found = False
while not found:
response = requests.get(CLICKHOUSE_TAGS_URL, {'page': page, 'per_page': 100})
if not response.ok:
raise Exception('Cannot load the list of tags from github: ' + response.reason)
releases_str = set(re.findall(VERSION_PATTERN, response.text))
if len(releases_str) == 0:
raise Exception('Cannot find previous release for ' + str(server_version) + ' server version')
releases = list(map(lambda x: ReleaseInfo(Version(x.split('-')[0]), x.split('-')[1]), releases_str))
found, previous_release = find_previous_release(server_version, releases)
page += 1
return previous_release
def download_packet(url, local_file_name):
response = requests.get(url)
print(url)
if response.ok:
open(PACKETS_DIR + local_file_name, 'wb').write(response.content)
def download_packets(release):
if not os.path.exists(PACKETS_DIR):
os.makedirs(PACKETS_DIR)
download_packet(CLICKHOUSE_COMMON_STATIC_DOWNLOAD_URL.format(version=release.version, type=release.type),
CLICKHOUSE_COMMON_STATIC_PACKET_NAME.format(version=release.version))
download_packet(CLICKHOUSE_COMMON_STATIC_DBG_DOWNLOAD_URL.format(version=release.version, type=release.type),
CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME.format(version=release.version))
download_packet(CLICKHOUSE_SERVER_DOWNLOAD_URL.format(version=release.version, type=release.type),
CLICKHOUSE_SERVER_PACKET_NAME.format(version=release.version))
download_packet(CLICKHOUSE_CLIENT_DOWNLOAD_URL.format(version=release.version, type=release.type),
CLICKHOUSE_CLIENT_PACKET_NAME.format(version=release.version))
if __name__ == '__main__':
server_version = Version(input())
previous_release = get_previous_release(server_version)
download_packets(previous_release)

View File

@ -22,15 +22,19 @@ export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_PROBABILITY=0.001
export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_PROBABILITY=0.001
export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_PROBABILITY=0.001
export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_TIME_US=10000
export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_TIME_US=10000
export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_TIME_US=10000
export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000
dpkg -i package_folder/clickhouse-common-static_*.deb
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb
dpkg -i package_folder/clickhouse-server_*.deb
dpkg -i package_folder/clickhouse-client_*.deb
function install_packages()
{
dpkg -i $1/clickhouse-common-static_*.deb
dpkg -i $1/clickhouse-common-static-dbg_*.deb
dpkg -i $1/clickhouse-server_*.deb
dpkg -i $1/clickhouse-client_*.deb
}
function configure()
{
@ -116,7 +120,7 @@ function start()
counter=0
until clickhouse-client --query "SELECT 1"
do
if [ "$counter" -gt 240 ]
if [ "$counter" -gt ${1:-240} ]
then
echo "Cannot start clickhouse-server"
cat /var/log/clickhouse-server/stdout.log
@ -127,6 +131,9 @@ function start()
# use root to match with current uid
clickhouse start --user root >/var/log/clickhouse-server/stdout.log 2>>/var/log/clickhouse-server/stderr.log
sleep 0.5
cat /var/log/clickhouse-server/stdout.log
tail -n200 /var/log/clickhouse-server/stderr.log
tail -n200 /var/log/clickhouse-server/clickhouse-server.log
counter=$((counter + 1))
done
@ -171,6 +178,8 @@ quit
time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||:
}
install_packages package_folder
configure
./setup_minio.sh
@ -246,6 +255,120 @@ zgrep -Fa "########################################" /test_output/* > /dev/null
zgrep -Fa " received signal " /test_output/gdb.log > /dev/null \
&& echo -e 'Found signal in gdb.log\tFAIL' >> /test_output/test_results.tsv
echo -e "Backward compatibility check\n"
echo "Download previous release server"
mkdir previous_release_package_folder
clickhouse-client --query="SELECT version()" | ./download_previous_release && echo -e 'Download script exit code\tOK' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Download script failed\tFAIL' >> /test_output/backward_compatibility_check_results.tsv
if [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ]
then
echo -e "Successfully downloaded previous release packets\tOK" >> /test_output/backward_compatibility_check_results.tsv
stop
# Uninstall current packages
dpkg --remove clickhouse-client
dpkg --remove clickhouse-server
dpkg --remove clickhouse-common-static-dbg
dpkg --remove clickhouse-common-static
rm -rf /var/lib/clickhouse/*
# Install previous release packages
install_packages previous_release_package_folder
# Start server from previous release
configure
start
clickhouse-client --query="SELECT 'Server version: ', version()"
# Install new package before running stress test because we should use new clickhouse-client and new clickhouse-test
install_packages package_folder
mkdir tmp_stress_output
./stress --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \
&& echo -e 'Test script exit code\tOK' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Test script failed\tFAIL' >> /test_output/backward_compatibility_check_results.tsv
rm -rf tmp_stress_output
clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables"
stop
# Start new server
configure
start 500
clickhouse-client --query "SELECT 'Server successfully started', 'OK'" >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Server failed to start\tFAIL' >> /test_output/backward_compatibility_check_results.tsv
clickhouse-client --query="SELECT 'Server version: ', version()"
# Let the server run for a while before checking log.
sleep 60
stop
# Error messages (we should ignore some errors)
zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \
-e "Code: 236. DB::Exception: Cancelled mutating parts" \
-e "REPLICA_IS_ALREADY_ACTIVE" \
-e "REPLICA_IS_ALREADY_EXIST" \
-e "DDLWorker: Cannot parse DDL task query" \
-e "RaftInstance: failed to accept a rpc connection due to error 125" \
-e "UNKNOWN_DATABASE" \
-e "NETWORK_ERROR" \
-e "UNKNOWN_TABLE" \
-e "ZooKeeperClient" \
-e "KEEPER_EXCEPTION" \
-e "DirectoryMonitor" \
-e "TABLE_IS_READ_ONLY" \
-e "Code: 1000, e.code() = 111, Connection refused" \
-e "UNFINISHED" \
-e "Renaming unexpected part" \
/var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "<Error>" > /dev/null \
&& echo -e 'Error message in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No Error messages in clickhouse-server.log\tOK' >> /test_output/backward_compatibility_check_results.tsv
# Sanitizer asserts
zgrep -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
zgrep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
zgrep -Fav "ASan doesn't fully support makecontext/swapcontext functions" /test_output/tmp > /dev/null \
&& echo -e 'Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No sanitizer asserts\tOK' >> /test_output/backward_compatibility_check_results.tsv
rm -f /test_output/tmp
# OOM
zgrep -Fa " <Fatal> Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No OOM messages in clickhouse-server.log\tOK' >> /test_output/backward_compatibility_check_results.tsv
# Logical errors
zgrep -Fa "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Logical error thrown (see clickhouse-server.log)\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No logical errors\tOK' >> /test_output/backward_compatibility_check_results.tsv
# Crash
zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Not crashed\tOK' >> /test_output/backward_compatibility_check_results.tsv
# It also checks for crash without stacktrace (printed by watchdog)
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Fatal message in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No fatal messages in clickhouse-server.log\tOK' >> /test_output/backward_compatibility_check_results.tsv
else
echo -e "Failed to download previous release packets\tFAIL" >> /test_output/backward_compatibility_check_results.tsv
fi
zgrep -Fa "FAIL" /test_output/backward_compatibility_check_results.tsv > /dev/null \
&& echo -e 'Backward compatibility check\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check\tOK' >> /test_output/test_results.tsv
# Put logs into /test_output/
for log_file in /var/log/clickhouse-server/clickhouse-server.log*
do

View File

@ -47,7 +47,8 @@ def get_options(i):
return ' '.join(options)
def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_time_limit):
def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_time_limit, backward_compatibility_check):
backward_compatibility_check_option = '--backward-compatibility-check' if backward_compatibility_check else ''
global_time_limit_option = ''
if global_time_limit:
global_time_limit_option = "--global_time_limit={}".format(global_time_limit)
@ -56,7 +57,7 @@ def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_t
pipes = []
for i in range(0, len(output_paths)):
f = open(output_paths[i], 'w')
full_command = "{} {} {} {}".format(cmd, get_options(i), global_time_limit_option, skip_tests_option)
full_command = "{} {} {} {} {}".format(cmd, get_options(i), global_time_limit_option, skip_tests_option, backward_compatibility_check_option)
logging.info("Run func tests '%s'", full_command)
p = Popen(full_command, shell=True, stdout=f, stderr=f)
pipes.append(p)
@ -168,6 +169,7 @@ if __name__ == "__main__":
parser.add_argument("--output-folder")
parser.add_argument("--global-time-limit", type=int, default=1800)
parser.add_argument("--num-parallel", type=int, default=cpu_count())
parser.add_argument('--backward-compatibility-check', action='store_true')
parser.add_argument('--hung-check', action='store_true', default=False)
# make sense only for hung check
parser.add_argument('--drop-databases', action='store_true', default=False)
@ -176,7 +178,7 @@ if __name__ == "__main__":
if args.drop_databases and not args.hung_check:
raise Exception("--drop-databases only used in hung check (--hung-check)")
func_pipes = []
func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests, args.global_time_limit)
func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests, args.global_time_limit, args.backward_compatibility_check)
logging.info("Will wait functests to finish")
while True:

View File

@ -402,7 +402,7 @@ Parsing allows the presence of the additional field `tskv` without the equal sig
Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)).
When formatting, rows are enclosed in double-quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double-quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first, the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double-quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
When formatting, strings are enclosed in double-quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double-quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first, the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double-quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
``` bash
$ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv
@ -410,7 +410,7 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR
\*By default, the delimiter is `,`. See the [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) setting for more information.
When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to the delimiter character or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) types are all supported.
When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Strings can also be arranged without quotes. In this case, they are parsed up to the delimiter character or line feed (CR or LF). In violation of the RFC, when parsing strings without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) types are all supported.
If setting [input_format_csv_empty_as_default](../operations/settings/settings.md#settings-input_format_csv_empty_as_default) is enabled,
empty unquoted input values are replaced with default values. For complex default expressions [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#settings-input_format_defaults_for_omitted_fields) must be enabled too.

View File

@ -820,6 +820,7 @@ void Client::addOptions(OptionsDescription & options_description)
("opentelemetry-tracestate", po::value<std::string>(), "OpenTelemetry tracestate header as described by W3C Trace Context recommendation")
("no-warnings", "disable warnings when client connects to server")
("fake-drop", "Ignore all DROP queries, should be used only for testing")
;
/// Commandline options related to external tables.
@ -952,6 +953,8 @@ void Client::processOptions(const OptionsDescription & options_description,
config().setBool("compression", options["compression"].as<bool>());
if (options.count("no-warnings"))
config().setBool("no-warnings", true);
if (options.count("fake-drop"))
fake_drop = true;
if ((query_fuzzer_runs = options["query-fuzzer-runs"].as<int>()))
{

View File

@ -0,0 +1,112 @@
#include <Backups/ArchiveBackup.h>
#include <Disks/IDisk.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/WriteBufferFromFileBase.h>
#include <IO/Archives/IArchiveReader.h>
#include <IO/Archives/IArchiveWriter.h>
#include <IO/Archives/createArchiveReader.h>
#include <IO/Archives/createArchiveWriter.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
ArchiveBackup::ArchiveBackup(
const String & backup_name_,
const DiskPtr & disk_,
const String & path_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_)
: BackupImpl(backup_name_, context_, base_backup_info_), disk(disk_), path(path_)
{
}
ArchiveBackup::~ArchiveBackup()
{
close();
}
bool ArchiveBackup::backupExists() const
{
return disk ? disk->exists(path) : fs::exists(path);
}
void ArchiveBackup::openImpl(OpenMode open_mode_)
{
/// mutex is already locked
if (open_mode_ == OpenMode::WRITE)
{
if (disk)
writer = createArchiveWriter(path, disk->writeFile(path));
else
writer = createArchiveWriter(path);
writer->setCompression(compression_method, compression_level);
writer->setPassword(password);
}
else if (open_mode_ == OpenMode::READ)
{
if (disk)
{
auto archive_read_function = [d = disk, p = path]() -> std::unique_ptr<SeekableReadBuffer> { return d->readFile(p); };
size_t archive_size = disk->getFileSize(path);
reader = createArchiveReader(path, archive_read_function, archive_size);
}
else
reader = createArchiveReader(path);
reader->setPassword(password);
}
}
void ArchiveBackup::closeImpl(bool writing_finalized_)
{
/// mutex is already locked
if (writer && writer->isWritingFile())
throw Exception("There is some writing unfinished on close", ErrorCodes::LOGICAL_ERROR);
writer.reset();
reader.reset();
if ((getOpenModeNoLock() == OpenMode::WRITE) && !writing_finalized_)
fs::remove(path);
}
std::unique_ptr<ReadBuffer> ArchiveBackup::readFileImpl(const String & file_name) const
{
/// mutex is already locked
return reader->readFile(file_name);
}
std::unique_ptr<WriteBuffer> ArchiveBackup::writeFileImpl(const String & file_name)
{
/// mutex is already locked
return writer->writeFile(file_name);
}
void ArchiveBackup::setCompression(const String & compression_method_, int compression_level_)
{
std::lock_guard lock{mutex};
compression_method = compression_method_;
compression_level = compression_level_;
if (writer)
writer->setCompression(compression_method, compression_level);
}
void ArchiveBackup::setPassword(const String & password_)
{
std::lock_guard lock{mutex};
password = password_;
if (writer)
writer->setPassword(password);
if (reader)
reader->setPassword(password);
}
}

View File

@ -0,0 +1,52 @@
#pragma once
#include <Backups/BackupImpl.h>
namespace DB
{
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
class IArchiveReader;
class IArchiveWriter;
/// Stores a backup as a single .zip file.
class ArchiveBackup : public BackupImpl
{
public:
/// `disk`_ is allowed to be nullptr and that means the `path_` is a path in the local filesystem.
ArchiveBackup(
const String & backup_name_,
const DiskPtr & disk_,
const String & path_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_ = {});
~ArchiveBackup() override;
static constexpr const int kDefaultCompressionLevel = -1;
/// Sets compression method and level.
void setCompression(const String & compression_method_, int compression_level_ = kDefaultCompressionLevel);
/// Sets password.
void setPassword(const String & password_);
private:
bool backupExists() const override;
void openImpl(OpenMode open_mode_) override;
void closeImpl(bool writing_finalized_) override;
bool supportsWritingInMultipleThreads() const override { return false; }
std::unique_ptr<ReadBuffer> readFileImpl(const String & file_name) const override;
std::unique_ptr<WriteBuffer> writeFileImpl(const String & file_name) override;
const DiskPtr disk;
const String path;
std::shared_ptr<IArchiveReader> reader;
std::shared_ptr<IArchiveWriter> writer;
String compression_method;
int compression_level = kDefaultCompressionLevel;
String password;
};
}

View File

@ -1,28 +0,0 @@
#include <Backups/BackupEntryConcat.h>
#include <IO/ConcatReadBuffer.h>
namespace DB
{
BackupEntryConcat::BackupEntryConcat(
BackupEntryPtr first_source_,
BackupEntryPtr second_source_,
const std::optional<UInt128> & checksum_)
: first_source(std::move(first_source_))
, second_source(std::move(second_source_))
, checksum(checksum_)
{
}
UInt64 BackupEntryConcat::getSize() const
{
if (!size)
size = first_source->getSize() + second_source->getSize();
return *size;
}
std::unique_ptr<ReadBuffer> BackupEntryConcat::getReadBuffer() const
{
return std::make_unique<ConcatReadBuffer>(*first_source->getReadBuffer(), *second_source->getReadBuffer());
}
}

View File

@ -1,30 +0,0 @@
#pragma once
#include <Backups/IBackupEntry.h>
namespace DB
{
/// Concatenates data of two backup entries.
class BackupEntryConcat : public IBackupEntry
{
public:
/// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data.
BackupEntryConcat(
BackupEntryPtr first_source_,
BackupEntryPtr second_source_,
const std::optional<UInt128> & checksum_ = {});
UInt64 getSize() const override;
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<ReadBuffer> getReadBuffer() const override;
private:
BackupEntryPtr first_source;
BackupEntryPtr second_source;
mutable std::optional<UInt64> size;
std::optional<UInt128> checksum;
};
}

View File

@ -29,7 +29,7 @@ BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile(
std::unique_ptr<ReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer() const
{
auto buf = BackupEntryFromImmutableFile::getReadBuffer();
return std::make_unique<LimitReadBuffer>(std::move(buf), limit, true);
return std::make_unique<LimitReadBuffer>(std::move(buf), limit, false);
}
}

View File

@ -1,31 +0,0 @@
#pragma once
#include <Backups/IBackupEntry.h>
namespace DB
{
/// Represents small preloaded data to be included in a backup.
class BackupEntryFromCallback : public IBackupEntry
{
public:
using ReadBufferCreator = std::function<std::unique_ptr<ReadBuffer>()>;
/// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data.
BackupEntryFromCallback(const ReadBufferCreator & callback_, size_t size_, const std::optional<UInt128> & checksum_ = {})
: callback(callback_), size(size_), checksum(checksum_)
{
}
UInt64 getSize() const override { return size; }
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<ReadBuffer> getReadBuffer() const override { return callback(); }
private:
const ReadBufferCreator callback;
const size_t size;
const std::optional<UInt128> checksum;
};
}

View File

@ -21,7 +21,9 @@ BackupMutablePtr BackupFactory::createBackup(const CreateParams & params) const
auto it = creators.find(engine_name);
if (it == creators.end())
throw Exception(ErrorCodes::BACKUP_ENGINE_NOT_FOUND, "Not found backup engine {}", engine_name);
return (it->second)(params);
BackupMutablePtr backup = (it->second)(params);
backup->open(params.open_mode);
return backup;
}
void BackupFactory::registerBackupEngine(const String & engine_name, const CreatorFn & creator_fn)
@ -31,7 +33,12 @@ void BackupFactory::registerBackupEngine(const String & engine_name, const Creat
creators[engine_name] = creator_fn;
}
void registerBackupEngines(BackupFactory & factory);
void registerBackupEnginesFileAndDisk(BackupFactory &);
void registerBackupEngines(BackupFactory & factory)
{
registerBackupEnginesFileAndDisk(factory);
}
BackupFactory::BackupFactory()
{

View File

@ -26,6 +26,9 @@ public:
OpenMode open_mode = OpenMode::WRITE;
BackupInfo backup_info;
std::optional<BackupInfo> base_backup_info;
String compression_method;
int compression_level = -1;
String password;
ContextPtr context;
};

View File

@ -1,13 +1,11 @@
#include <Backups/BackupImpl.h>
#include <Backups/BackupFactory.h>
#include <Backups/BackupEntryConcat.h>
#include <Backups/BackupEntryFromCallback.h>
#include <Backups/BackupEntryFromMemory.h>
#include <Backups/IBackupEntry.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/hex.h>
#include <Common/typeid_cast.h>
#include <Common/quoteString.h>
#include <IO/ConcatReadBuffer.h>
#include <IO/HashingReadBuffer.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/ReadHelpers.h>
@ -47,34 +45,82 @@ namespace
}
}
BackupImpl::BackupImpl(const String & backup_name_, OpenMode open_mode_, const ContextPtr & context_, const std::optional<BackupInfo> & base_backup_info_)
: backup_name(backup_name_), open_mode(open_mode_), context(context_), base_backup_info(base_backup_info_)
class BackupImpl::BackupEntryFromBackupImpl : public IBackupEntry
{
public:
BackupEntryFromBackupImpl(
const std::shared_ptr<const BackupImpl> & backup_,
const String & file_name_,
UInt64 size_,
const std::optional<UInt128> checksum_,
BackupEntryPtr base_backup_entry_ = {})
: backup(backup_), file_name(file_name_), size(size_), checksum(checksum_),
base_backup_entry(std::move(base_backup_entry_))
{
}
std::unique_ptr<ReadBuffer> getReadBuffer() const override
{
auto read_buffer = backup->readFileImpl(file_name);
if (base_backup_entry)
{
auto base_backup_read_buffer = base_backup_entry->getReadBuffer();
read_buffer = std::make_unique<ConcatReadBuffer>(std::move(base_backup_read_buffer), std::move(read_buffer));
}
return read_buffer;
}
UInt64 getSize() const override { return size; }
std::optional<UInt128> getChecksum() const override { return checksum; }
private:
const std::shared_ptr<const BackupImpl> backup;
const String file_name;
const UInt64 size;
const std::optional<UInt128> checksum;
BackupEntryPtr base_backup_entry;
};
BackupImpl::BackupImpl(const String & backup_name_, const ContextPtr & context_, const std::optional<BackupInfo> & base_backup_info_)
: backup_name(backup_name_), context(context_), base_backup_info_param(base_backup_info_)
{
}
BackupImpl::~BackupImpl() = default;
void BackupImpl::open()
void BackupImpl::open(OpenMode open_mode_)
{
if (open_mode == OpenMode::WRITE)
std::lock_guard lock{mutex};
if (open_mode == open_mode_)
return;
if (open_mode != OpenMode::NONE)
throw Exception("Backup is already opened", ErrorCodes::LOGICAL_ERROR);
if (open_mode_ == OpenMode::WRITE)
{
if (backupExists())
throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", getName());
timestamp = std::time(nullptr);
uuid = UUIDHelpers::generateV4();
startWriting();
writing_started = true;
writing_finalized = false;
}
if (open_mode == OpenMode::READ)
if (open_mode_ == OpenMode::READ)
{
if (!backupExists())
throw Exception(ErrorCodes::BACKUP_NOT_FOUND, "Backup {} not found", getName());
readBackupMetadata();
}
openImpl(open_mode_);
base_backup_info = base_backup_info_param;
if (open_mode_ == OpenMode::READ)
readBackupMetadata();
if (base_backup_info)
{
BackupFactory::CreateParams params;
@ -83,25 +129,43 @@ void BackupImpl::open()
params.context = context;
base_backup = BackupFactory::instance().createBackup(params);
if (open_mode == OpenMode::WRITE)
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 ({} != {})",
getName(), base_backup->getName(), toString(base_backup->getUUID()), (base_backup_uuid ? toString(*base_backup_uuid) : ""));
}
open_mode = open_mode_;
}
void BackupImpl::close()
{
if (open_mode == OpenMode::WRITE)
{
if (writing_started && !writing_finalized)
{
/// Creating of the backup wasn't finished correctly,
/// so the backup cannot be used and it's better to remove its files.
removeAllFilesAfterFailure();
}
}
std::lock_guard lock{mutex};
if (open_mode == OpenMode::NONE)
return;
closeImpl(writing_finalized);
uuid = UUIDHelpers::Nil;
timestamp = 0;
base_backup_info.reset();
base_backup.reset();
base_backup_uuid.reset();
file_infos.clear();
open_mode = OpenMode::NONE;
}
IBackup::OpenMode BackupImpl::getOpenMode() const
{
std::lock_guard lock{mutex};
return open_mode;
}
time_t BackupImpl::getTimestamp() const
{
std::lock_guard lock{mutex};
return timestamp;
}
void BackupImpl::writeBackupMetadata()
@ -112,9 +176,20 @@ void BackupImpl::writeBackupMetadata()
config->setString("uuid", toString(uuid));
if (base_backup_info)
config->setString("base_backup", base_backup_info->toString());
if (base_backup_uuid)
config->setString("base_backup_uuid", toString(*base_backup_uuid));
{
bool base_backup_in_use = false;
for (const auto & [name, info] : file_infos)
{
if (info.base_size)
base_backup_in_use = true;
}
if (base_backup_in_use)
{
config->setString("base_backup", base_backup_info->toString());
config->setString("base_backup_uuid", toString(*base_backup_uuid));
}
}
size_t index = 0;
for (const auto & [name, info] : file_infos)
@ -128,7 +203,7 @@ void BackupImpl::writeBackupMetadata()
if (info.base_size)
{
config->setUInt(prefix + "base_size", info.base_size);
if (info.base_size != info.size)
if (info.base_checksum != info.checksum)
config->setString(prefix + "base_checksum", getHexUIntLowercase(info.base_checksum));
}
}
@ -138,7 +213,7 @@ void BackupImpl::writeBackupMetadata()
std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
config->save(stream);
String str = stream.str();
auto out = addFileImpl(".backup");
auto out = writeFileImpl(".backup");
out->write(str.data(), str.size());
}
@ -161,7 +236,7 @@ void BackupImpl::readBackupMetadata()
if (config->has("base_backup") && !base_backup_info)
base_backup_info = BackupInfo::fromString(config->getString("base_backup"));
if (config->has("base_backup_uuid") && !base_backup_uuid)
if (config->has("base_backup_uuid"))
base_backup_uuid = parse<UUID>(config->getString("base_backup_uuid"));
file_infos.clear();
@ -173,20 +248,22 @@ void BackupImpl::readBackupMetadata()
{
String prefix = "contents." + key + ".";
String name = config->getString(prefix + "name");
FileInfo & info = file_infos.emplace(name, FileInfo{}).first->second;
FileInfo info;
info.size = config->getUInt(prefix + "size");
if (info.size)
{
info.checksum = unhexChecksum(config->getString(prefix + "checksum"));
if (config->has(prefix + "base_size"))
info.base_size = config->getUInt(prefix + "base_size", 0);
if (info.base_size)
{
info.base_size = config->getUInt(prefix + "base_size");
if (info.base_size == info.size)
info.base_checksum = info.checksum;
else
if (config->has(prefix + "base_checksum"))
info.base_checksum = unhexChecksum(config->getString(prefix + "base_checksum"));
else
info.base_checksum = info.checksum;
}
}
file_infos.emplace(name, info);
file_checksums.emplace(info.checksum, name);
}
}
}
@ -240,10 +317,22 @@ UInt128 BackupImpl::getFileChecksum(const String & file_name) const
return it->second.checksum;
}
std::optional<String> BackupImpl::findFileByChecksum(const UInt128 & checksum) const
{
std::lock_guard lock{mutex};
auto it = file_checksums.find(checksum);
if (it == file_checksums.end())
return std::nullopt;
return it->second;
}
BackupEntryPtr BackupImpl::readFile(const String & file_name) const
{
std::lock_guard lock{mutex};
if (open_mode != OpenMode::READ)
throw Exception("Backup is not opened for reading", ErrorCodes::LOGICAL_ERROR);
auto it = file_infos.find(file_name);
if (it == file_infos.end())
throw Exception(
@ -264,7 +353,8 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
if (!info.base_size)
{
/// Data goes completely from this backup, the base backup isn't used.
return std::make_unique<BackupEntryFromCallback>(read_callback, info.size, info.checksum);
return std::make_unique<BackupEntryFromBackupImpl>(
std::static_pointer_cast<const BackupImpl>(shared_from_this()), file_name, info.size, info.checksum);
}
if (info.size < info.base_size)
@ -283,7 +373,8 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
getName(), quoteString(file_name));
}
if (!base_backup->fileExists(file_name))
auto base_file_name = base_backup->findFileByChecksum(info.base_checksum);
if (!base_file_name)
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
@ -291,7 +382,7 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
getName(), quoteString(file_name));
}
auto base_entry = base_backup->readFile(file_name);
auto base_entry = base_backup->readFile(*base_file_name);
auto base_size = base_entry->getSize();
if (base_size != info.base_size)
{
@ -301,15 +392,6 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
getName(), quoteString(file_name), base_backup->getName(), base_size, info.base_size);
}
auto base_checksum = base_entry->getChecksum();
if (base_checksum && (*base_checksum != info.base_checksum))
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: Entry {} has unexpected checksum in the base backup {}",
getName(), quoteString(file_name), base_backup->getName());
}
if (info.size == info.base_size)
{
/// Data goes completely from the base backup (nothing goes from this backup).
@ -318,18 +400,16 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
/// The beginning of the data goes from the base backup,
/// and the ending goes from this backup.
return std::make_unique<BackupEntryConcat>(
std::move(base_entry),
std::make_unique<BackupEntryFromCallback>(read_callback, info.size - info.base_size),
info.checksum);
return std::make_unique<BackupEntryFromBackupImpl>(
static_pointer_cast<const BackupImpl>(shared_from_this()), file_name, info.size, info.checksum, std::move(base_entry));
}
void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry)
void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
{
std::lock_guard lock{mutex};
if (open_mode != OpenMode::WRITE)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal operation: Cannot write to a backup opened for reading");
throw Exception("Backup is not opened for writing", ErrorCodes::LOGICAL_ERROR);
if (file_infos.contains(file_name))
throw Exception(
@ -356,44 +436,60 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry)
}
std::unique_ptr<ReadBuffer> read_buffer; /// We'll set that later.
UInt64 read_pos = 0; /// Current position in read_buffer.
std::optional<HashingReadBuffer> hashing_read_buffer;
UInt64 hashing_pos = 0; /// Current position in `hashing_read_buffer`.
/// Determine whether it's possible to receive this entry's data from the base backup completely or partly.
bool use_base = false;
if (base_exists && base_size)
if (base_exists && base_size && (size >= base_size))
{
if (size == base_size)
if (checksum && (size == base_size))
{
/// The size is the same, we need to compare checksums to find out
/// if the entry's data has not been changed since the base backup.
if (!checksum)
{
read_buffer = entry->getReadBuffer();
HashingReadBuffer hashing_read_buffer{*read_buffer};
hashing_read_buffer.ignore(size);
read_pos = size;
checksum = hashing_read_buffer.getHash();
}
if (checksum == base_checksum)
use_base = true; /// The data has not been changed.
/// if the entry's data has not changed since the base backup.
use_base = (*checksum == base_checksum);
}
else if (size > base_size)
else
{
/// The size has been increased, we need to calculate a partial checksum to find out
/// if the entry's data has been only appended since the base backup.
/// The size has increased, we need to calculate a partial checksum to find out
/// if the entry's data has only appended since the base backup.
read_buffer = entry->getReadBuffer();
HashingReadBuffer hashing_read_buffer{*read_buffer};
hashing_read_buffer.ignore(base_size);
UInt128 partial_checksum = hashing_read_buffer.getHash();
read_pos = base_size;
if (!checksum)
{
hashing_read_buffer.ignore(size - base_size);
checksum = hashing_read_buffer.getHash();
read_pos = size;
}
hashing_read_buffer.emplace(*read_buffer);
hashing_read_buffer->ignore(base_size);
hashing_pos = base_size;
UInt128 partial_checksum = hashing_read_buffer->getHash();
if (size == base_size)
checksum = partial_checksum;
if (partial_checksum == base_checksum)
use_base = true; /// The data has been appended.
use_base = true;
}
}
/// Finish calculating the checksum.
if (!checksum)
{
if (!read_buffer)
read_buffer = entry->getReadBuffer();
if (!hashing_read_buffer)
hashing_read_buffer.emplace(*read_buffer);
hashing_read_buffer->ignore(size - hashing_pos);
checksum = hashing_read_buffer->getHash();
}
hashing_read_buffer.reset();
/// Check if a entry with the same checksum exists in the base backup.
if (base_backup && !use_base)
{
if (auto base_file_name = base_backup->findFileByChecksum(*checksum))
{
if (size == base_backup->getFileSize(*base_file_name))
{
/// The entry's data has not changed since the base backup,
/// but the entry itself has been moved or renamed.
base_size = size;
base_checksum = *checksum;
use_base = true;
}
}
}
@ -401,74 +497,61 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry)
{
/// The entry's data has not been changed since the base backup.
FileInfo info;
info.size = base_size;
info.checksum = base_checksum;
info.size = size;
info.checksum = *checksum;
info.base_size = base_size;
info.base_checksum = base_checksum;
file_infos.emplace(file_name, info);
file_checksums.emplace(*checksum, file_name);
return;
}
/// Either the entry wasn't exist in the base backup
/// or the entry has data appended to the end of the data from the base backup.
/// In both those cases we have to copy data to this backup.
/// Find out where the start position to copy data is.
auto copy_pos = use_base ? base_size : 0;
/// Move the current read position to the start position to copy data.
/// If `read_buffer` is seekable it's easier, otherwise we can use ignore().
if (auto * seekable_buffer = dynamic_cast<SeekableReadBuffer *>(read_buffer.get()))
{
/// Either the entry wasn't exist in the base backup
/// or the entry has data appended to the end of the data from the base backup.
/// In both those cases we have to copy data to this backup.
/// Find out where the start position to copy data is.
auto copy_pos = use_base ? base_size : 0;
/// Move the current read position to the start position to copy data.
/// If `read_buffer` is seekable it's easier, otherwise we can use ignore().
if (auto * seekable_buffer = dynamic_cast<SeekableReadBuffer *>(read_buffer.get()))
{
if (read_pos != copy_pos)
seekable_buffer->seek(copy_pos, SEEK_SET);
}
else
{
if (read_pos > copy_pos)
{
read_buffer.reset();
read_pos = 0;
}
if (!read_buffer)
read_buffer = entry->getReadBuffer();
if (read_pos < copy_pos)
read_buffer->ignore(copy_pos - read_pos);
}
/// If we haven't received or calculated a checksum yet, calculate it now.
ReadBuffer * maybe_hashing_read_buffer = read_buffer.get();
std::optional<HashingReadBuffer> hashing_read_buffer;
if (!checksum)
maybe_hashing_read_buffer = &hashing_read_buffer.emplace(*read_buffer);
/// Copy the entry's data after `copy_pos`.
auto out = addFileImpl(file_name);
copyData(*maybe_hashing_read_buffer, *out);
if (hashing_read_buffer)
checksum = hashing_read_buffer->getHash();
/// Done!
FileInfo info;
info.size = size;
info.checksum = *checksum;
if (use_base)
{
info.base_size = base_size;
info.base_checksum = base_checksum;
}
file_infos.emplace(file_name, info);
seekable_buffer->seek(copy_pos, SEEK_SET);
}
else
{
read_buffer = entry->getReadBuffer();
read_buffer->ignore(copy_pos);
}
/// Copy the entry's data after `copy_pos`.
auto out = writeFileImpl(file_name);
copyData(*read_buffer, *out);
/// Done!
FileInfo info;
info.size = size;
info.checksum = *checksum;
if (use_base)
{
info.base_size = base_size;
info.base_checksum = base_checksum;
}
file_infos.emplace(file_name, info);
file_checksums.emplace(*checksum, file_name);
}
void BackupImpl::finalizeWriting()
{
std::lock_guard lock{mutex};
if (writing_finalized)
return;
if (open_mode != OpenMode::WRITE)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal operation: Cannot write to a backup opened for reading");
throw Exception("Backup is not opened for writing", ErrorCodes::LOGICAL_ERROR);
writeBackupMetadata();
writing_finalized = true;
}

View File

@ -4,6 +4,7 @@
#include <Backups/BackupInfo.h>
#include <map>
#include <mutex>
#include <unordered_map>
namespace DB
@ -20,29 +21,33 @@ class BackupImpl : public IBackup
public:
BackupImpl(
const String & backup_name_,
OpenMode open_mode_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_ = {});
~BackupImpl() override;
const String & getName() const override { return backup_name; }
OpenMode getOpenMode() const override { return open_mode; }
time_t getTimestamp() const override { return timestamp; }
void open(OpenMode open_mode_) override;
OpenMode getOpenMode() const override;
void close() override;
time_t getTimestamp() const override;
UUID getUUID() const override { return uuid; }
Strings listFiles(const String & prefix, const String & terminator) const override;
bool fileExists(const String & file_name) const override;
size_t getFileSize(const String & file_name) const override;
UInt128 getFileChecksum(const String & file_name) const override;
std::optional<String> findFileByChecksum(const UInt128 & checksum) const override;
BackupEntryPtr readFile(const String & file_name) const override;
void addFile(const String & file_name, BackupEntryPtr entry) override;
void writeFile(const String & file_name, BackupEntryPtr entry) override;
void finalizeWriting() override;
protected:
/// Should be called in the constructor of a derived class.
void open();
/// Checks if this backup exists.
virtual bool backupExists() const = 0;
/// Should be called in the destructor of a derived class.
void close();
virtual void openImpl(OpenMode open_mode_) = 0;
OpenMode getOpenModeNoLock() const { return open_mode; }
virtual void closeImpl(bool writing_finalized_) = 0;
/// Read a file from the backup.
/// Low level: the function doesn't check base backup or checksums.
@ -50,18 +55,9 @@ protected:
/// Add a file to the backup.
/// Low level: the function doesn't check base backup or checksums.
virtual std::unique_ptr<WriteBuffer> addFileImpl(const String & file_name) = 0;
virtual std::unique_ptr<WriteBuffer> writeFileImpl(const String & file_name) = 0;
/// Checks if this backup exists.
virtual bool backupExists() const = 0;
/// Starts writing of this backup, only used if `open_mode == OpenMode::WRITE`.
/// After calling this function `backupExists()` should return true.
virtual void startWriting() = 0;
/// Removes all the backup files, called if something goes wrong while we're writing the backup.
/// This function is called by `close()` if `startWriting()` was called and `finalizeWriting()` wasn't.
virtual void removeAllFilesAfterFailure() = 0;
mutable std::mutex mutex;
private:
void writeBackupMetadata();
@ -77,18 +73,20 @@ private:
UInt128 base_checksum{0, 0};
};
class BackupEntryFromBackupImpl;
const String backup_name;
const OpenMode open_mode;
UUID uuid;
time_t timestamp = 0;
ContextPtr context;
const std::optional<BackupInfo> base_backup_info_param;
OpenMode open_mode = OpenMode::NONE;
UUID uuid = {};
time_t timestamp = 0;
std::optional<BackupInfo> base_backup_info;
std::shared_ptr<const IBackup> base_backup;
std::optional<UUID> base_backup_uuid;
std::map<String, FileInfo> file_infos;
bool writing_started = false;
std::map<String, FileInfo> file_infos; /// Should be ordered alphabetically, see listFiles().
std::unordered_map<UInt128, String> file_checksums;
bool writing_finalized = false;
mutable std::mutex mutex;
};
}

View File

@ -1,160 +0,0 @@
#include <Backups/BackupInDirectory.h>
#include <Backups/BackupFactory.h>
#include <Common/quoteString.h>
#include <Disks/DiskSelector.h>
#include <Disks/IDisk.h>
#include <Disks/DiskLocal.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
/// Checks multiple keys "key", "key[1]", "key[2]", and so on in the configuration
/// and find out if some of them have matching value.
bool findConfigKeyWithMatchingValue(const Poco::Util::AbstractConfiguration & config, const String & key, const std::function<bool(const String & value)> & match_function)
{
String current_key = key;
size_t counter = 0;
while (config.has(current_key))
{
if (match_function(config.getString(current_key)))
return true;
current_key = key + "[" + std::to_string(++counter) + "]";
}
return false;
}
bool isDiskAllowed(const String & disk_name, const Poco::Util::AbstractConfiguration & config)
{
return findConfigKeyWithMatchingValue(config, "backups.allowed_disk", [&](const String & value) { return value == disk_name; });
}
bool isPathAllowed(const String & path, const Poco::Util::AbstractConfiguration & config)
{
return findConfigKeyWithMatchingValue(config, "backups.allowed_path", [&](const String & value) { return path.starts_with(value); });
}
}
BackupInDirectory::BackupInDirectory(
const String & backup_name_,
OpenMode open_mode_,
const DiskPtr & disk_,
const String & path_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_)
: BackupImpl(backup_name_, open_mode_, context_, base_backup_info_)
, disk(disk_), path(path_)
{
/// Path to backup must end with '/'
if (path.back() != '/')
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to backup must end with '/', but {} doesn't.", getName(), quoteString(path));
dir_path = fs::path(path).parent_path(); /// get path without terminating slash
/// If `disk` is not specified, we create an internal instance of `DiskLocal` here.
if (!disk)
{
auto fspath = fs::path{dir_path};
if (!fspath.has_filename())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to a backup must be a directory path.", getName(), quoteString(path));
path = fspath.filename() / "";
dir_path = fs::path(path).parent_path(); /// get path without terminating slash
String disk_path = fspath.remove_filename();
disk = std::make_shared<DiskLocal>(disk_path, disk_path, 0);
}
open();
}
BackupInDirectory::~BackupInDirectory()
{
close();
}
bool BackupInDirectory::backupExists() const
{
return disk->isDirectory(dir_path);
}
void BackupInDirectory::startWriting()
{
disk->createDirectories(dir_path);
}
void BackupInDirectory::removeAllFilesAfterFailure()
{
if (disk->isDirectory(dir_path))
disk->removeRecursive(dir_path);
}
std::unique_ptr<ReadBuffer> BackupInDirectory::readFileImpl(const String & file_name) const
{
String file_path = path + file_name;
return disk->readFile(file_path);
}
std::unique_ptr<WriteBuffer> BackupInDirectory::addFileImpl(const String & file_name)
{
String file_path = path + file_name;
disk->createDirectories(fs::path(file_path).parent_path());
return disk->writeFile(file_path);
}
void registerBackupEngineFile(BackupFactory & factory)
{
auto creator_fn = [](const BackupFactory::CreateParams & params)
{
String backup_name = params.backup_info.toString();
const String & engine_name = params.backup_info.backup_engine_name;
const auto & args = params.backup_info.args;
DiskPtr disk;
String path;
if (engine_name == "File")
{
if (args.size() != 1)
{
throw Exception(
"Backup engine 'File' requires 1 argument (path)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
path = args[0].safeGet<String>();
if (!isPathAllowed(path, params.context->getConfigRef()))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} is not allowed for backups", path);
}
else if (engine_name == "Disk")
{
if (args.size() != 2)
{
throw Exception(
"Backup engine 'Disk' requires 2 arguments (disk_name, path)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
String disk_name = args[0].safeGet<String>();
disk = params.context->getDisk(disk_name);
path = args[1].safeGet<String>();
if (!isDiskAllowed(disk_name, params.context->getConfigRef()))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} is not allowed for backups", disk_name);
}
return std::make_shared<BackupInDirectory>(backup_name, params.open_mode, disk, path, params.context, params.base_backup_info);
};
factory.registerBackupEngine("File", creator_fn);
factory.registerBackupEngine("Disk", creator_fn);
}
}

View File

@ -1,89 +0,0 @@
#include <Backups/BackupRenamingConfig.h>
#include <Parsers/ASTBackupQuery.h>
namespace DB
{
using Kind = ASTBackupQuery::Kind;
using ElementType = ASTBackupQuery::ElementType;
void BackupRenamingConfig::setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name)
{
old_to_new_table_names[old_table_name] = new_table_name;
}
void BackupRenamingConfig::setNewDatabaseName(const String & old_database_name, const String & new_database_name)
{
old_to_new_database_names[old_database_name] = new_database_name;
}
void BackupRenamingConfig::setNewTemporaryTableName(const String & old_temporary_table_name, const String & new_temporary_table_name)
{
old_to_new_temporary_table_names[old_temporary_table_name] = new_temporary_table_name;
}
void BackupRenamingConfig::setFromBackupQuery(const ASTBackupQuery & backup_query)
{
setFromBackupQueryElements(backup_query.elements);
}
void BackupRenamingConfig::setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements)
{
for (const auto & element : backup_query_elements)
{
switch (element.type)
{
case ElementType::TABLE: [[fallthrough]];
case ElementType::DICTIONARY:
{
const auto & new_name = element.new_name.second.empty() ? element.name : element.new_name;
setNewTableName(element.name, new_name);
break;
}
case ASTBackupQuery::DATABASE:
{
const auto & new_name = element.new_name.first.empty() ? element.name.first : element.new_name.first;
setNewDatabaseName(element.name.first, new_name);
break;
}
case ASTBackupQuery::TEMPORARY_TABLE:
{
const auto & new_name = element.new_name.second.empty() ? element.name.second : element.new_name.second;
setNewTemporaryTableName(element.name.second, new_name);
break;
}
case ASTBackupQuery::ALL_DATABASES: break;
case ASTBackupQuery::ALL_TEMPORARY_TABLES: break;
case ASTBackupQuery::EVERYTHING: break;
}
}
}
DatabaseAndTableName BackupRenamingConfig::getNewTableName(const DatabaseAndTableName & old_table_name) const
{
auto it = old_to_new_table_names.find(old_table_name);
if (it != old_to_new_table_names.end())
return it->second;
return {getNewDatabaseName(old_table_name.first), old_table_name.second};
}
const String & BackupRenamingConfig::getNewDatabaseName(const String & old_database_name) const
{
auto it = old_to_new_database_names.find(old_database_name);
if (it != old_to_new_database_names.end())
return it->second;
return old_database_name;
}
const String & BackupRenamingConfig::getNewTemporaryTableName(const String & old_temporary_table_name) const
{
auto it = old_to_new_temporary_table_names.find(old_temporary_table_name);
if (it != old_to_new_temporary_table_names.end())
return it->second;
return old_temporary_table_name;
}
}

View File

@ -1,39 +0,0 @@
#pragma once
#include <Parsers/ASTBackupQuery.h>
#include <Core/Types.h>
#include <map>
#include <unordered_map>
namespace DB
{
using DatabaseAndTableName = std::pair<String, String>;
/// Keeps information about renamings of databases or tables being processed
/// while we're making a backup or while we're restoring from a backup.
class BackupRenamingConfig
{
public:
BackupRenamingConfig() = default;
void setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name);
void setNewDatabaseName(const String & old_database_name, const String & new_database_name);
void setNewTemporaryTableName(const String & old_temporary_table_name, const String & new_temporary_table_name);
void setFromBackupQuery(const ASTBackupQuery & backup_query);
void setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements);
/// Changes names according to the renaming.
DatabaseAndTableName getNewTableName(const DatabaseAndTableName & old_table_name) const;
const String & getNewDatabaseName(const String & old_database_name) const;
const String & getNewTemporaryTableName(const String & old_temporary_table_name) const;
private:
std::map<DatabaseAndTableName, DatabaseAndTableName> old_to_new_table_names;
std::unordered_map<String, String> old_to_new_database_names;
std::unordered_map<String, String> old_to_new_temporary_table_names;
};
using BackupRenamingConfigPtr = std::shared_ptr<const BackupRenamingConfig>;
}

View File

@ -1,6 +1,43 @@
#include <Backups/BackupSettings.h>
#include <Backups/BackupInfo.h>
#include <Core/SettingsFields.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTSetQuery.h>
namespace DB
{
IMPLEMENT_SETTINGS_TRAITS(BackupSettingsTraits, LIST_OF_BACKUP_SETTINGS)
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
}
BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query)
{
BackupSettings res;
if (query.base_backup_name)
res.base_backup_info = BackupInfo::fromAST(*query.base_backup_name);
if (query.settings)
{
const auto & settings = query.settings->as<const ASTSetQuery &>().changes;
for (const auto & setting : settings)
{
if (setting.name == "compression_method")
res.compression_method = SettingFieldString{setting.value};
else if (setting.name == "compression_level")
res.compression_level = SettingFieldInt64{setting.value};
else if (setting.name == "password")
res.password = SettingFieldString{setting.value};
else if (setting.name == "structure_only")
res.structure_only = SettingFieldBool{setting.value};
else
throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name);
}
}
return res;
}
}

View File

@ -1,16 +1,31 @@
#pragma once
#include <Core/BaseSettings.h>
#include <Backups/BackupInfo.h>
#include <optional>
namespace DB
{
class ASTBackupQuery;
#define LIST_OF_BACKUP_SETTINGS(M) \
M(Bool, dummy, false, "", 0) \
/// Settings specified in the "SETTINGS" clause of a BACKUP query.
struct BackupSettings
{
/// Base backup, if it's set an incremental backup will be built.
std::optional<BackupInfo> base_backup_info;
DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(BackupSettingsTraits, LIST_OF_BACKUP_SETTINGS)
/// Compression method and level for writing the backup (when applicable).
String compression_method; /// "" means default method
int compression_level = -1; /// -1 means default level
struct BackupSettings : public BaseSettings<BackupSettingsTraits> {};
/// Password used to encrypt the backup.
String password;
/// If this is set to true then only create queries will be written to backup,
/// without the data of tables.
bool structure_only = false;
static BackupSettings fromBackupQuery(const ASTBackupQuery & query);
};
}

File diff suppressed because it is too large Load Diff

View File

@ -5,35 +5,28 @@
namespace DB
{
class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
using BackupMutablePtr = std::shared_ptr<IBackup>;
class IBackupEntry;
using BackupEntryPtr = std::unique_ptr<IBackupEntry>;
using BackupEntries = std::vector<std::pair<String, BackupEntryPtr>>;
using RestoreDataTask = std::function<void()>;
using RestoreDataTasks = std::vector<RestoreDataTask>;
using RestoreObjectTask = std::function<RestoreDataTasks()>;
using RestoreObjectsTasks = std::vector<RestoreObjectTask>;
struct BackupSettings;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
using ContextMutablePtr = std::shared_ptr<Context>;
/// Prepares backup entries.
BackupEntries makeBackupEntries(const ASTBackupQuery::Elements & elements, const ContextPtr & context);
/// Estimate total size of the backup which would be written from the specified entries.
UInt64 estimateBackupSize(const BackupEntries & backup_entries, const BackupPtr & base_backup);
BackupEntries makeBackupEntries(const ContextPtr & context, const ASTBackupQuery::Elements & elements, const BackupSettings & backup_settings);
/// Write backup entries to an opened backup.
void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads);
/// Prepare restore tasks.
RestoreObjectsTasks makeRestoreTasks(const ASTBackupQuery::Elements & elements, ContextMutablePtr context, const BackupPtr & backup);
/// Returns the path to metadata in backup.
String getMetadataPathInBackup(const DatabaseAndTableName & table_name);
String getMetadataPathInBackup(const String & database_name);
String getMetadataPathInBackup(const IAST & create_query);
/// Execute restore tasks.
void executeRestoreTasks(RestoreObjectsTasks && restore_tasks, size_t num_threads);
/// Returns the path to table's data in backup.
String getDataPathInBackup(const DatabaseAndTableName & table_name);
String getDataPathInBackup(const IAST & create_query);
}

View File

@ -0,0 +1,87 @@
#include <Backups/DDLCompareUtils.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/formatAST.h>
namespace DB
{
namespace
{
std::shared_ptr<const ASTCreateQuery> prepareDDLToCompare(const ASTCreateQuery & ast)
{
auto res = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(ast.shared_from_this());
std::shared_ptr<ASTCreateQuery> clone;
auto get_clone = [&]
{
if (!clone)
{
clone = typeid_cast<std::shared_ptr<ASTCreateQuery>>(res->clone());
res = clone;
}
return clone;
};
/// Remove UUIDs.
if (res->uuid != UUIDHelpers::Nil)
get_clone()->uuid = UUIDHelpers::Nil;
if (res->to_inner_uuid != UUIDHelpers::Nil)
get_clone()->to_inner_uuid = UUIDHelpers::Nil;
/// Clear IF NOT EXISTS flag.
if (res->if_not_exists)
get_clone()->if_not_exists = false;
return res;
}
}
bool areTableDefinitionsSame(const IAST & table1, const IAST & table2)
{
auto ast1 = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(table1.shared_from_this());
if (!ast1 || !ast1->table)
return false;
auto ast2 = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(table2.shared_from_this());
if (!ast2 || !ast2->table)
return false;
if ((ast1->uuid != ast2->uuid) || (ast1->to_inner_uuid != ast2->to_inner_uuid) ||
(ast1->if_not_exists != ast2->if_not_exists))
{
ast1 = prepareDDLToCompare(*ast1);
ast2 = prepareDDLToCompare(*ast2);
}
return serializeAST(*ast1) == serializeAST(*ast1);
}
bool areDatabaseDefinitionsSame(const IAST & database1, const IAST & database2)
{
auto ast1 = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(database1.shared_from_this());
if (!ast1 || ast1->table || !ast1->database)
return false;
auto ast2 = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(database2.shared_from_this());
if (!ast2 || ast2->table || !ast2->database)
return false;
if ((ast1->uuid != ast2->uuid) || (ast1->if_not_exists != ast2->if_not_exists))
{
ast1 = prepareDDLToCompare(*ast1);
ast2 = prepareDDLToCompare(*ast2);
}
return serializeAST(*ast1) == serializeAST(*ast1);
}
bool areTableDataCompatible(const IAST & src_table, const IAST & dest_table)
{
return areTableDefinitionsSame(src_table, dest_table);
}
}

View File

@ -0,0 +1,17 @@
#pragma once
namespace DB
{
class IAST;
/// Checks that two table definitions are actually the same.
bool areTableDefinitionsSame(const IAST & table1, const IAST & table2);
/// Checks that two database definitions are actually the same.
bool areDatabaseDefinitionsSame(const IAST & database1, const IAST & database2);
/// Whether the data from the first table can be attached to the second table.
bool areTableDataCompatible(const IAST & src_table, const IAST & dest_table);
}

View File

@ -0,0 +1,379 @@
#include <Backups/DDLRenamingVisitor.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <TableFunctions/TableFunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int WRONG_DDL_RENAMING_SETTINGS;
extern const int LOGICAL_ERROR;
}
namespace
{
/// Replaces names of tables and databases used in a CREATE query, which can be either CREATE TABLE or
/// CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query.
void visitCreateQuery(ASTCreateQuery & create, const DDLRenamingVisitor::Data & data)
{
if (create.table)
{
DatabaseAndTableName table_name;
table_name.second = create.getTable();
if (create.temporary)
table_name.first = DatabaseCatalog::TEMPORARY_DATABASE;
else if (create.database)
table_name.first = create.getDatabase();
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE TABLE query must not be empty");
table_name = data.renaming_settings.getNewTableName(table_name);
if (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
{
create.temporary = true;
create.setDatabase("");
}
else
{
create.temporary = false;
create.setDatabase(table_name.first);
}
create.setTable(table_name.second);
}
else if (create.database)
{
String database_name = create.getDatabase();
database_name = data.renaming_settings.getNewDatabaseName(database_name);
create.setDatabase(database_name);
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty");
if (!create.as_table.empty() && !create.as_database.empty())
std::tie(create.as_database, create.as_table) = data.renaming_settings.getNewTableName({create.as_database, create.as_table});
if (!create.to_table_id.table_name.empty() && !create.to_table_id.database_name.empty())
{
auto to_table = data.renaming_settings.getNewTableName({create.to_table_id.database_name, create.to_table_id.table_name});
create.to_table_id = StorageID{to_table.first, to_table.second};
}
}
/// Replaces names of a database and a table in a expression like `db`.`table`
void visitTableExpression(ASTTableExpression & expr, const DDLRenamingVisitor::Data & data)
{
if (!expr.database_and_table_name)
return;
ASTIdentifier * id = expr.database_and_table_name->as<ASTIdentifier>();
if (!id)
return;
auto table_id = id->createTable();
if (!table_id)
return;
const String & db_name = table_id->getDatabaseName();
const String & table_name = table_id->shortName();
if (db_name.empty() || table_name.empty())
return;
String new_db_name, new_table_name;
std::tie(new_db_name, new_table_name) = data.renaming_settings.getNewTableName({db_name, table_name});
if ((new_db_name == db_name) && (new_table_name == table_name))
return;
expr.database_and_table_name = std::make_shared<ASTIdentifier>(Strings{new_db_name, new_table_name});
expr.children.push_back(expr.database_and_table_name);
}
/// Replaces a database's name passed via an argument of the function merge() or the table engine Merge.
void visitFunctionMerge(ASTFunction & function, const DDLRenamingVisitor::Data & data)
{
if (!function.arguments)
return;
/// The first argument is a database's name and we can rename it.
/// The second argument is a regular expression and we can do nothing about it.
auto & args = function.arguments->as<ASTExpressionList &>().children;
size_t db_name_arg_index = 0;
if (args.size() <= db_name_arg_index)
return;
String db_name = evaluateConstantExpressionForDatabaseName(args[db_name_arg_index], data.context)->as<ASTLiteral &>().value.safeGet<String>();
if (db_name.empty())
return;
String new_db_name = data.renaming_settings.getNewDatabaseName(db_name);
if (new_db_name == db_name)
return;
args[db_name_arg_index] = std::make_shared<ASTLiteral>(new_db_name);
}
/// Replaces names of a table and a database passed via arguments of the function remote() or cluster() or the table engine Distributed.
void visitFunctionRemote(ASTFunction & function, const DDLRenamingVisitor::Data & data)
{
if (!function.arguments)
return;
/// The first argument is an address or cluster's name, so we skip it.
/// The second argument can be either 'db.name' or just 'db' followed by the third argument 'table'.
auto & args = function.arguments->as<ASTExpressionList &>().children;
const auto * second_arg_as_function = args[1]->as<ASTFunction>();
if (second_arg_as_function && TableFunctionFactory::instance().isTableFunctionName(second_arg_as_function->name))
return;
size_t db_name_index = 1;
if (args.size() <= db_name_index)
return;
String name = evaluateConstantExpressionForDatabaseName(args[db_name_index], data.context)->as<ASTLiteral &>().value.safeGet<String>();
size_t table_name_index = static_cast<size_t>(-1);
QualifiedTableName qualified_name;
if (function.name == "Distributed")
qualified_name.table = name;
else
qualified_name = QualifiedTableName::parseFromString(name);
if (qualified_name.database.empty())
{
std::swap(qualified_name.database, qualified_name.table);
table_name_index = 2;
if (args.size() <= table_name_index)
return;
qualified_name.table = evaluateConstantExpressionForDatabaseName(args[table_name_index], data.context)->as<ASTLiteral &>().value.safeGet<String>();
}
const String & db_name = qualified_name.database;
const String & table_name = qualified_name.table;
if (db_name.empty() || table_name.empty())
return;
String new_db_name, new_table_name;
std::tie(new_db_name, new_table_name) = data.renaming_settings.getNewTableName({db_name, table_name});
if ((new_db_name == db_name) && (new_table_name == table_name))
return;
if (table_name_index != static_cast<size_t>(-1))
{
if (new_db_name != db_name)
args[db_name_index] = std::make_shared<ASTLiteral>(new_db_name);
if (new_table_name != table_name)
args[table_name_index] = std::make_shared<ASTLiteral>(new_table_name);
}
else
{
args[db_name_index] = std::make_shared<ASTLiteral>(new_db_name);
args.insert(args.begin() + db_name_index + 1, std::make_shared<ASTLiteral>(new_table_name));
}
}
/// Replaces names of tables and databases used in arguments of a table function or a table engine.
void visitFunction(ASTFunction & function, const DDLRenamingVisitor::Data & data)
{
if ((function.name == "merge") || (function.name == "Merge"))
{
visitFunctionMerge(function, data);
}
else if ((function.name == "remote") || (function.name == "remoteSecure") || (function.name == "cluster") ||
(function.name == "clusterAllReplicas") || (function.name == "Distributed"))
{
visitFunctionRemote(function, data);
}
}
/// Replaces names of a table and a database used in source parameters of a dictionary.
void visitDictionary(ASTDictionary & dictionary, const DDLRenamingVisitor::Data & data)
{
if (!dictionary.source || dictionary.source->name != "clickhouse" || !dictionary.source->elements)
return;
auto & elements = dictionary.source->elements->as<ASTExpressionList &>().children;
String db_name, table_name;
size_t db_name_index = static_cast<size_t>(-1);
size_t table_name_index = static_cast<size_t>(-1);
for (size_t i = 0; i != elements.size(); ++i)
{
auto & pair = elements[i]->as<ASTPair &>();
if (pair.first == "db")
{
if (db_name_index != static_cast<size_t>(-1))
return;
db_name = pair.second->as<ASTLiteral &>().value.safeGet<String>();
db_name_index = i;
}
else if (pair.first == "table")
{
if (table_name_index != static_cast<size_t>(-1))
return;
table_name = pair.second->as<ASTLiteral &>().value.safeGet<String>();
table_name_index = i;
}
}
if (db_name.empty() || table_name.empty())
return;
String new_db_name, new_table_name;
std::tie(new_db_name, new_table_name) = data.renaming_settings.getNewTableName({db_name, table_name});
if ((new_db_name == db_name) && (new_table_name == table_name))
return;
if (new_db_name != db_name)
{
auto & pair = elements[db_name_index]->as<ASTPair &>();
pair.replace(pair.second, std::make_shared<ASTLiteral>(new_db_name));
}
if (new_table_name != table_name)
{
auto & pair = elements[table_name_index]->as<ASTPair &>();
pair.replace(pair.second, std::make_shared<ASTLiteral>(new_table_name));
}
}
}
void DDLRenamingSettings::setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name)
{
auto it = old_to_new_table_names.find(old_table_name);
if ((it != old_to_new_table_names.end()))
{
if (it->second == new_table_name)
return;
throw Exception(ErrorCodes::WRONG_DDL_RENAMING_SETTINGS, "Wrong renaming: it's specified that table {}.{} should be renamed to {}.{} and to {}.{} at the same time",
backQuoteIfNeed(old_table_name.first), backQuoteIfNeed(old_table_name.second),
backQuoteIfNeed(it->second.first), backQuoteIfNeed(it->second.second),
backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second));
}
old_to_new_table_names[old_table_name] = new_table_name;
}
void DDLRenamingSettings::setNewDatabaseName(const String & old_database_name, const String & new_database_name)
{
auto it = old_to_new_database_names.find(old_database_name);
if ((it != old_to_new_database_names.end()))
{
if (it->second == new_database_name)
return;
throw Exception(ErrorCodes::WRONG_DDL_RENAMING_SETTINGS, "Wrong renaming: it's specified that database {} should be renamed to {} and to {} at the same time",
backQuoteIfNeed(old_database_name), backQuoteIfNeed(it->second), backQuoteIfNeed(new_database_name));
}
old_to_new_database_names[old_database_name] = new_database_name;
}
void DDLRenamingSettings::setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database)
{
setFromBackupQuery(backup_query.elements, current_database);
}
void DDLRenamingSettings::setFromBackupQuery(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database)
{
old_to_new_table_names.clear();
old_to_new_database_names.clear();
using ElementType = ASTBackupQuery::ElementType;
for (const auto & element : backup_query_elements)
{
switch (element.type)
{
case ElementType::TABLE:
{
const String & table_name = element.name.second;
String database_name = element.name.first;
if (element.name_is_in_temp_db)
database_name = DatabaseCatalog::TEMPORARY_DATABASE;
else if (database_name.empty())
database_name = current_database;
const String & new_table_name = element.new_name.second;
String new_database_name = element.new_name.first;
if (element.new_name_is_in_temp_db)
new_database_name = DatabaseCatalog::TEMPORARY_DATABASE;
else if (new_database_name.empty())
new_database_name = current_database;
setNewTableName({database_name, table_name}, {new_database_name, new_table_name});
break;
}
case ASTBackupQuery::DATABASE:
{
String database_name = element.name.first;
if (element.name_is_in_temp_db)
database_name = DatabaseCatalog::TEMPORARY_DATABASE;
String new_database_name = element.new_name.first;
if (element.new_name_is_in_temp_db)
new_database_name = DatabaseCatalog::TEMPORARY_DATABASE;
setNewDatabaseName(database_name, new_database_name);
break;
}
case ASTBackupQuery::ALL_DATABASES: break;
}
}
}
DatabaseAndTableName DDLRenamingSettings::getNewTableName(const DatabaseAndTableName & old_table_name) const
{
auto it = old_to_new_table_names.find(old_table_name);
if (it != old_to_new_table_names.end())
return it->second;
return {getNewDatabaseName(old_table_name.first), old_table_name.second};
}
const String & DDLRenamingSettings::getNewDatabaseName(const String & old_database_name) const
{
auto it = old_to_new_database_names.find(old_database_name);
if (it != old_to_new_database_names.end())
return it->second;
return old_database_name;
}
bool DDLRenamingVisitor::needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
void DDLRenamingVisitor::visit(ASTPtr & ast, const Data & data)
{
if (auto * create = ast->as<ASTCreateQuery>())
visitCreateQuery(*create, data);
else if (auto * expr = ast->as<ASTTableExpression>())
visitTableExpression(*expr, data);
else if (auto * function = ast->as<ASTFunction>())
visitFunction(*function, data);
else if (auto * dictionary = ast->as<ASTDictionary>())
visitDictionary(*dictionary, data);
}
void renameInCreateQuery(ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings)
{
try
{
DDLRenamingVisitor::Data data{renaming_settings, global_context};
DDLRenamingVisitor::Visitor{data}.visit(ast);
}
catch (...)
{
tryLogCurrentException("Backup", "Error while renaming in AST");
}
}
}

View File

@ -0,0 +1,61 @@
#pragma once
#include <Core/Types.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Parsers/ASTBackupQuery.h>
#include <map>
#include <memory>
#include <unordered_map>
namespace DB
{
using DatabaseAndTableName = std::pair<String, String>;
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
/// Keeps information about renamings of databases or tables being processed
/// while we're making a backup or while we're restoring from a backup.
class DDLRenamingSettings
{
public:
DDLRenamingSettings() = default;
void setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name);
void setNewDatabaseName(const String & old_database_name, const String & new_database_name);
void setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database);
void setFromBackupQuery(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database);
/// Changes names according to the renaming.
DatabaseAndTableName getNewTableName(const DatabaseAndTableName & old_table_name) const;
const String & getNewDatabaseName(const String & old_database_name) const;
private:
std::map<DatabaseAndTableName, DatabaseAndTableName> old_to_new_table_names;
std::unordered_map<String, String> old_to_new_database_names;
};
/// Changes names in AST according to the renaming settings.
void renameInCreateQuery(ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings);
/// Visits ASTCreateQuery and changes names of tables and databases according to passed DDLRenamingConfig.
class DDLRenamingVisitor
{
public:
struct Data
{
const DDLRenamingSettings & renaming_settings;
ContextPtr context;
};
using Visitor = InDepthNodeVisitor<DDLRenamingVisitor, false>;
static bool needChildVisit(ASTPtr &, const ASTPtr &);
static void visit(ASTPtr & ast, const Data & data);
};
}

View File

@ -0,0 +1,82 @@
#include <Backups/DirectoryBackup.h>
#include <Common/quoteString.h>
#include <Disks/IDisk.h>
#include <Disks/DiskLocal.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
DirectoryBackup::DirectoryBackup(
const String & backup_name_,
const DiskPtr & disk_,
const String & path_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_)
: BackupImpl(backup_name_, context_, base_backup_info_)
, disk(disk_), path(path_)
{
/// Path to backup must end with '/'
if (!path.ends_with("/"))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to backup must end with '/', but {} doesn't.", getName(), quoteString(path));
dir_path = fs::path(path).parent_path(); /// get path without terminating slash
/// If `disk` is not specified, we create an internal instance of `DiskLocal` here.
if (!disk)
{
auto fspath = fs::path{dir_path};
if (!fspath.has_filename())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to a backup must be a directory path.", getName(), quoteString(path));
path = fspath.filename() / "";
dir_path = fs::path(path).parent_path(); /// get path without terminating slash
String disk_path = fspath.remove_filename();
disk = std::make_shared<DiskLocal>(disk_path, disk_path, 0);
}
}
DirectoryBackup::~DirectoryBackup()
{
close();
}
bool DirectoryBackup::backupExists() const
{
return disk->isDirectory(dir_path);
}
void DirectoryBackup::openImpl(OpenMode open_mode_)
{
if (open_mode_ == OpenMode::WRITE)
disk->createDirectories(dir_path);
}
void DirectoryBackup::closeImpl(bool writing_finalized_)
{
if ((getOpenModeNoLock() == OpenMode::WRITE) && !writing_finalized_ && disk->isDirectory(dir_path))
{
/// Creating of the backup wasn't finished correctly,
/// so the backup cannot be used and it's better to remove its files.
disk->removeRecursive(dir_path);
}
}
std::unique_ptr<ReadBuffer> DirectoryBackup::readFileImpl(const String & file_name) const
{
String file_path = path + file_name;
return disk->readFile(file_path);
}
std::unique_ptr<WriteBuffer> DirectoryBackup::writeFileImpl(const String & file_name)
{
String file_path = path + file_name;
disk->createDirectories(fs::path(file_path).parent_path());
return disk->writeFile(file_path);
}
}

View File

@ -10,25 +10,24 @@ using DiskPtr = std::shared_ptr<IDisk>;
/// Represents a backup stored on a disk.
/// A backup is stored as a directory, each entry is stored as a file in that directory.
class BackupInDirectory : public BackupImpl
class DirectoryBackup : public BackupImpl
{
public:
/// `disk`_ is allowed to be nullptr and that means the `path_` is a path in the local filesystem.
BackupInDirectory(
DirectoryBackup(
const String & backup_name_,
OpenMode open_mode_,
const DiskPtr & disk_,
const String & path_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_ = {});
~BackupInDirectory() override;
~DirectoryBackup() override;
private:
bool backupExists() const override;
void startWriting() override;
void removeAllFilesAfterFailure() override;
void openImpl(OpenMode open_mode_) override;
void closeImpl(bool writing_finalized_) override;
std::unique_ptr<ReadBuffer> readFileImpl(const String & file_name) const override;
std::unique_ptr<WriteBuffer> addFileImpl(const String & file_name) override;
std::unique_ptr<WriteBuffer> writeFileImpl(const String & file_name) override;
DiskPtr disk;
String path;

View File

@ -1,8 +1,8 @@
#pragma once
#include <Core/Types.h>
#include <Common/TypePromotion.h>
#include <memory>
#include <optional>
namespace DB
@ -13,7 +13,7 @@ using BackupEntryPtr = std::unique_ptr<IBackupEntry>;
/// Represents a backup, i.e. a storage of BackupEntries which can be accessed by their names.
/// A backup can be either incremental or non-incremental. An incremental backup doesn't store
/// the data of the entries which are not changed compared to its base backup.
class IBackup : public std::enable_shared_from_this<IBackup>, public TypePromotion<IBackup>
class IBackup : public std::enable_shared_from_this<IBackup>
{
public:
IBackup() = default;
@ -24,13 +24,18 @@ public:
enum class OpenMode
{
NONE,
READ,
WRITE,
};
/// A backup can be open either in READ or WRITE mode.
/// Opens the backup and start its reading or writing depending on `open_mode`.
virtual void open(OpenMode open_mode) = 0;
virtual OpenMode getOpenMode() const = 0;
/// Closes the backup and ends its reading or writing.
virtual void close() = 0;
/// Returns the time point when this backup was created.
virtual time_t getTimestamp() const = 0;
@ -57,17 +62,20 @@ public:
/// This function does the same as `read(file_name)->getCheckum()` but faster.
virtual UInt128 getFileChecksum(const String & file_name) const = 0;
/// Finds a file by its checksum, returns nullopt if not found.
virtual std::optional<String> findFileByChecksum(const UInt128 & checksum) const = 0;
/// Reads an entry from the backup.
virtual BackupEntryPtr readFile(const String & file_name) const = 0;
/// Puts a new entry to the backup.
virtual void addFile(const String & file_name, BackupEntryPtr entry) = 0;
/// Whether it's possible to add new entries to the backup in multiple threads.
virtual bool supportsWritingInMultipleThreads() const { return true; }
virtual void writeFile(const String & file_name, BackupEntryPtr entry) = 0;
/// Finalizes writing the backup, should be called after all entries have been successfully written.
virtual void finalizeWriting() = 0;
/// Whether it's possible to add new entries to the backup in multiple threads.
virtual bool supportsWritingInMultipleThreads() const { return true; }
};
using BackupPtr = std::shared_ptr<const IBackup>;

View File

@ -0,0 +1,37 @@
#include <Backups/IBackupEntriesBatch.h>
#include <IO/ReadBuffer.h>
namespace DB
{
class IBackupEntriesBatch::BackupEntryFromBatch : public IBackupEntry
{
public:
BackupEntryFromBatch(const std::shared_ptr<IBackupEntriesBatch> & generator_, size_t index_) : batch(generator_), index(index_)
{
assert(batch);
}
UInt64 getSize() const override { return batch->getSize(index); }
std::optional<UInt128> getChecksum() const override { return batch->getChecksum(index); }
std::unique_ptr<ReadBuffer> getReadBuffer() const override { return batch->getReadBuffer(index); }
private:
const std::shared_ptr<IBackupEntriesBatch> batch;
const size_t index;
};
BackupEntries IBackupEntriesBatch::getBackupEntries()
{
BackupEntries res;
res.reserve(entry_names.size());
for (size_t i = 0; i != entry_names.size(); ++i)
{
res.emplace_back(entry_names[i], std::make_unique<BackupEntryFromBatch>(shared_from_this(), i));
}
return res;
}
}

View File

@ -0,0 +1,29 @@
#pragma once
#include <Backups/IBackupEntry.h>
#include <mutex>
namespace DB
{
/// Helper class designed to generate multiple backup entries from one source.
class IBackupEntriesBatch : public std::enable_shared_from_this<IBackupEntriesBatch>
{
public:
BackupEntries getBackupEntries();
virtual ~IBackupEntriesBatch() = default;
protected:
IBackupEntriesBatch(const Strings & entry_names_) : entry_names(entry_names_) {}
virtual std::unique_ptr<ReadBuffer> getReadBuffer(size_t index) = 0;
virtual UInt64 getSize(size_t index) = 0;
virtual std::optional<UInt128> getChecksum(size_t) { return {}; }
private:
class BackupEntryFromBatch;
const Strings entry_names;
};
}

View File

@ -0,0 +1,31 @@
#pragma once
#include <memory>
#include <vector>
namespace DB
{
/// Represents a task of restoring something (database / table / table's part) from backup.
class IRestoreTask
{
public:
IRestoreTask() = default;
virtual ~IRestoreTask() = default;
/// Perform restoring, the function also can return a list of nested tasks that should be run later.
virtual std::vector<std::unique_ptr<IRestoreTask>> run() = 0;
/// Is it necessary to run this task sequentially?
/// Sequential tasks are executed first and strictly in one thread.
virtual bool isSequential() const { return false; }
/// Reverts the effect of run(). If that's not possible, the function does nothing.
virtual void rollback() {}
};
using RestoreTaskPtr = std::unique_ptr<IRestoreTask>;
using RestoreTasks = std::vector<RestoreTaskPtr>;
}

View File

@ -0,0 +1,47 @@
#include <Backups/RestoreSettings.h>
#include <Backups/BackupInfo.h>
#include <Core/SettingsFields.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTSetQuery.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
}
RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
{
RestoreSettings res;
if (query.base_backup_name)
res.base_backup_info = BackupInfo::fromAST(*query.base_backup_name);
if (query.settings)
{
const auto & settings = query.settings->as<const ASTSetQuery &>().changes;
for (const auto & setting : settings)
{
if (setting.name == "password")
res.password = SettingFieldString{setting.value};
else if (setting.name == "structure_only")
res.structure_only = SettingFieldBool{setting.value};
else if (setting.name == "throw_if_database_exists")
res.throw_if_database_exists = SettingFieldBool{setting.value};
else if (setting.name == "throw_if_table_exists")
res.throw_if_table_exists = SettingFieldBool{setting.value};
else if (setting.name == "throw_if_database_def_differs")
res.throw_if_database_def_differs = SettingFieldBool{setting.value};
else if (setting.name == "throw_if_table_def_differs")
res.throw_if_table_def_differs = SettingFieldBool{setting.value};
else
throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name);
}
}
return res;
}
}

View File

@ -0,0 +1,47 @@
#pragma once
#include <Backups/BackupInfo.h>
#include <optional>
namespace DB
{
class ASTBackupQuery;
struct StorageRestoreSettings
{
};
/// Settings specified in the "SETTINGS" clause of a RESTORE query.
struct RestoreSettings : public StorageRestoreSettings
{
/// Base backup, with this setting we can override the location of the base backup while restoring.
/// Any incremental backup keeps inside the information about its base backup,
/// so using this setting is optional.
std::optional<BackupInfo> base_backup_info;
/// Password used to decrypt the backup.
String password;
/// If this is set to true then only create queries will be read from backup,
/// without the data of tables.
bool structure_only = false;
/// Whether RESTORE DATABASE must throw an exception if a destination database already exists.
bool throw_if_database_exists = true;
/// Whether RESTORE TABLE must throw an exception if a destination table already exists.
bool throw_if_table_exists = true;
/// Whether RESTORE DATABASE must throw an exception if a destination database has
/// a different definition comparing with the definition read from backup.
bool throw_if_database_def_differs = true;
/// Whether RESTORE TABLE must throw an exception if a destination table has
/// a different definition comparing with the definition read from backup.
bool throw_if_table_def_differs = true;
static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query);
};
}

View File

@ -0,0 +1,685 @@
#include <Backups/RestoreUtils.h>
#include <Backups/BackupUtils.h>
#include <Backups/DDLCompareUtils.h>
#include <Backups/DDLRenamingVisitor.h>
#include <Backups/IBackup.h>
#include <Backups/IBackupEntry.h>
#include <Backups/IRestoreTask.h>
#include <Backups/RestoreSettings.h>
#include <Backups/formatTableNameOrTemporaryTableName.h>
#include <Common/escapeForFileName.h>
#include <Databases/IDatabase.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Storages/IStorage.h>
#include <boost/range/adaptor/reversed.hpp>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_RESTORE_TABLE;
extern const int CANNOT_RESTORE_DATABASE;
}
namespace
{
using Kind = ASTBackupQuery::Kind;
using Element = ASTBackupQuery::Element;
using Elements = ASTBackupQuery::Elements;
using ElementType = ASTBackupQuery::ElementType;
using RestoreSettingsPtr = std::shared_ptr<const RestoreSettings>;
/// Restores a database (without tables inside), should be executed before executing
/// RestoreTableTask.
class RestoreDatabaseTask : public IRestoreTask
{
public:
RestoreDatabaseTask(
ContextMutablePtr context_,
const ASTPtr & create_query_,
const RestoreSettingsPtr & restore_settings_,
bool ignore_if_database_def_differs_)
: context(context_)
, create_query(typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query_))
, restore_settings(restore_settings_)
, ignore_if_database_def_differs(ignore_if_database_def_differs_)
{
}
RestoreTasks run() override
{
createDatabase();
getDatabase();
checkDatabaseCreateQuery();
return {};
}
bool isSequential() const override { return true; }
private:
void createDatabase()
{
/// We need to call clone() for `create_query` because the interpreter can decide
/// to change a passed AST a little bit.
InterpreterCreateQuery create_interpreter{create_query->clone(), context};
create_interpreter.execute();
}
DatabasePtr getDatabase()
{
if (!database)
database = DatabaseCatalog::instance().getDatabase(create_query->getDatabase());
return database;
}
ASTPtr getDatabaseCreateQuery()
{
if (!database_create_query)
database_create_query = getDatabase()->getCreateDatabaseQuery();
return database_create_query;
}
void checkDatabaseCreateQuery()
{
if (ignore_if_database_def_differs || !restore_settings->throw_if_database_def_differs)
return;
getDatabaseCreateQuery();
if (areDatabaseDefinitionsSame(*create_query, *database_create_query))
return;
throw Exception(
ErrorCodes::CANNOT_RESTORE_DATABASE,
"The database {} already exists but has a different definition: {}, "
"compare to its definition in the backup: {}",
backQuoteIfNeed(create_query->getDatabase()),
serializeAST(*database_create_query),
serializeAST(*create_query));
}
ContextMutablePtr context;
std::shared_ptr<ASTCreateQuery> create_query;
RestoreSettingsPtr restore_settings;
bool ignore_if_database_def_differs = false;
DatabasePtr database;
ASTPtr database_create_query;
};
/// Restores a table and fills it with data.
class RestoreTableTask : public IRestoreTask
{
public:
RestoreTableTask(
ContextMutablePtr context_,
const ASTPtr & create_query_,
const ASTs & partitions_,
const BackupPtr & backup_,
const DatabaseAndTableName & table_name_in_backup_,
const RestoreSettingsPtr & restore_settings_)
: context(context_), create_query(typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query_)),
partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_),
restore_settings(restore_settings_)
{
table_name = DatabaseAndTableName{create_query->getDatabase(), create_query->getTable()};
if (create_query->temporary)
table_name.first = DatabaseCatalog::TEMPORARY_DATABASE;
}
RestoreTasks run() override
{
createStorage();
getStorage();
checkStorageCreateQuery();
RestoreTasks tasks;
if (auto task = insertData())
tasks.push_back(std::move(task));
return tasks;
}
bool isSequential() const override { return true; }
private:
void createStorage()
{
/// We need to call clone() for `create_query` because the interpreter can decide
/// to change a passed AST a little bit.
InterpreterCreateQuery create_interpreter{create_query->clone(), context};
create_interpreter.execute();
}
StoragePtr getStorage()
{
if (!storage)
std::tie(database, storage) = DatabaseCatalog::instance().getDatabaseAndTable({table_name.first, table_name.second}, context);
return storage;
}
ASTPtr getStorageCreateQuery()
{
if (!storage_create_query)
{
getStorage();
storage_create_query = database->getCreateTableQuery(table_name.second, context);
}
return storage_create_query;
}
void checkStorageCreateQuery()
{
if (!restore_settings->throw_if_table_def_differs)
return;
getStorageCreateQuery();
if (areTableDefinitionsSame(*create_query, *storage_create_query))
return;
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,
"The {} already exists but has a different definition: {}, "
"compare to its definition in the backup: {}",
formatTableNameOrTemporaryTableName(table_name),
serializeAST(*storage_create_query),
serializeAST(*create_query));
}
bool hasData()
{
if (has_data)
return *has_data;
has_data = false;
if (restore_settings->structure_only)
return false;
data_path_in_backup = getDataPathInBackup(table_name_in_backup);
if (backup->listFiles(data_path_in_backup).empty())
return false;
getStorageCreateQuery();
if (!areTableDataCompatible(*create_query, *storage_create_query))
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,
"Cannot attach data of the {} in the backup to the existing {} because of they are not compatible. "
"Here is the definition of the {} in the backup: {}, and here is the definition of the existing {}: {}",
formatTableNameOrTemporaryTableName(table_name_in_backup),
formatTableNameOrTemporaryTableName(table_name),
formatTableNameOrTemporaryTableName(table_name_in_backup),
serializeAST(*create_query),
formatTableNameOrTemporaryTableName(table_name),
serializeAST(*storage_create_query));
/// We check for INSERT privilege only if we're going to write into table.
context->checkAccess(AccessType::INSERT, table_name.first, table_name.second);
has_data = true;
return true;
}
RestoreTaskPtr insertData()
{
if (!hasData())
return {};
return storage->restoreData(context, partitions, backup, data_path_in_backup, *restore_settings);
}
ContextMutablePtr context;
std::shared_ptr<ASTCreateQuery> create_query;
DatabaseAndTableName table_name;
ASTs partitions;
BackupPtr backup;
DatabaseAndTableName table_name_in_backup;
RestoreSettingsPtr restore_settings;
DatabasePtr database;
StoragePtr storage;
ASTPtr storage_create_query;
std::optional<bool> has_data;
String data_path_in_backup;
};
/// Makes tasks for restoring databases and tables according to the elements of ASTBackupQuery.
/// Keep this class consistent with BackupEntriesBuilder.
class RestoreTasksBuilder
{
public:
RestoreTasksBuilder(ContextMutablePtr context_, const BackupPtr & backup_, const RestoreSettings & restore_settings_)
: context(context_), backup(backup_), restore_settings(restore_settings_) {}
/// Prepares internal structures for making tasks for restoring.
void prepare(const ASTBackupQuery::Elements & elements)
{
String current_database = context->getCurrentDatabase();
renaming_settings.setFromBackupQuery(elements, current_database);
for (const auto & element : elements)
{
switch (element.type)
{
case ElementType::TABLE:
{
const String & table_name = element.name.second;
String database_name = element.name.first;
if (database_name.empty())
database_name = current_database;
prepareToRestoreTable(DatabaseAndTableName{database_name, table_name}, element.partitions);
break;
}
case ElementType::DATABASE:
{
const String & database_name = element.name.first;
prepareToRestoreDatabase(database_name, element.except_list);
break;
}
case ElementType::ALL_DATABASES:
{
prepareToRestoreAllDatabases(element.except_list);
break;
}
}
}
}
/// Makes tasks for restoring, should be called after prepare().
RestoreTasks makeTasks() const
{
/// Check that there are not `different_create_query`. (If it's set it means error.)
for (const auto & info : databases | boost::adaptors::map_values)
{
if (info.different_create_query)
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE,
"Cannot restore a database because two different create queries were generated for it: {} and {}",
serializeAST(*info.create_query), serializeAST(*info.different_create_query));
}
auto restore_settings_ptr = std::make_shared<const RestoreSettings>(restore_settings);
RestoreTasks res;
for (const auto & info : databases | boost::adaptors::map_values)
res.push_back(std::make_unique<RestoreDatabaseTask>(context, info.create_query, restore_settings_ptr,
/* ignore_if_database_def_differs = */ !info.is_explicit));
/// TODO: We need to restore tables according to their dependencies.
for (const auto & info : tables | boost::adaptors::map_values)
res.push_back(std::make_unique<RestoreTableTask>(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings_ptr));
return res;
}
private:
/// Prepares to restore a single table and probably its database's definition.
void prepareToRestoreTable(const DatabaseAndTableName & table_name_, const ASTs & partitions_)
{
/// Check that we are not trying to restore the same table again.
DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_);
if (tables.contains(new_table_name))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore the {} twice", formatTableNameOrTemporaryTableName(new_table_name));
/// Make a create query for this table.
auto create_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_));
create_query->if_not_exists = !restore_settings.throw_if_table_exists;
CreateTableInfo info;
info.create_query = create_query;
info.name_in_backup = table_name_;
info.partitions = partitions_;
tables[new_table_name] = std::move(info);
/// If it's not system or temporary database then probably we need to restore the database's definition too.
if (!isSystemOrTemporaryDatabase(new_table_name.first))
{
if (!databases.contains(new_table_name.first))
{
/// Add a create query for restoring the database if we haven't done it yet.
std::shared_ptr<ASTCreateQuery> create_db_query;
String db_name_in_backup = table_name_.first;
if (hasCreateQueryInBackup(db_name_in_backup))
{
create_db_query = renameInCreateQuery(readCreateQueryFromBackup(db_name_in_backup));
}
else
{
create_db_query = std::make_shared<ASTCreateQuery>();
db_name_in_backup.clear();
}
create_db_query->setDatabase(new_table_name.first);
create_db_query->if_not_exists = true;
CreateDatabaseInfo info_db;
info_db.create_query = create_db_query;
info_db.name_in_backup = std::move(db_name_in_backup);
info_db.is_explicit = false;
databases[new_table_name.first] = std::move(info_db);
}
else
{
/// We already have added a create query for restoring the database,
/// set `different_create_query` if it's not the same.
auto & info_db = databases[new_table_name.first];
if (!info_db.is_explicit && (info_db.name_in_backup != table_name_.first) && !info_db.different_create_query)
{
std::shared_ptr<ASTCreateQuery> create_db_query;
if (hasCreateQueryInBackup(table_name_.first))
create_db_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_.first));
else
create_db_query = std::make_shared<ASTCreateQuery>();
create_db_query->setDatabase(new_table_name.first);
create_db_query->if_not_exists = true;
if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query))
info_db.different_create_query = create_db_query;
}
}
}
}
/// Prepares to restore a database and all tables in it.
void prepareToRestoreDatabase(const String & database_name_, const std::set<String> & except_list_)
{
/// Check that we are not trying to restore the same database again.
String new_database_name = renaming_settings.getNewDatabaseName(database_name_);
if (databases.contains(new_database_name) && databases[new_database_name].is_explicit)
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} twice", backQuoteIfNeed(new_database_name));
Strings table_metadata_filenames = backup->listFiles("metadata/" + escapeForFileName(database_name_) + "/", "/");
bool throw_if_no_create_database_query = table_metadata_filenames.empty();
if (throw_if_no_create_database_query && !hasCreateQueryInBackup(database_name_))
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} because there is no such database in the backup", backQuoteIfNeed(database_name_));
/// Of course we're not going to restore the definition of the system or the temporary database.
if (!isSystemOrTemporaryDatabase(new_database_name))
{
/// Make a create query for this database.
std::shared_ptr<ASTCreateQuery> create_db_query;
String db_name_in_backup = database_name_;
if (hasCreateQueryInBackup(db_name_in_backup))
{
create_db_query = renameInCreateQuery(readCreateQueryFromBackup(db_name_in_backup));
}
else
{
create_db_query = std::make_shared<ASTCreateQuery>();
create_db_query->setDatabase(database_name_);
db_name_in_backup.clear();
}
create_db_query->if_not_exists = !restore_settings.throw_if_database_exists;
CreateDatabaseInfo info_db;
info_db.create_query = create_db_query;
info_db.name_in_backup = std::move(db_name_in_backup);
info_db.is_explicit = true;
databases[new_database_name] = std::move(info_db);
}
/// Restore tables in this database.
for (const String & table_metadata_filename : table_metadata_filenames)
{
String table_name = unescapeForFileName(fs::path{table_metadata_filename}.stem());
if (except_list_.contains(table_name))
continue;
prepareToRestoreTable(DatabaseAndTableName{database_name_, table_name}, ASTs{});
}
}
/// Prepares to restore all the databases contained in the backup.
void prepareToRestoreAllDatabases(const std::set<String> & except_list_)
{
Strings database_metadata_filenames = backup->listFiles("metadata/", "/");
for (const String & database_metadata_filename : database_metadata_filenames)
{
String database_name = unescapeForFileName(fs::path{database_metadata_filename}.stem());
if (except_list_.contains(database_name))
continue;
prepareToRestoreDatabase(database_name, std::set<String>{});
}
}
/// Reads a create query for creating a specified table from the backup.
std::shared_ptr<ASTCreateQuery> readCreateQueryFromBackup(const DatabaseAndTableName & table_name) const
{
String create_query_path = getMetadataPathInBackup(table_name);
if (!backup->fileExists(create_query_path))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore the {} because there is no such table in the backup",
formatTableNameOrTemporaryTableName(table_name));
auto read_buffer = backup->readFile(create_query_path)->getReadBuffer();
String create_query_str;
readStringUntilEOF(create_query_str, *read_buffer);
read_buffer.reset();
ParserCreateQuery create_parser;
return typeid_cast<std::shared_ptr<ASTCreateQuery>>(parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
}
/// Reads a create query for creating a specified database from the backup.
std::shared_ptr<ASTCreateQuery> readCreateQueryFromBackup(const String & database_name) const
{
String create_query_path = getMetadataPathInBackup(database_name);
if (!backup->fileExists(create_query_path))
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} because there is no such database in the backup", backQuoteIfNeed(database_name));
auto read_buffer = backup->readFile(create_query_path)->getReadBuffer();
String create_query_str;
readStringUntilEOF(create_query_str, *read_buffer);
read_buffer.reset();
ParserCreateQuery create_parser;
return typeid_cast<std::shared_ptr<ASTCreateQuery>>(parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
}
/// Whether there is a create query for creating a specified database in the backup.
bool hasCreateQueryInBackup(const String & database_name) const
{
String create_query_path = getMetadataPathInBackup(database_name);
return backup->fileExists(create_query_path);
}
/// Do renaming in the create query according to the renaming config.
std::shared_ptr<ASTCreateQuery> renameInCreateQuery(const ASTPtr & ast) const
{
ASTPtr query = ast;
::DB::renameInCreateQuery(query, context, renaming_settings);
auto create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query);
return create_query;
}
static bool isSystemOrTemporaryDatabase(const String & database_name)
{
return (database_name == DatabaseCatalog::SYSTEM_DATABASE) || (database_name == DatabaseCatalog::TEMPORARY_DATABASE);
}
/// Information which is used to make an instance of RestoreTableTask.
struct CreateTableInfo
{
ASTPtr create_query;
DatabaseAndTableName name_in_backup;
ASTs partitions;
};
/// Information which is used to make an instance of RestoreDatabaseTask.
struct CreateDatabaseInfo
{
ASTPtr create_query;
String name_in_backup;
/// Whether the creation of this database is specified explicitly, via RESTORE DATABASE or
/// RESTORE ALL DATABASES.
/// It's false if the creation of this database is caused by creating a table contained in it.
bool is_explicit = false;
/// If this is set it means the following error:
/// it means that for implicitly created database there were two different create query
/// generated so we cannot restore the database.
ASTPtr different_create_query;
};
ContextMutablePtr context;
BackupPtr backup;
RestoreSettings restore_settings;
DDLRenamingSettings renaming_settings;
std::map<String, CreateDatabaseInfo> databases;
std::map<DatabaseAndTableName, CreateTableInfo> tables;
};
/// Reverts completed restore tasks (in reversed order).
void rollbackRestoreTasks(RestoreTasks && restore_tasks)
{
for (auto & restore_task : restore_tasks | boost::adaptors::reversed)
{
try
{
std::move(restore_task)->rollback();
}
catch (...)
{
tryLogCurrentException("Restore", "Couldn't rollback changes after failed RESTORE");
}
}
}
}
RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements, const RestoreSettings & restore_settings)
{
RestoreTasksBuilder builder{context, backup, restore_settings};
builder.prepare(elements);
return builder.makeTasks();
}
void executeRestoreTasks(RestoreTasks && restore_tasks, size_t num_threads)
{
if (!num_threads)
num_threads = 1;
RestoreTasks completed_tasks;
bool need_rollback_completed_tasks = true;
SCOPE_EXIT({
if (need_rollback_completed_tasks)
rollbackRestoreTasks(std::move(completed_tasks));
});
std::deque<std::unique_ptr<IRestoreTask>> sequential_tasks;
std::deque<std::unique_ptr<IRestoreTask>> enqueued_tasks;
/// There are two kinds of restore tasks: sequential and non-sequential ones.
/// Sequential tasks are executed first and always in one thread.
for (auto & task : restore_tasks)
{
if (task->isSequential())
sequential_tasks.push_back(std::move(task));
else
enqueued_tasks.push_back(std::move(task));
}
/// Sequential tasks.
while (!sequential_tasks.empty())
{
auto current_task = std::move(sequential_tasks.front());
sequential_tasks.pop_front();
RestoreTasks new_tasks = current_task->run();
completed_tasks.push_back(std::move(current_task));
for (auto & task : new_tasks)
{
if (task->isSequential())
sequential_tasks.push_back(std::move(task));
else
enqueued_tasks.push_back(std::move(task));
}
}
/// Non-sequential tasks.
std::unordered_map<IRestoreTask *, std::unique_ptr<IRestoreTask>> running_tasks;
std::vector<ThreadFromGlobalPool> threads;
std::mutex mutex;
std::condition_variable cond;
std::exception_ptr exception;
while (true)
{
IRestoreTask * current_task = nullptr;
{
std::unique_lock lock{mutex};
cond.wait(lock, [&]
{
if (exception)
return true;
if (enqueued_tasks.empty())
return running_tasks.empty();
return (running_tasks.size() < num_threads);
});
if (exception || enqueued_tasks.empty())
break;
auto current_task_ptr = std::move(enqueued_tasks.front());
current_task = current_task_ptr.get();
enqueued_tasks.pop_front();
running_tasks[current_task] = std::move(current_task_ptr);
}
assert(current_task);
threads.emplace_back([current_task, &mutex, &cond, &enqueued_tasks, &running_tasks, &completed_tasks, &exception]() mutable
{
{
std::lock_guard lock{mutex};
if (exception)
return;
}
RestoreTasks new_tasks;
std::exception_ptr new_exception;
try
{
new_tasks = current_task->run();
}
catch (...)
{
new_exception = std::current_exception();
}
{
std::lock_guard lock{mutex};
auto current_task_it = running_tasks.find(current_task);
auto current_task_ptr = std::move(current_task_it->second);
running_tasks.erase(current_task_it);
if (!new_exception)
{
completed_tasks.push_back(std::move(current_task_ptr));
enqueued_tasks.insert(
enqueued_tasks.end(), std::make_move_iterator(new_tasks.begin()), std::make_move_iterator(new_tasks.end()));
}
if (!exception)
exception = new_exception;
cond.notify_all();
}
});
}
for (auto & thread : threads)
thread.join();
if (exception)
std::rethrow_exception(exception);
else
need_rollback_completed_tasks = false;
}
}

View File

@ -0,0 +1,24 @@
#pragma once
#include <Parsers/ASTBackupQuery.h>
namespace DB
{
class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
class IRestoreTask;
using RestoreTaskPtr = std::unique_ptr<IRestoreTask>;
using RestoreTasks = std::vector<RestoreTaskPtr>;
struct RestoreSettings;
class Context;
using ContextMutablePtr = std::shared_ptr<Context>;
/// Prepares restore tasks.
RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements, const RestoreSettings & restore_settings);
/// Executes restore tasks.
void executeRestoreTasks(RestoreTasks && tasks, size_t num_threads);
}

View File

@ -0,0 +1,17 @@
#include <Backups/formatTableNameOrTemporaryTableName.h>
#include <Common/quoteString.h>
#include <Interpreters/DatabaseCatalog.h>
namespace DB
{
String formatTableNameOrTemporaryTableName(const DatabaseAndTableName & table_name)
{
if (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
return "temporary table " + backQuoteIfNeed(table_name.second);
else
return "table " + backQuoteIfNeed(table_name.first) + "." + backQuoteIfNeed(table_name.second);
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <base/types.h>
namespace DB
{
using DatabaseAndTableName = std::pair<String, String>;
/// Outputs either "table db_name.table_name" or "temporary table table_name".
String formatTableNameOrTemporaryTableName(const DatabaseAndTableName & table_name);
}

View File

@ -1,22 +0,0 @@
#include <Backups/hasCompatibleDataToRestoreTable.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/formatAST.h>
namespace DB
{
bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCreateQuery & query2)
{
/// TODO: Write more subtle condition here.
auto q1 = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query1.clone());
auto q2 = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query2.clone());
/// Remove UUIDs.
q1->uuid = UUIDHelpers::Nil;
q2->uuid = UUIDHelpers::Nil;
return serializeAST(*q1) == serializeAST(*q2);
}
}

View File

@ -1,11 +0,0 @@
#pragma once
namespace DB
{
class ASTCreateQuery;
/// Whether the data of the first table can be inserted to the second table.
bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCreateQuery & query2);
}

View File

@ -1,14 +0,0 @@
namespace DB
{
class BackupFactory;
void registerBackupEngineFile(BackupFactory &);
void registerBackupEngines(BackupFactory & factory)
{
registerBackupEngineFile(factory);
}
}

View File

@ -0,0 +1,168 @@
#include <Backups/BackupFactory.h>
#include <Backups/DirectoryBackup.h>
#include <Backups/ArchiveBackup.h>
#include <Common/quoteString.h>
#include <IO/Archives/hasRegisteredArchiveFileExtension.h>
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <filesystem>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int INVALID_CONFIG_PARAMETER;
}
namespace
{
namespace fs = std::filesystem;
[[noreturn]] void throwDiskIsAllowed(const String & disk_name)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} is not allowed for backups", disk_name);
}
[[noreturn]] void throwPathNotAllowed(const fs::path & path)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} is not allowed for backups", quoteString(String{path}));
}
void checkAllowedPathInConfigIsValid(const String & key, const fs::path & value)
{
if (value.empty() || value.is_relative())
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Configuration parameter {} has a wrong value {}", key, String{value});
}
/// Checks that a disk name and a path specified as parameters of Disk() are valid.
void checkDiskNameAndPath(const String & disk_name, fs::path & path, const Poco::Util::AbstractConfiguration & config)
{
String key = "backups.allowed_disk";
bool disk_name_found = false;
size_t counter = 0;
while (config.has(key))
{
if (config.getString(key) == disk_name)
{
disk_name_found = true;
break;
}
key = "backups.allowed_disk[" + std::to_string(++counter) + "]";
}
if (!disk_name_found)
throwDiskIsAllowed(disk_name);
path = path.lexically_normal();
if (!path.is_relative() || path.empty() || (*path.begin() == ".."))
throwPathNotAllowed(path);
}
/// Checks that a path specified as a parameter of File() is valid.
void checkPath(fs::path & path, const Poco::Util::AbstractConfiguration & config)
{
String key = "backups.allowed_path";
path = path.lexically_normal();
if (path.empty())
throwPathNotAllowed(path);
if (path.is_relative())
{
if (*path.begin() == "..")
throwPathNotAllowed(path);
auto base = fs::path(config.getString(key, ""));
checkAllowedPathInConfigIsValid(key, base);
path = base / path;
return;
}
bool path_found_in_config = false;
size_t counter = 0;
while (config.has(key))
{
auto base = fs::path(config.getString(key));
checkAllowedPathInConfigIsValid(key, base);
auto rel = path.lexically_relative(base);
if (!rel.empty() && (*rel.begin() != ".."))
{
path_found_in_config = true;
break;
}
key = "backups.allowed_path[" + std::to_string(++counter) + "]";
}
if (!path_found_in_config)
throwPathNotAllowed(path);
}
}
void registerBackupEnginesFileAndDisk(BackupFactory & factory)
{
auto creator_fn = [](const BackupFactory::CreateParams & params) -> std::unique_ptr<IBackup>
{
String backup_name = params.backup_info.toString();
const String & engine_name = params.backup_info.backup_engine_name;
const auto & args = params.backup_info.args;
DiskPtr disk;
fs::path path;
if (engine_name == "File")
{
if (args.size() != 1)
{
throw Exception(
"Backup engine 'File' requires 1 argument (path)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
path = args[0].safeGet<String>();
checkPath(path, params.context->getConfigRef());
}
else if (engine_name == "Disk")
{
if (args.size() != 2)
{
throw Exception(
"Backup engine 'Disk' requires 2 arguments (disk_name, path)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
String disk_name = args[0].safeGet<String>();
path = args[1].safeGet<String>();
checkDiskNameAndPath(disk_name, path, params.context->getConfigRef());
disk = params.context->getDisk(disk_name);
}
std::unique_ptr<IBackup> backup;
if (!path.has_filename() && !path.empty())
{
if (!params.password.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password is not applicable, backup cannot be encrypted");
backup = std::make_unique<DirectoryBackup>(backup_name, disk, path, params.context, params.base_backup_info);
}
else if (hasRegisteredArchiveFileExtension(path))
{
auto archive_backup = std::make_unique<ArchiveBackup>(backup_name, disk, path, params.context, params.base_backup_info);
archive_backup->setCompression(params.compression_method, params.compression_level);
archive_backup->setPassword(params.password);
backup = std::move(archive_backup);
}
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to backup must be either a directory or a path to an archive");
return backup;
};
factory.registerBackupEngine("File", creator_fn);
factory.registerBackupEngine("Disk", creator_fn);
}
}

View File

@ -1,282 +0,0 @@
#include <Backups/BackupRenamingConfig.h>
#include <Backups/renameInCreateQuery.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <TableFunctions/TableFunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
class RenameInCreateQueryTransformMatcher
{
public:
struct Data
{
BackupRenamingConfigPtr renaming_config;
ContextPtr context;
};
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
static void visit(ASTPtr & ast, const Data & data)
{
if (auto * create = ast->as<ASTCreateQuery>())
visitCreateQuery(*create, data);
else if (auto * expr = ast->as<ASTTableExpression>())
visitTableExpression(*expr, data);
else if (auto * function = ast->as<ASTFunction>())
visitFunction(*function, data);
else if (auto * dictionary = ast->as<ASTDictionary>())
visitDictionary(*dictionary, data);
}
private:
/// Replaces names of tables and databases used in a CREATE query, which can be either CREATE TABLE or
/// CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query.
static void visitCreateQuery(ASTCreateQuery & create, const Data & data)
{
if (create.temporary)
{
if (!create.table)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table name specified in the CREATE TEMPORARY TABLE query must not be empty");
create.setTable(data.renaming_config->getNewTemporaryTableName(create.getTable()));
}
else if (!create.table)
{
if (!create.database)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty");
create.setDatabase(data.renaming_config->getNewDatabaseName(create.getDatabase()));
}
else
{
if (!create.database)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE TABLE query must not be empty");
auto table_and_database_name = data.renaming_config->getNewTableName({create.getDatabase(), create.getTable()});
create.setDatabase(table_and_database_name.first);
create.setTable(table_and_database_name.second);
}
create.uuid = UUIDHelpers::Nil;
if (!create.as_table.empty() && !create.as_database.empty())
std::tie(create.as_database, create.as_table) = data.renaming_config->getNewTableName({create.as_database, create.as_table});
if (!create.to_table_id.table_name.empty() && !create.to_table_id.database_name.empty())
{
auto to_table = data.renaming_config->getNewTableName({create.to_table_id.database_name, create.to_table_id.table_name});
create.to_table_id = StorageID{to_table.first, to_table.second};
}
}
/// Replaces names of a database and a table in a expression like `db`.`table`
static void visitTableExpression(ASTTableExpression & expr, const Data & data)
{
if (!expr.database_and_table_name)
return;
ASTIdentifier * id = expr.database_and_table_name->as<ASTIdentifier>();
if (!id)
return;
auto table_id = id->createTable();
if (!table_id)
return;
const String & db_name = table_id->getDatabaseName();
const String & table_name = table_id->shortName();
if (db_name.empty() || table_name.empty())
return;
String new_db_name, new_table_name;
std::tie(new_db_name, new_table_name) = data.renaming_config->getNewTableName({db_name, table_name});
if ((new_db_name == db_name) && (new_table_name == table_name))
return;
expr.database_and_table_name = std::make_shared<ASTIdentifier>(Strings{new_db_name, new_table_name});
expr.children.push_back(expr.database_and_table_name);
}
/// Replaces names of tables and databases used in arguments of a table function or a table engine.
static void visitFunction(ASTFunction & function, const Data & data)
{
if ((function.name == "merge") || (function.name == "Merge"))
{
visitFunctionMerge(function, data);
}
else if ((function.name == "remote") || (function.name == "remoteSecure") || (function.name == "cluster") ||
(function.name == "clusterAllReplicas") || (function.name == "Distributed"))
{
visitFunctionRemote(function, data);
}
}
/// Replaces a database's name passed via an argument of the function merge() or the table engine Merge.
static void visitFunctionMerge(ASTFunction & function, const Data & data)
{
if (!function.arguments)
return;
/// The first argument is a database's name and we can rename it.
/// The second argument is a regular expression and we can do nothing about it.
auto & args = function.arguments->as<ASTExpressionList &>().children;
size_t db_name_arg_index = 0;
if (args.size() <= db_name_arg_index)
return;
String db_name = evaluateConstantExpressionForDatabaseName(args[db_name_arg_index], data.context)->as<ASTLiteral &>().value.safeGet<String>();
if (db_name.empty())
return;
String new_db_name = data.renaming_config->getNewDatabaseName(db_name);
if (new_db_name == db_name)
return;
args[db_name_arg_index] = std::make_shared<ASTLiteral>(new_db_name);
}
/// Replaces names of a table and a database passed via arguments of the function remote() or cluster() or the table engine Distributed.
static void visitFunctionRemote(ASTFunction & function, const Data & data)
{
if (!function.arguments)
return;
/// The first argument is an address or cluster's name, so we skip it.
/// The second argument can be either 'db.name' or just 'db' followed by the third argument 'table'.
auto & args = function.arguments->as<ASTExpressionList &>().children;
const auto * second_arg_as_function = args[1]->as<ASTFunction>();
if (second_arg_as_function && TableFunctionFactory::instance().isTableFunctionName(second_arg_as_function->name))
return;
size_t db_name_index = 1;
if (args.size() <= db_name_index)
return;
String name = evaluateConstantExpressionForDatabaseName(args[db_name_index], data.context)->as<ASTLiteral &>().value.safeGet<String>();
size_t table_name_index = static_cast<size_t>(-1);
QualifiedTableName qualified_name;
if (function.name == "Distributed")
qualified_name.table = name;
else
qualified_name = QualifiedTableName::parseFromString(name);
if (qualified_name.database.empty())
{
std::swap(qualified_name.database, qualified_name.table);
table_name_index = 2;
if (args.size() <= table_name_index)
return;
qualified_name.table = evaluateConstantExpressionForDatabaseName(args[table_name_index], data.context)->as<ASTLiteral &>().value.safeGet<String>();
}
const String & db_name = qualified_name.database;
const String & table_name = qualified_name.table;
if (db_name.empty() || table_name.empty())
return;
String new_db_name, new_table_name;
std::tie(new_db_name, new_table_name) = data.renaming_config->getNewTableName({db_name, table_name});
if ((new_db_name == db_name) && (new_table_name == table_name))
return;
if (table_name_index != static_cast<size_t>(-1))
{
if (new_db_name != db_name)
args[db_name_index] = std::make_shared<ASTLiteral>(new_db_name);
if (new_table_name != table_name)
args[table_name_index] = std::make_shared<ASTLiteral>(new_table_name);
}
else
{
args[db_name_index] = std::make_shared<ASTLiteral>(new_db_name);
args.insert(args.begin() + db_name_index + 1, std::make_shared<ASTLiteral>(new_table_name));
}
}
/// Replaces names of a table and a database used in source parameters of a dictionary.
static void visitDictionary(ASTDictionary & dictionary, const Data & data)
{
if (!dictionary.source || dictionary.source->name != "clickhouse" || !dictionary.source->elements)
return;
auto & elements = dictionary.source->elements->as<ASTExpressionList &>().children;
String db_name, table_name;
size_t db_name_index = static_cast<size_t>(-1);
size_t table_name_index = static_cast<size_t>(-1);
for (size_t i = 0; i != elements.size(); ++i)
{
auto & pair = elements[i]->as<ASTPair &>();
if (pair.first == "db")
{
if (db_name_index != static_cast<size_t>(-1))
return;
db_name = pair.second->as<ASTLiteral &>().value.safeGet<String>();
db_name_index = i;
}
else if (pair.first == "table")
{
if (table_name_index != static_cast<size_t>(-1))
return;
table_name = pair.second->as<ASTLiteral &>().value.safeGet<String>();
table_name_index = i;
}
}
if (db_name.empty() || table_name.empty())
return;
String new_db_name, new_table_name;
std::tie(new_db_name, new_table_name) = data.renaming_config->getNewTableName({db_name, table_name});
if ((new_db_name == db_name) && (new_table_name == table_name))
return;
if (new_db_name != db_name)
{
auto & pair = elements[db_name_index]->as<ASTPair &>();
pair.replace(pair.second, std::make_shared<ASTLiteral>(new_db_name));
}
if (new_table_name != table_name)
{
auto & pair = elements[table_name_index]->as<ASTPair &>();
pair.replace(pair.second, std::make_shared<ASTLiteral>(new_table_name));
}
}
};
using RenameInCreateQueryTransformVisitor = InDepthNodeVisitor<RenameInCreateQueryTransformMatcher, false>;
}
ASTPtr renameInCreateQuery(const ASTPtr & ast, const BackupRenamingConfigPtr & renaming_config, const ContextPtr & context)
{
auto new_ast = ast->clone();
try
{
RenameInCreateQueryTransformVisitor::Data data{renaming_config, context};
RenameInCreateQueryTransformVisitor{data}.visit(new_ast);
return new_ast;
}
catch (...)
{
tryLogCurrentException("Backup", "Error while renaming in AST");
return ast;
}
}
}

View File

@ -1,16 +0,0 @@
#pragma once
#include <memory>
namespace DB
{
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
class BackupRenamingConfig;
using BackupRenamingConfigPtr = std::shared_ptr<const BackupRenamingConfig>;
/// Changes names in AST according to the renaming settings.
ASTPtr renameInCreateQuery(const ASTPtr & ast, const BackupRenamingConfigPtr & renaming_config, const ContextPtr & context);
}

View File

@ -647,6 +647,12 @@ void ClientBase::processTextAsSingleQuery(const String & full_query)
void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr parsed_query)
{
if (fake_drop)
{
if (parsed_query->as<ASTDropQuery>())
return;
}
/// Rewrite query only when we have query parameters.
/// Note that if query is rewritten, comments in query are lost.
/// But the user often wants to see comments in server logs, query log, processlist, etc.
@ -1970,25 +1976,23 @@ void ClientBase::readArguments(
for (int arg_num = 1; arg_num < argc; ++arg_num)
{
const char * arg = argv[arg_num];
std::string_view arg = argv[arg_num];
if (arg == "--external"sv)
if (arg == "--external")
{
in_external_group = true;
external_tables_arguments.emplace_back(Arguments{""});
}
/// Options with value after equal sign.
else if (in_external_group
&& (0 == strncmp(arg, "--file=", strlen("--file=")) || 0 == strncmp(arg, "--name=", strlen("--name="))
|| 0 == strncmp(arg, "--format=", strlen("--format=")) || 0 == strncmp(arg, "--structure=", strlen("--structure="))
|| 0 == strncmp(arg, "--types=", strlen("--types="))))
else if (
in_external_group
&& (arg.starts_with("--file=") || arg.starts_with("--name=") || arg.starts_with("--format=") || arg.starts_with("--structure=")
|| arg.starts_with("--types=")))
{
external_tables_arguments.back().emplace_back(arg);
}
/// Options with value after whitespace.
else if (in_external_group
&& (arg == "--file"sv || arg == "--name"sv || arg == "--format"sv
|| arg == "--structure"sv || arg == "--types"sv))
else if (in_external_group && (arg == "--file" || arg == "--name" || arg == "--format" || arg == "--structure" || arg == "--types"))
{
if (arg_num + 1 < argc)
{
@ -2005,20 +2009,12 @@ void ClientBase::readArguments(
in_external_group = false;
/// Parameter arg after underline.
if (startsWith(arg, "--param_"))
if (arg.starts_with("--param_"))
{
const char * param_continuation = arg + strlen("--param_");
const char * equal_pos = strchr(param_continuation, '=');
auto param_continuation = arg.substr(strlen("--param_"));
auto equal_pos = param_continuation.find_first_of('=');
if (equal_pos == param_continuation)
throw Exception("Parameter name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
if (equal_pos)
{
/// param_name=value
query_parameters.emplace(String(param_continuation, equal_pos), String(equal_pos + 1));
}
else
if (equal_pos == std::string::npos)
{
/// param_name value
++arg_num;
@ -2027,12 +2023,20 @@ void ClientBase::readArguments(
arg = argv[arg_num];
query_parameters.emplace(String(param_continuation), String(arg));
}
else
{
if (equal_pos == 0)
throw Exception("Parameter name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
/// param_name=value
query_parameters.emplace(param_continuation.substr(0, equal_pos), param_continuation.substr(equal_pos + 1));
}
}
else if (startsWith(arg, "--host") || startsWith(arg, "-h"))
else if (arg.starts_with("--host") || arg.starts_with("-h"))
{
std::string host_arg;
/// --host host
if (arg == "--host"sv || arg == "-h"sv)
if (arg == "--host" || arg == "-h")
{
++arg_num;
if (arg_num >= argc)
@ -2059,11 +2063,11 @@ void ClientBase::readArguments(
prev_host_arg = host_arg;
}
}
else if (startsWith(arg, "--port"))
else if (arg.starts_with("--port"))
{
std::string port_arg = arg;
auto port_arg = String{arg};
/// --port port
if (arg == "--port"sv)
if (arg == "--port")
{
port_arg.push_back('=');
++arg_num;
@ -2088,7 +2092,7 @@ void ClientBase::readArguments(
prev_port_arg = port_arg;
}
}
else if (arg == "--allow_repeated_settings"sv)
else if (arg == "--allow_repeated_settings")
allow_repeated_settings = true;
else
common_arguments.emplace_back(arg);

View File

@ -252,6 +252,8 @@ protected:
QueryProcessingStage::Enum query_processing_stage;
bool fake_drop = false;
struct HostAndPort
{
String host;

View File

@ -60,7 +60,8 @@ public:
return ColumnArray::create(nested_column->assumeMutable());
}
template <typename ... Args, typename = typename std::enable_if<IsMutableColumns<Args ...>::value>::type>
template <typename ... Args>
requires (IsMutableColumns<Args ...>::value)
static MutablePtr create(Args &&... args) { return Base::create(std::forward<Args>(args)...); }
/** On the index i there is an offset to the beginning of the i + 1 -th element. */

View File

@ -36,7 +36,8 @@ public:
static Ptr create(const ColumnPtr & column) { return ColumnMap::create(column->assumeMutable()); }
static Ptr create(ColumnPtr && arg) { return create(arg); }
template <typename ... Args, typename = typename std::enable_if<IsMutableColumns<Args ...>::value>::type>
template <typename ... Args>
requires (IsMutableColumns<Args ...>::value)
static MutablePtr create(Args &&... args) { return Base::create(std::forward<Args>(args)...); }
std::string getName() const override;

View File

@ -41,7 +41,8 @@ public:
return ColumnNullable::create(nested_column_->assumeMutable(), null_map_->assumeMutable());
}
template <typename ... Args, typename = typename std::enable_if<IsMutableColumns<Args ...>::value>::type>
template <typename ... Args>
requires (IsMutableColumns<Args ...>::value)
static MutablePtr create(Args &&... args) { return Base::create(std::forward<Args>(args)...); }
const char * getFamilyName() const override { return "Nullable"; }

View File

@ -37,7 +37,8 @@ public:
return Base::create(values_->assumeMutable(), offsets_->assumeMutable(), size_);
}
template <typename TColumnPtr, typename = typename std::enable_if<IsMutableColumns<TColumnPtr>::value>::type>
template <typename TColumnPtr>
requires IsMutableColumns<TColumnPtr>::value
static MutablePtr create(TColumnPtr && values_, TColumnPtr && offsets_, size_t size_)
{
return Base::create(std::forward<TColumnPtr>(values_), std::forward<TColumnPtr>(offsets_), size_);
@ -48,7 +49,8 @@ public:
return Base::create(values_->assumeMutable());
}
template <typename TColumnPtr, typename = typename std::enable_if<IsMutableColumns<TColumnPtr>::value>::type>
template <typename TColumnPtr>
requires IsMutableColumns<TColumnPtr>::value
static MutablePtr create(TColumnPtr && values_)
{
return Base::create(std::forward<TColumnPtr>(values_));

View File

@ -35,7 +35,8 @@ public:
static Ptr create(const TupleColumns & columns);
static Ptr create(Columns && arg) { return create(arg); }
template <typename Arg, typename = typename std::enable_if<std::is_rvalue_reference<Arg &&>::value>::type>
template <typename Arg>
requires std::is_rvalue_reference_v<Arg &&>
static MutablePtr create(Arg && arg) { return Base::create(std::forward<Arg>(arg)); }
std::string getName() const override;

View File

@ -122,7 +122,8 @@ const uint32_t kMaxAbbreviationEntries = 1000;
// Read (bitwise) one object of type T
template <typename T>
std::enable_if_t<std::is_trivial_v<T> && std::is_standard_layout_v<T>, T> read(std::string_view & sp)
requires std::is_trivial_v<T> && std::is_standard_layout_v<T>
T read(std::string_view & sp)
{
SAFE_CHECK(sp.size() >= sizeof(T), "underflow");
T x;

View File

@ -575,7 +575,7 @@
M(604, BACKUP_ENTRY_ALREADY_EXISTS) \
M(605, BACKUP_ENTRY_NOT_FOUND) \
M(606, BACKUP_IS_EMPTY) \
M(607, BACKUP_ELEMENT_DUPLICATE) \
M(607, CANNOT_RESTORE_DATABASE) \
M(608, CANNOT_RESTORE_TABLE) \
M(609, FUNCTION_ALREADY_EXISTS) \
M(610, CANNOT_DROP_FUNCTION) \
@ -614,6 +614,9 @@
M(643, CANNOT_UNPACK_ARCHIVE) \
M(644, REMOTE_FS_OBJECT_CACHE_ERROR) \
M(645, NUMBER_OF_DIMENSIONS_MISMATHED) \
M(646, CANNOT_BACKUP_DATABASE) \
M(647, CANNOT_BACKUP_TABLE) \
M(648, WRONG_DDL_RENAMING_SETTINGS) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -1,13 +1,15 @@
#include <base/JSON.h>
#include <Common/FileChecker.h>
#include <Common/escapeForFileName.h>
#include <Disks/IDisk.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Common/escapeForFileName.h>
#include <base/JSON.h>
#include <Common/FileChecker.h>
namespace fs = std::filesystem;
namespace DB
{
@ -19,6 +21,10 @@ namespace ErrorCodes
}
FileChecker::FileChecker(const String & file_info_path_) : FileChecker(nullptr, file_info_path_)
{
}
FileChecker::FileChecker(DiskPtr disk_, const String & file_info_path_) : disk(std::move(disk_))
{
setPath(file_info_path_);
@ -45,8 +51,8 @@ String FileChecker::getPath() const
void FileChecker::update(const String & full_file_path)
{
bool exists = disk->exists(full_file_path);
auto real_size = exists ? disk->getFileSize(full_file_path) : 0; /// No race condition assuming no one else is working with these files.
bool exists = fileReallyExists(full_file_path);
auto real_size = exists ? getRealFileSize(full_file_path) : 0; /// No race condition assuming no one else is working with these files.
map[fileName(full_file_path)] = real_size;
}
@ -74,8 +80,8 @@ CheckResults FileChecker::check() const
{
const String & name = name_size.first;
String path = parentPath(files_info_path) + name;
bool exists = disk->exists(path);
auto real_size = exists ? disk->getFileSize(path) : 0; /// No race condition assuming no one else is working with these files.
bool exists = fileReallyExists(path);
auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files.
if (real_size != name_size.second)
{
@ -99,8 +105,8 @@ void FileChecker::repair()
const String & name = name_size.first;
size_t expected_size = name_size.second;
String path = parentPath(files_info_path) + name;
bool exists = disk->exists(path);
auto real_size = exists ? disk->getFileSize(path) : 0; /// No race condition assuming no one else is working with these files.
bool exists = fileReallyExists(path);
auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files.
if (real_size < expected_size)
throw Exception(ErrorCodes::UNEXPECTED_END_OF_FILE, "Size of {} is less than expected. Size is {} but should be {}.",
@ -119,7 +125,7 @@ void FileChecker::save() const
std::string tmp_files_info_path = parentPath(files_info_path) + "tmp_" + fileName(files_info_path);
{
std::unique_ptr<WriteBuffer> out = disk->writeFile(tmp_files_info_path);
std::unique_ptr<WriteBuffer> out = disk ? disk->writeFile(tmp_files_info_path) : std::make_unique<WriteBufferFromFile>(tmp_files_info_path);
/// So complex JSON structure - for compatibility with the old format.
writeCString("{\"clickhouse\":{", *out);
@ -141,17 +147,20 @@ void FileChecker::save() const
out->next();
}
disk->replaceFile(tmp_files_info_path, files_info_path);
if (disk)
disk->replaceFile(tmp_files_info_path, files_info_path);
else
fs::rename(tmp_files_info_path, files_info_path);
}
void FileChecker::load()
{
map.clear();
if (!disk->exists(files_info_path))
if (!fileReallyExists(files_info_path))
return;
std::unique_ptr<ReadBuffer> in = disk->readFile(files_info_path);
std::unique_ptr<ReadBuffer> in = disk ? disk->readFile(files_info_path) : std::make_unique<ReadBufferFromFile>(files_info_path);
WriteBufferFromOwnString out;
/// The JSON library does not support whitespace. We delete them. Inefficient.
@ -169,4 +178,14 @@ void FileChecker::load()
map[unescapeForFileName(file.getName())] = file.getValue()["size"].toUInt();
}
bool FileChecker::fileReallyExists(const String & path_) const
{
return disk ? disk->exists(path_) : fs::exists(path_);
}
size_t FileChecker::getRealFileSize(const String & path_) const
{
return disk ? disk->getFileSize(path_) : fs::file_size(path_);
}
}

View File

@ -2,16 +2,19 @@
#include <base/logger_useful.h>
#include <Storages/CheckResults.h>
#include <Disks/IDisk.h>
namespace DB
{
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
/// Stores the sizes of all columns, and can check whether the columns are corrupted.
class FileChecker
{
public:
FileChecker(const String & file_info_path_);
FileChecker(DiskPtr disk_, const String & file_info_path_);
void setPath(const String & file_info_path_);
@ -36,6 +39,9 @@ public:
private:
void load();
bool fileReallyExists(const String & path_) const;
size_t getRealFileSize(const String & path_) const;
const DiskPtr disk;
const Poco::Logger * log = &Poco::Logger::get("FileChecker");

View File

@ -73,8 +73,8 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value)
}
template <typename T>
inline typename std::enable_if<(sizeof(T) > sizeof(DB::UInt64)), DB::UInt64>::type
intHashCRC32(const T & x, DB::UInt64 updated_value)
requires (sizeof(T) > sizeof(DB::UInt64))
inline DB::UInt64 intHashCRC32(const T & x, DB::UInt64 updated_value)
{
const auto * begin = reinterpret_cast<const char *>(&x);
for (size_t i = 0; i < sizeof(T); i += sizeof(UInt64))
@ -155,7 +155,8 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 up
}
template <typename T>
inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) <= sizeof(UInt64)), T> key)
requires (sizeof(T) <= sizeof(UInt64))
inline size_t DefaultHash64(T key)
{
union
{
@ -169,7 +170,8 @@ inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) <= sizeof(UInt64)), T> k
template <typename T>
inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) > sizeof(UInt64)), T> key)
requires (sizeof(T) > sizeof(UInt64))
inline size_t DefaultHash64(T key)
{
if constexpr (is_big_int_v<T> && sizeof(T) == 16)
{
@ -217,7 +219,8 @@ struct DefaultHash<T>
template <typename T> struct HashCRC32;
template <typename T>
inline size_t hashCRC32(std::enable_if_t<(sizeof(T) <= sizeof(UInt64)), T> key)
requires (sizeof(T) <= sizeof(UInt64))
inline size_t hashCRC32(T key)
{
union
{
@ -230,7 +233,8 @@ inline size_t hashCRC32(std::enable_if_t<(sizeof(T) <= sizeof(UInt64)), T> key)
}
template <typename T>
inline size_t hashCRC32(std::enable_if_t<(sizeof(T) > sizeof(UInt64)), T> key)
requires (sizeof(T) > sizeof(UInt64))
inline size_t hashCRC32(T key)
{
return intHashCRC32(key, -1);
}

View File

@ -78,8 +78,7 @@ template <UInt64 MaxValue> struct MinCounterType
};
/// Denominator of expression for HyperLogLog algorithm.
template <UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
DenominatorMode denominator_mode, typename Enable = void>
template <UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType, DenominatorMode denominator_mode>
class Denominator;
/// Returns true if rank storage is big.
@ -89,11 +88,12 @@ constexpr bool isBigRankStore(UInt8 precision)
}
/// Used to deduce denominator type depending on options provided.
template <typename HashValueType, typename DenominatorType, DenominatorMode denominator_mode, typename Enable = void>
template <typename HashValueType, typename DenominatorType, DenominatorMode denominator_mode>
struct IntermediateDenominator;
template <typename DenominatorType, DenominatorMode denominator_mode>
struct IntermediateDenominator<UInt32, DenominatorType, denominator_mode, std::enable_if_t<denominator_mode != DenominatorMode::ExactType>>
requires (denominator_mode != DenominatorMode::ExactType)
struct IntermediateDenominator<UInt32, DenominatorType, denominator_mode>
{
using Type = double;
};
@ -113,11 +113,9 @@ struct IntermediateDenominator<HashValueType, DenominatorType, DenominatorMode::
/// "Lightweight" implementation of expression's denominator for HyperLogLog algorithm.
/// Uses minimum amount of memory, but estimates may be unstable.
/// Satisfiable when rank storage is small enough.
template <UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
DenominatorMode denominator_mode>
class __attribute__((__packed__)) Denominator<precision, max_rank, HashValueType, DenominatorType,
denominator_mode,
std::enable_if_t<!details::isBigRankStore(precision) || !(denominator_mode == DenominatorMode::StableIfBig)>>
template <UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType, DenominatorMode denominator_mode>
requires (!details::isBigRankStore(precision)) || (!(denominator_mode == DenominatorMode::StableIfBig))
class __attribute__((__packed__)) Denominator<precision, max_rank, HashValueType, DenominatorType, denominator_mode>
{
private:
using T = typename IntermediateDenominator<HashValueType, DenominatorType, denominator_mode>::Type;
@ -156,11 +154,9 @@ private:
/// Fully-functional version of expression's denominator for HyperLogLog algorithm.
/// Spends more space that lightweight version. Estimates will always be stable.
/// Used when rank storage is big.
template <UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
DenominatorMode denominator_mode>
class __attribute__((__packed__)) Denominator<precision, max_rank, HashValueType, DenominatorType,
denominator_mode,
std::enable_if_t<details::isBigRankStore(precision) && denominator_mode == DenominatorMode::StableIfBig>>
template <UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType, DenominatorMode denominator_mode>
requires (details::isBigRankStore(precision)) && (denominator_mode == DenominatorMode::StableIfBig)
class __attribute__((__packed__)) Denominator<precision, max_rank, HashValueType, DenominatorType, denominator_mode>
{
public:
Denominator(DenominatorType initial_value) /// NOLINT

View File

@ -129,7 +129,7 @@ public:
IntervalTree() { nodes.resize(1); }
template <typename TValue = Value, std::enable_if_t<std::is_same_v<TValue, IntervalTreeVoidValue>, bool> = true>
template <typename TValue = Value>
requires std::is_same_v<Value, IntervalTreeVoidValue>
ALWAYS_INLINE bool emplace(Interval interval)
{
@ -157,19 +157,22 @@ public:
return true;
}
template <typename TValue = Value, std::enable_if_t<std::is_same_v<TValue, IntervalTreeVoidValue>, bool> = true>
template <typename TValue = Value>
requires std::is_same_v<TValue, IntervalTreeVoidValue>
bool insert(Interval interval)
{
return emplace(interval);
}
template <typename TValue = Value, std::enable_if_t<!std::is_same_v<TValue, IntervalTreeVoidValue>, bool> = true>
template <typename TValue = Value>
requires (!std::is_same_v<TValue, IntervalTreeVoidValue>)
bool insert(Interval interval, const Value & value)
{
return emplace(interval, value);
}
template <typename TValue = Value, std::enable_if_t<!std::is_same_v<TValue, IntervalTreeVoidValue>, bool> = true>
template <typename TValue = Value>
requires (!std::is_same_v<TValue, IntervalTreeVoidValue>)
bool insert(Interval interval, Value && value)
{
return emplace(interval, std::move(value));

View File

@ -127,7 +127,14 @@ PoolWithFailover::Entry PoolWithFailover::get()
/// If we cannot connect to some replica due to pool overflow, than we will wait and connect.
PoolPtr * full_pool = nullptr;
std::map<std::string, std::tuple<std::string, int>> error_detail;
struct ErrorDetail
{
int code;
std::string description;
};
std::unordered_map<std::string, ErrorDetail> replica_name_to_error_detail;
for (size_t try_no = 0; try_no < max_tries; ++try_no)
{
@ -161,15 +168,8 @@ PoolWithFailover::Entry PoolWithFailover::get()
}
app.logger().warning("Connection to " + pool->getDescription() + " failed: " + e.displayText());
//save all errors to error_detail
if (error_detail.contains(pool->getDescription()))
{
error_detail[pool->getDescription()] = {e.displayText(), e.code()};
}
else
{
error_detail.insert({pool->getDescription(), {e.displayText(), e.code()}});
}
replica_name_to_error_detail.insert_or_assign(pool->getDescription(), ErrorDetail{e.code(), e.displayText()});
continue;
}
@ -189,15 +189,19 @@ PoolWithFailover::Entry PoolWithFailover::get()
DB::WriteBufferFromOwnString message;
message << "Connections to all replicas failed: ";
for (auto it = replicas_by_priority.begin(); it != replicas_by_priority.end(); ++it)
{
for (auto jt = it->second.begin(); jt != it->second.end(); ++jt)
{
message << (it == replicas_by_priority.begin() && jt == it->second.begin() ? "" : ", ") << (*jt)->getDescription();
if (error_detail.contains((*jt)->getDescription()))
if (auto error_detail_it = replica_name_to_error_detail.find(((*jt)->getDescription()));
error_detail_it != replica_name_to_error_detail.end())
{
std::tuple<std::string, int> error_and_code = error_detail[(*jt)->getDescription()];
message << ", ERROR " << std::get<1>(error_and_code) << " : " << std::get<0>(error_and_code);
const auto & [code, description] = error_detail_it->second;
message << ", ERROR " << code << " : " << description;
}
}
}
throw Poco::Exception(message.str());
}

View File

@ -259,7 +259,8 @@ namespace details
{
// To avoid stack overflow when converting to type with no appropriate c-tor,
// resulting in endless recursive calls from `Value::get<T>()` to `Value::operator T()` to `Value::get<T>()` to ...
template <typename T, typename std::enable_if_t<std::is_constructible_v<T, Value>>>
template <typename T>
requires std::is_constructible_v<T, Value>
inline T contructFromValue(const Value & val)
{
return T(val);

View File

@ -240,7 +240,8 @@ template <> struct NearestFieldTypeImpl<AggregateFunctionStateData> { using Type
// For enum types, use the field type that corresponds to their underlying type.
template <typename T>
struct NearestFieldTypeImpl<T, std::enable_if_t<std::is_enum_v<T>>>
requires std::is_enum_v<T>
struct NearestFieldTypeImpl<T>
{
using Type = NearestFieldType<std::underlying_type_t<T>>;
};
@ -669,7 +670,8 @@ private:
}
template <typename CharT>
std::enable_if_t<sizeof(CharT) == 1> assignString(const CharT * data, size_t size)
requires (sizeof(CharT) == 1)
void assignString(const CharT * data, size_t size)
{
assert(which == Types::String);
String * ptr = reinterpret_cast<String *>(&storage);
@ -704,7 +706,8 @@ private:
}
template <typename CharT>
std::enable_if_t<sizeof(CharT) == 1> create(const CharT * data, size_t size)
requires (sizeof(CharT) == 1)
void create(const CharT * data, size_t size)
{
new (&storage) String(reinterpret_cast<const char *>(data), size);
which = Types::String;

View File

@ -17,7 +17,8 @@ struct MultiEnum
: MultiEnum((toBitFlag(v) | ... | 0u))
{}
template <typename ValueType, typename = std::enable_if_t<std::is_convertible_v<ValueType, StorageType>>>
template <typename ValueType>
requires std::is_convertible_v<ValueType, StorageType>
constexpr explicit MultiEnum(ValueType v)
: bitset(v)
{

View File

@ -50,6 +50,9 @@ public:
void alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override;
/// This database can contain tables to backup.
bool hasTablesToBackup() const override { return true; }
private:
String data_path;
using NameToASTCreate = std::unordered_map<String, ASTPtr>;

View File

@ -36,6 +36,9 @@ public:
const StorageID & table_id,
const StorageInMemoryMetadata & metadata) override;
/// This database can contain tables to backup.
bool hasTablesToBackup() const override { return true; }
protected:
virtual void commitAlterTable(
const StorageID & table_id,

View File

@ -289,12 +289,6 @@ public:
throw Exception(getEngineName() + ": RENAME DATABASE is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
/// Whether the contained tables should be written to a backup.
virtual DatabaseTablesIteratorPtr getTablesIteratorForBackup(ContextPtr context) const
{
return getTablesIterator(context); /// By default we backup each table.
}
/// Returns path for persistent data storage if the database supports it, empty string otherwise
virtual String getDataPath() const { return {}; }
@ -335,6 +329,10 @@ public:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database engine {} does not run a replication thread!", getEngineName());
}
/// Returns true if the backup of the database is hollow, which means it doesn't contain
/// any tables which can be stored to a backup.
virtual bool hasTablesToBackup() const { return false; }
virtual ~IDatabase() = default;
protected:

View File

@ -791,7 +791,8 @@ static ColumnUInt8::MutablePtr copyNullMap(ColumnPtr col)
}
template <typename FromDataType, typename Name>
struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType, DataTypeString>, DataTypeString>, Name, ConvertDefaultBehaviorTag>
requires (!std::is_same_v<FromDataType, DataTypeString>)
struct ConvertImpl<FromDataType, DataTypeString, Name, ConvertDefaultBehaviorTag>
{
using FromFieldType = typename FromDataType::FieldType;
using ColVecType = ColumnVectorOrDecimal<FromFieldType>;
@ -1324,19 +1325,23 @@ struct ConvertThroughParsing
template <typename ToDataType, typename Name>
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeString>, DataTypeString>, ToDataType, Name, ConvertDefaultBehaviorTag>
requires (!std::is_same_v<ToDataType, DataTypeString>)
struct ConvertImpl<DataTypeString, ToDataType, Name, ConvertDefaultBehaviorTag>
: ConvertThroughParsing<DataTypeString, ToDataType, Name, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::Normal> {};
template <typename ToDataType, typename Name>
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeFixedString>, DataTypeFixedString>, ToDataType, Name, ConvertDefaultBehaviorTag>
requires (!std::is_same_v<ToDataType, DataTypeFixedString>)
struct ConvertImpl<DataTypeFixedString, ToDataType, Name, ConvertDefaultBehaviorTag>
: ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::Normal> {};
template <typename ToDataType, typename Name>
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeString>, DataTypeString>, ToDataType, Name, ConvertReturnNullOnErrorTag>
requires (!std::is_same_v<ToDataType, DataTypeString>)
struct ConvertImpl<DataTypeString, ToDataType, Name, ConvertReturnNullOnErrorTag>
: ConvertThroughParsing<DataTypeString, ToDataType, Name, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::Normal> {};
template <typename ToDataType, typename Name>
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeFixedString>, DataTypeFixedString>, ToDataType, Name, ConvertReturnNullOnErrorTag>
requires (!std::is_same_v<ToDataType, DataTypeFixedString>)
struct ConvertImpl<DataTypeFixedString, ToDataType, Name, ConvertReturnNullOnErrorTag>
: ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::Normal> {};
/// Generic conversion of any type from String. Used for complex types: Array and Tuple or types with custom serialization.
@ -1391,7 +1396,8 @@ struct ConvertImpl<DataTypeString, DataTypeUInt32, NameToUnixTimestamp, ConvertR
/** If types are identical, just take reference to column.
*/
template <typename T, typename Name>
struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name, ConvertDefaultBehaviorTag>
requires (!T::is_parametric)
struct ConvertImpl<T, T, Name, ConvertDefaultBehaviorTag>
{
template <typename Additions = void *>
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/,
@ -2739,8 +2745,8 @@ private:
}
template <typename ToDataType>
std::enable_if_t<IsDataTypeDecimal<ToDataType>, WrapperType>
createDecimalWrapper(const DataTypePtr & from_type, const ToDataType * to_type, bool requested_result_is_nullable) const
requires IsDataTypeDecimal<ToDataType>
WrapperType createDecimalWrapper(const DataTypePtr & from_type, const ToDataType * to_type, bool requested_result_is_nullable) const
{
TypeIndex type_index = from_type->getTypeId();
UInt32 scale = to_type->getScale();

View File

@ -23,7 +23,6 @@ public:
{
UInt64 uncompressed_size;
UInt64 compressed_size;
int compression_method;
bool is_encrypted;
};

View File

@ -29,7 +29,7 @@ public:
/// Sets compression method and level.
/// Changing them will affect next file in the archive.
virtual void setCompression(int /* compression_method */, int /* compression_level */ = kDefaultCompressionLevel) {} /// NOLINT
virtual void setCompression(const String & /* compression_method */, int /* compression_level */ = kDefaultCompressionLevel) {}
/// Sets password. If the password is not empty it will enable encryption in the archive.
virtual void setPassword(const String & /* password */) {}

View File

@ -1,6 +1,7 @@
#include <IO/Archives/ZipArchiveReader.h>
#if USE_MINIZIP
#include <IO/Archives/ZipArchiveWriter.h>
#include <IO/ReadBufferFromFileBase.h>
#include <Common/quoteString.h>
#include <unzip.h>
@ -18,6 +19,20 @@ namespace ErrorCodes
using RawHandle = unzFile;
namespace
{
void checkCompressionMethodIsEnabled(int compression_method_)
{
ZipArchiveWriter::checkCompressionMethodIsEnabled(compression_method_);
}
void checkEncryptionIsEnabled()
{
ZipArchiveWriter::checkEncryptionIsEnabled();
}
}
/// Holds a raw handle, calls acquireRawHandle() in the constructor and releaseRawHandle() in the destructor.
class ZipArchiveReader::HandleHolder
{
@ -108,7 +123,7 @@ public:
return *file_name;
}
const FileInfo & getFileInfo() const
const FileInfoImpl & getFileInfo() const
{
if (!file_info)
retrieveFileInfo();
@ -161,7 +176,7 @@ private:
std::shared_ptr<ZipArchiveReader> reader;
RawHandle raw_handle = nullptr;
mutable std::optional<String> file_name;
mutable std::optional<FileInfo> file_info;
mutable std::optional<FileInfoImpl> file_info;
};
@ -174,7 +189,7 @@ public:
, handle(std::move(handle_))
{
const auto & file_info = handle.getFileInfo();
checkCompressionMethodIsEnabled(static_cast<CompressionMethod>(file_info.compression_method));
checkCompressionMethodIsEnabled(file_info.compression_method);
const char * password_cstr = nullptr;
if (file_info.is_encrypted)
@ -227,7 +242,7 @@ public:
if (new_pos > static_cast<off_t>(file_info.uncompressed_size))
throw Exception("Seek position is out of bound", ErrorCodes::SEEK_POSITION_OUT_OF_BOUND);
if (file_info.compression_method == static_cast<int>(CompressionMethod::kStore))
if (file_info.compression_method == MZ_COMPRESS_METHOD_STORE)
{
/// unzSeek64() works only for non-compressed files.
checkResult(unzSeek64(raw_handle, off, whence));

View File

@ -4,7 +4,6 @@
#if USE_MINIZIP
#include <IO/Archives/IArchiveReader.h>
#include <IO/Archives/ZipArchiveWriter.h>
#include <base/shared_ptr_helper.h>
#include <mutex>
#include <vector>
@ -20,8 +19,6 @@ class SeekableReadBuffer;
class ZipArchiveReader : public shared_ptr_helper<ZipArchiveReader>, public IArchiveReader
{
public:
using CompressionMethod = ZipArchiveWriter::CompressionMethod;
~ZipArchiveReader() override;
/// Returns true if there is a specified file in the archive.
@ -45,11 +42,6 @@ public:
/// Sets password used to decrypt the contents of the files in the archive.
void setPassword(const String & password_) override;
/// Utility functions.
static CompressionMethod parseCompressionMethod(const String & str) { return ZipArchiveWriter::parseCompressionMethod(str); }
static void checkCompressionMethodIsEnabled(CompressionMethod method) { ZipArchiveWriter::checkCompressionMethodIsEnabled(method); }
static void checkEncryptionIsEnabled() { ZipArchiveWriter::checkEncryptionIsEnabled(); }
private:
/// Constructs an archive's reader that will read from a file in the local filesystem.
explicit ZipArchiveReader(const String & path_to_archive_);
@ -66,6 +58,11 @@ private:
void init();
struct FileInfoImpl : public FileInfo
{
int compression_method;
};
HandleHolder acquireHandle();
RawHandle acquireRawHandle();
void releaseRawHandle(RawHandle handle_);

View File

@ -80,7 +80,7 @@ public:
{
auto compress_method = handle.getWriter()->compression_method;
auto compress_level = handle.getWriter()->compression_level;
checkCompressionMethodIsEnabled(static_cast<CompressionMethod>(compress_method));
checkCompressionMethodIsEnabled(compress_method);
const char * password_cstr = nullptr;
const String & password_str = handle.getWriter()->password;
@ -238,7 +238,7 @@ ZipArchiveWriter::ZipArchiveWriter(const String & path_to_archive_)
}
ZipArchiveWriter::ZipArchiveWriter(const String & path_to_archive_, std::unique_ptr<WriteBuffer> archive_write_buffer_)
: path_to_archive(path_to_archive_)
: path_to_archive(path_to_archive_), compression_method(MZ_COMPRESS_METHOD_DEFLATE)
{
if (archive_write_buffer_)
handle = StreamFromWriteBuffer::open(std::move(archive_write_buffer_));
@ -246,6 +246,7 @@ ZipArchiveWriter::ZipArchiveWriter(const String & path_to_archive_, std::unique_
handle = zipOpen64(path_to_archive.c_str(), /* append= */ false);
if (!handle)
throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Couldn't create zip archive {}", quoteString(path_to_archive));
}
ZipArchiveWriter::~ZipArchiveWriter()
@ -274,10 +275,10 @@ bool ZipArchiveWriter::isWritingFile() const
return !handle;
}
void ZipArchiveWriter::setCompression(int compression_method_, int compression_level_)
void ZipArchiveWriter::setCompression(const String & compression_method_, int compression_level_)
{
std::lock_guard lock{mutex};
compression_method = compression_method_;
compression_method = compressionMethodToInt(compression_method_);
compression_level = compression_level_;
}
@ -287,48 +288,62 @@ void ZipArchiveWriter::setPassword(const String & password_)
password = password_;
}
ZipArchiveWriter::CompressionMethod ZipArchiveWriter::parseCompressionMethod(const String & str)
int ZipArchiveWriter::compressionMethodToInt(const String & compression_method_)
{
if (str.empty())
return CompressionMethod::kDeflate; /// Default compression method is DEFLATE.
else if (boost::iequals(str, "store"))
return CompressionMethod::kStore;
else if (boost::iequals(str, "deflate"))
return CompressionMethod::kDeflate;
else if (boost::iequals(str, "bzip2"))
return CompressionMethod::kBzip2;
else if (boost::iequals(str, "lzma"))
return CompressionMethod::kLzma;
else if (boost::iequals(str, "zstd"))
return CompressionMethod::kZstd;
else if (boost::iequals(str, "xz"))
return CompressionMethod::kXz;
if (compression_method_.empty())
return MZ_COMPRESS_METHOD_DEFLATE; /// By default the compression method is "deflate".
else if (compression_method_ == kStore)
return MZ_COMPRESS_METHOD_STORE;
else if (compression_method_ == kDeflate)
return MZ_COMPRESS_METHOD_DEFLATE;
else if (compression_method_ == kBzip2)
return MZ_COMPRESS_METHOD_BZIP2;
else if (compression_method_ == kLzma)
return MZ_COMPRESS_METHOD_LZMA;
else if (compression_method_ == kZstd)
return MZ_COMPRESS_METHOD_ZSTD;
else if (compression_method_ == kXz)
return MZ_COMPRESS_METHOD_XZ;
else
throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", str);
throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", compression_method_);
}
String ZipArchiveWriter::intToCompressionMethod(int compression_method_)
{
switch (compression_method_)
{
case MZ_COMPRESS_METHOD_STORE: return kStore;
case MZ_COMPRESS_METHOD_DEFLATE: return kDeflate;
case MZ_COMPRESS_METHOD_BZIP2: return kBzip2;
case MZ_COMPRESS_METHOD_LZMA: return kLzma;
case MZ_COMPRESS_METHOD_ZSTD: return kZstd;
case MZ_COMPRESS_METHOD_XZ: return kXz;
}
throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", compression_method_);
}
/// Checks that a passed compression method can be used.
void ZipArchiveWriter::checkCompressionMethodIsEnabled(CompressionMethod method)
void ZipArchiveWriter::checkCompressionMethodIsEnabled(int compression_method_)
{
switch (method)
switch (compression_method_)
{
case CompressionMethod::kStore: [[fallthrough]];
case CompressionMethod::kDeflate:
case CompressionMethod::kLzma:
case CompressionMethod::kXz:
case CompressionMethod::kZstd:
case MZ_COMPRESS_METHOD_STORE: [[fallthrough]];
case MZ_COMPRESS_METHOD_DEFLATE:
case MZ_COMPRESS_METHOD_LZMA:
case MZ_COMPRESS_METHOD_ZSTD:
case MZ_COMPRESS_METHOD_XZ:
return;
case CompressionMethod::kBzip2:
case MZ_COMPRESS_METHOD_BZIP2:
{
#if USE_BZIP2
return;
#else
throw Exception("BZIP2 compression method is disabled", ErrorCodes::SUPPORT_IS_DISABLED);
throw Exception("bzip2 compression method is disabled", ErrorCodes::SUPPORT_IS_DISABLED);
#endif
}
}
throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", static_cast<int>(method));
throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", compression_method_);
}
/// Checks that encryption is enabled.

View File

@ -31,16 +31,12 @@ public:
bool isWritingFile() const override;
/// Supported compression methods.
enum class CompressionMethod
{
/// See mz.h
kStore = 0,
kDeflate = 8,
kBzip2 = 12,
kLzma = 14,
kZstd = 93,
kXz = 95,
};
static constexpr const char kStore[] = "store";
static constexpr const char kDeflate[] = "deflate";
static constexpr const char kBzip2[] = "bzip2";
static constexpr const char kLzma[] = "lzma";
static constexpr const char kZstd[] = "zstd";
static constexpr const char kXz[] = "xz";
/// Some compression levels.
enum class CompressionLevels
@ -53,7 +49,7 @@ public:
/// Sets compression method and level.
/// Changing them will affect next file in the archive.
void setCompression(int compression_method_, int compression_level_) override;
void setCompression(const String & compression_method_, int compression_level_) override;
/// Sets password. Only contents of the files are encrypted,
/// names of files are not encrypted.
@ -61,8 +57,9 @@ public:
void setPassword(const String & password_) override;
/// Utility functions.
static CompressionMethod parseCompressionMethod(const String & str);
static void checkCompressionMethodIsEnabled(CompressionMethod method);
static int compressionMethodToInt(const String & compression_method_);
static String intToCompressionMethod(int compression_method_);
static void checkCompressionMethodIsEnabled(int compression_method_);
static void checkEncryptionIsEnabled();
private:
@ -85,7 +82,7 @@ private:
[[noreturn]] void showError(const String & message) const;
const String path_to_archive;
int compression_method = static_cast<int>(CompressionMethod::kDeflate);
int compression_method; /// By default the compression method is "deflate".
int compression_level = kDefaultCompressionLevel;
String password;
RawHandle handle = nullptr;

View File

@ -0,0 +1,12 @@
#include <IO/Archives/hasRegisteredArchiveFileExtension.h>
namespace DB
{
bool hasRegisteredArchiveFileExtension(const String & path)
{
return path.ends_with(".zip") || path.ends_with(".zipx");
}
}

View File

@ -0,0 +1,12 @@
#pragma once
#include <base/types.h>
namespace DB
{
/// Returns true if a specified path has one of the registered file extensions for an archive.
bool hasRegisteredArchiveFileExtension(const String & path);
}

View File

@ -23,6 +23,12 @@ public:
assert(!buffers.empty());
}
ConcatReadBuffer(std::unique_ptr<ReadBuffer> buf1, std::unique_ptr<ReadBuffer> buf2) : ConcatReadBuffer()
{
appendBuffer(std::move(buf1));
appendBuffer(std::move(buf2));
}
ConcatReadBuffer(ReadBuffer & buf1, ReadBuffer & buf2) : ConcatReadBuffer()
{
appendBuffer(wrapReadBufferReference(buf1));

View File

@ -12,7 +12,8 @@ namespace DB
class ReadBufferFromMemory : public SeekableReadBuffer
{
public:
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
template <typename CharT>
requires (sizeof(CharT) == 1)
ReadBufferFromMemory(const CharT * buf, size_t size)
: SeekableReadBuffer(const_cast<char *>(reinterpret_cast<const char *>(buf)), size, 0) {}

View File

@ -869,8 +869,8 @@ inline void writeDateTimeUnixTimestamp(DateTime64 datetime64, UInt32 scale, Writ
/// Methods for output in binary format.
template <typename T>
inline std::enable_if_t<is_arithmetic_v<T>, void>
writeBinary(const T & x, WriteBuffer & buf) { writePODBinary(x, buf); }
requires is_arithmetic_v<T>
inline void writeBinary(const T & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const String & x, WriteBuffer & buf) { writeStringBinary(x, buf); }
inline void writeBinary(const StringRef & x, WriteBuffer & buf) { writeStringBinary(x, buf); }
@ -988,8 +988,8 @@ void writeText(Decimal<T> x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer
/// String, date, datetime are in single quotes with C-style escaping. Numbers - without.
template <typename T>
inline std::enable_if_t<is_arithmetic_v<T>, void>
writeQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf); }
requires is_arithmetic_v<T>
inline void writeQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf); }
inline void writeQuoted(const String & x, WriteBuffer & buf) { writeQuotedString(x, buf); }
@ -1021,8 +1021,8 @@ inline void writeQuoted(const UUID & x, WriteBuffer & buf)
/// String, date, datetime are in double quotes with C-style escaping. Numbers - without.
template <typename T>
inline std::enable_if_t<is_arithmetic_v<T>, void>
writeDoubleQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf); }
requires is_arithmetic_v<T>
inline void writeDoubleQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf); }
inline void writeDoubleQuoted(const String & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); }
@ -1054,8 +1054,8 @@ inline void writeDoubleQuoted(const UUID & x, WriteBuffer & buf)
/// String - in double quotes and with CSV-escaping; date, datetime - in double quotes. Numbers - without.
template <typename T>
inline std::enable_if_t<is_arithmetic_v<T>, void>
writeCSV(const T & x, WriteBuffer & buf) { writeText(x, buf); }
requires is_arithmetic_v<T>
inline void writeCSV(const T & x, WriteBuffer & buf) { writeText(x, buf); }
inline void writeCSV(const String & x, WriteBuffer & buf) { writeCSVString<>(x, buf); }
inline void writeCSV(const LocalDate & x, WriteBuffer & buf) { writeDoubleQuoted(x, buf); }
@ -1124,8 +1124,8 @@ inline void writeNullTerminatedString(const String & s, WriteBuffer & buffer)
}
template <typename T>
inline std::enable_if_t<is_arithmetic_v<T> && (sizeof(T) <= 8), void>
writeBinaryBigEndian(T x, WriteBuffer & buf) /// Assuming little endian architecture.
requires is_arithmetic_v<T> && (sizeof(T) <= 8)
inline void writeBinaryBigEndian(T x, WriteBuffer & buf) /// Assuming little endian architecture.
{
if constexpr (sizeof(x) == 2)
x = __builtin_bswap16(x);
@ -1138,8 +1138,8 @@ writeBinaryBigEndian(T x, WriteBuffer & buf) /// Assuming little endian archi
}
template <typename T>
inline std::enable_if_t<is_big_int_v<T>, void>
writeBinaryBigEndian(const T & x, WriteBuffer & buf) /// Assuming little endian architecture.
requires is_big_int_v<T>
inline void writeBinaryBigEndian(const T & x, WriteBuffer & buf) /// Assuming little endian architecture.
{
for (size_t i = 0; i != std::size(x.items); ++i)
{

View File

@ -25,6 +25,7 @@ namespace ErrorCodes
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int LOGICAL_ERROR;
extern const int SHARD_HAS_NO_CONNECTIONS;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int SYNTAX_ERROR;
}
@ -97,7 +98,6 @@ Cluster::Address::Address(
, replica_index(replica_index_)
{
host_name = config.getString(config_prefix + ".host");
port = static_cast<UInt16>(config.getInt(config_prefix + ".port"));
if (config.has(config_prefix + ".user"))
user_specified = true;
@ -106,7 +106,14 @@ Cluster::Address::Address(
default_database = config.getString(config_prefix + ".default_database", "");
secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable;
priority = config.getInt(config_prefix + ".priority", 1);
const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port";
auto default_port = config.getInt(port_type, 0);
port = static_cast<UInt16>(config.getInt(config_prefix + ".port", default_port));
if (!port)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Port is not specified in cluster configuration: {}", config_prefix + ".port");
is_local = isLocal(config.getInt(port_type, 0));
/// By default compression is disabled if address looks like localhost.

View File

@ -1,10 +1,12 @@
#include <Interpreters/InterpreterBackupQuery.h>
#include <Backups/IBackup.h>
#include <Backups/IBackupEntry.h>
#include <Backups/IRestoreTask.h>
#include <Backups/BackupFactory.h>
#include <Backups/BackupSettings.h>
#include <Backups/BackupUtils.h>
#include <Backups/IBackup.h>
#include <Backups/IBackupEntry.h>
#include <Parsers/ASTSetQuery.h>
#include <Backups/RestoreSettings.h>
#include <Backups/RestoreUtils.h>
#include <Interpreters/Context.h>
@ -12,40 +14,43 @@ namespace DB
{
namespace
{
BackupMutablePtr createBackup(const ASTBackupQuery & query, const ContextPtr & context)
BackupMutablePtr createBackup(const BackupInfo & backup_info, const BackupSettings & backup_settings, const ContextPtr & context)
{
BackupFactory::CreateParams params;
params.open_mode = (query.kind == ASTBackupQuery::BACKUP) ? IBackup::OpenMode::WRITE : IBackup::OpenMode::READ;
params.open_mode = IBackup::OpenMode::WRITE;
params.context = context;
params.backup_info = BackupInfo::fromAST(*query.backup_name);
if (query.base_backup_name)
params.base_backup_info = BackupInfo::fromAST(*query.base_backup_name);
params.backup_info = backup_info;
params.base_backup_info = backup_settings.base_backup_info;
params.compression_method = backup_settings.compression_method;
params.compression_level = backup_settings.compression_level;
params.password = backup_settings.password;
return BackupFactory::instance().createBackup(params);
}
#if 0
void getBackupSettings(const ASTBackupQuery & query, BackupSettings & settings, std::optional<BaseBackupInfo> & base_backup)
BackupMutablePtr openBackup(const BackupInfo & backup_info, const RestoreSettings & restore_settings, const ContextPtr & context)
{
settings = {};
if (query.settings)
settings.applyChanges(query.settings->as<const ASTSetQuery &>().changes);
return settings;
BackupFactory::CreateParams params;
params.open_mode = IBackup::OpenMode::READ;
params.context = context;
params.backup_info = backup_info;
params.base_backup_info = restore_settings.base_backup_info;
params.password = restore_settings.password;
return BackupFactory::instance().createBackup(params);
}
#endif
void executeBackup(const ASTBackupQuery & query, const ContextPtr & context)
void executeBackup(const ContextPtr & context, const ASTBackupQuery & query)
{
BackupMutablePtr backup = createBackup(query, context);
auto backup_entries = makeBackupEntries(query.elements, context);
auto backup_settings = BackupSettings::fromBackupQuery(query);
BackupMutablePtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), backup_settings, context);
auto backup_entries = makeBackupEntries(context, query.elements, backup_settings);
writeBackupEntries(backup, std::move(backup_entries), context->getSettingsRef().max_backup_threads);
}
void executeRestore(const ASTBackupQuery & query, ContextMutablePtr context)
void executeRestore(ContextMutablePtr context, const ASTBackupQuery & query)
{
BackupPtr backup = createBackup(query, context);
auto restore_tasks = makeRestoreTasks(query.elements, context, backup);
auto restore_settings = RestoreSettings::fromRestoreQuery(query);
BackupPtr backup = openBackup(BackupInfo::fromAST(*query.backup_name), restore_settings, context);
auto restore_tasks = makeRestoreTasks(context, backup, query.elements, restore_settings);
executeRestoreTasks(std::move(restore_tasks), context->getSettingsRef().max_backup_threads);
}
}
@ -54,9 +59,9 @@ BlockIO InterpreterBackupQuery::execute()
{
const auto & query = query_ptr->as<const ASTBackupQuery &>();
if (query.kind == ASTBackupQuery::BACKUP)
executeBackup(query, context);
executeBackup(context, query);
else if (query.kind == ASTBackupQuery::RESTORE)
executeRestore(query, context);
executeRestore(context, query);
return {};
}

View File

@ -11,31 +11,50 @@ namespace
using Element = ASTBackupQuery::Element;
using ElementType = ASTBackupQuery::ElementType;
void formatName(const DatabaseAndTableName & name, ElementType type, const IAST::FormatSettings & format)
void formatTypeWithName(const DatabaseAndTableName & name, bool name_is_in_temp_db, ElementType type, bool show_type, const IAST::FormatSettings & format)
{
switch (type)
{
case ElementType::TABLE: [[fallthrough]];
case ElementType::DICTIONARY:
case ElementType::TABLE:
{
if (show_type)
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "");
if (name_is_in_temp_db)
format.ostr << " TEMPORARY TABLE";
else
format.ostr << " TABLE";
format.ostr << (format.hilite ? IAST::hilite_none : "");
}
format.ostr << " ";
if (!name.first.empty())
if (!name_is_in_temp_db && !name.first.empty())
format.ostr << backQuoteIfNeed(name.first) << ".";
format.ostr << backQuoteIfNeed(name.second);
break;
}
case ElementType::DATABASE:
{
format.ostr << " " << backQuoteIfNeed(name.first);
if (show_type)
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "");
if (name_is_in_temp_db)
format.ostr << " ALL TEMPORARY TABLES";
else
format.ostr << " DATABASE";
format.ostr << (format.hilite ? IAST::hilite_none : "");
}
if (!name_is_in_temp_db)
format.ostr << " " << backQuoteIfNeed(name.first);
break;
}
case ElementType::TEMPORARY_TABLE:
case ElementType::ALL_DATABASES:
{
format.ostr << " " << backQuoteIfNeed(name.second);
if (show_type)
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " ALL DATABASES" << (format.hilite ? IAST::hilite_none : "");
break;
}
default:
break;
}
}
@ -55,32 +74,36 @@ namespace
}
}
void formatExceptList(const std::set<String> & except_list, const IAST::FormatSettings & format)
{
if (except_list.empty())
return;
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " EXCEPT "
<< (format.hilite ? IAST::hilite_none : "");
bool need_comma = false;
for (const auto & item : except_list)
{
if (std::exchange(need_comma, true))
format.ostr << ",";
format.ostr << " " << backQuoteIfNeed(item);
}
}
void formatElement(const Element & element, Kind kind, const IAST::FormatSettings & format)
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " ";
switch (element.type)
{
case ElementType::TABLE: format.ostr << "TABLE"; break;
case ElementType::DICTIONARY: format.ostr << "DICTIONARY"; break;
case ElementType::DATABASE: format.ostr << "DATABASE"; break;
case ElementType::ALL_DATABASES: format.ostr << "ALL DATABASES"; break;
case ElementType::TEMPORARY_TABLE: format.ostr << "TEMPORARY TABLE"; break;
case ElementType::ALL_TEMPORARY_TABLES: format.ostr << "ALL TEMPORARY TABLES"; break;
case ElementType::EVERYTHING: format.ostr << "EVERYTHING"; break;
}
format.ostr << (format.hilite ? IAST::hilite_none : "");
formatTypeWithName(element.name, element.name_is_in_temp_db, element.type, true, format);
formatName(element.name, element.type, format);
formatPartitions(element.partitions, format);
formatExceptList(element.except_list, format);
bool under_another_name = !element.new_name.first.empty() || !element.new_name.second.empty();
if (under_another_name)
bool new_name_is_different = (element.new_name != element.name) || (element.new_name_is_in_temp_db != element.name_is_in_temp_db);
if (new_name_is_different)
{
format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " " << ((kind == Kind::BACKUP) ? "AS" : "INTO")
<< (format.hilite ? IAST::hilite_none : "");
formatName(element.new_name, element.type, format);
bool show_type = (element.new_name_is_in_temp_db != element.name_is_in_temp_db);
formatTypeWithName(element.new_name, element.new_name_is_in_temp_db, element.type, show_type, format);
}
formatPartitions(element.partitions, format);
}
void formatElements(const std::vector<Element> & elements, Kind kind, const IAST::FormatSettings & format)

View File

@ -11,22 +11,20 @@ using DatabaseAndTableName = std::pair<String, String>;
/** BACKUP { TABLE [db.]table_name [AS [db.]table_name_in_backup] [PARTITION[S] partition_expr [,...]] |
* DICTIONARY [db.]dictionary_name [AS [db.]dictionary_name_in_backup] |
* DATABASE database_name [AS database_name_in_backup] |
* ALL DATABASES |
* TEMPORARY TABLE table_name [AS table_name_in_backup]
* ALL TEMPORARY TABLES |
* EVERYTHING } [,...]
* TEMPORARY TABLE table_name [AS table_name_in_backup] |
* ALL TEMPORARY TABLES [EXCEPT ...] |
* DATABASE database_name [EXCEPT ...] [AS database_name_in_backup] |
* ALL DATABASES [EXCEPT ...] } [,...]
* TO { File('path/') |
* Disk('disk_name', 'path/')
* [SETTINGS base_backup = {File(...) | Disk(...)}]
*
* RESTORE { TABLE [db.]table_name_in_backup [INTO [db.]table_name] [PARTITION[S] partition_expr [,...]] |
* DICTIONARY [db.]dictionary_name_in_backup [INTO [db.]dictionary_name] |
* DATABASE database_name_in_backup [INTO database_name] |
* ALL DATABASES |
* TEMPORARY TABLE table_name_in_backup [INTO table_name] |
* ALL TEMPORARY TABLES |
* EVERYTHING } [,...]
* ALL TEMPORARY TABLES [EXCEPT ...] |
* DATABASE database_name_in_backup [EXCEPT ...] [INTO database_name] |
* ALL DATABASES [EXCEPT ...] } [,...]
* FROM {File(...) | Disk(...)}
*
* Notes:
@ -57,12 +55,8 @@ public:
enum ElementType
{
TABLE,
DICTIONARY,
DATABASE,
ALL_DATABASES,
TEMPORARY_TABLE,
ALL_TEMPORARY_TABLES,
EVERYTHING,
};
struct Element
@ -70,6 +64,8 @@ public:
ElementType type;
DatabaseAndTableName name;
DatabaseAndTableName new_name;
bool name_is_in_temp_db = false;
bool new_name_is_in_temp_db = false;
ASTs partitions;
std::set<String> except_list;
};

View File

@ -18,38 +18,109 @@ namespace
using Element = ASTBackupQuery::Element;
using ElementType = ASTBackupQuery::ElementType;
bool parseName(IParser::Pos & pos, Expected & expected, ElementType type, DatabaseAndTableName & name)
bool parseType(IParser::Pos & pos, Expected & expected, ElementType & type, bool & name_is_in_temp_db)
{
name_is_in_temp_db = false;
if (ParserKeyword{"TABLE"}.ignore(pos, expected) || ParserKeyword{"DICTIONARY"}.ignore(pos, expected))
{
type = ElementType::TABLE;
return true;
}
if (ParserKeyword{"TEMPORARY TABLE"}.ignore(pos, expected))
{
type = ElementType::TABLE;
name_is_in_temp_db = true;
return true;
}
if (ParserKeyword{"DATABASE"}.ignore(pos, expected))
{
type = ElementType::DATABASE;
return true;
}
if (ParserKeyword{"ALL TEMPORARY TABLES"}.ignore(pos, expected))
{
type = ElementType::DATABASE;
name_is_in_temp_db = true;
return true;
}
if (ParserKeyword{"ALL DATABASES"}.ignore(pos, expected))
{
type = ElementType::ALL_DATABASES;
return true;
}
return false;
}
bool parseTempDBFlag(IParser::Pos & pos, Expected & expected, ElementType type, bool & temp_db_flag)
{
temp_db_flag = false;
switch (type)
{
case ElementType::TABLE: [[fallthrough]];
case ElementType::DICTIONARY:
case ElementType::TABLE:
{
if (ParserKeyword{"TABLE"}.ignore(pos, expected) || ParserKeyword{"DICTIONARY"}.ignore(pos, expected))
{
return true;
}
if (ParserKeyword{"TEMPORARY TABLE"}.ignore(pos, expected))
{
temp_db_flag = true;
return true;
}
return false;
}
case ElementType::DATABASE:
{
if (ParserKeyword{"DATABASE"}.ignore(pos, expected))
{
return true;
}
if (ParserKeyword{"ALL TEMPORARY TABLES"}.ignore(pos, expected))
{
temp_db_flag = true;
return true;
}
return false;
}
default:
return false;
}
}
bool parseName(IParser::Pos & pos, Expected & expected, ElementType type, bool name_is_in_temp_db, DatabaseAndTableName & name)
{
name.first.clear();
name.second.clear();
switch (type)
{
case ElementType::TABLE:
{
if (name_is_in_temp_db)
{
ASTPtr ast;
if (!ParserIdentifier{}.parse(pos, ast, expected))
return false;
name.second = getIdentifierName(ast);
return true;
}
return parseDatabaseAndTableName(pos, expected, name.first, name.second);
}
case ElementType::DATABASE:
{
if (name_is_in_temp_db)
return false;
ASTPtr ast;
if (!ParserIdentifier{}.parse(pos, ast, expected))
return false;
name.first = getIdentifierName(ast);
name.second.clear();
return true;
}
case ElementType::TEMPORARY_TABLE:
{
ASTPtr ast;
if (!ParserIdentifier{}.parse(pos, ast, expected))
return false;
name.second = getIdentifierName(ast);
name.first.clear();
return true;
}
default:
return true;
return false;
}
}
@ -64,7 +135,7 @@ namespace
ASTPtr ast;
if (!ParserPartition{}.parse(pos, ast, expected))
return false;
result.emplace_back(ast);
result.push_back(ast);
return true;
};
if (!ParserList::parseUtil(pos, expected, parse_list_element, false))
@ -74,50 +145,72 @@ namespace
return true;
}
bool parseExceptList(IParser::Pos & pos, Expected & expected, std::set<String> & except_list)
{
if (!ParserKeyword{"EXCEPT"}.ignore(pos, expected))
return false;
std::set<String> result;
auto parse_list_element = [&]
{
ASTPtr ast;
if (!ParserIdentifier{}.parse(pos, ast, expected))
return false;
result.insert(getIdentifierName(ast));
return true;
};
if (!ParserList::parseUtil(pos, expected, parse_list_element, false))
return false;
except_list = std::move(result);
return true;
}
bool parseElement(IParser::Pos & pos, Expected & expected, Element & entry)
{
return IParserBase::wrapParseImpl(pos, [&]
{
ElementType type;
if (ParserKeyword{"TABLE"}.ignore(pos, expected))
type = ElementType::TABLE;
else if (ParserKeyword{"DICTIONARY"}.ignore(pos, expected))
type = ElementType::DICTIONARY;
else if (ParserKeyword{"DATABASE"}.ignore(pos, expected))
type = ElementType::DATABASE;
else if (ParserKeyword{"ALL DATABASES"}.ignore(pos, expected))
type = ElementType::ALL_DATABASES;
else if (ParserKeyword{"TEMPORARY TABLE"}.ignore(pos, expected))
type = ElementType::TEMPORARY_TABLE;
else if (ParserKeyword{"ALL TEMPORARY TABLES"}.ignore(pos, expected))
type = ElementType::ALL_TEMPORARY_TABLES;
else if (ParserKeyword{"EVERYTHING"}.ignore(pos, expected))
type = ElementType::EVERYTHING;
else
bool name_is_in_temp_db = false;
if (!parseType(pos, expected, type, name_is_in_temp_db))
return false;
DatabaseAndTableName name;
if (!parseName(pos, expected, type, name))
return false;
if ((type == ElementType::TABLE) || (type == ElementType::DATABASE && !name_is_in_temp_db))
{
if (!parseName(pos, expected, type, name_is_in_temp_db, name))
return false;
}
bool new_name_is_in_temp_db = name_is_in_temp_db;
DatabaseAndTableName new_name = name;
if (ParserKeyword{"AS"}.ignore(pos, expected) || ParserKeyword{"INTO"}.ignore(pos, expected))
{
if (!parseTempDBFlag(pos, expected, type, new_name_is_in_temp_db))
new_name_is_in_temp_db = name_is_in_temp_db;
if ((type == ElementType::TABLE) || (type == ElementType::DATABASE && !new_name_is_in_temp_db))
{
if (!parseName(pos, expected, type, new_name_is_in_temp_db, new_name))
new_name = name;
}
}
ASTs partitions;
if (type == ElementType::TABLE)
parsePartitions(pos, expected, partitions);
DatabaseAndTableName new_name;
if (ParserKeyword{"AS"}.ignore(pos, expected) || ParserKeyword{"INTO"}.ignore(pos, expected))
{
if (!parseName(pos, expected, type, new_name))
return false;
}
if ((type == ElementType::TABLE) && partitions.empty())
parsePartitions(pos, expected, partitions);
std::set<String> except_list;
if (type != ElementType::TABLE)
parseExceptList(pos, expected, except_list);
entry.type = type;
entry.name = std::move(name);
entry.new_name = std::move(new_name);
entry.name_is_in_temp_db = name_is_in_temp_db;
entry.new_name_is_in_temp_db = new_name_is_in_temp_db;
entry.partitions = std::move(partitions);
entry.except_list = std::move(except_list);
return true;
});
}

View File

@ -8,22 +8,20 @@ namespace DB
/** Parses queries like
* BACKUP { TABLE [db.]table_name [AS [db.]table_name_in_backup] [PARTITION[S] partition_expr [,...]] |
* DICTIONARY [db.]dictionary_name [AS [db.]dictionary_name_in_backup] |
* DATABASE database_name [AS database_name_in_backup] |
* ALL DATABASES |
* TEMPORARY TABLE table_name [AS table_name_in_backup]
* ALL TEMPORARY TABLES |
* EVERYTHING } [,...]
* ALL TEMPORARY TABLES [EXCEPT ...] |
* DATABASE database_name [AS database_name_in_backup] [EXCEPT ...] |
* ALL DATABASES [EXCEPT ...] } [,...]
* TO { File('path/') |
* Disk('disk_name', 'path/')
* [SETTINGS base_backup = {FILE(...) | DISK(...)}]
*
* RESTORE { TABLE [db.]table_name_in_backup [INTO [db.]table_name] [PARTITION[S] partition_expr [,...]] |
* DICTIONARY [db.]dictionary_name_in_backup [INTO [db.]dictionary_name] |
* DATABASE database_name_in_backup [INTO database_name] |
* ALL DATABASES |
* TEMPORARY TABLE table_name_in_backup [INTO table_name] |
* ALL TEMPORARY TABLES |
* EVERYTHING } [,...]
* ALL TEMPORARY TABLES [EXCEPT ...] |
* DATABASE database_name_in_backup [EXCEPT ...] [INTO database_name] |
* ALL DATABASES [EXCEPT ...] } [,...]
* FROM {File(...) | Disk(...)}
*/
class ParserBackupQuery : public IParserBase

View File

@ -497,11 +497,17 @@ static void checkStatus(const arrow::Status & status, const String & column_name
throw Exception{ErrorCodes::UNKNOWN_EXCEPTION, "Error with a {} column '{}': {}.", format_name, column_name, status.ToString()};
}
Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name)
Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name, const Block * hint_header)
{
ColumnsWithTypeAndName sample_columns;
std::unordered_set<String> nested_table_names;
if (hint_header)
nested_table_names = Nested::getAllTableNames(*hint_header);
for (const auto & field : schema.fields())
{
if (hint_header && !hint_header->has(field->name()) && !nested_table_names.contains(field->name()))
continue;
/// Create empty arrow column by it's type and convert it to ClickHouse column.
arrow::MemoryPool * pool = arrow::default_memory_pool();
std::unique_ptr<arrow::ArrayBuilder> array_builder;
@ -650,7 +656,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr &
std::vector<size_t> ArrowColumnToCHColumn::getMissingColumns(const arrow::Schema & schema) const
{
std::vector<size_t> missing_columns;
auto block_from_arrow = arrowSchemaToCHHeader(schema, format_name);
auto block_from_arrow = arrowSchemaToCHHeader(schema, format_name, &header);
auto flatten_block_from_arrow = Nested::flatten(block_from_arrow);
for (size_t i = 0, columns = header.columns(); i < columns; ++i)
{

View File

@ -35,7 +35,9 @@ public:
/// Get missing columns that exists in header but not in arrow::Schema
std::vector<size_t> getMissingColumns(const arrow::Schema & schema) const;
static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name);
/// Transform arrow schema to ClickHouse header. If hint_header is provided,
/// we will skip columns in schema that are not in hint_header.
static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name, const Block * hint_header = nullptr);
private:
const Block & header;

View File

@ -187,7 +187,7 @@ void AddingDefaultsTransform::transform(Chunk & chunk)
{
const String & column_name = column_def.name;
if (column_defaults.count(column_name) == 0)
if (column_defaults.count(column_name) == 0 || !res.has(column_name))
continue;
size_t block_column_position = res.getPositionByName(column_name);

View File

@ -216,14 +216,14 @@ bool IStorage::isStaticStorage() const
return false;
}
BackupEntries IStorage::backup(const ASTs &, ContextPtr)
BackupEntries IStorage::backupData(ContextPtr, const ASTs &)
{
throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED);
}
RestoreDataTasks IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr)
RestoreTaskPtr IStorage::restoreData(ContextMutablePtr, const ASTs &, const BackupPtr &, const String &, const StorageRestoreSettings &)
{
throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED);
throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED);
}
std::string PrewhereInfo::dump() const

View File

@ -72,7 +72,9 @@ class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
class IBackupEntry;
using BackupEntries = std::vector<std::pair<String, std::unique_ptr<IBackupEntry>>>;
using RestoreDataTasks = std::vector<std::function<void()>>;
class IRestoreTask;
using RestoreTaskPtr = std::unique_ptr<IRestoreTask>;
struct StorageRestoreSettings;
struct ColumnSize
{
@ -216,11 +218,14 @@ public:
NameDependencies getDependentViewsByColumn(ContextPtr context) const;
/// Returns true if the backup is hollow, which means it doesn't contain any data.
virtual bool hasDataToBackup() const { return false; }
/// Prepares entries to backup data of the storage.
virtual BackupEntries backup(const ASTs & partitions, ContextPtr context);
virtual BackupEntries backupData(ContextPtr context, const ASTs & partitions);
/// Extract data from the backup and put it to the storage.
virtual RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context);
virtual RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings);
/// Returns whether the column is virtual - by default all columns are real.
/// Initially reserved virtual column name may be shadowed by real column.

View File

@ -715,18 +715,12 @@ bool KeyCondition::transformConstantWithValidFunctions(
if (is_valid_chain)
{
/// Here we cast constant to the input type.
/// It is not clear, why this works in general.
/// I can imagine the case when expression like `column < const` is legal,
/// but `type(column)` and `type(const)` are of different types,
/// and const cannot be casted to column type.
/// (There could be `superType(type(column), type(const))` which is used for comparison).
///
/// However, looks like this case newer happenes (I could not find such).
/// Let's assume that any two comparable types are castable to each other.
auto const_type = cur_node->result_type;
auto const_column = out_type->createColumnConst(1, out_value);
auto const_value = (*castColumn({const_column, out_type, ""}, const_type))[0];
auto const_value = (*castColumnAccurateOrNull({const_column, out_type, ""}, const_type))[0];
if (const_value.isNull())
return false;
while (!chain.empty())
{

View File

@ -3,6 +3,7 @@
#include <Backups/BackupEntryFromImmutableFile.h>
#include <Backups/BackupEntryFromSmallFile.h>
#include <Backups/IBackup.h>
#include <Backups/IRestoreTask.h>
#include <Compression/CompressedReadBuffer.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDate.h>
@ -3629,7 +3630,7 @@ Pipe MergeTreeData::alterPartition(
}
BackupEntries MergeTreeData::backup(const ASTs & partitions, ContextPtr local_context)
BackupEntries MergeTreeData::backupData(ContextPtr local_context, const ASTs & partitions)
{
DataPartsVector data_parts;
if (partitions.empty())
@ -3681,40 +3682,78 @@ BackupEntries MergeTreeData::backupDataParts(const DataPartsVector & data_parts)
}
RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup,
const std::unordered_set<String> & partition_ids,
SimpleIncrement * increment)
class MergeTreeDataRestoreTask : public IRestoreTask
{
RestoreDataTasks restore_tasks;
Strings part_names = backup->listFiles(data_path_in_backup);
for (const String & part_name : part_names)
public:
MergeTreeDataRestoreTask(
const std::shared_ptr<MergeTreeData> & storage_,
const BackupPtr & backup_,
const String & data_path_in_backup_,
const std::unordered_set<String> & partition_ids_,
SimpleIncrement * increment_)
: storage(storage_)
, backup(backup_)
, data_path_in_backup(data_path_in_backup_)
, partition_ids(partition_ids_)
, increment(increment_)
{
auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version);
}
if (!part_info)
continue;
if (!partition_ids.empty() && !partition_ids.contains(part_info->partition_id))
continue;
UInt64 total_size_of_part = 0;
Strings filenames = backup->listFiles(data_path_in_backup + part_name + "/", "");
for (const String & filename : filenames)
total_size_of_part += backup->getFileSize(data_path_in_backup + part_name + "/" + filename);
std::shared_ptr<IReservation> reservation = getStoragePolicy()->reserveAndCheck(total_size_of_part);
auto restore_task = [this,
backup,
data_path_in_backup,
part_name,
part_info = std::move(part_info),
filenames = std::move(filenames),
reservation,
increment]()
RestoreTasks run() override
{
RestoreTasks restore_part_tasks;
Strings part_names = backup->listFiles(data_path_in_backup);
for (const String & part_name : part_names)
{
const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, storage->format_version);
if (!part_info)
continue;
if (!partition_ids.empty() && !partition_ids.contains(part_info->partition_id))
continue;
restore_part_tasks.push_back(
std::make_unique<RestorePartTask>(storage, backup, data_path_in_backup, part_name, *part_info, increment));
}
return restore_part_tasks;
}
private:
std::shared_ptr<MergeTreeData> storage;
BackupPtr backup;
String data_path_in_backup;
std::unordered_set<String> partition_ids;
SimpleIncrement * increment;
class RestorePartTask : public IRestoreTask
{
public:
RestorePartTask(
const std::shared_ptr<MergeTreeData> & storage_,
const BackupPtr & backup_,
const String & data_path_in_backup_,
const String & part_name_,
const MergeTreePartInfo & part_info_,
SimpleIncrement * increment_)
: storage(storage_)
, backup(backup_)
, data_path_in_backup(data_path_in_backup_)
, part_name(part_name_)
, part_info(part_info_)
, increment(increment_)
{
}
RestoreTasks run() override
{
UInt64 total_size_of_part = 0;
Strings filenames = backup->listFiles(data_path_in_backup + part_name + "/", "");
for (const String & filename : filenames)
total_size_of_part += backup->getFileSize(data_path_in_backup + part_name + "/" + filename);
std::shared_ptr<IReservation> reservation = storage->getStoragePolicy()->reserveAndCheck(total_size_of_part);
auto disk = reservation->getDisk();
String relative_data_path = storage->getRelativeDataPath();
auto temp_part_dir_owner = std::make_shared<TemporaryFileOnDisk>(disk, relative_data_path + "restoring_" + part_name + "_");
String temp_part_dir = temp_part_dir_owner->getPath();
@ -3729,18 +3768,33 @@ RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & bac
auto read_buffer = backup_entry->getReadBuffer();
auto write_buffer = disk->writeFile(temp_part_dir + "/" + filename);
copyData(*read_buffer, *write_buffer);
reservation->update(reservation->getSize() - backup_entry->getSize());
}
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
auto part = createPart(part_name, *part_info, single_disk_volume, relative_temp_part_dir);
auto part = storage->createPart(part_name, part_info, single_disk_volume, relative_temp_part_dir);
part->loadColumnsChecksumsIndexes(false, true);
renameTempPartAndAdd(part, increment);
};
storage->renameTempPartAndAdd(part, increment);
return {};
}
restore_tasks.emplace_back(std::move(restore_task));
}
private:
std::shared_ptr<MergeTreeData> storage;
BackupPtr backup;
String data_path_in_backup;
String part_name;
MergeTreePartInfo part_info;
SimpleIncrement * increment;
};
};
return restore_tasks;
RestoreTaskPtr MergeTreeData::restoreDataParts(const std::unordered_set<String> & partition_ids,
const BackupPtr & backup, const String & data_path_in_backup,
SimpleIncrement * increment)
{
return std::make_unique<MergeTreeDataRestoreTask>(
std::static_pointer_cast<MergeTreeData>(shared_from_this()), backup, data_path_in_backup, partition_ids, increment);
}

View File

@ -674,15 +674,18 @@ public:
ContextPtr context,
TableLockHolder & table_lock_holder);
/// Storage has data to backup.
bool hasDataToBackup() const override { return true; }
/// Prepares entries to backup data of the storage.
BackupEntries backup(const ASTs & partitions, ContextPtr context) override;
BackupEntries backupData(ContextPtr context, const ASTs & partitions) override;
static BackupEntries backupDataParts(const DataPartsVector & data_parts);
/// Extract data from the backup and put it to the storage.
RestoreDataTasks restoreDataPartsFromBackup(
RestoreTaskPtr restoreDataParts(
const std::unordered_set<String> & partition_ids,
const BackupPtr & backup,
const String & data_path_in_backup,
const std::unordered_set<String> & partition_ids,
SimpleIncrement * increment);
/// Moves partition to specified Disk

View File

@ -240,7 +240,7 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts()
return;
/// Firstly, remove parts from ZooKeeper
storage.tryRemovePartsFromZooKeeperWithRetries(failed_parts);
storage.removePartsFromZooKeeperWithRetries(failed_parts);
for (const auto & part_name : failed_parts)
{

View File

@ -25,9 +25,10 @@
#include <QueryPipeline/Pipe.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <Backups/BackupEntryFromImmutableFile.h>
#include <Backups/BackupEntryFromAppendOnlyFile.h>
#include <Backups/BackupEntryFromSmallFile.h>
#include <Backups/IBackup.h>
#include <Backups/IRestoreTask.h>
#include <Disks/TemporaryFileOnDisk.h>
#include <cassert>
@ -887,7 +888,7 @@ IStorage::ColumnSizeByName StorageLog::getColumnSizes() const
}
BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context)
BackupEntries StorageLog::backupData(ContextPtr context, const ASTs & partitions)
{
if (!partitions.empty())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
@ -913,12 +914,12 @@ BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context)
{
/// We make a copy of the data file because it can be changed later in write() or in truncate().
String data_file_name = fileName(data_file.path);
String temp_file_path = temp_dir + "/" + data_file_name;
disk->copy(data_file.path, disk, temp_file_path);
String hardlink_file_path = temp_dir + "/" + data_file_name;
disk->createHardLink(data_file.path, hardlink_file_path);
backup_entries.emplace_back(
data_file_name,
std::make_unique<BackupEntryFromImmutableFile>(
disk, temp_file_path, file_checker.getFileSize(data_file.path), std::nullopt, temp_dir_owner));
std::make_unique<BackupEntryFromAppendOnlyFile>(
disk, hardlink_file_path, file_checker.getFileSize(data_file.path), std::nullopt, temp_dir_owner));
}
/// __marks.mrk
@ -926,12 +927,12 @@ BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context)
{
/// We make a copy of the data file because it can be changed later in write() or in truncate().
String marks_file_name = fileName(marks_file_path);
String temp_file_path = temp_dir + "/" + marks_file_name;
disk->copy(marks_file_path, disk, temp_file_path);
String hardlink_file_path = temp_dir + "/" + marks_file_name;
disk->createHardLink(marks_file_path, hardlink_file_path);
backup_entries.emplace_back(
marks_file_name,
std::make_unique<BackupEntryFromImmutableFile>(
disk, temp_file_path, file_checker.getFileSize(marks_file_path), std::nullopt, temp_dir_owner));
std::make_unique<BackupEntryFromAppendOnlyFile>(
disk, hardlink_file_path, file_checker.getFileSize(marks_file_path), std::nullopt, temp_dir_owner));
}
/// sizes.json
@ -952,43 +953,57 @@ BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context)
return backup_entries;
}
RestoreDataTasks StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context)
class LogRestoreTask : public IRestoreTask
{
if (!partitions.empty())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
using WriteLock = StorageLog::WriteLock;
using Mark = StorageLog::Mark;
auto restore_task = [this, backup, data_path_in_backup, context]()
public:
LogRestoreTask(
std::shared_ptr<StorageLog> storage_, const BackupPtr & backup_, const String & data_path_in_backup_, ContextMutablePtr context_)
: storage(storage_), backup(backup_), data_path_in_backup(data_path_in_backup_), context(context_)
{
}
RestoreTasks run() override
{
auto lock_timeout = getLockTimeout(context);
WriteLock lock{rwlock, lock_timeout};
WriteLock lock{storage->rwlock, lock_timeout};
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
const auto num_data_files = storage->num_data_files;
if (!num_data_files)
return;
return {};
auto & file_checker = storage->file_checker;
/// Load the marks if not loaded yet. We have to do that now because we're going to update these marks.
loadMarks(lock);
storage->loadMarks(lock);
/// If there were no files, save zero file sizes to be able to rollback in case of error.
saveFileSizes(lock);
storage->saveFileSizes(lock);
try
{
/// Append data files.
auto & data_files = storage->data_files;
for (const auto & data_file : data_files)
{
String file_path_in_backup = data_path_in_backup + fileName(data_file.path);
auto backup_entry = backup->readFile(file_path_in_backup);
const auto & disk = storage->disk;
auto in = backup_entry->getReadBuffer();
auto out = disk->writeFile(data_file.path, max_compress_block_size, WriteMode::Append);
auto out = disk->writeFile(data_file.path, storage->max_compress_block_size, WriteMode::Append);
copyData(*in, *out);
}
const bool use_marks_file = storage->use_marks_file;
if (use_marks_file)
{
/// Append marks.
size_t num_extra_marks = 0;
const auto & marks_file_path = storage->marks_file_path;
String file_path_in_backup = data_path_in_backup + fileName(marks_file_path);
size_t file_size = backup->getFileSize(file_path_in_backup);
if (file_size % (num_data_files * sizeof(Mark)) != 0)
@ -1027,19 +1042,34 @@ RestoreDataTasks StorageLog::restoreFromBackup(const BackupPtr & backup, const S
}
/// Finish writing.
saveMarks(lock);
saveFileSizes(lock);
storage->saveMarks(lock);
storage->saveFileSizes(lock);
}
catch (...)
{
/// Rollback partial writes.
file_checker.repair();
removeUnsavedMarks(lock);
storage->removeUnsavedMarks(lock);
throw;
}
};
return {restore_task};
return {};
}
private:
std::shared_ptr<StorageLog> storage;
BackupPtr backup;
String data_path_in_backup;
ContextMutablePtr context;
};
RestoreTaskPtr StorageLog::restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
{
if (!partitions.empty())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
return std::make_unique<LogRestoreTask>(
typeid_cast<std::shared_ptr<StorageLog>>(shared_from_this()), backup, data_path_in_backup, context);
}

View File

@ -23,6 +23,7 @@ class StorageLog final : public shared_ptr_helper<StorageLog>, public IStorage
{
friend class LogSource;
friend class LogSink;
friend class LogRestoreTask;
friend struct shared_ptr_helper<StorageLog>;
public:
@ -51,8 +52,9 @@ public:
bool supportsSubcolumns() const override { return true; }
ColumnSizeByName getColumnSizes() const override;
BackupEntries backup(const ASTs & partitions, ContextPtr context) override;
RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override;
bool hasDataToBackup() const override { return true; }
BackupEntries backupData(ContextPtr context, const ASTs & partitions) override;
RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override;
protected:
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),

View File

@ -25,6 +25,9 @@
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <Backups/IBackupEntry.h>
#include <Backups/IRestoreTask.h>
namespace DB
{
@ -430,6 +433,20 @@ Strings StorageMaterializedView::getDataPaths() const
return {};
}
BackupEntries StorageMaterializedView::backupData(ContextPtr context_, const ASTs & partitions_)
{
if (!hasInnerTable())
return {};
return getTargetTable()->backupData(context_, partitions_);
}
RestoreTaskPtr StorageMaterializedView::restoreData(ContextMutablePtr context_, const ASTs & partitions_, const BackupPtr & backup_, const String & data_path_in_backup_, const StorageRestoreSettings & restore_settings_)
{
if (!hasInnerTable())
return {};
return getTargetTable()->restoreData(context_, partitions_, backup_, data_path_in_backup_, restore_settings_);
}
ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type)
{
if (has_inner_table)

View File

@ -97,6 +97,10 @@ public:
Strings getDataPaths() const override;
bool hasDataToBackup() const override { return hasInnerTable(); }
BackupEntries backupData(ContextPtr context_, const ASTs & partitions_) override;
RestoreTaskPtr restoreData(ContextMutablePtr context_, const ASTs & partitions_, const BackupPtr & backup, const String & data_path_in_backup_, const StorageRestoreSettings & restore_settings_) override;
private:
/// Will be initialized in constructor
StorageID target_table_id = StorageID::createEmpty();

View File

@ -17,6 +17,17 @@
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Parsers/ASTCreateQuery.h>
#include <Common/FileChecker.h>
#include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedReadBufferFromFile.h>
#include <Compression/CompressedWriteBuffer.h>
#include <Backups/IBackup.h>
#include <Backups/IBackupEntriesBatch.h>
#include <Backups/IRestoreTask.h>
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <IO/copyData.h>
#include <Poco/TemporaryFile.h>
namespace DB
{
@ -24,6 +35,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NOT_IMPLEMENTED;
}
@ -364,6 +376,193 @@ void StorageMemory::truncate(
total_size_rows.store(0, std::memory_order_relaxed);
}
class MemoryBackupEntriesBatch : public shared_ptr_helper<MemoryBackupEntriesBatch>, public IBackupEntriesBatch
{
private:
friend struct shared_ptr_helper<MemoryBackupEntriesBatch>;
MemoryBackupEntriesBatch(
const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr<const Blocks> blocks_, UInt64 max_compress_block_size_)
: IBackupEntriesBatch({"data.bin", "index.mrk", "sizes.json"})
, metadata_snapshot(metadata_snapshot_)
, blocks(blocks_)
, max_compress_block_size(max_compress_block_size_)
{
}
static constexpr const size_t kDataBinPos = 0;
static constexpr const size_t kIndexMrkPos = 1;
static constexpr const size_t kSizesJsonPos = 2;
static constexpr const size_t kSize = 3;
void initialize()
{
std::call_once(initialized_flag, [this]()
{
temp_dir_owner.emplace();
auto temp_dir = temp_dir_owner->path();
fs::create_directories(temp_dir);
/// Writing data.bin
constexpr char data_file_name[] = "data.bin";
String data_file_path = temp_dir + "/" + data_file_name;
IndexForNativeFormat index;
{
auto data_out_compressed = std::make_unique<WriteBufferFromFile>(data_file_path);
CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size};
NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index};
for (const auto & block : *blocks)
block_out.write(block);
}
/// Writing index.mrk
constexpr char index_file_name[] = "index.mrk";
String index_file_path = temp_dir + "/" + index_file_name;
{
auto index_out_compressed = std::make_unique<WriteBufferFromFile>(index_file_path);
CompressedWriteBuffer index_out{*index_out_compressed};
index.write(index_out);
}
/// Writing sizes.json
constexpr char sizes_file_name[] = "sizes.json";
String sizes_file_path = temp_dir + "/" + sizes_file_name;
FileChecker file_checker{sizes_file_path};
file_checker.update(data_file_path);
file_checker.update(index_file_path);
file_checker.save();
file_paths[kDataBinPos] = data_file_path;
file_sizes[kDataBinPos] = file_checker.getFileSize(data_file_path);
file_paths[kIndexMrkPos] = index_file_path;
file_sizes[kIndexMrkPos] = file_checker.getFileSize(index_file_path);
file_paths[kSizesJsonPos] = sizes_file_path;
file_sizes[kSizesJsonPos] = fs::file_size(sizes_file_path);
/// We don't need to keep `blocks` any longer.
blocks.reset();
metadata_snapshot.reset();
});
}
std::unique_ptr<ReadBuffer> getReadBuffer(size_t index) override
{
initialize();
return createReadBufferFromFileBase(file_paths[index], {});
}
UInt64 getSize(size_t index) override
{
initialize();
return file_sizes[index];
}
StorageMetadataPtr metadata_snapshot;
std::shared_ptr<const Blocks> blocks;
UInt64 max_compress_block_size;
std::once_flag initialized_flag;
std::optional<Poco::TemporaryFile> temp_dir_owner;
std::array<String, kSize> file_paths;
std::array<UInt64, kSize> file_sizes;
};
BackupEntries StorageMemory::backupData(ContextPtr context, const ASTs & partitions)
{
if (!partitions.empty())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
return MemoryBackupEntriesBatch::create(getInMemoryMetadataPtr(), data.get(), context->getSettingsRef().max_compress_block_size)
->getBackupEntries();
}
class MemoryRestoreTask : public IRestoreTask
{
public:
MemoryRestoreTask(
std::shared_ptr<StorageMemory> storage_, const BackupPtr & backup_, const String & data_path_in_backup_, ContextMutablePtr context_)
: storage(storage_), backup(backup_), data_path_in_backup(data_path_in_backup_), context(context_)
{
}
RestoreTasks run() override
{
/// Our data are in the StripeLog format.
/// Reading index.mrk
IndexForNativeFormat index;
{
String index_file_path = data_path_in_backup + "index.mrk";
auto backup_entry = backup->readFile(index_file_path);
auto in = backup_entry->getReadBuffer();
CompressedReadBuffer compressed_in{*in};
index.read(compressed_in);
}
/// Reading data.bin
Blocks new_blocks;
size_t new_bytes = 0;
size_t new_rows = 0;
{
String data_file_path = data_path_in_backup + "data.bin";
auto backup_entry = backup->readFile(data_file_path);
std::unique_ptr<ReadBuffer> in = backup_entry->getReadBuffer();
std::optional<Poco::TemporaryFile> temp_data_copy;
if (!dynamic_cast<ReadBufferFromFileBase *>(in.get()))
{
temp_data_copy.emplace();
auto temp_data_copy_out = std::make_unique<WriteBufferFromFile>(temp_data_copy->path());
copyData(*in, *temp_data_copy_out);
temp_data_copy_out.reset();
in = createReadBufferFromFileBase(temp_data_copy->path(), {});
}
std::unique_ptr<ReadBufferFromFileBase> in_from_file{static_cast<ReadBufferFromFileBase *>(in.release())};
CompressedReadBufferFromFile compressed_in{std::move(in_from_file)};
NativeReader block_in{compressed_in, 0, index.blocks.begin(), index.blocks.end()};
while (auto block = block_in.read())
{
new_bytes += block.bytes();
new_rows += block.rows();
new_blocks.push_back(std::move(block));
}
}
/// Append old blocks with the new ones.
auto old_blocks = storage->data.get();
Blocks old_and_new_blocks = *old_blocks;
old_and_new_blocks.insert(old_and_new_blocks.end(), std::make_move_iterator(new_blocks.begin()), std::make_move_iterator(new_blocks.end()));
/// Finish restoring.
storage->data.set(std::make_unique<Blocks>(std::move(old_and_new_blocks)));
storage->total_size_bytes += new_bytes;
storage->total_size_rows += new_rows;
return {};
}
private:
std::shared_ptr<StorageMemory> storage;
BackupPtr backup;
String data_path_in_backup;
ContextMutablePtr context;
};
RestoreTaskPtr StorageMemory::restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
{
if (!partitions.empty())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
return std::make_unique<MemoryRestoreTask>(
typeid_cast<std::shared_ptr<StorageMemory>>(shared_from_this()), backup, data_path_in_backup, context);
}
std::optional<UInt64> StorageMemory::totalRows(const Settings &) const
{
/// All modifications of these counters are done under mutex which automatically guarantees synchronization/consistency

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