From 1d83c596f86d98738e676f58f4e63f675f54a9a0 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Fri, 5 Mar 2021 09:50:26 +0000 Subject: [PATCH 01/85] RFC: Throw exception if removing parts from ZooKeeper fails. This is used for removing part metadata from ZooKeeper when executing queue events like `DROP_RANGE` triggered when a user tries to drop a part or a partition. There are other uses but I'll focus only on this one. Before this change the method was giving up silently if it was unable to remove parts from ZooKeeper and this behaviour seems to be problematic. It could lead to operation being reported as successful at first but data reappearing later (very rarely) or "stuck" events in replication queue. Here is one particular scenario which I think we've hit: * Execute a DETACH PARTITION * DROP_RANGE event put in the queue * Replicas try to execute dropRange but some of them get disconnected from ZK and 5 retries aren't enough (ZK is miss-behaving), return code (false) is ignored and log pointer advances. * One of the replica where dropRange failed is restarted. * checkParts is executed and it finds parts that weren't removed from ZK, logs `Removing locally missing part from ZooKeeper and queueing a fetch` and puts GET_PART on the queue. * Few things can happen from here: * There is a lagging replica that din't execute DROP_RANGE yet: part will be fetched. The other replica will execute DROP_RANGE later and we'll get diverging set of parts on replicas. * Another replica also silently failed to remove parts from ZK: both of them are left with GET_PART in the queue and none of them can make progress, logging: `No active replica has part ... or covering part`. --- .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 21 ++++++++++--------- src/Storages/StorageReplicatedMergeTree.h | 5 +++-- 3 files changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index b3cb7c92def..ce73281cfe8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -208,7 +208,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) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 68f3b6d80d1..d95c1f12aa8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1954,7 +1954,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) } /// Forcibly remove parts from ZooKeeper - tryRemovePartsFromZooKeeperWithRetries(parts_to_remove); + removePartsFromZooKeeperWithRetries(parts_to_remove); if (entry.detach) LOG_DEBUG(log, "Detached {} parts inside {}.", parts_to_remove.size(), entry.new_part_name); @@ -2047,7 +2047,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (parts_to_add.empty()) { LOG_INFO(log, "All parts from REPLACE PARTITION command have been already attached"); - tryRemovePartsFromZooKeeperWithRetries(parts_to_remove); + removePartsFromZooKeeperWithRetries(parts_to_remove); return true; } @@ -2284,7 +2284,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) throw; } - tryRemovePartsFromZooKeeperWithRetries(parts_to_remove); + removePartsFromZooKeeperWithRetries(parts_to_remove); res_parts.clear(); parts_to_remove.clear(); cleanup_thread.wakeup(); @@ -2408,7 +2408,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo LOG_WARNING(log, "Source replica does not have part {}. Removing it from ZooKeeper.", part); } } - tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_from_zk); + removePartsFromZooKeeperWithRetries(parts_to_remove_from_zk); auto local_active_parts = getDataParts(); DataPartsVector parts_to_remove_from_working_set; @@ -5470,16 +5470,16 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() } -bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(DataPartsVector & parts, size_t max_retries) +void StorageReplicatedMergeTree::removePartsFromZooKeeperWithRetries(DataPartsVector & parts, size_t max_retries) { Strings part_names_to_remove; for (const auto & part : parts) part_names_to_remove.emplace_back(part->name); - return tryRemovePartsFromZooKeeperWithRetries(part_names_to_remove, max_retries); + return removePartsFromZooKeeperWithRetries(part_names_to_remove, max_retries); } -bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries) +void StorageReplicatedMergeTree::removePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries) { size_t num_tries = 0; bool success = false; @@ -5544,7 +5544,8 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St std::this_thread::sleep_for(std::chrono::milliseconds(1000)); } - return success; + if (!success) + throw Exception(ErrorCodes::UNFINISHED, "Failed to remove parts from ZooKeeper after {} retries", num_tries); } void StorageReplicatedMergeTree::removePartsFromZooKeeper( @@ -5843,7 +5844,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( lock.assumeUnlocked(); /// Forcibly remove replaced parts from ZooKeeper - tryRemovePartsFromZooKeeperWithRetries(parts_to_remove); + removePartsFromZooKeeperWithRetries(parts_to_remove); /// Speedup removing of replaced parts from filesystem parts_to_remove.clear(); @@ -6028,7 +6029,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta for (auto & lock : ephemeral_locks) lock.assumeUnlocked(); - tryRemovePartsFromZooKeeperWithRetries(parts_to_remove); + removePartsFromZooKeeperWithRetries(parts_to_remove); parts_to_remove.clear(); cleanup_thread.wakeup(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index a1a70ada9b2..2591fa2d3dc 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -388,8 +388,9 @@ private: void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, NameSet * parts_should_be_retried = nullptr); - bool tryRemovePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries = 5); - bool tryRemovePartsFromZooKeeperWithRetries(DataPartsVector & parts, size_t max_retries = 5); + /// Remove parts from ZooKeeper, throw exception if unable to do so after max_retries. + void removePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries = 5); + void removePartsFromZooKeeperWithRetries(DataPartsVector & parts, size_t max_retries = 5); /// Removes a part from ZooKeeper and adds a task to the queue to download it. It is supposed to do this with broken parts. void removePartAndEnqueueFetch(const String & part_name); From 2db11bc6c56e156823dd7cf34e6519306696086b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 20 Aug 2021 15:17:51 +0300 Subject: [PATCH 02/85] Add backward compatibility check in stress test --- docker/test/stress/Dockerfile | 1 + docker/test/stress/download_previous_release | 103 ++++++++++++++++ docker/test/stress/run.sh | 112 +++++++++++++++++- docker/test/stress/stress | 8 +- programs/client/Client.cpp | 3 + src/Client/ClientBase.cpp | 6 + src/Client/ClientBase.h | 2 + tests/clickhouse-test | 8 ++ ...788_update_nested_type_subcolumn_check.sql | 4 + .../0_stateless/01889_sqlite_read_write.sh | 1 - .../01943_non_deterministic_order_key.sql | 2 + 11 files changed, 241 insertions(+), 9 deletions(-) create mode 100755 docker/test/stress/download_previous_release diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index 3fe1b790d5a..f3497ba9f4a 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -23,6 +23,7 @@ RUN apt-get update -y \ brotli COPY ./stress /stress +COPY ./download_previous_release /download_previous_release COPY run.sh / ENV DATASETS="hits visits" diff --git a/docker/test/stress/download_previous_release b/docker/test/stress/download_previous_release new file mode 100755 index 00000000000..ad3b5ed2123 --- /dev/null +++ b/docker/test/stress/download_previous_release @@ -0,0 +1,103 @@ +#!/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_TEST_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-test_{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" +CLICKHOUSE_TEST_PACKET_NAME = "clickhouse-test_{version}_all.deb" + +PACKETS_DIR = "previous_release_package_folder/" +VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-lts*)" + + +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)) + + download_packet(CLICKHOUSE_TEST_DOWNLOAD_URL.format(version=release.version, type=release.type), + CLICKHOUSE_TEST_PACKET_NAME.format(version=release.version)) + + + +if __name__ == '__main__': + server_version = Version(input()) + previous_release = get_previous_release(server_version) + download_packets(previous_release) + diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index afdc026732f..60831a78c2d 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -21,16 +21,20 @@ 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 +LONG 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 -dpkg -i package_folder/clickhouse-test_*.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 + dpkg -i $1/clickhouse-test_*.deb +} function configure() { @@ -107,6 +111,8 @@ quit gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & } +install_packages package_folder + configure start @@ -175,6 +181,102 @@ zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.log* > /dev/n zgrep -Fa "########################################" /test_output/* > /dev/null \ && echo -e 'Killed by signal (output files)\tFAIL' >> /test_output/test_results.tsv +echo -e "Backward compatibility check\n" + +echo "Download previous release server" +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-test + dpkg --remove clickhouse-client + dpkg --remove clickhouse-server + dpkg --remove clickhouse-common-static-dbg + dpkg --remove clickhouse-common-static + + # 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=1800 \ + && 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 + 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 ignore Cancelled merging parts, REPLICA_IS_ALREADY_ACTIVE and errors) + zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -e "RaftInstance: failed to accept a rpc connection due to error 125" \ + /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /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 " 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 " " /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 diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 8fc4ade2da6..cfe8ae9ec6d 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -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) @@ -140,6 +141,7 @@ if __name__ == "__main__": parser.add_argument("--output-folder") parser.add_argument("--global-time-limit", type=int, default=3600) parser.add_argument("--num-parallel", type=int, default=cpu_count()) + parser.add_argument('--backward-compatibility-check', action='store_true', default=False) 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) @@ -148,7 +150,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: diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 34081168429..88eb9f3d157 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1028,6 +1028,7 @@ void Client::addAndCheckOptions(OptionsDescription & options_description, po::va ("no-warnings", "disable warnings when client connects to server") ("max_memory_usage_in_client", po::value(), "sets memory limit in client") + ("fake-drop", "Ignore all DROP queries") ; /// Commandline options related to external tables. @@ -1148,6 +1149,8 @@ void Client::processOptions(const OptionsDescription & options_description, server_logs_file = options["server_logs_file"].as(); 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())) { diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7e402028aa5..56dcc38b37b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -439,6 +439,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()) + 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. diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index bf9e8fdfe47..9d6bf408b79 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -212,6 +212,8 @@ protected: int query_fuzzer_runs = 0; QueryProcessingStage::Enum query_processing_stage; + + bool fake_drop = false; }; } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 19080f3934f..ebba7704af4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -265,6 +265,7 @@ class FailureReason(enum.Enum): NO_LONG = "not running long tests" REPLICATED_DB = "replicated-database" BUILD = "not running for current build" + BACKWARD_INCOMPATIBLE = "test is backward incompatible" # UNKNOWN reasons NO_REFERENCE = "no reference file" @@ -402,6 +403,9 @@ class TestCase: elif tags and ('no-replicated-database' in tags) and args.replicated_database: return FailureReason.REPLICATED_DB + elif tags and ('backward-incompatible' in tags) and args.backward_compatibility_check: + return FailureReason.BACKWARD_INCOMPATIBLE + elif tags: for build_flag in args.build_flags: if 'no-' + build_flag in tags: @@ -1384,6 +1388,7 @@ if __name__ == '__main__': group.add_argument('--shard', action='store_true', default=None, dest='shard', help='Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)') group.add_argument('--no-shard', action='store_false', default=None, dest='shard', help='Do not run shard related tests') + group.add_argument('--backward-compatibility-check', action='store_true', default=False, help='Ignore all drop queries in tests') args = parser.parse_args() if args.queries and not os.path.isdir(args.queries): @@ -1442,6 +1447,9 @@ if __name__ == '__main__': if os.getenv("CLICKHOUSE_DATABASE"): args.client += ' --database=' + os.getenv("CLICKHOUSE_DATABASE") + if args.backward_compatibility_check: + args.client += ' --fake-drop' + if args.client_option: # Set options for client if 'CLICKHOUSE_CLIENT_OPT' in os.environ: diff --git a/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql index accb785ba03..efd8ea2a565 100644 --- a/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql +++ b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql @@ -31,6 +31,8 @@ select * from test_wide_nested; alter table test_wide_nested update `info.id` = [100,200], `info.age` = [10,20,30], `info.name` = ['a','b','c'] where id = 0; -- { serverError 341 } +kill mutation where table = 'test_wide_nested' and database = currentDatabase() format Null; + -- Recreate table, because KILL MUTATION is not suitable for parallel tests execution. SELECT '********* test 2 **********'; DROP TABLE test_wide_nested; @@ -54,6 +56,8 @@ select * from test_wide_nested; alter table test_wide_nested update `info.id` = [100,200,300], `info.age` = [10,20,30] where id = 1; -- { serverError 341 } +kill mutation where table = 'test_wide_nested' and database = currentDatabase() format Null; + DROP TABLE test_wide_nested; SELECT '********* test 3 **********'; diff --git a/tests/queries/0_stateless/01889_sqlite_read_write.sh b/tests/queries/0_stateless/01889_sqlite_read_write.sh index 247f44b61e7..fc87aa08fa7 100755 --- a/tests/queries/0_stateless/01889_sqlite_read_write.sh +++ b/tests/queries/0_stateless/01889_sqlite_read_write.sh @@ -19,7 +19,6 @@ DB_PATH2=$CUR_DIR/${CURR_DATABASE}_db2 function cleanup() { ${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS ${CURR_DATABASE}" - rm -r "${DB_PATH}" "${DB_PATH2}" } trap cleanup EXIT diff --git a/tests/queries/0_stateless/01943_non_deterministic_order_key.sql b/tests/queries/0_stateless/01943_non_deterministic_order_key.sql index 200a88ec677..0929dcda601 100644 --- a/tests/queries/0_stateless/01943_non_deterministic_order_key.sql +++ b/tests/queries/0_stateless/01943_non_deterministic_order_key.sql @@ -1,3 +1,5 @@ +-- Tags: backward-incompatible + CREATE TABLE a (number UInt64) ENGINE = MergeTree ORDER BY if(now() > toDateTime('2020-06-01 13:31:40'), toInt64(number), -number); -- { serverError 36 } CREATE TABLE b (number UInt64) ENGINE = MergeTree ORDER BY now() > toDateTime(number); -- { serverError 36 } CREATE TABLE c (number UInt64) ENGINE = MergeTree ORDER BY now(); -- { serverError 36 } From 6fc043026eae1747ed43ebce8ad9fa2f44a79dc0 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 28 Sep 2021 14:09:14 +0300 Subject: [PATCH 03/85] Update run.sh --- docker/test/stress/run.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 60831a78c2d..a7c08dd04ee 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -199,6 +199,8 @@ then 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 @@ -234,8 +236,8 @@ then stop - # Error messages (we ignore Cancelled merging parts, REPLICA_IS_ALREADY_ACTIVE and errors) - zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -e "RaftInstance: failed to accept a rpc connection due to error 125" \ + # Error messages (we should ignore some errors) + zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -e "DDLWorker: Cannot parse DDL task query" -e "RaftInstance: failed to accept a rpc connection due to error 125" \ /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /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 From 821daa21da9b5101e4ea99c2701aee95e35ca009 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 28 Sep 2021 14:17:50 +0300 Subject: [PATCH 04/85] Update run.sh --- docker/test/stress/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index a7c08dd04ee..67febb00ae5 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -221,6 +221,7 @@ then rm -rf tmp_stress_output clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables" + stop # Start new server From 31302afe81cf9b51f3b30fa7351aacb9c0adf71d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 29 Sep 2021 14:07:02 +0300 Subject: [PATCH 05/85] Update run.sh --- docker/test/stress/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 67febb00ae5..c09080050b7 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -21,7 +21,7 @@ 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 -LONG + 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 @@ -238,7 +238,7 @@ then stop # Error messages (we should ignore some errors) - zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -e "DDLWorker: Cannot parse DDL task query" -e "RaftInstance: failed to accept a rpc connection due to error 125" \ + zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -e "DDLWorker: Cannot parse DDL task query" -e "RaftInstance: failed to accept a rpc connection due to error 125" -e "UNKNOWN_DATABASE" \ /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /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 From fd7d1131ec5cd39d06bec160d8ba7ed9ce57c304 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Oct 2021 13:17:48 +0300 Subject: [PATCH 06/85] Update run.sh --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index c09080050b7..561ecce5303 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -238,7 +238,7 @@ then stop # Error messages (we should ignore some errors) - zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -e "DDLWorker: Cannot parse DDL task query" -e "RaftInstance: failed to accept a rpc connection due to error 125" -e "UNKNOWN_DATABASE" \ + zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -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" \ /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /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 From 2110a0fbd1a16cf4fa5f96cb9d7fff16b3aec034 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 08:43:06 +0300 Subject: [PATCH 07/85] Add version mark in backward-incompatible tag --- tests/clickhouse-test | 26 ++++++++++++++++++- .../01942_create_table_with_sample.sql | 2 ++ .../01943_non_deterministic_order_key.sql | 2 +- 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 51b4bce0616..fed5e1fa5cb 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -57,6 +57,8 @@ MAX_RETRIES = 3 TEST_FILE_EXTENSIONS = ['.sql', '.sql.j2', '.sh', '.py', '.expect'] +VERSION_PATTERN = r"^((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+)$" + class HTTPError(Exception): def __init__(self, message=None, code=None): self.message = message @@ -406,6 +408,28 @@ class TestCase: self.testcase_args = None self.runs_count = 0 + # Check if test contains tag "backward-incompatible" and we should skip it + def check_backward_incompatible_tag(self, tags) -> bool: + for tag in tags: + if tag.startswith("backward-incompatible"): + split = tag.split(':') + + # If version is not specified in tag, always skip this test. + if len(split) == 1: + return True + version_from_tag = split[1] + + # Check if extracted string from tag is a real ClickHouse version, if not - always skip test. + if re.match(VERSION_PATTERN, version_from_tag) is None: + return True + + server_version = str(clickhouse_execute(args, "SELECT version()").decode()) + # If server version is less or equal from the version specified in tag, we should skip this test. + if list(map(int, server_version.split('.'))) <= list(map(int, version_from_tag.split('.'))): + return True + + return False + # should skip test, should increment skipped_total, skip reason def should_skip_test(self, suite) -> Optional[FailureReason]: tags = self.tags @@ -438,7 +462,7 @@ class TestCase: elif tags and ('no-replicated-database' in tags) and args.replicated_database: return FailureReason.REPLICATED_DB - elif tags and ('backward-incompatible' in tags) and args.backward_compatibility_check: + elif args.backward_compatibility_check and self.check_backward_incompatible_tag(tags): return FailureReason.BACKWARD_INCOMPATIBLE elif tags: diff --git a/tests/queries/0_stateless/01942_create_table_with_sample.sql b/tests/queries/0_stateless/01942_create_table_with_sample.sql index 6320edd7a31..bd14baf2c8f 100644 --- a/tests/queries/0_stateless/01942_create_table_with_sample.sql +++ b/tests/queries/0_stateless/01942_create_table_with_sample.sql @@ -1,3 +1,5 @@ +-- Tags: backward-incompatible:21.9.1.1 + CREATE TABLE IF NOT EXISTS sample_incorrect (`x` UUID) ENGINE = MergeTree diff --git a/tests/queries/0_stateless/01943_non_deterministic_order_key.sql b/tests/queries/0_stateless/01943_non_deterministic_order_key.sql index 0929dcda601..8a949cd36de 100644 --- a/tests/queries/0_stateless/01943_non_deterministic_order_key.sql +++ b/tests/queries/0_stateless/01943_non_deterministic_order_key.sql @@ -1,4 +1,4 @@ --- Tags: backward-incompatible +-- Tags: backward-incompatible:21.9.1.1 CREATE TABLE a (number UInt64) ENGINE = MergeTree ORDER BY if(now() > toDateTime('2020-06-01 13:31:40'), toInt64(number), -number); -- { serverError 36 } CREATE TABLE b (number UInt64) ENGINE = MergeTree ORDER BY now() > toDateTime(number); -- { serverError 36 } From ca02f9757c74d22e1bea333def29b29de6e37af1 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 13:19:21 +0300 Subject: [PATCH 08/85] Chenge test 02097_default_dict_get_add_database a bit --- .../0_stateless/02097_default_dict_get_add_database.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02097_default_dict_get_add_database.sql b/tests/queries/0_stateless/02097_default_dict_get_add_database.sql index af177566476..892389b3062 100644 --- a/tests/queries/0_stateless/02097_default_dict_get_add_database.sql +++ b/tests/queries/0_stateless/02097_default_dict_get_add_database.sql @@ -29,8 +29,8 @@ SOURCE(CLICKHOUSE(TABLE 'test_table')); DROP TABLE IF EXISTS test_table_default; CREATE TABLE test_table_default ( - data_1 DEFAULT dictGetUInt64('test_dictionary', 'data_column_1', toUInt64(0)), - data_2 DEFAULT dictGet(test_dictionary, 'data_column_2', toUInt64(0)) + data_1 DEFAULT dictGetUInt64('02097_db.test_dictionary', 'data_column_1', toUInt64(0)), + data_2 DEFAULT dictGet(02097_db.test_dictionary, 'data_column_2', toUInt64(0)) ) ENGINE=TinyLog; From 5e5b6ade00a52d80db45c7734408ea6c413cbd38 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 27 Oct 2021 14:46:53 +0300 Subject: [PATCH 09/85] Update clickhouse-test --- tests/clickhouse-test | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fed5e1fa5cb..bc260954246 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -409,8 +409,8 @@ class TestCase: self.runs_count = 0 # Check if test contains tag "backward-incompatible" and we should skip it - def check_backward_incompatible_tag(self, tags) -> bool: - for tag in tags: + def check_backward_incompatible_tag(self) -> bool: + for tag in self.tags: if tag.startswith("backward-incompatible"): split = tag.split(':') @@ -462,7 +462,7 @@ class TestCase: elif tags and ('no-replicated-database' in tags) and args.replicated_database: return FailureReason.REPLICATED_DB - elif args.backward_compatibility_check and self.check_backward_incompatible_tag(tags): + elif args.backward_compatibility_check and self.check_backward_incompatible_tag(): return FailureReason.BACKWARD_INCOMPATIBLE elif tags: From 482154940f3294d18a8d6fa31bc9b2a39b129041 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 2 Nov 2021 17:48:40 +0300 Subject: [PATCH 10/85] Update 01079_parallel_alter_detach_table_zookeeper.sh --- .../0_stateless/01079_parallel_alter_detach_table_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh index c95554d26bb..22c4de28cd1 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel, no-fasttest +# Tags: zookeeper, no-parallel, no-fasttest, backward-incompatible CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 03f1e3c67d2fb1e2171c773c001cf600c18d036b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 8 Nov 2021 14:02:36 +0300 Subject: [PATCH 11/85] Update run.sh --- docker/test/stress/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 5e4ef036c9d..ca1c5c3e830 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -90,6 +90,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 From 08af98b4a1a5d642e22977eaf79d8c5f7d9cf1c7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Nov 2021 16:34:13 +0300 Subject: [PATCH 12/85] Update download_previous_release --- docker/test/stress/download_previous_release | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/download_previous_release b/docker/test/stress/download_previous_release index ad3b5ed2123..fc95b196b14 100755 --- a/docker/test/stress/download_previous_release +++ b/docker/test/stress/download_previous_release @@ -20,7 +20,7 @@ CLICKHOUSE_CLIENT_PACKET_NAME = "clickhouse-client_{version}_all.deb" CLICKHOUSE_TEST_PACKET_NAME = "clickhouse-test_{version}_all.deb" PACKETS_DIR = "previous_release_package_folder/" -VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-lts*)" +VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+*)" class Version: From 87df3c8272e94f36e76bc898166560b9ee053d6f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 Nov 2021 15:03:48 +0300 Subject: [PATCH 13/85] Fix --- docker/test/stress/download_previous_release | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/download_previous_release b/docker/test/stress/download_previous_release index fc95b196b14..73ffb2afd9a 100755 --- a/docker/test/stress/download_previous_release +++ b/docker/test/stress/download_previous_release @@ -20,7 +20,7 @@ CLICKHOUSE_CLIENT_PACKET_NAME = "clickhouse-client_{version}_all.deb" CLICKHOUSE_TEST_PACKET_NAME = "clickhouse-test_{version}_all.deb" PACKETS_DIR = "previous_release_package_folder/" -VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+*)" +VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+.*)" class Version: From 6180aedab6b13ffea4cea7d52f6bc84fccff25d2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 Nov 2021 23:16:35 +0300 Subject: [PATCH 14/85] Update download_previous_release --- docker/test/stress/download_previous_release | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/download_previous_release b/docker/test/stress/download_previous_release index 73ffb2afd9a..3d4c649d091 100755 --- a/docker/test/stress/download_previous_release +++ b/docker/test/stress/download_previous_release @@ -20,7 +20,7 @@ CLICKHOUSE_CLIENT_PACKET_NAME = "clickhouse-client_{version}_all.deb" CLICKHOUSE_TEST_PACKET_NAME = "clickhouse-test_{version}_all.deb" PACKETS_DIR = "previous_release_package_folder/" -VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+.*)" +VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)" class Version: From 6bf9a170ab53071c084cf67262f1e7948c6d1921 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 Nov 2021 13:48:39 +0300 Subject: [PATCH 15/85] Update 01160_table_dependencies.sh --- tests/queries/0_stateless/01160_table_dependencies.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01160_table_dependencies.sh b/tests/queries/0_stateless/01160_table_dependencies.sh index a0a3f05c6a9..00121e9d989 100755 --- a/tests/queries/0_stateless/01160_table_dependencies.sh +++ b/tests/queries/0_stateless/01160_table_dependencies.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: backward-incompatible:21.12.1.8761 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From a1631e92687c7779d852b22017ed9089b2552416 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 Nov 2021 13:53:38 +0300 Subject: [PATCH 16/85] Update 02022_storage_filelog_one_file.sh --- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index 8ae0ce0ec1c..76fce0162c6 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: backward-incompatible set -eu From 18e60919c55d89fb674d92d6b66908a1f5433ea6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 Nov 2021 16:49:59 +0300 Subject: [PATCH 17/85] Update run.sh --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index ca1c5c3e830..ed1dacf3d1a 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -241,7 +241,7 @@ then stop # Error messages (we should ignore some errors) - zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -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" \ + zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -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" \ /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /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 From b0a607d95a0fea5be1c69e286f29776e4f690513 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 Nov 2021 17:22:59 +0300 Subject: [PATCH 18/85] Update 01747_system_session_log_long.sh --- tests/queries/0_stateless/01747_system_session_log_long.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh index b41bf077b57..452f1d3222e 100755 --- a/tests/queries/0_stateless/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel, no-fasttest +# Tags: long, no-parallel, no-fasttest, backward-incompatible # Tag no-fasttest: Accesses CH via mysql table function (which is unavailable) ################################################################################################## @@ -369,4 +369,4 @@ GROUP BY user_name, interface, type ORDER BY user_name, interface, type; -EOF \ No newline at end of file +EOF From aaed515493e88ba201e0afed362331365a2a705b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 Nov 2021 17:26:06 +0300 Subject: [PATCH 19/85] Update 01747_system_session_log_long.sh --- tests/queries/0_stateless/01747_system_session_log_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh index 452f1d3222e..9b127e0b48d 100755 --- a/tests/queries/0_stateless/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel, no-fasttest, backward-incompatible +# Tags: long, no-parallel, no-fasttest # Tag no-fasttest: Accesses CH via mysql table function (which is unavailable) ################################################################################################## From 653eeca8a7acf1bc2291c67912ce3a394812c6e6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 Nov 2021 17:29:31 +0300 Subject: [PATCH 20/85] Update 01555_system_distribution_queue_mask.sql --- .../0_stateless/01555_system_distribution_queue_mask.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql index 0143b8e46ed..ac977d8491b 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql @@ -1,3 +1,5 @@ +-- Tags: backward-incompatible + -- force data path with the user/pass in it set use_compact_format_in_distributed_parts_names=0; -- use async send even for localhost From 4124e690f414c40a83d59fec91b7062da354b0f3 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 Nov 2021 15:56:04 +0300 Subject: [PATCH 21/85] Update 01191_rename_dictionary.sql --- tests/queries/0_stateless/01191_rename_dictionary.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01191_rename_dictionary.sql b/tests/queries/0_stateless/01191_rename_dictionary.sql index e9fed1dd6b2..fa100b21a80 100644 --- a/tests/queries/0_stateless/01191_rename_dictionary.sql +++ b/tests/queries/0_stateless/01191_rename_dictionary.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel +-- Tags: no-parallel, backward-incompatible DROP DATABASE IF EXISTS test_01191; CREATE DATABASE test_01191 ENGINE=Atomic; From d5bde26bc10b4df135995339c9d093bc331bb940 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 20 Jan 2022 15:22:00 +0300 Subject: [PATCH 22/85] Update run.sh --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 3fe5f1be849..b695695460f 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -307,7 +307,7 @@ then stop # Error messages (we should ignore some errors) - zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -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" \ + zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -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" \ /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /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 From f3f414ae75885db8b8e1936cc38be77eeb75a95b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 27 Jan 2022 13:59:39 +0300 Subject: [PATCH 23/85] Fix style --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e8345c01a42..fe8350f7b16 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -67,7 +67,7 @@ def stringhash(s): # only during process invocation https://stackoverflow.com/a/42089311 return zlib.crc32(s.encode('utf-8')) - + class HTTPError(Exception): def __init__(self, message=None, code=None): self.message = message From e004dae25b3fcd526755193b2c7b3210a3e66fe9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 27 Jan 2022 18:15:03 +0300 Subject: [PATCH 24/85] Update run.sh --- docker/test/stress/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index e1d5827fbce..6706160e670 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -254,6 +254,7 @@ zgrep -Fa " received signal " /test_output/gdb.log > /dev/null \ 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 From 74087470c23967448744d0530a34411f97a7ecf4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 18 Feb 2022 12:52:56 +0300 Subject: [PATCH 25/85] Update run.sh --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index cdf7f847022..16b0930f610 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -309,7 +309,7 @@ then stop # Error messages (we should ignore some errors) - zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -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" \ + zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -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 "Code: 1000, e.code() = 111, Connection refused" \ /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /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 From 9d0e1e301467e4f2509bd4bc7e049f2b3e24dfd2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 18 Feb 2022 12:57:31 +0300 Subject: [PATCH 26/85] Update 00626_replace_partition_from_table_zookeeper.sh --- .../0_stateless/00626_replace_partition_from_table_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index cd0a4cdb640..0316ec682c2 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel, no-s3-storage +# Tags: zookeeper, no-parallel, no-s3-storage, backward-incompatible # Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem # (it tries to do it as quick as possible, but it still performed in separate thread asynchronously) From 71e7668fa668956523ee3436e2005f9b9fd85589 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 18 Feb 2022 13:02:49 +0300 Subject: [PATCH 27/85] Remove downloading clickhouse-test --- docker/test/stress/download_previous_release | 6 ------ 1 file changed, 6 deletions(-) diff --git a/docker/test/stress/download_previous_release b/docker/test/stress/download_previous_release index 3d4c649d091..364c863423b 100755 --- a/docker/test/stress/download_previous_release +++ b/docker/test/stress/download_previous_release @@ -10,14 +10,12 @@ CLICKHOUSE_COMMON_STATIC_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHous 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_TEST_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-test_{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" -CLICKHOUSE_TEST_PACKET_NAME = "clickhouse-test_{version}_all.deb" PACKETS_DIR = "previous_release_package_folder/" VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)" @@ -91,10 +89,6 @@ def download_packets(release): download_packet(CLICKHOUSE_CLIENT_DOWNLOAD_URL.format(version=release.version, type=release.type), CLICKHOUSE_CLIENT_PACKET_NAME.format(version=release.version)) - download_packet(CLICKHOUSE_TEST_DOWNLOAD_URL.format(version=release.version, type=release.type), - CLICKHOUSE_TEST_PACKET_NAME.format(version=release.version)) - - if __name__ == '__main__': server_version = Version(input()) From 10c45bef6758652e564011ae9b5bdbce6d36d97f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 18 Feb 2022 13:03:23 +0300 Subject: [PATCH 28/85] Remove clickhouse-test package --- docker/test/stress/run.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 16b0930f610..5bb9fc2949d 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -264,7 +264,6 @@ then stop # Uninstall current packages - dpkg --remove clickhouse-test dpkg --remove clickhouse-client dpkg --remove clickhouse-server dpkg --remove clickhouse-common-static-dbg From 1b4c5990eb4bfa8140ecb5c5a6d20ee2a3184b21 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 18 Feb 2022 13:14:10 +0300 Subject: [PATCH 29/85] Update 01780_column_sparse_alter.sql --- tests/queries/0_stateless/01780_column_sparse_alter.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01780_column_sparse_alter.sql b/tests/queries/0_stateless/01780_column_sparse_alter.sql index 7f9558bfc18..d3538ffe56b 100644 --- a/tests/queries/0_stateless/01780_column_sparse_alter.sql +++ b/tests/queries/0_stateless/01780_column_sparse_alter.sql @@ -1,3 +1,5 @@ +-- Tags: backward-incompatible + SET mutations_sync = 2; DROP TABLE IF EXISTS t_sparse_alter; From de888db506e91e1cf6058d8b129c7245d8ab2f2d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 18 Feb 2022 16:36:38 +0300 Subject: [PATCH 30/85] Update docker/test/stress/stress Co-authored-by: Vladimir C --- docker/test/stress/stress | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 0b4f4a8b35b..86f8edf5980 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -169,7 +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', default=False) + 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) From 500ed56061cccf515fd6bc813ab29baad20e367a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 18 Feb 2022 16:36:48 +0300 Subject: [PATCH 31/85] Update docker/test/stress/run.sh Co-authored-by: Vladimir C --- docker/test/stress/run.sh | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 5bb9fc2949d..aad83e6de72 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -308,7 +308,17 @@ then stop # Error messages (we should ignore some errors) - zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" -e "REPLICA_IS_ALREADY_ACTIVE" -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 "Code: 1000, e.code() = 111, Connection refused" \ + zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ + -e "REPLICA_IS_ALREADY_ACTIVE" \ + -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 "Code: 1000, e.code() = 111, Connection refused" \ /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /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 From ba7f7239612db095b4dc7d6326729a4fd07fd279 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Feb 2022 13:54:21 +0000 Subject: [PATCH 32/85] Fix comments --- programs/client/Client.cpp | 2 +- tests/clickhouse-test | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 0b988dcc22d..e4d7d5b22a0 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1014,7 +1014,7 @@ void Client::addOptions(OptionsDescription & options_description) ("opentelemetry-tracestate", po::value(), "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") + ("fake-drop", "Ignore all DROP queries, should be used only for testing") ; /// Commandline options related to external tables. diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 7cf8c58ce36..b516fd37f3b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -59,7 +59,7 @@ MAX_RETRIES = 3 TEST_FILE_EXTENSIONS = ['.sql', '.sql.j2', '.sh', '.py', '.expect'] -VERSION_PATTERN = r"^((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+)$" +VERSION_PATTERN = r"^((\d+\.)?(\d+\.)?(\d+\.)?\d+)$" def stringhash(s): @@ -1414,7 +1414,7 @@ if __name__ == '__main__': group.add_argument('--shard', action='store_true', default=None, dest='shard', help='Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)') group.add_argument('--no-shard', action='store_false', default=None, dest='shard', help='Do not run shard related tests') - group.add_argument('--backward-compatibility-check', action='store_true', default=False, help='Ignore all drop queries in tests') + group.add_argument('--backward-compatibility-check', action='store_true', help='Run tests for further backwoard compatibility testing by ignoring all drop queries in tests for collecting data from new version of server') args = parser.parse_args() if args.queries and not os.path.isdir(args.queries): From 522c1c1397fbf5f3e979bd65678ce6e9964ee4bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Feb 2022 19:27:14 +0100 Subject: [PATCH 33/85] Allow to use default port in cluster configuration --- src/Interpreters/Cluster.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index c01b19d81de..d558d1cfd67 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -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(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(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. From 625258892259a31b1a3dffd4e0e31772de893f24 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 28 Feb 2022 13:33:10 +0300 Subject: [PATCH 34/85] Update run.sh --- docker/test/stress/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index aad83e6de72..083a48df958 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -318,6 +318,7 @@ then -e "ZooKeeperClient" \ -e "KEEPER_EXCEPTION" \ -e "DirectoryMonitor" \ + -e "TABLE_IS_READ_ONLY" \ -e "Code: 1000, e.code() = 111, Connection refused" \ /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /dev/null \ && echo -e 'Error message in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \ From 8f6ba99fec13cc2c24399e945fb9296426364d04 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 1 Mar 2022 15:20:02 +0300 Subject: [PATCH 35/85] Update run.sh --- docker/test/stress/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 083a48df958..9a6bc8b2f5d 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -294,6 +294,8 @@ then stop + rm -rf /var/lib/clickhouse/coordination + # Start new server configure start From 7252c18ff008cceb8ba7485e97a7a52333287137 Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 4 Mar 2022 15:04:43 +0800 Subject: [PATCH 36/85] to #34890_fix_input_format_null_as_default_bug --- src/Processors/Transforms/AddingDefaultsTransform.cpp | 2 +- .../0_stateless/01016_input_null_as_default.reference | 6 ++++++ .../0_stateless/01016_input_null_as_default.sh | 11 +++++++++++ 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/AddingDefaultsTransform.cpp b/src/Processors/Transforms/AddingDefaultsTransform.cpp index 01048354b82..82c235d9034 100644 --- a/src/Processors/Transforms/AddingDefaultsTransform.cpp +++ b/src/Processors/Transforms/AddingDefaultsTransform.cpp @@ -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); diff --git a/tests/queries/0_stateless/01016_input_null_as_default.reference b/tests/queries/0_stateless/01016_input_null_as_default.reference index d7010f42d4e..7a84006caf7 100644 --- a/tests/queries/0_stateless/01016_input_null_as_default.reference +++ b/tests/queries/0_stateless/01016_input_null_as_default.reference @@ -33,3 +33,9 @@ Values 1 world 3 2019-07-23 [1,2,3] ('tuple',3.14) 2 Hello 123 2019-06-19 [] ('test',2.71828) 3 Hello 42 2019-06-19 [1,2,3] ('default',0.75) +default_by_other_column +1 2 ('tuple',3) +10 10 ('default',2.5) +100 100 ('default',25) +100 100 ('default',25) +100 100 ('default',25) diff --git a/tests/queries/0_stateless/01016_input_null_as_default.sh b/tests/queries/0_stateless/01016_input_null_as_default.sh index bfeaca0fcac..24d93b2703c 100755 --- a/tests/queries/0_stateless/01016_input_null_as_default.sh +++ b/tests/queries/0_stateless/01016_input_null_as_default.sh @@ -5,7 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS null_as_default"; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS default_by_other_column"; $CLICKHOUSE_CLIENT --query="CREATE TABLE null_as_default (i Int8, s String DEFAULT 'Hello', n UInt64 DEFAULT 42, d Date DEFAULT '2019-06-19', a Array(UInt8) DEFAULT [1, 2, 3], t Tuple(String, Float64) DEFAULT ('default', i / 4)) ENGINE = Memory"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE default_by_other_column (a Float32 DEFAULT 100, b Float64 DEFAULT a, c Tuple(String, Float64) DEFAULT ('default', b / 4)) ENGINE = Memory"; echo 'CSV' echo '\N, 1, \N, "2019-07-22", "[10, 20, 30]", \N @@ -62,3 +64,12 @@ echo '(NULL, '\''1'\'', (null), '\''2019-07-22'\'', ([10, 20, 30]), (NuLl)), (3, null, (null), null, (null), (null))' | $CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO null_as_default VALUES"; $CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i"; $CLICKHOUSE_CLIENT --query="DROP TABLE null_as_default"; + +echo 'default_by_other_column' +$CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO default_by_other_column(c) VALUES(null)"; +$CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO default_by_other_column(b, c) VALUES(null, null)"; +$CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO default_by_other_column(a, b, c) VALUES(null, null, null)"; +$CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO default_by_other_column(a) VALUES(10)"; +$CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO default_by_other_column(a, b, c) VALUES(1, 2, ('tuple', 3))"; +$CLICKHOUSE_CLIENT --query="SELECT * FROM default_by_other_column ORDER BY a"; +$CLICKHOUSE_CLIENT --query="DROP TABLE default_by_other_column"; From 0f90996bb8b77e8d256fc959beeac9b7e959c14a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 4 Mar 2022 14:24:24 +0300 Subject: [PATCH 37/85] Update run.sh --- docker/test/stress/run.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 9a6bc8b2f5d..fcf55eec75b 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -292,9 +292,7 @@ then clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables" - stop - - rm -rf /var/lib/clickhouse/coordination + stop # Start new server configure From 607ec6c884988e1e26962708d2812739b388b1ac Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 5 Mar 2022 12:05:41 +0300 Subject: [PATCH 38/85] Fix style --- tests/clickhouse-test | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 88b0f4a8d99..f1723fb09e1 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1471,7 +1471,8 @@ if __name__ == '__main__': group.add_argument('--shard', action='store_true', default=None, dest='shard', help='Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)') group.add_argument('--no-shard', action='store_false', default=None, dest='shard', help='Do not run shard related tests') - group.add_argument('--backward-compatibility-check', action='store_true', help='Run tests for further backwoard compatibility testing by ignoring all drop queries in tests for collecting data from new version of server') + group.add_argument('--backward-compatibility-check', action='store_true', help='Run tests for further backwoard compatibility testing by ignoring all' + 'drop queries in tests for collecting data from new version of server') args = parser.parse_args() if args.queries and not os.path.isdir(args.queries): From 4135f2ddad667455e2e60cbf268d9be031f94c87 Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Thu, 10 Mar 2022 11:12:53 +0800 Subject: [PATCH 39/85] to /#34890_fix_input_format_null_as_default_bug fix failed test --- tests/queries/0_stateless/02205_ephemeral_1.reference | 1 + tests/queries/0_stateless/02205_ephemeral_1.sql | 4 +--- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02205_ephemeral_1.reference b/tests/queries/0_stateless/02205_ephemeral_1.reference index bad56205b46..d18a7abd2eb 100644 --- a/tests/queries/0_stateless/02205_ephemeral_1.reference +++ b/tests/queries/0_stateless/02205_ephemeral_1.reference @@ -2,6 +2,7 @@ x UInt32 DEFAULT y y UInt32 EPHEMERAL 17 z UInt32 DEFAULT 5 1 2 +17 2 17 5 7 5 21 5 diff --git a/tests/queries/0_stateless/02205_ephemeral_1.sql b/tests/queries/0_stateless/02205_ephemeral_1.sql index 60411a92583..178e037f910 100644 --- a/tests/queries/0_stateless/02205_ephemeral_1.sql +++ b/tests/queries/0_stateless/02205_ephemeral_1.sql @@ -11,9 +11,7 @@ SELECT * FROM t_ephemeral_02205_1; TRUNCATE TABLE t_ephemeral_02205_1; -# Attempt to use ephemeral through default of ordinary column should end up with error 10 - -# ephemeral only considered when explicitly mentioned in the columns list -INSERT INTO t_ephemeral_02205_1 VALUES (DEFAULT, 2); -- { clientError 10 } +INSERT INTO t_ephemeral_02205_1 VALUES (DEFAULT, 2); # Test INSERT using ephemerals default INSERT INTO t_ephemeral_02205_1 (x, y) VALUES (DEFAULT, DEFAULT); From 91b9c119390bfcc1de26fd196628153e1c82637f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 10 Mar 2022 14:38:39 +0300 Subject: [PATCH 40/85] Increase the number of attempts to start server --- docker/test/stress/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index fcf55eec75b..7f8468cda73 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -120,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 @@ -296,7 +296,7 @@ then # Start new server configure - start + 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 From 3d36eced2dda370cd6ed1cc21ac712fca4325dd1 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 10 Mar 2022 17:55:45 +0300 Subject: [PATCH 41/85] Update run.sh --- docker/test/stress/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 7f8468cda73..9775c19e009 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -309,6 +309,7 @@ then # 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 "DDLWorker: Cannot parse DDL task query" \ -e "RaftInstance: failed to accept a rpc connection due to error 125" \ From b9496af3b368d1a284dbb0163248efe018107d11 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 14 Mar 2022 16:40:17 +0000 Subject: [PATCH 42/85] Fix tests --- docker/test/stress/run.sh | 1 + tests/clickhouse-test | 4 ++-- .../00626_replace_partition_from_table_zookeeper.sh | 2 +- .../00993_system_parts_race_condition_drop_zookeeper.sh | 2 +- .../01079_parallel_alter_detach_table_zookeeper.sh | 2 +- tests/queries/0_stateless/01160_table_dependencies.sh | 2 +- tests/queries/0_stateless/01191_rename_dictionary.sql | 2 +- .../0_stateless/01555_system_distribution_queue_mask.sql | 2 +- .../0_stateless/01576_alter_low_cardinality_and_select.sh | 1 + tests/queries/0_stateless/01780_column_sparse_alter.sql | 2 +- tests/queries/0_stateless/01942_create_table_with_sample.sql | 2 +- .../queries/0_stateless/01943_non_deterministic_order_key.sql | 2 +- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 2 +- 13 files changed, 14 insertions(+), 12 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 9775c19e009..0f487f23365 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -311,6 +311,7 @@ then 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" \ diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5d328625df8..01992f24bfa 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -482,10 +482,10 @@ class TestCase: self.base_url_params = os.environ['CLICKHOUSE_URL_PARAMS'] if 'CLICKHOUSE_URL_PARAMS' in os.environ else '' self.base_client_options = os.environ['CLICKHOUSE_CLIENT_OPT'] if 'CLICKHOUSE_CLIENT_OPT' in os.environ else '' - # Check if test contains tag "backward-incompatible" and we should skip it + # Check if test contains tag "no-backward-compatibility-check" and we should skip it def check_backward_incompatible_tag(self) -> bool: for tag in self.tags: - if tag.startswith("backward-incompatible"): + if tag.startswith("no-backward-compatibility-check"): split = tag.split(':') # If version is not specified in tag, always skip this test. diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 0316ec682c2..1aa02864815 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel, no-s3-storage, backward-incompatible +# Tags: zookeeper, no-parallel, no-s3-storage, no-backward-compatibility-check # Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem # (it tries to do it as quick as possible, but it still performed in separate thread asynchronously) diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index 6b1df1d45a0..d5d43d3c293 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, zookeeper, no-parallel +# Tags: race, zookeeper, no-parallel, no-backward-compatibility-check CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh index 22c4de28cd1..1f316b4b389 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel, no-fasttest, backward-incompatible +# Tags: zookeeper, no-parallel, no-fasttest, no-backward-compatibility-check CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01160_table_dependencies.sh b/tests/queries/0_stateless/01160_table_dependencies.sh index 00121e9d989..4cfad526f24 100755 --- a/tests/queries/0_stateless/01160_table_dependencies.sh +++ b/tests/queries/0_stateless/01160_table_dependencies.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: backward-incompatible:21.12.1.8761 +# Tags: no-backward-compatibility-check:21.12.1.8761 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01191_rename_dictionary.sql b/tests/queries/0_stateless/01191_rename_dictionary.sql index fa100b21a80..ed9bc8af61b 100644 --- a/tests/queries/0_stateless/01191_rename_dictionary.sql +++ b/tests/queries/0_stateless/01191_rename_dictionary.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, backward-incompatible +-- Tags: no-parallel, no-backward-compatibility-check DROP DATABASE IF EXISTS test_01191; CREATE DATABASE test_01191 ENGINE=Atomic; diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql index ac977d8491b..bdcde1adbad 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql @@ -1,4 +1,4 @@ --- Tags: backward-incompatible +-- Tags: no-backward-compatibility-check -- force data path with the user/pass in it set use_compact_format_in_distributed_parts_names=0; diff --git a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh index fcea7f57cd3..27de10ab16a 100755 --- a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh +++ b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-backward-compatibility-check CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01780_column_sparse_alter.sql b/tests/queries/0_stateless/01780_column_sparse_alter.sql index d3538ffe56b..1c9a4c6c0c2 100644 --- a/tests/queries/0_stateless/01780_column_sparse_alter.sql +++ b/tests/queries/0_stateless/01780_column_sparse_alter.sql @@ -1,4 +1,4 @@ --- Tags: backward-incompatible +-- Tags: no-backward-compatibility-check SET mutations_sync = 2; diff --git a/tests/queries/0_stateless/01942_create_table_with_sample.sql b/tests/queries/0_stateless/01942_create_table_with_sample.sql index bd14baf2c8f..c4ededae4ca 100644 --- a/tests/queries/0_stateless/01942_create_table_with_sample.sql +++ b/tests/queries/0_stateless/01942_create_table_with_sample.sql @@ -1,4 +1,4 @@ --- Tags: backward-incompatible:21.9.1.1 +-- Tags: no-backward-compatibility-check:21.9.1.1 CREATE TABLE IF NOT EXISTS sample_incorrect (`x` UUID) diff --git a/tests/queries/0_stateless/01943_non_deterministic_order_key.sql b/tests/queries/0_stateless/01943_non_deterministic_order_key.sql index 8a949cd36de..8b6abebe4da 100644 --- a/tests/queries/0_stateless/01943_non_deterministic_order_key.sql +++ b/tests/queries/0_stateless/01943_non_deterministic_order_key.sql @@ -1,4 +1,4 @@ --- Tags: backward-incompatible:21.9.1.1 +-- Tags: no-backward-compatibility-check:21.9.1.1 CREATE TABLE a (number UInt64) ENGINE = MergeTree ORDER BY if(now() > toDateTime('2020-06-01 13:31:40'), toInt64(number), -number); -- { serverError 36 } CREATE TABLE b (number UInt64) ENGINE = MergeTree ORDER BY now() > toDateTime(number); -- { serverError 36 } diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index 76fce0162c6..2f47001eda9 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: backward-incompatible +# Tags: no-backward-compatibility-check set -eu From b249a1a24a9ae66e3f5d1ccf8204f1f066f65d5e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 15 Mar 2022 13:09:46 +0100 Subject: [PATCH 43/85] Update 00942_dataparts_500.sh --- tests/queries/0_stateless/00942_dataparts_500.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00942_dataparts_500.sh b/tests/queries/0_stateless/00942_dataparts_500.sh index 19cb1138aa8..7e1a7f15810 100755 --- a/tests/queries/0_stateless/00942_dataparts_500.sh +++ b/tests/queries/0_stateless/00942_dataparts_500.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash - +# Tags: no-backward-compatibility-check # Test fix for issue #5066 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From f955920146efc7a22d1c8d7514ddbf6a0dbbcba8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 15 Mar 2022 13:11:41 +0100 Subject: [PATCH 44/85] Update 00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql --- ...32_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql index a1859220c6c..e8d923389e5 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql @@ -1,4 +1,4 @@ --- Tags: long, zookeeper, no-replicated-database +-- Tags: long, zookeeper, no-replicated-database, no-backward-compatibility-check -- Tag no-replicated-database: Fails due to additional replicas or shards SET send_logs_level = 'fatal'; From 9058e776e5715bfae6ffacc122442048ab904b44 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 15 Mar 2022 13:17:08 +0100 Subject: [PATCH 45/85] Update 02222_create_table_without_columns_metadata.sh --- .../0_stateless/02222_create_table_without_columns_metadata.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh index 1ba67fa77ea..842c32cf243 100755 --- a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh +++ b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest, no-parallel, no-backward-compatibility-check CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e68a66ddaad46677dd87a9ae0dd594085db97bff Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 15 Mar 2022 13:25:50 +0100 Subject: [PATCH 46/85] Update 02025_storage_filelog_virtual_col.sh --- tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index a475913b7d2..f0faafe55d5 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-backward-compatibility-check set -eu From 6b3a25e20dd7b79f7caa915cfdea74f867299682 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 15 Mar 2022 17:15:17 +0100 Subject: [PATCH 47/85] Update run.sh --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 0f487f23365..265cf9d964e 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -285,7 +285,7 @@ then mkdir tmp_stress_output - ./stress --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1800 \ + ./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 From 6e192c45a7770bd4ba9707f375e44a8bbf175276 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 15 Mar 2022 17:21:00 +0100 Subject: [PATCH 48/85] Fix tests --- tests/queries/0_stateless/02205_ephemeral_1.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02205_ephemeral_1.reference b/tests/queries/0_stateless/02205_ephemeral_1.reference index d18a7abd2eb..6e98ffd2495 100644 --- a/tests/queries/0_stateless/02205_ephemeral_1.reference +++ b/tests/queries/0_stateless/02205_ephemeral_1.reference @@ -2,7 +2,7 @@ x UInt32 DEFAULT y y UInt32 EPHEMERAL 17 z UInt32 DEFAULT 5 1 2 -17 2 +17 2 17 5 7 5 21 5 From fa6d963f831d207c09804a9ac74b79e892d79852 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 16 Mar 2022 12:30:45 +0100 Subject: [PATCH 49/85] Update run.sh --- docker/test/stress/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 265cf9d964e..b09a8d8c3a0 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -322,6 +322,8 @@ then -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 "" > /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 From 719cf23185e45f16a0c00a8e20929cfd6b84a7db Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 16 Mar 2022 18:06:30 +0100 Subject: [PATCH 50/85] Fix test --- tests/queries/0_stateless/02205_ephemeral_1.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02205_ephemeral_1.sql b/tests/queries/0_stateless/02205_ephemeral_1.sql index 178e037f910..5d0565cbfc0 100644 --- a/tests/queries/0_stateless/02205_ephemeral_1.sql +++ b/tests/queries/0_stateless/02205_ephemeral_1.sql @@ -12,6 +12,9 @@ SELECT * FROM t_ephemeral_02205_1; TRUNCATE TABLE t_ephemeral_02205_1; INSERT INTO t_ephemeral_02205_1 VALUES (DEFAULT, 2); +SELECT * FROM t_ephemeral_02205_1; + +TRUNCATE TABLE t_ephemeral_02205_1; # Test INSERT using ephemerals default INSERT INTO t_ephemeral_02205_1 (x, y) VALUES (DEFAULT, DEFAULT); From df73c754566161f26cfb13ac0320ab660994d7d7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Mar 2022 19:06:50 +0100 Subject: [PATCH 51/85] Replace more uses of std::enable_if for SFINAE by concepts - enable_if is usually regarded as fragile and unreadable - C++20 concepts are much easier to read and produce more expressive error messages - this is follow-up to PR #35347 but this time most of the remaining and more complex usages of enable_if in the codebase were replaced. --- src/Columns/ColumnArray.h | 3 ++- src/Columns/ColumnMap.h | 3 ++- src/Columns/ColumnNullable.h | 3 ++- src/Columns/ColumnSparse.h | 6 ++++-- src/Columns/ColumnTuple.h | 3 ++- src/Common/Dwarf.cpp | 3 ++- src/Common/HashTable/Hash.h | 16 ++++++++++------ src/Common/HashTable/HashTable.h | 10 ++++++---- src/Common/HyperLogLogCounter.h | 24 ++++++++++-------------- src/Common/IntervalTree.h | 11 +++++++---- src/Common/mysqlxx/mysqlxx/Value.h | 3 ++- src/Core/Field.h | 9 ++++++--- src/Core/MultiEnum.h | 3 ++- src/Functions/FunctionsConversion.h | 22 ++++++++++++++-------- src/IO/ReadBufferFromMemory.h | 3 ++- src/IO/WriteHelpers.h | 24 ++++++++++++------------ 16 files changed, 85 insertions(+), 61 deletions(-) diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 3f41ae9cd8a..1d88b2e6a26 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -60,7 +60,8 @@ public: return ColumnArray::create(nested_column->assumeMutable()); } - template ::value>::type> + template + requires (IsMutableColumns::value) static MutablePtr create(Args &&... args) { return Base::create(std::forward(args)...); } /** On the index i there is an offset to the beginning of the i + 1 -th element. */ diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index ed013dc03f9..95838e70d10 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -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 ::value>::type> + template + requires (IsMutableColumns::value) static MutablePtr create(Args &&... args) { return Base::create(std::forward(args)...); } std::string getName() const override; diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 52e5e43fa48..a0c72182f63 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -41,7 +41,8 @@ public: return ColumnNullable::create(nested_column_->assumeMutable(), null_map_->assumeMutable()); } - template ::value>::type> + template + requires (IsMutableColumns::value) static MutablePtr create(Args &&... args) { return Base::create(std::forward(args)...); } const char * getFamilyName() const override { return "Nullable"; } diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 0c35961bd18..aa0ab8a0cbc 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -37,7 +37,8 @@ public: return Base::create(values_->assumeMutable(), offsets_->assumeMutable(), size_); } - template ::value>::type> + template + requires IsMutableColumns::value static MutablePtr create(TColumnPtr && values_, TColumnPtr && offsets_, size_t size_) { return Base::create(std::forward(values_), std::forward(offsets_), size_); @@ -48,7 +49,8 @@ public: return Base::create(values_->assumeMutable()); } - template ::value>::type> + template + requires IsMutableColumns::value static MutablePtr create(TColumnPtr && values_) { return Base::create(std::forward(values_)); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 430e57e404c..e6797c2582c 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -35,7 +35,8 @@ public: static Ptr create(const TupleColumns & columns); static Ptr create(Columns && arg) { return create(arg); } - template ::value>::type> + template + requires std::is_rvalue_reference_v static MutablePtr create(Arg && arg) { return Base::create(std::forward(arg)); } std::string getName() const override; diff --git a/src/Common/Dwarf.cpp b/src/Common/Dwarf.cpp index ce8a0974870..6b952d0d093 100644 --- a/src/Common/Dwarf.cpp +++ b/src/Common/Dwarf.cpp @@ -122,7 +122,8 @@ const uint32_t kMaxAbbreviationEntries = 1000; // Read (bitwise) one object of type T template -std::enable_if_t && std::is_standard_layout_v, T> read(std::string_view & sp) +requires std::is_trivial_v && std::is_standard_layout_v +T read(std::string_view & sp) { SAFE_CHECK(sp.size() >= sizeof(T), "underflow"); T x; diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index 7250f05a325..5dbeeecf96b 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -73,8 +73,8 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value) } template -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(&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 -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 -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 && sizeof(T) == 16) { @@ -217,7 +219,8 @@ struct DefaultHash template struct HashCRC32; template -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 -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); } diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 5e439591ff2..6dee485550b 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -1036,14 +1036,16 @@ public: return const_cast *>(this)->find(x, hash_value); } - std::enable_if_t - ALWAYS_INLINE erase(const Key & x) + template + requires (Grower_::performs_linear_probing_with_single_step) + bool ALWAYS_INLINE erase(const Key & x) { return erase(x, hash(x)); } - std::enable_if_t - ALWAYS_INLINE erase(const Key & x, size_t hash_value) + template + requires (Grower_::performs_linear_probing_with_single_step) + bool ALWAYS_INLINE erase(const Key & x, size_t hash_value) { /** Deletion from open addressing hash table without tombstones * diff --git a/src/Common/HyperLogLogCounter.h b/src/Common/HyperLogLogCounter.h index 70a83a481a4..4644d18ecf3 100644 --- a/src/Common/HyperLogLogCounter.h +++ b/src/Common/HyperLogLogCounter.h @@ -78,8 +78,7 @@ template struct MinCounterType }; /// Denominator of expression for HyperLogLog algorithm. -template +template 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 +template struct IntermediateDenominator; template -struct IntermediateDenominator> +requires (denominator_mode != DenominatorMode::ExactType) +struct IntermediateDenominator { using Type = double; }; @@ -113,11 +113,9 @@ struct IntermediateDenominator -class __attribute__((__packed__)) Denominator> +template +requires (!details::isBigRankStore(precision)) || (!(denominator_mode == DenominatorMode::StableIfBig)) +class __attribute__((__packed__)) Denominator { private: using T = typename IntermediateDenominator::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 -class __attribute__((__packed__)) Denominator> +template +requires (details::isBigRankStore(precision)) && (denominator_mode == DenominatorMode::StableIfBig) +class __attribute__((__packed__)) Denominator { public: Denominator(DenominatorType initial_value) /// NOLINT diff --git a/src/Common/IntervalTree.h b/src/Common/IntervalTree.h index e11edde53a9..2214a4e842d 100644 --- a/src/Common/IntervalTree.h +++ b/src/Common/IntervalTree.h @@ -129,7 +129,7 @@ public: IntervalTree() { nodes.resize(1); } - template , bool> = true> + template requires std::is_same_v ALWAYS_INLINE bool emplace(Interval interval) { @@ -157,19 +157,22 @@ public: return true; } - template , bool> = true> + template + requires std::is_same_v bool insert(Interval interval) { return emplace(interval); } - template , bool> = true> + template + requires (!std::is_same_v) bool insert(Interval interval, const Value & value) { return emplace(interval, value); } - template , bool> = true> + template + requires (!std::is_same_v) bool insert(Interval interval, Value && value) { return emplace(interval, std::move(value)); diff --git a/src/Common/mysqlxx/mysqlxx/Value.h b/src/Common/mysqlxx/mysqlxx/Value.h index 27845ac5c7d..2e6471692a2 100644 --- a/src/Common/mysqlxx/mysqlxx/Value.h +++ b/src/Common/mysqlxx/mysqlxx/Value.h @@ -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()` to `Value::operator T()` to `Value::get()` to ... -template >> +template +requires std::is_constructible_v inline T contructFromValue(const Value & val) { return T(val); diff --git a/src/Core/Field.h b/src/Core/Field.h index 2f4e648d379..2a925a9e2a6 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -240,7 +240,8 @@ template <> struct NearestFieldTypeImpl { using Type // For enum types, use the field type that corresponds to their underlying type. template -struct NearestFieldTypeImpl>> +requires std::is_enum_v +struct NearestFieldTypeImpl { using Type = NearestFieldType>; }; @@ -669,7 +670,8 @@ private: } template - std::enable_if_t 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(&storage); @@ -704,7 +706,8 @@ private: } template - std::enable_if_t create(const CharT * data, size_t size) + requires (sizeof(CharT) == 1) + void create(const CharT * data, size_t size) { new (&storage) String(reinterpret_cast(data), size); which = Types::String; diff --git a/src/Core/MultiEnum.h b/src/Core/MultiEnum.h index 1c013cc0dc1..32aae93c6d5 100644 --- a/src/Core/MultiEnum.h +++ b/src/Core/MultiEnum.h @@ -17,7 +17,8 @@ struct MultiEnum : MultiEnum((toBitFlag(v) | ... | 0u)) {} - template >> + template + requires std::is_convertible_v constexpr explicit MultiEnum(ValueType v) : bitset(v) { diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index bba94409fb9..d1564008dfe 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -791,7 +791,8 @@ static ColumnUInt8::MutablePtr copyNullMap(ColumnPtr col) } template -struct ConvertImpl, DataTypeString>, Name, ConvertDefaultBehaviorTag> +requires (!std::is_same_v) +struct ConvertImpl { using FromFieldType = typename FromDataType::FieldType; using ColVecType = ColumnVectorOrDecimal; @@ -1324,19 +1325,23 @@ struct ConvertThroughParsing template -struct ConvertImpl, DataTypeString>, ToDataType, Name, ConvertDefaultBehaviorTag> +requires (!std::is_same_v) +struct ConvertImpl : ConvertThroughParsing {}; template -struct ConvertImpl, DataTypeFixedString>, ToDataType, Name, ConvertDefaultBehaviorTag> +requires (!std::is_same_v) +struct ConvertImpl : ConvertThroughParsing {}; template -struct ConvertImpl, DataTypeString>, ToDataType, Name, ConvertReturnNullOnErrorTag> +requires (!std::is_same_v) +struct ConvertImpl : ConvertThroughParsing {}; template -struct ConvertImpl, DataTypeFixedString>, ToDataType, Name, ConvertReturnNullOnErrorTag> +requires (!std::is_same_v) +struct ConvertImpl : ConvertThroughParsing {}; /// 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 -struct ConvertImpl, T, Name, ConvertDefaultBehaviorTag> +requires (!T::is_parametric) +struct ConvertImpl { template static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/, @@ -2739,8 +2745,8 @@ private: } template - std::enable_if_t, WrapperType> - createDecimalWrapper(const DataTypePtr & from_type, const ToDataType * to_type, bool requested_result_is_nullable) const + requires IsDataTypeDecimal + 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(); diff --git a/src/IO/ReadBufferFromMemory.h b/src/IO/ReadBufferFromMemory.h index 28d78f75f1c..dc5c464604b 100644 --- a/src/IO/ReadBufferFromMemory.h +++ b/src/IO/ReadBufferFromMemory.h @@ -12,7 +12,8 @@ namespace DB class ReadBufferFromMemory : public SeekableReadBuffer { public: - template > + template + requires (sizeof(CharT) == 1) ReadBufferFromMemory(const CharT * buf, size_t size) : SeekableReadBuffer(const_cast(reinterpret_cast(buf)), size, 0) {} diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 447a3ed0480..7c6abf2aec7 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -869,8 +869,8 @@ inline void writeDateTimeUnixTimestamp(DateTime64 datetime64, UInt32 scale, Writ /// Methods for output in binary format. template -inline std::enable_if_t, void> -writeBinary(const T & x, WriteBuffer & buf) { writePODBinary(x, buf); } +requires is_arithmetic_v +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 x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer /// String, date, datetime are in single quotes with C-style escaping. Numbers - without. template -inline std::enable_if_t, void> -writeQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf); } +requires is_arithmetic_v +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 -inline std::enable_if_t, void> -writeDoubleQuoted(const T & x, WriteBuffer & buf) { writeText(x, buf); } +requires is_arithmetic_v +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 -inline std::enable_if_t, void> -writeCSV(const T & x, WriteBuffer & buf) { writeText(x, buf); } +requires is_arithmetic_v +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 -inline std::enable_if_t && (sizeof(T) <= 8), void> -writeBinaryBigEndian(T x, WriteBuffer & buf) /// Assuming little endian architecture. +requires is_arithmetic_v && (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 -inline std::enable_if_t, void> -writeBinaryBigEndian(const T & x, WriteBuffer & buf) /// Assuming little endian architecture. +requires is_big_int_v +inline void writeBinaryBigEndian(const T & x, WriteBuffer & buf) /// Assuming little endian architecture. { for (size_t i = 0; i != std::size(x.items); ++i) { From 14963f17768136fe9187b16ff39e7e0d383072a6 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 16 Mar 2022 21:42:22 +0800 Subject: [PATCH 52/85] client nit fix --- src/Client/ClientBase.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 4f1c1f4539e..fdcc98f016f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1979,9 +1979,8 @@ void ClientBase::readArguments( } /// 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=")))) + && (arg == "--file="sv || arg == "--name="sv || arg == "--format="sv + || arg == "--structure="sv || arg == "--types="sv)) { external_tables_arguments.back().emplace_back(arg); } From 10996b1434a6abfd8f7f3aeb80df5561007bb876 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 16 Mar 2022 22:31:15 +0800 Subject: [PATCH 53/85] Fix mixed constant type during partition pruning --- src/Storages/MergeTree/KeyCondition.cpp | 16 +++++++--------- ...artition_pruner_mixed_constant_type.reference | 4 ++++ ...2232_partition_pruner_mixed_constant_type.sql | 7 +++++++ 3 files changed, 18 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02232_partition_pruner_mixed_constant_type.reference create mode 100644 tests/queries/0_stateless/02232_partition_pruner_mixed_constant_type.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index c17eb5a981e..c3664252297 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -715,16 +715,14 @@ 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; + + /// Try cast constant to the least super type of out_type and const_type. If there + /// is no super type, monotonic chains can be broken. Return false immediately. + const_type = tryGetLeastSupertype({out_type, const_type}); + if (!const_type) + return false; + auto const_column = out_type->createColumnConst(1, out_value); auto const_value = (*castColumn({const_column, out_type, ""}, const_type))[0]; diff --git a/tests/queries/0_stateless/02232_partition_pruner_mixed_constant_type.reference b/tests/queries/0_stateless/02232_partition_pruner_mixed_constant_type.reference new file mode 100644 index 00000000000..6fcbc14234d --- /dev/null +++ b/tests/queries/0_stateless/02232_partition_pruner_mixed_constant_type.reference @@ -0,0 +1,4 @@ +1647353101000 +1647353101001 +1647353101002 +1647353101003 diff --git a/tests/queries/0_stateless/02232_partition_pruner_mixed_constant_type.sql b/tests/queries/0_stateless/02232_partition_pruner_mixed_constant_type.sql new file mode 100644 index 00000000000..a0b58271764 --- /dev/null +++ b/tests/queries/0_stateless/02232_partition_pruner_mixed_constant_type.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS broken; + +CREATE TABLE broken (time UInt64) ENGINE = MergeTree PARTITION BY toYYYYMMDD(toDate(time / 1000)) ORDER BY time; +INSERT INTO broken (time) VALUES (1647353101000), (1647353101001), (1647353101002), (1647353101003); +SELECT * FROM broken WHERE time>-1; + +DROP TABLE broken; From 4d85c2fc8bdea0f73d9076aa61855dbbd2d2cecc Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 18 Mar 2022 16:18:29 +0800 Subject: [PATCH 54/85] Use castColumnAccurateOrNull --- src/Storages/MergeTree/KeyCondition.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index c3664252297..eeff7e4c875 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -716,15 +716,11 @@ bool KeyCondition::transformConstantWithValidFunctions( if (is_valid_chain) { auto const_type = cur_node->result_type; - - /// Try cast constant to the least super type of out_type and const_type. If there - /// is no super type, monotonic chains can be broken. Return false immediately. - const_type = tryGetLeastSupertype({out_type, const_type}); - if (!const_type) - return false; - 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()) { From cffa2096de6d43939cd54d19201fde555a20eeb0 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Mar 2022 13:06:22 +0000 Subject: [PATCH 55/85] Fix working with unneeded columns in Arrow/Parquet/ORC formats --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 7 +++-- .../Formats/Impl/ArrowColumnToCHColumn.h | 4 ++- .../02241_parquet_bad_column.reference | 1 + .../0_stateless/02241_parquet_bad_column.sh | 27 +++++++++++++++++++ 4 files changed, 36 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02241_parquet_bad_column.reference create mode 100755 tests/queries/0_stateless/02241_parquet_bad_column.sh diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 5c367bb69f0..06f8616ea8c 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -484,11 +484,14 @@ 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; for (const auto & field : schema.fields()) { + if (hint_header && !hint_header->has(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 array_builder; @@ -603,7 +606,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & std::vector ArrowColumnToCHColumn::getMissingColumns(const arrow::Schema & schema) const { std::vector 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) { diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 07e7fb36404..cf4f6bb3ff3 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -34,7 +34,9 @@ public: /// Get missing columns that exists in header but not in arrow::Schema std::vector 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; diff --git a/tests/queries/0_stateless/02241_parquet_bad_column.reference b/tests/queries/0_stateless/02241_parquet_bad_column.reference new file mode 100644 index 00000000000..c7c7a69d906 --- /dev/null +++ b/tests/queries/0_stateless/02241_parquet_bad_column.reference @@ -0,0 +1 @@ +1000448 diff --git a/tests/queries/0_stateless/02241_parquet_bad_column.sh b/tests/queries/0_stateless/02241_parquet_bad_column.sh new file mode 100755 index 00000000000..a160671a088 --- /dev/null +++ b/tests/queries/0_stateless/02241_parquet_bad_column.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_02241" +$CLICKHOUSE_CLIENT -q "create table test_02241 (image_path Nullable(String), + caption Nullable(String), + NSFW Nullable(String), + similarity Nullable(Float64), + LICENSE Nullable(String), + url Nullable(String), + key Nullable(UInt64), + shard_id Nullable(UInt64), + status Nullable(String), + width Nullable(UInt32), + height Nullable(UInt32), + exif Nullable(String), + original_width Nullable(UInt32), + original_height Nullable(UInt32)) engine=Memory" + +cat $CUR_DIR/data_parquet_bad_column/metadata_0.parquet | $CLICKHOUSE_CLIENT -q "insert into test_02241 format Parquet" + +$CLICKHOUSE_CLIENT -q "select count() from test_02241" +$CLICKHOUSE_CLIENT -q "drop table test_02241" From 411ffbc600584ffe47bbdce29981dfda7353c25e Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Mar 2022 14:15:39 +0000 Subject: [PATCH 56/85] Add data file for test --- .../02241_parquet_bad_column.reference | 2 +- .../data_parquet_bad_column/metadata_0.parquet | Bin 0 -> 14818 bytes 2 files changed, 1 insertion(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/data_parquet_bad_column/metadata_0.parquet diff --git a/tests/queries/0_stateless/02241_parquet_bad_column.reference b/tests/queries/0_stateless/02241_parquet_bad_column.reference index c7c7a69d906..f599e28b8ab 100644 --- a/tests/queries/0_stateless/02241_parquet_bad_column.reference +++ b/tests/queries/0_stateless/02241_parquet_bad_column.reference @@ -1 +1 @@ -1000448 +10 diff --git a/tests/queries/0_stateless/data_parquet_bad_column/metadata_0.parquet b/tests/queries/0_stateless/data_parquet_bad_column/metadata_0.parquet new file mode 100644 index 0000000000000000000000000000000000000000..8dd5f1e76f7f561c9ac7c45db65f809e3988e35d GIT binary patch literal 14818 zcmeG@S!^Tcc|)1pr8d&8yk5$bqjhHKEVG*xIR|fb?KMg9lC_eSI(RrPJ)9YmBh5i` zhNQHM1>C@J1EX-$ASi;MNc|A_C56$4Al-+e3HngDKv4vU(;6+127WNy!U(npj3jW2 ze*ZtbN%>ms0xdky4FCC$@BY7IX0#d2G7hWbr76ctmNmyYhxIH$eBClPO%MbVWMhUAwa{jTf&yQEo<`HlZ4$|@EB?QlL!f|3T zNL-rkjD`KqZj%lGmbt4_a6MkdjKq8(PKC$u&+Dm&mN9>mE(@BY~W?whD8-G*IGlxwdLGuQx+- z3b`eVVjY&cT;!V6g2walyg86+7fo?!mpKixO~lRQ^9Bu&M(|IVrT zw@)zr4FcWEe&Ej~a=h8OAfb02x z+4}i4_$A7#KeO4buRLv=n|bj!Km14Aob}yr+3Yhf{?YIKgJsVCOX^Q;r2YBJe`BNR z=RLRHLQ9_iaq8#SM$N^EI7R@j;T8gWZ5}8b?}eTTivGZ{=s4B&bGoERGS`rdL(KI* zP0hbK<$n8|`;A8yKLwtNk~rl!-PMw9&H}U`PDa&Zx#qMWP6rio$JtlN$+b*8vA&hS z)T#3=_xq2!|N2p+5p(54%=*&s(m0%d%G0>zZNvn&64; zY^)yg0oo|i)Q(mxMdO zO-+X@lFA;jlOd$b?=A5f45$V~G_jG`uq#x?^4Q(v_F-N0urv)a4>KNyx)_QvF*+QI z`4XOBDCzh3qin(x3&vOv8%@MRFvbP~k&B^~j8cVl2*nCP-1v(*Rc!0tS=}xfE!#ia z&eW-jB#TI+Vb&j}xCW?R7E8^#z!_rc!p~-4VI^|m5-7^=ZC4r~R@=(m4Y@-315}OE zYZMa-gd!kLf6&J=6p+TUW~{oxDOb9&3OcWt!3usV;uYlkubANqk@nK;UUbL8JCUJE zl|$P;q1%42r~@?G7Y=b>BbdW8ch`67kdl_DSFXq?#WkHb1qqT<2! ziRf@*M56|UksWPPQSsqQhqXpu~D8zjxZVJ0nhe zcMgh#Ol1 zuA9V;MnR{&JJN3Ke4!f?Q_5B10Kc!9RYqK z5A%-4x;%X0(n%meZ}L3k{G)6HiAyItL(Xnvgy4S*^|!(a6#1GuCih#G`R`iXFNfSe z4Wix9UEK#WkubtBwFX3>(h*t4X~)@q;s;$PojApV!tU+JqGPI~=2@t@v_?q*2B09p z*f-~Q-LFO5Uyod#RKN|XkU(h+9HYELs?-_jw~&m-dtvdQ3SYh2SD~#f37vA&)ZhOs zyZ?R}rD2~G_Czt=Uk1K;t0SkQbDfdsGyk`#Hop!u*Vht*Kr0V~)V!htF zy=Xac;L4yfo!o*_xeaBWYngPVG_5L&uG<;V{=?*@jfMmP9&#*AwLFneYYpip#C)jy zE~#1rD!|LGCDJv$=FDeBU6Y#+FWH#@y|73W_7;hSFV5^hxoD|g+LR7NS*J28Ut5S< zx3q;HtV9K^3{@@8*C3FRA@6zFxdL$<@<7{LX-ESC%M82hqe&*X?DH)%%tDq-97q+Y zOLfuK1Ux~@7t`A)OHw&WCgYk?heSk|raug?ZYEOXB~2C}iXF!(+{7+5aZ2 zAvU;15QWy4h=s?C4_Ou@%kMAYN}8{siZprQW!q2eY{VZli0oI1b8DhnHEI_`yWbbG zG5$n8lLS=o@L;RX@fU+jJ1Wn~Qn>+?3zlu$6Eld$i~nlRNy2RGG8=rH2%8{~aD)D^ z>oyoa6kz?m20M1@)@|qCT5omKF(7la=m?>ks_V!d=?BdP;Uk5ZV+B(u6UX{PNJVJr zfR6eg1}$S;V*|K4n!_eEL3e$KJr!c4phPq@L5n6oBZ|b<)}K7We25)}HNyYs@W+av zj|h=5Xd}q>B_W5neECMp$ZthJ;|^DM(sanP5&v4XFu|{PEE~0&HR6 zBxjb)0)5Ixd>LH$FE;y;!zuz$>}1}1Wd4;o_pg)gx1L6E<8vz<^Ch2KA@X;h(F!Mq zQ^;ZW`Z?^6A&0e|wM^Ry6(zFoL*n`?IDZHy4`*nWzWO>!7~nYF&mBgR*ibLN^X60n z-HuaeZU4brXI1AxS9cOZfXc=0y%pU!`l=0|CJ}w$N2== z{Xm}1y6^1~tNZh38s-Z=_x*i>;C=uVzWW%9@_VSX-S@i^cHj3(_46bZm$@jrH|w|0 zOQl|=)2q0Ty;K^{R@@IPBYc1<*!+*u^XJ_5`EN(h5s;xdPt@Us1gAUytSj zlz4c2qo_L6n9tsF?L!}`HsQ5JRYY1)IR%McZOSs-!L!E(s()$R*6Pq?%(jTzx1WOn z6JQBL7dvo=!N8XhvQQfCGwCWB@D2$P0N)$%nqZR9RO*Ml0j<0TUqK+!(AzM00mQ-d zTP{(B>F9d{Si7^G^uY6$0b+)!a7n$7R1lmYYDUKHi7F(t+e<)Phf`PQ>ws~j0|Up4 z5xtflN7wB=UR?dmd>rHO*a%*D>&qEI{Cq?m()GZ=fU?KL+Nl7>u?`H3p!{^^HHH!D zi*an7&f~=dmCHD$PUi?Fcv(A1h`sKQ5@Gi($T+USk>f;&-(pNa*Xtiac8?kxdVMjD zc5HYAyQtI+tcYO)o#>3ltOY+)7{}S`9>LX9*f6eM_ebf`tm4OU_4|(z8G4>Lj;h-` zg6JOg*LzOUliIM`9v>gU-xJ?wkZKQjUyMg!?@zc#^f-_refLSEWSyCB@SI+zE>7_Z;UxNZ~9nP@hP+;;-lk@3PSxFkjEm)fhFy&NR&%{?1F1M=}laXd2$H-`2S>4skae$-7fG)tz zL})P|!($nBx5?$!TgB9lglK`e2A`HILRxNPxbf9i0lq;k_->fvxm~)D%hrTc0`Ua$ z*liZr$Ug9{V&3K5ovcLrGb%GodtXjHxS}*64 zYH>HxF66Tsmg({Hfpt>>S%V%2=v%Mx>1@5s`j4V*uaJ+?TrSdlpmgmAwzD#S6ipjD zbTR)x=~34~N3veXuhxb1{!w(@#UdP_X?NKSGA~j*9 zKw}sRBMYe{?9ptkXX|NN&!h+TMtt@9*3NozTV79^Yl`(^?qEO9cK2pf+D^zT*&UGU zRtD{%&c0tC+y_iqk@w1Mw!5#Hay}Nw`?3KZ_73c2sm!jvP|U4=8oC%>X&)u8-WZ=s zg3n3b9Kht!G4*xxI3_~m59VCyL&<$j~hjh55S-cJzsGUS&O6n+m`{y z?wWd0#^uu+o3!`9C-(ILerQB4;2Zk-9QGfR_no?~fh~c4TlG=3b(?9`_$)R2p*C zFq|s|l*XzkolQsM&_Sd7IJ)0JX>MFYxi610+{6vIhda3UK_mJM%70@B2DHv zD7cA5Yq%39$!KBc*hIf&-)g`XEQ=awvxfA>jEpJM?t$ zL-7EInd+mKneS`G9N2vtV8!;J=op}X<>S`x0F0QqR}i?w$of=oeQYxSNQbd8AB{j2 z5dVSR`XpXInf{xQ3(*3qSFrr41l1bSACKS-=nvOB1F;khN107veL&tY z2#&tIp@fP?Q}F+0H;|#Vk>SAm2!6y57t>x35bEeYx(x6ObtBk6D;r$y0P+W!2Aa_UCR9cBB>dtd9$ou5 z{Aghe=h}G1D#A|{y7*&R+=~rx06!L8flaoBl-w#;R}yGG2my=-R2xADfa)%N@=3z^r8?Jmp(z?3zFzm>dK~_B4;SO`MI%(=@M<4EZ207V#L@-$ KS3&si&i@7d41KQv literal 0 HcmV?d00001 From 58f2aca120e565226137611ebca7b6e48f3df781 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Mar 2022 19:04:16 +0000 Subject: [PATCH 57/85] Fix tests --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 06f8616ea8c..14c81a0d90d 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -487,9 +487,12 @@ static void checkStatus(const arrow::Status & status, const String & column_name Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name, const Block * hint_header) { ColumnsWithTypeAndName sample_columns; + std::unordered_set 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())) + 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. From 7617053c582517d81b3ca898220215c7cdf1ce52 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 19 Mar 2022 15:44:21 +0100 Subject: [PATCH 58/85] Revert changes in HashTable.h --- src/Common/HashTable/HashTable.h | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 6dee485550b..5e439591ff2 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -1036,16 +1036,14 @@ public: return const_cast *>(this)->find(x, hash_value); } - template - requires (Grower_::performs_linear_probing_with_single_step) - bool ALWAYS_INLINE erase(const Key & x) + std::enable_if_t + ALWAYS_INLINE erase(const Key & x) { return erase(x, hash(x)); } - template - requires (Grower_::performs_linear_probing_with_single_step) - bool ALWAYS_INLINE erase(const Key & x, size_t hash_value) + std::enable_if_t + ALWAYS_INLINE erase(const Key & x, size_t hash_value) { /** Deletion from open addressing hash table without tombstones * From 7a63feb3f7be76733489139f438d6b70a0b3103d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 01:55:40 +0700 Subject: [PATCH 59/85] Make restore tasks explicit. --- src/Backups/BackupUtils.cpp | 374 +--------- src/Backups/BackupUtils.h | 16 +- src/Backups/IRestoreFromBackupTask.h | 31 + src/Backups/RestoreFromBackupUtils.cpp | 637 ++++++++++++++++++ src/Backups/RestoreFromBackupUtils.h | 23 + .../hasCompatibleDataToRestoreTable.cpp | 4 + src/Common/ErrorCodes.cpp | 1 + src/Interpreters/InterpreterBackupQuery.cpp | 4 +- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 124 +++- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageLog.cpp | 58 +- src/Storages/StorageLog.h | 3 +- src/Storages/StorageMergeTree.cpp | 4 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 60 +- src/Storages/StorageStripeLog.h | 3 +- 18 files changed, 930 insertions(+), 423 deletions(-) create mode 100644 src/Backups/IRestoreFromBackupTask.h create mode 100644 src/Backups/RestoreFromBackupUtils.cpp create mode 100644 src/Backups/RestoreFromBackupUtils.h diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index c26eec440e6..35c2fb33631 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -2,24 +2,17 @@ #include #include #include -#include #include #include +#include #include -#include #include -#include #include -#include -#include #include #include #include #include #include -#include - -namespace fs = std::filesystem; namespace DB @@ -29,8 +22,6 @@ namespace ErrorCodes extern const int BACKUP_ELEMENT_DUPLICATE; extern const int BACKUP_IS_EMPTY; extern const int LOGICAL_ERROR; - extern const int TABLE_ALREADY_EXISTS; - extern const int CANNOT_RESTORE_TABLE; } namespace @@ -265,28 +256,6 @@ namespace elements[database.index].except_list.emplace(table_name); } - /// Reorder the elements: databases should be before tables and dictionaries they contain. - for (auto & [database_name, database] : databases) - { - if (database.index == static_cast(-1)) - continue; - size_t min_index = std::numeric_limits::max(); - auto min_index_it = database.tables.end(); - for (auto it = database.tables.begin(); it != database.tables.end(); ++it) - { - if (min_index > it->second) - { - min_index = it->second; - min_index_it = it; - } - } - if (database.index > min_index) - { - std::swap(elements[database.index], elements[min_index]); - std::swap(database.index, min_index_it->second); - } - } - for (auto skip_index : skip_indices | boost::adaptors::reversed) elements.erase(elements.begin() + skip_index); } @@ -302,48 +271,6 @@ namespace return res; } - String getDataPathInBackup(const DatabaseAndTableName & table_name) - { - if (table_name.first.empty() || table_name.second.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty"); - assert(!table_name.first.empty() && !table_name.second.empty()); - return String{"data/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + "/"; - } - - String getDataPathInBackup(const IAST & create_query) - { - const auto & create = create_query.as(); - if (!create.table) - return {}; - if (create.temporary) - return getDataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}); - return getDataPathInBackup({create.getDatabase(), create.getTable()}); - } - - String getMetadataPathInBackup(const DatabaseAndTableName & table_name) - { - if (table_name.first.empty() || table_name.second.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty"); - return String{"metadata/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + ".sql"; - } - - String getMetadataPathInBackup(const String & database_name) - { - if (database_name.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name must not be empty"); - return String{"metadata/"} + escapeForFileName(database_name) + ".sql"; - } - - String getMetadataPathInBackup(const IAST & create_query) - { - const auto & create = create_query.as(); - if (!create.table) - return getMetadataPathInBackup(create.getDatabase()); - if (create.temporary) - return getMetadataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}); - return getMetadataPathInBackup({create.getDatabase(), create.getTable()}); - } - void backupCreateQuery(const IAST & create_query, BackupEntries & backup_entries) { auto metadata_entry = std::make_unique(serializeAST(create_query)); @@ -412,179 +339,6 @@ namespace backupDatabase(database, {}, context, renaming_config, backup_entries); } } - - void makeDatabaseIfNotExists(const String & database_name, ContextMutablePtr context) - { - if (DatabaseCatalog::instance().isDatabaseExist(database_name)) - return; - - /// We create and execute `create` query for the database name. - auto create_query = std::make_shared(); - create_query->setDatabase(database_name); - create_query->if_not_exists = true; - InterpreterCreateQuery create_interpreter{create_query, context}; - create_interpreter.execute(); - } - - ASTPtr readCreateQueryFromBackup(const DatabaseAndTableName & table_name, const BackupPtr & backup) - { - String create_query_path = getMetadataPathInBackup(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 parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - } - - ASTPtr readCreateQueryFromBackup(const String & database_name, const BackupPtr & backup) - { - String create_query_path = getMetadataPathInBackup(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 parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - } - - void restoreTable( - const DatabaseAndTableName & table_name, - const ASTs & partitions, - ContextMutablePtr context, - const BackupPtr & backup, - const BackupRenamingConfigPtr & renaming_config, - RestoreObjectsTasks & restore_tasks) - { - ASTPtr create_query = readCreateQueryFromBackup(table_name, backup); - auto new_create_query = typeid_cast>(renameInCreateQuery(create_query, renaming_config, context)); - - restore_tasks.emplace_back([table_name, new_create_query, partitions, context, backup]() -> RestoreDataTasks - { - DatabaseAndTableName new_table_name{new_create_query->getDatabase(), new_create_query->getTable()}; - if (new_create_query->temporary) - new_table_name.first = DatabaseCatalog::TEMPORARY_DATABASE; - - context->checkAccess(AccessType::INSERT, new_table_name.first, new_table_name.second); - - StoragePtr storage; - for (size_t try_index = 0; try_index != 10; ++try_index) - { - if (DatabaseCatalog::instance().isTableExist({new_table_name.first, new_table_name.second}, context)) - { - DatabasePtr existing_database; - StoragePtr existing_storage; - std::tie(existing_database, existing_storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable({new_table_name.first, new_table_name.second}, context); - if (existing_storage) - { - if (auto existing_table_create_query = existing_database->tryGetCreateTableQuery(new_table_name.second, context)) - { - if (hasCompatibleDataToRestoreTable(*new_create_query, existing_table_create_query->as())) - { - storage = existing_storage; - break; - } - else - { - String error_message = (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - ? ("Temporary table " + backQuoteIfNeed(new_table_name.second) + " already exists") - : ("Table " + backQuoteIfNeed(new_table_name.first) + "." + backQuoteIfNeed(new_table_name.second) - + " already exists"); - throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); - } - } - } - } - - makeDatabaseIfNotExists(new_table_name.first, context); - - try - { - InterpreterCreateQuery create_interpreter{new_create_query, context}; - create_interpreter.execute(); - } - catch (Exception & e) - { - if (e.code() != ErrorCodes::TABLE_ALREADY_EXISTS) - throw; - } - } - - if (!storage) - { - String error_message = (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - ? ("Could not create temporary table " + backQuoteIfNeed(new_table_name.second) + " for restoring") - : ("Could not create table " + backQuoteIfNeed(new_table_name.first) + "." + backQuoteIfNeed(new_table_name.second) - + " for restoring"); - throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); - } - - String data_path_in_backup = getDataPathInBackup(table_name); - RestoreDataTasks restore_data_tasks = storage->restoreFromBackup(backup, data_path_in_backup, partitions, context); - - /// Keep `storage` alive while we're executing `restore_data_tasks`. - for (auto & restore_data_task : restore_data_tasks) - restore_data_task = [restore_data_task, storage]() { restore_data_task(); }; - - return restore_data_tasks; - }); - } - - void restoreDatabase(const String & database_name, const std::set & except_list, ContextMutablePtr context, const BackupPtr & backup, const BackupRenamingConfigPtr & renaming_config, RestoreObjectsTasks & restore_tasks) - { - ASTPtr create_query = readCreateQueryFromBackup(database_name, backup); - auto new_create_query = typeid_cast>(renameInCreateQuery(create_query, renaming_config, context)); - - restore_tasks.emplace_back([database_name, new_create_query, except_list, context, backup, renaming_config]() -> RestoreDataTasks - { - const String & new_database_name = new_create_query->getDatabase(); - context->checkAccess(AccessType::SHOW_TABLES, new_database_name); - - if (!DatabaseCatalog::instance().isDatabaseExist(new_database_name)) - { - /// We create and execute `create` query for the database name. - new_create_query->if_not_exists = true; - InterpreterCreateQuery create_interpreter{new_create_query, context}; - create_interpreter.execute(); - } - - RestoreObjectsTasks restore_objects_tasks; - Strings table_metadata_filenames = backup->listFiles("metadata/" + escapeForFileName(database_name) + "/", "/"); - 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; - restoreTable({database_name, table_name}, {}, context, backup, renaming_config, restore_objects_tasks); - } - - RestoreDataTasks restore_data_tasks; - for (auto & restore_object_task : restore_objects_tasks) - insertAtEnd(restore_data_tasks, std::move(restore_object_task)()); - return restore_data_tasks; - }); - } - - void restoreAllDatabases(const std::set & except_list, ContextMutablePtr context, const BackupPtr & backup, const BackupRenamingConfigPtr & renaming_config, RestoreObjectsTasks & restore_tasks) - { - restore_tasks.emplace_back([except_list, context, backup, renaming_config]() -> RestoreDataTasks - { - RestoreObjectsTasks restore_objects_tasks; - 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; - restoreDatabase(database_name, {}, context, backup, renaming_config, restore_objects_tasks); - } - - RestoreDataTasks restore_data_tasks; - for (auto & restore_object_task : restore_objects_tasks) - insertAtEnd(restore_data_tasks, std::move(restore_object_task)()); - return restore_data_tasks; - }); - } } @@ -728,106 +482,46 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries backup->finalizeWriting(); } - -RestoreObjectsTasks makeRestoreTasks(const Elements & elements, ContextMutablePtr context, const BackupPtr & backup) +String getDataPathInBackup(const DatabaseAndTableName & table_name) { - RestoreObjectsTasks restore_tasks; - - auto elements2 = adjustElements(elements, context->getCurrentDatabase()); - auto renaming_config = std::make_shared(); - renaming_config->setFromBackupQueryElements(elements2); - - for (const auto & element : elements2) - { - switch (element.type) - { - case ElementType::TABLE: - { - const String & database_name = element.name.first; - const String & table_name = element.name.second; - restoreTable({database_name, table_name}, element.partitions, context, backup, renaming_config, restore_tasks); - break; - } - - case ElementType::DATABASE: - { - const String & database_name = element.name.first; - restoreDatabase(database_name, element.except_list, context, backup, renaming_config, restore_tasks); - break; - } - - case ElementType::ALL_DATABASES: - { - restoreAllDatabases(element.except_list, context, backup, renaming_config, restore_tasks); - break; - } - - default: - throw Exception("Unexpected element type", ErrorCodes::LOGICAL_ERROR); /// other element types have been removed in deduplicateElements() - } - } - - return restore_tasks; + if (table_name.first.empty() || table_name.second.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty"); + assert(!table_name.first.empty() && !table_name.second.empty()); + return String{"data/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + "/"; } - -void executeRestoreTasks(RestoreObjectsTasks && restore_tasks, size_t num_threads) +String getDataPathInBackup(const IAST & create_query) { - if (!num_threads) - num_threads = 1; + const auto & create = create_query.as(); + if (!create.table) + return {}; + if (create.temporary) + return getDataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}); + return getDataPathInBackup({create.getDatabase(), create.getTable()}); +} - RestoreDataTasks restore_data_tasks; - for (auto & restore_object_task : restore_tasks) - insertAtEnd(restore_data_tasks, std::move(restore_object_task)()); - restore_tasks.clear(); +String getMetadataPathInBackup(const DatabaseAndTableName & table_name) +{ + if (table_name.first.empty() || table_name.second.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty"); + return String{"metadata/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + ".sql"; +} - std::vector threads; - size_t num_active_threads = 0; - std::mutex mutex; - std::condition_variable cond; - std::exception_ptr exception; +String getMetadataPathInBackup(const String & database_name) +{ + if (database_name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name must not be empty"); + return String{"metadata/"} + escapeForFileName(database_name) + ".sql"; +} - for (auto & restore_data_task : restore_data_tasks) - { - { - std::unique_lock lock{mutex}; - if (exception) - break; - cond.wait(lock, [&] { return num_active_threads < num_threads; }); - if (exception) - break; - ++num_active_threads; - } - - threads.emplace_back([&restore_data_task, &mutex, &cond, &num_active_threads, &exception]() mutable - { - try - { - restore_data_task(); - restore_data_task = {}; - } - catch (...) - { - std::lock_guard lock{mutex}; - if (!exception) - exception = std::current_exception(); - } - - { - std::lock_guard lock{mutex}; - --num_active_threads; - cond.notify_all(); - } - }); - } - - for (auto & thread : threads) - thread.join(); - - restore_data_tasks.clear(); - - if (exception) - std::rethrow_exception(exception); +String getMetadataPathInBackup(const IAST & create_query) +{ + const auto & create = create_query.as(); + if (!create.table) + return getMetadataPathInBackup(create.getDatabase()); + if (create.temporary) + return getMetadataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}); + return getMetadataPathInBackup({create.getDatabase(), create.getTable()}); } } diff --git a/src/Backups/BackupUtils.h b/src/Backups/BackupUtils.h index 70f080cf6e9..c2f56044d5d 100644 --- a/src/Backups/BackupUtils.h +++ b/src/Backups/BackupUtils.h @@ -12,13 +12,8 @@ using BackupMutablePtr = std::shared_ptr; class IBackupEntry; using BackupEntryPtr = std::unique_ptr; using BackupEntries = std::vector>; -using RestoreDataTask = std::function; -using RestoreDataTasks = std::vector; -using RestoreObjectTask = std::function; -using RestoreObjectsTasks = std::vector; class Context; using ContextPtr = std::shared_ptr; -using ContextMutablePtr = std::shared_ptr; /// Prepares backup entries. @@ -30,10 +25,13 @@ UInt64 estimateBackupSize(const BackupEntries & backup_entries, const BackupPtr /// 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); } diff --git a/src/Backups/IRestoreFromBackupTask.h b/src/Backups/IRestoreFromBackupTask.h new file mode 100644 index 00000000000..93e08b035ee --- /dev/null +++ b/src/Backups/IRestoreFromBackupTask.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/// Represents a task of restoring something (database / table / table's part) from backup. +class IRestoreFromBackupTask +{ +public: + IRestoreFromBackupTask() = default; + virtual ~IRestoreFromBackupTask() = default; + + /// Perform restoring, the function also can return a list of nested tasks that should be run later. + virtual std::vector> 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 RestoreFromBackupTaskPtr = std::unique_ptr; +using RestoreFromBackupTasks = std::vector; + +} diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreFromBackupUtils.cpp new file mode 100644 index 00000000000..af3cc926a7e --- /dev/null +++ b/src/Backups/RestoreFromBackupUtils.cpp @@ -0,0 +1,637 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +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; + + + /// Replaces an empty database with the current database. + void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database) + { + for (auto & element : elements) + { + if (element.type == ElementType::TABLE) + { + if (element.name.first.empty() && !element.name.second.empty()) + element.name.first = current_database; + if (element.new_name.first.empty() && !element.new_name.second.empty()) + element.new_name.first = current_database; + } + } + } + + + /// Restores a database (without tables inside), should be executed before executing + /// RestoreTableFromBackupTask. + class RestoreDatabaseFromBackupTask : public IRestoreFromBackupTask + { + public: + RestoreDatabaseFromBackupTask(ContextMutablePtr context_, const ASTPtr & create_query_) + : context(context_), create_query(typeid_cast>(create_query_)) + { + } + + RestoreFromBackupTasks run() override + { + createDatabase(); + return {}; + } + + bool isSequential() const override { return true; } + + private: + void createDatabase() + { + InterpreterCreateQuery create_interpreter{create_query, context}; + create_interpreter.execute(); + } + + ContextMutablePtr context; + std::shared_ptr create_query; + }; + + + /// Restores a table and fills it with data. + class RestoreTableFromBackupTask : public IRestoreFromBackupTask + { + public: + RestoreTableFromBackupTask( + ContextMutablePtr context_, + const ASTPtr & create_query_, + const ASTs & partitions_, + const BackupPtr & backup_, + const DatabaseAndTableName & table_name_in_backup_) + : context(context_), create_query(typeid_cast>(create_query_)), + partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_) + { + table_name = DatabaseAndTableName{create_query->getDatabase(), create_query->getTable()}; + if (create_query->temporary) + table_name.first = DatabaseCatalog::TEMPORARY_DATABASE; + } + + RestoreFromBackupTasks run() override + { + createStorage(); + auto storage = getStorage(); + RestoreFromBackupTasks tasks; + if (auto task = insertDataIntoStorage(storage)) + tasks.push_back(std::move(task)); + return tasks; + } + + bool isSequential() const override { return true; } + + private: + void createStorage() + { + InterpreterCreateQuery create_interpreter{create_query, context}; + create_interpreter.execute(); + } + + StoragePtr tryGetStorage() + { + if (!DatabaseCatalog::instance().isTableExist({table_name.first, table_name.second}, context)) + return nullptr; + + DatabasePtr existing_database; + StoragePtr existing_storage; + std::tie(existing_database, existing_storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable({table_name.first, table_name.second}, context); + if (!existing_storage) + return nullptr; + + auto existing_table_create_query = existing_database->tryGetCreateTableQuery(table_name.second, context); + if (!existing_table_create_query) + return nullptr; + + if (!hasCompatibleDataToRestoreTable(*create_query, existing_table_create_query->as())) + throw Exception( + ErrorCodes::CANNOT_RESTORE_TABLE, + "Table {}.{} from backup is incompatible with existing table {}.{}. " + "The create query of the table from backup: {}." + "The create query of the existing table: {}", + backQuoteIfNeed(table_name_in_backup.first), + backQuoteIfNeed(table_name_in_backup.second), + backQuoteIfNeed(table_name.first), + backQuoteIfNeed(table_name.second), + serializeAST(*create_query), + serializeAST(*existing_table_create_query)); + + return existing_storage; + } + + StoragePtr getStorage() + { + if (auto storage = tryGetStorage()) + return storage; + + String error_message = (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) + ? ("Could not create temporary table " + backQuoteIfNeed(table_name.second) + " for restoring") + : ("Could not create table " + backQuoteIfNeed(table_name.first) + "." + backQuoteIfNeed(table_name.second) + + " for restoring"); + throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); + } + + RestoreFromBackupTaskPtr insertDataIntoStorage(StoragePtr storage) + { + context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); + String data_path_in_backup = getDataPathInBackup(table_name_in_backup); + return storage->restoreFromBackup(backup, data_path_in_backup, partitions, context); + } + + ContextMutablePtr context; + std::shared_ptr create_query; + DatabaseAndTableName table_name; + ASTs partitions; + BackupPtr backup; + DatabaseAndTableName table_name_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_) + : context(context_), backup(backup_) {} + + /// Prepares internal structures for making tasks for restoring. + void prepare(const ASTBackupQuery::Elements & elements) + { + auto elements2 = elements; + replaceEmptyDatabaseWithCurrentDatabase(elements2, context->getCurrentDatabase()); + + auto new_renaming_config = std::make_shared(); + new_renaming_config->setFromBackupQueryElements(elements2); + renaming_config = new_renaming_config; + + for (const auto & element : elements2) + { + switch (element.type) + { + case ElementType::TABLE: [[fallthrough]]; + case ElementType::DICTIONARY: + { + const String & database_name = element.name.first; + const String & table_name = element.name.second; + prepareToRestoreTable(DatabaseAndTableName{database_name, table_name}, element.partitions); + break; + } + + case ElementType::TEMPORARY_TABLE: + { + String database_name = DatabaseCatalog::TEMPORARY_DATABASE; + const String & table_name = element.name.second; + 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_TEMPORARY_TABLES: + { + prepareToRestoreDatabase(DatabaseCatalog::TEMPORARY_DATABASE, element.except_list); + break; + } + + case ElementType::ALL_DATABASES: + { + prepareToRestoreAllDatabases(element.except_list); + break; + } + + case ElementType::EVERYTHING: + { + prepareToRestoreAllDatabases({}); + break; + } + } + } + } + + /// Makes tasks for restoring, should be called after prepare(). + RestoreFromBackupTasks makeTasks() const + { + /// Check that there are not `different_create_query`. (If it's set it means error.) + for (auto & info : databases | boost::adaptors::map_values) + { + if (info.different_create_query) + throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, + "Couldn't restore a database because two different create queries were generated for it: {} and {}", + serializeAST(*info.create_query), serializeAST(*info.different_create_query)); + } + + RestoreFromBackupTasks res; + for (auto & info : databases | boost::adaptors::map_values) + res.push_back(std::make_unique(context, info.create_query)); + + /// TODO: We need to restore tables according to their dependencies. + for (auto & info : tables | boost::adaptors::map_values) + res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup)); + + 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_config->getNewTableName(table_name_); + if (tables.contains(new_table_name)) + { + String message; + if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) + message = fmt::format("Couldn't restore temporary table {} twice", backQuoteIfNeed(new_table_name.second)); + else + message = fmt::format("Couldn't restore table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second)); + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, message); + } + + /// Make a create query for this table. + auto create_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_)); + create_query->if_not_exists = true; + + 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 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(); + 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 create_db_query; + if (hasCreateQueryInBackup(table_name_.first)) + create_db_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_.first)); + else + create_db_query = std::make_shared(); + create_db_query->setDatabase(new_table_name.first); + create_db_query->if_not_exists = true; + if (serializeAST(*info_db.create_query) != serializeAST(*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 & except_list_) + { + /// Check that we are not trying to restore the same database again. + String new_database_name = renaming_config->getNewDatabaseName(database_name_); + if (databases.contains(new_database_name) && databases[new_database_name].is_explicit) + throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Couldn't restore 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, "Could not restore 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 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(); + create_db_query->setDatabase(database_name_); + db_name_in_backup.clear(); + } + + 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 = 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 & 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{}); + } + } + + /// Reads a create query for creating a specified table from the backup. + std::shared_ptr 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, "Could not restore table {}.{} because there is no such table in the backup", backQuoteIfNeed(table_name.first), backQuoteIfNeed(table_name.second)); + 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>(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 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, "Could not restore 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>(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 renameInCreateQuery(const ASTPtr & ast) const + { + return typeid_cast>(::DB::renameInCreateQuery(ast, renaming_config, context)); + } + + 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 RestoreTableFromBackupTask. + struct CreateTableInfo + { + ASTPtr create_query; + DatabaseAndTableName name_in_backup; + ASTs partitions; + }; + + /// Information which is used to make an instance of RestoreDatabaseFromBackupTask. + 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; + BackupRenamingConfigPtr renaming_config; + std::map databases; + std::map tables; + }; + + + /// Reverts completed restore tasks (in reversed order). + void rollbackRestoreTasks(RestoreFromBackupTasks && 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"); + } + } + } +} + + +RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements) +{ + RestoreTasksBuilder builder{context, backup}; + builder.prepare(elements); + return builder.makeTasks(); +} + + +void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_threads) +{ + if (!num_threads) + num_threads = 1; + + RestoreFromBackupTasks completed_tasks; + bool need_rollback_completed_tasks = true; + + SCOPE_EXIT({ + if (need_rollback_completed_tasks) + rollbackRestoreTasks(std::move(completed_tasks)); + }); + + std::deque> sequential_tasks; + std::deque> 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(); + + RestoreFromBackupTasks 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> running_tasks; + std::vector threads; + std::mutex mutex; + std::condition_variable cond; + std::exception_ptr exception; + + while (true) + { + IRestoreFromBackupTask * 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; + } + + RestoreFromBackupTasks 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; +} + +} diff --git a/src/Backups/RestoreFromBackupUtils.h b/src/Backups/RestoreFromBackupUtils.h new file mode 100644 index 00000000000..75fe96495d6 --- /dev/null +++ b/src/Backups/RestoreFromBackupUtils.h @@ -0,0 +1,23 @@ +#pragma once + +#include + + +namespace DB +{ + +class IBackup; +using BackupPtr = std::shared_ptr; +class IRestoreFromBackupTask; +using RestoreFromBackupTaskPtr = std::unique_ptr; +using RestoreFromBackupTasks = std::vector; +class Context; +using ContextMutablePtr = std::shared_ptr; + +/// Prepares restore tasks. +RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements); + +/// Executes restore tasks. +void executeRestoreTasks(RestoreFromBackupTasks && tasks, size_t num_threads); + +} diff --git a/src/Backups/hasCompatibleDataToRestoreTable.cpp b/src/Backups/hasCompatibleDataToRestoreTable.cpp index 9c11d371bb0..8ac2fda93d8 100644 --- a/src/Backups/hasCompatibleDataToRestoreTable.cpp +++ b/src/Backups/hasCompatibleDataToRestoreTable.cpp @@ -16,6 +16,10 @@ bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCre q1->uuid = UUIDHelpers::Nil; q2->uuid = UUIDHelpers::Nil; + /// Clear IF NOT EXISTS flag. + q1->if_not_exists = false; + q2->if_not_exists = false; + return serializeAST(*q1) == serializeAST(*q2); } diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 809830913fe..8bdae71b822 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -615,6 +615,7 @@ M(644, REMOTE_FS_OBJECT_CACHE_ERROR) \ M(645, NUMBER_OF_DIMENSIONS_MISMATHED) \ M(646, RBAC_VERSION_IS_TOO_NEW) \ + M(647, CANNOT_RESTORE_DATABASE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index 45eb8e48599..de8a9c273cd 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -2,8 +2,10 @@ #include #include #include +#include #include #include +#include #include #include @@ -45,7 +47,7 @@ namespace void executeRestore(const ASTBackupQuery & query, ContextMutablePtr context) { BackupPtr backup = createBackup(query, context); - auto restore_tasks = makeRestoreTasks(query.elements, context, backup); + auto restore_tasks = makeRestoreTasks(context, backup, query.elements); executeRestoreTasks(std::move(restore_tasks), context->getSettingsRef().max_backup_threads); } } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 89403a773b3..4b174e9fd6d 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -221,7 +221,7 @@ BackupEntries IStorage::backup(const ASTs &, ContextPtr) throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } -RestoreDataTasks IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) +RestoreFromBackupTaskPtr IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) { throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index e9969859d5f..e77dd215ce9 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -72,7 +72,8 @@ class IBackup; using BackupPtr = std::shared_ptr; class IBackupEntry; using BackupEntries = std::vector>>; -using RestoreDataTasks = std::vector>; +class IRestoreFromBackupTask; +using RestoreFromBackupTaskPtr = std::unique_ptr; struct ColumnSize { @@ -220,7 +221,7 @@ public: virtual BackupEntries backup(const ASTs & partitions, ContextPtr context); /// 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 RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); /// Returns whether the column is virtual - by default all columns are real. /// Initially reserved virtual column name may be shadowed by real column. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4fbc3376b7e..6ca1f10b8f2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -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 & partition_ids, - SimpleIncrement * increment) +class MergeTreeDataRestoreTask : public IRestoreFromBackupTask { - 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 & storage_, + const BackupPtr & backup_, + const String & data_path_in_backup_, + const std::unordered_set & 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 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]() + RestoreFromBackupTasks run() override + { + RestoreFromBackupTasks 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(storage, backup, data_path_in_backup, part_name, *part_info, increment)); + } + return restore_part_tasks; + } + +private: + std::shared_ptr storage; + BackupPtr backup; + String data_path_in_backup; + std::unordered_set partition_ids; + SimpleIncrement * increment; + + class RestorePartTask : public IRestoreFromBackupTask + { + public: + RestorePartTask( + const std::shared_ptr & 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_) + { + } + + RestoreFromBackupTasks 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 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(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(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 storage; + BackupPtr backup; + String data_path_in_backup; + String part_name; + MergeTreePartInfo part_info; + SimpleIncrement * increment; + }; +}; - return restore_tasks; + +RestoreFromBackupTaskPtr MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup, + const std::unordered_set & partition_ids, + SimpleIncrement * increment) +{ + return std::make_unique( + std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup, partition_ids, increment); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fca052d5538..9b3df89ac78 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -679,7 +679,7 @@ public: static BackupEntries backupDataParts(const DataPartsVector & data_parts); /// Extract data from the backup and put it to the storage. - RestoreDataTasks restoreDataPartsFromBackup( + RestoreFromBackupTaskPtr restoreDataPartsFromBackup( const BackupPtr & backup, const String & data_path_in_backup, const std::unordered_set & partition_ids, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index d3923a190a1..6ac73eb01ee 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include @@ -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 IRestoreFromBackupTask { - 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 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_) + { + } + + RestoreFromBackupTasks 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 storage; + BackupPtr backup; + String data_path_in_backup; + ContextMutablePtr context; +}; + +RestoreFromBackupTaskPtr StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + return std::make_unique( + typeid_cast>(shared_from_this()), backup, data_path_in_backup, context); } diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index d6f3208f693..3c337bbd4fa 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -23,6 +23,7 @@ class StorageLog final : public shared_ptr_helper, public IStorage { friend class LogSource; friend class LogSink; + friend class LogRestoreTask; friend struct shared_ptr_helper; public: @@ -52,7 +53,7 @@ public: 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; + RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; protected: /** Attach the table with the appropriate name, along the appropriate path (with / at the end), diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 812e2264adb..33893aa8a2d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -3,7 +3,7 @@ #include #include - +#include #include #include #include @@ -1644,7 +1644,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ } -RestoreDataTasks StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) +RestoreFromBackupTaskPtr StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) { return restoreDataPartsFromBackup(backup, data_path_in_backup, getPartitionIDsFromQuery(partitions, local_context), &increment); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index abdaf6934d6..9a089d0074a 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -97,7 +97,7 @@ public: CheckResults checkData(const ASTPtr & query, ContextPtr context) override; - RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index f1f84a88c36..b4ad3cfc7c5 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include @@ -552,37 +553,51 @@ BackupEntries StorageStripeLog::backup(const ASTs & partitions, ContextPtr conte return backup_entries; } -RestoreDataTasks StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +class StripeLogRestoreTask : public IRestoreFromBackupTask { - if (!partitions.empty()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + using WriteLock = StorageStripeLog::WriteLock; - auto restore_task = [this, backup, data_path_in_backup, context]() +public: + StripeLogRestoreTask( + const std::shared_ptr 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_) { - WriteLock lock{rwlock, getLockTimeout(context)}; + } + + RestoreFromBackupTasks run() override + { + WriteLock lock{storage->rwlock, getLockTimeout(context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + auto & file_checker = storage->file_checker; + /// Load the indices if not loaded yet. We have to do that now because we're going to update these indices. - loadIndices(lock); + storage->loadIndices(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 the data file. - auto old_data_size = file_checker.getFileSize(data_file_path); + auto old_data_size = file_checker.getFileSize(storage->data_file_path); { + const auto & data_file_path = storage->data_file_path; 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); } /// Append the index. { + const auto & index_file_path = storage->index_file_path; String index_path_in_backup = data_path_in_backup + fileName(index_file_path); IndexForNativeFormat extra_indices; auto backup_entry = backup->readFile(index_path_in_backup); @@ -597,23 +612,38 @@ RestoreDataTasks StorageStripeLog::restoreFromBackup(const BackupPtr & backup, c column.location.offset_in_compressed_file += old_data_size; } - insertAtEnd(indices.blocks, std::move(extra_indices.blocks)); + insertAtEnd(storage->indices.blocks, std::move(extra_indices.blocks)); } /// Finish writing. - saveIndices(lock); - saveFileSizes(lock); + storage->saveIndices(lock); + storage->saveFileSizes(lock); + return {}; } catch (...) { /// Rollback partial writes. file_checker.repair(); - removeUnsavedIndices(lock); + storage->removeUnsavedIndices(lock); throw; } + } - }; - return {restore_task}; +private: + std::shared_ptr storage; + BackupPtr backup; + String data_path_in_backup; + ContextMutablePtr context; +}; + + +RestoreFromBackupTaskPtr StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + return std::make_unique( + typeid_cast>(shared_from_this()), backup, data_path_in_backup, context); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index bab5116cfc1..8fbd94f9492 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -23,6 +23,7 @@ class StorageStripeLog final : public shared_ptr_helper, publi { friend class StripeLogSource; friend class StripeLogSink; + friend class StripeLogRestoreTask; friend struct shared_ptr_helper; public: @@ -51,7 +52,7 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) 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; + RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; protected: StorageStripeLog( From c160494f438d77641c4db0d475dcd298bd52f620 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 16:44:42 +0700 Subject: [PATCH 60/85] Make BACKUP more consistent with RESTORE. --- src/Backups/BackupUtils.cpp | 601 +++++++++++++++--------------------- src/Backups/BackupUtils.h | 6 - src/Common/ErrorCodes.cpp | 5 +- 3 files changed, 256 insertions(+), 356 deletions(-) diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 35c2fb33631..820a7f86f7f 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -10,18 +10,15 @@ #include #include #include -#include -#include -#include namespace DB { namespace ErrorCodes { - extern const int BACKUP_ELEMENT_DUPLICATE; + extern const int CANNOT_BACKUP_TABLE; + extern const int CANNOT_BACKUP_DATABASE; extern const int BACKUP_IS_EMPTY; - extern const int LOGICAL_ERROR; } namespace @@ -31,33 +28,6 @@ namespace using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; - /// Replace elements of types DICTIONARY or EVERYTHING with elements of other types. - void replaceElementTypesWithBaseElementTypes(Elements & elements) - { - for (size_t i = 0; i != elements.size(); ++i) - { - auto & element = elements[i]; - switch (element.type) - { - case ElementType::DICTIONARY: - { - element.type = ElementType::TABLE; - break; - } - - case ElementType::EVERYTHING: - { - element.type = ElementType::ALL_DATABASES; - auto & new_element = elements.emplace_back(); - new_element.type = ElementType::ALL_TEMPORARY_TABLES; - break; - } - - default: - break; - } - } - } /// Replaces an empty database with the current database. void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database) @@ -74,352 +44,286 @@ namespace } } - /// Replaces elements of types TEMPORARY_TABLE or ALL_TEMPORARY_TABLES with elements of type TABLE or DATABASE. - void replaceTemporaryTablesWithTemporaryDatabase(Elements & elements) + + /// Makes backup entries to backup databases and tables according to the elements of ASTBackupQuery. + /// Keep this class consistent with RestoreTasksBuilder. + class BackupEntriesBuilder { - for (auto & element : elements) + public: + BackupEntriesBuilder(ContextPtr context_) : context(context_) { } + + /// Prepares internal structures for making backup entries. + void prepare(const ASTBackupQuery::Elements & elements) { - switch (element.type) + auto elements2 = elements; + replaceEmptyDatabaseWithCurrentDatabase(elements2, context->getCurrentDatabase()); + + auto new_renaming_config = std::make_shared(); + new_renaming_config->setFromBackupQueryElements(elements2); + renaming_config = new_renaming_config; + + for (const auto & element : elements2) { - case ElementType::TEMPORARY_TABLE: + switch (element.type) { - element.type = ElementType::TABLE; - element.name.first = DatabaseCatalog::TEMPORARY_DATABASE; - if (element.new_name.first.empty() && !element.new_name.second.empty()) - element.new_name.first = DatabaseCatalog::TEMPORARY_DATABASE; - break; - } + case ElementType::TABLE: [[fallthrough]]; + case ElementType::DICTIONARY: + { + const String & database_name = element.name.first; + const String & table_name = element.name.second; + prepareToBackupTable(DatabaseAndTableName{database_name, table_name}, element.partitions); + break; + } - case ElementType::ALL_TEMPORARY_TABLES: - { - element.type = ElementType::DATABASE; - element.name.first = DatabaseCatalog::TEMPORARY_DATABASE; - break; - } + case ElementType::TEMPORARY_TABLE: + { + String database_name = DatabaseCatalog::TEMPORARY_DATABASE; + const String & table_name = element.name.second; + prepareToBackupTable(DatabaseAndTableName{database_name, table_name}, element.partitions); + break; + } - default: - break; + case ElementType::DATABASE: + { + const String & database_name = element.name.first; + prepareToBackupDatabase(database_name, element.except_list); + break; + } + + case ElementType::ALL_TEMPORARY_TABLES: + { + prepareToBackupDatabase(DatabaseCatalog::TEMPORARY_DATABASE, element.except_list); + break; + } + + case ElementType::ALL_DATABASES: + { + prepareToBackupAllDatabases(element.except_list); + break; + } + + case ElementType::EVERYTHING: + { + prepareToBackupAllDatabases({}); + prepareToBackupDatabase(DatabaseCatalog::TEMPORARY_DATABASE, {}); + break; + } + } } } - } - /// Set new names if they are not specified. - void setNewNamesIfNotSet(Elements & elements) - { - for (auto & element : elements) + /// Makes backup entries, should be called after prepare(). + BackupEntries makeBackupEntries() const { - switch (element.type) + /// Check that there are not `different_create_query`. (If it's set it means error.) + for (auto & info : databases | boost::adaptors::map_values) { - case ElementType::TABLE: - { - if (element.new_name.second.empty()) - element.new_name = element.name; - break; - } + if (info.different_create_query) + throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, + "Couldn't backup a database because two different create queries were generated for it: {} and {}", + serializeAST(*info.create_query), serializeAST(*info.different_create_query)); + } - case ElementType::DATABASE: - { - if (element.new_name.first.empty()) - element.new_name = element.name; - break; - } + BackupEntries res; + for (auto & info : databases | boost::adaptors::map_values) + res.push_back(makeBackupEntryForMetadata(*info.create_query)); - default: - break; + for (auto & info : tables | boost::adaptors::map_values) + { + res.push_back(makeBackupEntryForMetadata(*info.create_query)); + + auto data_backup = info.storage->backup(info.partitions, context); + if (!data_backup.empty()) + { + String data_path = getDataPathInBackup(*info.create_query); + for (auto & [path_in_backup, backup_entry] : data_backup) + res.emplace_back(data_path + path_in_backup, std::move(backup_entry)); + } + } + + /// A backup cannot be empty. + if (res.empty()) + throw Exception("Backup must not be empty", ErrorCodes::BACKUP_IS_EMPTY); + + return res; + } + + private: + /// Prepares to backup a single table and probably its database's definition. + void prepareToBackupTable(const DatabaseAndTableName & table_name_, const ASTs & partitions_) + { + auto [database, storage] = DatabaseCatalog::instance().getDatabaseAndTable({table_name_.first, table_name_.second}, context); + prepareToBackupTable(table_name_, {database, storage}, partitions_); + } + + void prepareToBackupTable(const DatabaseAndTableName & table_name_, const DatabaseAndTable & table_, const ASTs & partitions_) + { + context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); + + /// Check that we are not trying to backup the same table again. + DatabaseAndTableName new_table_name = renaming_config->getNewTableName(table_name_); + if (tables.contains(new_table_name)) + { + String message; + if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) + message = fmt::format("Couldn't backup temporary table {} twice", backQuoteIfNeed(new_table_name.second)); + else + message = fmt::format("Couldn't backup table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second)); + throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, message); + } + + /// Make a create query for this table. + auto create_query = renameInCreateQuery(table_.first->getCreateTableQuery(table_name_.second, context)); + + CreateTableInfo info; + info.create_query = create_query; + info.storage = table_.second; + info.name_in_backup = new_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 backup the database's definition too. + if (!isSystemOrTemporaryDatabase(table_name_.first)) + { + if (!databases.contains(new_table_name.first)) + { + /// Add a create query to backup the database if we haven't done it yet. + auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); + create_db_query->setDatabase(new_table_name.first); + + CreateDatabaseInfo info_db; + info_db.create_query = create_db_query; + info_db.original_name = table_name_.first; + info_db.is_explicit = false; + databases[new_table_name.first] = std::move(info_db); + } + else + { + /// We already have added a create query to backup 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.original_name != table_name_.first) && !info_db.different_create_query) + { + auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); + create_db_query->setDatabase(new_table_name.first); + if (serializeAST(*info_db.create_query) != serializeAST(*create_db_query)) + info_db.different_create_query = create_db_query; + } + } } } - } - /// Removes duplications in the elements of a backup query by removing some excessive elements and by updating except_lists. - /// This function helps deduplicate elements in queries like "BACKUP ALL DATABASES, DATABASE xxx USING NAME yyy" - /// (we need a deduplication for that query because `ALL DATABASES` includes `xxx` however we don't want - /// to backup/restore the same database twice while executing the same query). - /// Also this function slightly reorders elements: it puts databases before tables and dictionaries they contain. - void deduplicateAndReorderElements(Elements & elements) - { - std::set skip_indices; /// Indices of elements which should be removed in the end of this function. - size_t index_all_databases = static_cast(-1); /// Index of the first element of type ALL_DATABASES or -1 if not found. - - struct DatabaseInfo + /// Prepares to restore a database and all tables in it. + void prepareToBackupDatabase(const String & database_name_, const std::set & except_list_) { - size_t index = static_cast(-1); - std::unordered_map tables; + auto database = DatabaseCatalog::instance().getDatabase(database_name_, context); + prepareToBackupDatabase(database_name_, database, except_list_); + } + + void prepareToBackupDatabase(const String & database_name_, const DatabasePtr & database_, const std::set & except_list_) + { + context->checkAccess(AccessType::SHOW_DATABASES, database_name_); + + /// Check that we are not trying to restore the same database again. + String new_database_name = renaming_config->getNewDatabaseName(database_name_); + if (databases.contains(new_database_name) && databases[new_database_name].is_explicit) + throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Couldn't backup database {} twice", backQuoteIfNeed(new_database_name)); + + /// Of course we're not going to backup the definition of the system or the temporary database. + if (!isSystemOrTemporaryDatabase(database_name_)) + { + /// Make a create query for this database. + auto create_db_query = renameInCreateQuery(database_->getCreateDatabaseQuery()); + + CreateDatabaseInfo info_db; + info_db.create_query = create_db_query; + info_db.original_name = database_name_; + info_db.is_explicit = true; + databases[new_database_name] = std::move(info_db); + } + + /// Backup tables in this database. + for (auto it = database_->getTablesIteratorForBackup(context); it->isValid(); it->next()) + { + if (except_list_.contains(it->name())) + continue; + prepareToBackupTable({database_name_, it->name()}, {database_, it->table()}, {}); + } + } + + /// Prepares to backup all the databases contained in the backup. + void prepareToBackupAllDatabases(const std::set & except_list_) + { + for (const auto & [database_name, database] : DatabaseCatalog::instance().getDatabases()) + { + if (except_list_.contains(database_name)) + continue; + if (isSystemOrTemporaryDatabase(database_name)) + continue; + prepareToBackupDatabase(database_name, database, {}); + } + } + + /// Do renaming in the create query according to the renaming config. + std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const + { + return typeid_cast>(::DB::renameInCreateQuery(ast, renaming_config, context)); + } + + static bool isSystemOrTemporaryDatabase(const String & database_name) + { + return (database_name == DatabaseCatalog::SYSTEM_DATABASE) || (database_name == DatabaseCatalog::TEMPORARY_DATABASE); + } + + static std::pair makeBackupEntryForMetadata(const IAST & create_query) + { + auto metadata_entry = std::make_unique(serializeAST(create_query)); + String metadata_path = getMetadataPathInBackup(create_query); + return {metadata_path, std::move(metadata_entry)}; + } + + /// Information which is used to make an instance of RestoreTableFromBackupTask. + struct CreateTableInfo + { + ASTPtr create_query; + StoragePtr storage; + DatabaseAndTableName name_in_backup; + ASTs partitions; }; - std::unordered_map databases; /// Found databases and tables. - for (size_t i = 0; i != elements.size(); ++i) + /// Information which is used to make an instance of RestoreDatabaseFromBackupTask. + struct CreateDatabaseInfo { - auto & element = elements[i]; - switch (element.type) - { - case ElementType::TABLE: - { - auto & tables = databases.emplace(element.name.first, DatabaseInfo{}).first->second.tables; - auto it = tables.find(element.name.second); - if (it == tables.end()) - { - tables.emplace(element.name.second, i); - } - else - { - size_t prev_index = it->second; - if ((elements[i].new_name == elements[prev_index].new_name) - && (elements[i].partitions.empty() == elements[prev_index].partitions.empty())) - { - insertAtEnd(elements[prev_index].partitions, elements[i].partitions); - skip_indices.emplace(i); - } - else - { - throw Exception( - "Table " + backQuote(element.name.first) + "." + backQuote(element.name.second) + " was specified twice", - ErrorCodes::BACKUP_ELEMENT_DUPLICATE); - } - } - break; - } + ASTPtr create_query; + String original_name; - case ElementType::DATABASE: - { - auto it = databases.find(element.name.first); - if (it == databases.end()) - { - DatabaseInfo new_db_info; - new_db_info.index = i; - databases.emplace(element.name.first, new_db_info); - } - else if (it->second.index == static_cast(-1)) - { - it->second.index = i; - } - else - { - size_t prev_index = it->second.index; - if ((elements[i].new_name == elements[prev_index].new_name) - && (elements[i].except_list == elements[prev_index].except_list)) - { - skip_indices.emplace(i); - } - else - { - throw Exception("Database " + backQuote(element.name.first) + " was specified twice", ErrorCodes::BACKUP_ELEMENT_DUPLICATE); - } + /// 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; - } - break; - } + /// 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; + }; - case ElementType::ALL_DATABASES: - { - if (index_all_databases == static_cast(-1)) - { - index_all_databases = i; - } - else - { - size_t prev_index = index_all_databases; - if (elements[i].except_list == elements[prev_index].except_list) - skip_indices.emplace(i); - else - throw Exception("The tag ALL DATABASES was specified twice", ErrorCodes::BACKUP_ELEMENT_DUPLICATE); - } - break; - } - - default: - /// replaceElementTypesWithBaseElementTypes() and replaceTemporaryTablesWithTemporaryDatabase() should have removed all other element types. - throw Exception("Unexpected element type: " + std::to_string(static_cast(element.type)), ErrorCodes::LOGICAL_ERROR); - } - } - - if (index_all_databases != static_cast(-1)) - { - for (auto & [database_name, database] : databases) - { - elements[index_all_databases].except_list.emplace(database_name); - if (database.index == static_cast(-1)) - { - auto & new_element = elements.emplace_back(); - new_element.type = ElementType::DATABASE; - new_element.name.first = database_name; - new_element.new_name = new_element.name; - database.index = elements.size() - 1; - } - } - } - - for (auto & [database_name, database] : databases) - { - if (database.index == static_cast(-1)) - continue; - for (const auto & [table_name, table_index] : database.tables) - elements[database.index].except_list.emplace(table_name); - } - - for (auto skip_index : skip_indices | boost::adaptors::reversed) - elements.erase(elements.begin() + skip_index); - } - - Elements adjustElements(const Elements & elements, const String & current_database) - { - auto res = elements; - replaceElementTypesWithBaseElementTypes(res); - replaceEmptyDatabaseWithCurrentDatabase(res, current_database); - replaceTemporaryTablesWithTemporaryDatabase(res); - setNewNamesIfNotSet(res); - deduplicateAndReorderElements(res); - return res; - } - - void backupCreateQuery(const IAST & create_query, BackupEntries & backup_entries) - { - auto metadata_entry = std::make_unique(serializeAST(create_query)); - String metadata_path = getMetadataPathInBackup(create_query); - backup_entries.emplace_back(metadata_path, std::move(metadata_entry)); - } - - void backupTable( - const DatabaseAndTable & database_and_table, - const String & table_name, - const ASTs & partitions, - const ContextPtr & context, - const BackupRenamingConfigPtr & renaming_config, - BackupEntries & backup_entries) - { - const auto & database = database_and_table.first; - const auto & storage = database_and_table.second; - context->checkAccess(AccessType::SELECT, database->getDatabaseName(), table_name); - - auto create_query = database->getCreateTableQuery(table_name, context); - ASTPtr new_create_query = renameInCreateQuery(create_query, renaming_config, context); - backupCreateQuery(*new_create_query, backup_entries); - - auto data_backup = storage->backup(partitions, context); - if (!data_backup.empty()) - { - String data_path = getDataPathInBackup(*new_create_query); - for (auto & [path_in_backup, backup_entry] : data_backup) - backup_entries.emplace_back(data_path + path_in_backup, std::move(backup_entry)); - } - } - - void backupDatabase( - const DatabasePtr & database, - const std::set & except_list, - const ContextPtr & context, - const BackupRenamingConfigPtr & renaming_config, - BackupEntries & backup_entries) - { - context->checkAccess(AccessType::SHOW_TABLES, database->getDatabaseName()); - - auto create_query = database->getCreateDatabaseQuery(); - ASTPtr new_create_query = renameInCreateQuery(create_query, renaming_config, context); - backupCreateQuery(*new_create_query, backup_entries); - - for (auto it = database->getTablesIteratorForBackup(context); it->isValid(); it->next()) - { - if (except_list.contains(it->name())) - continue; - backupTable({database, it->table()}, it->name(), {}, context, renaming_config, backup_entries); - } - } - - void backupAllDatabases( - const std::set & except_list, - const ContextPtr & context, - const BackupRenamingConfigPtr & renaming_config, - BackupEntries & backup_entries) - { - for (const auto & [database_name, database] : DatabaseCatalog::instance().getDatabases()) - { - if (except_list.contains(database_name)) - continue; - if (database_name == DatabaseCatalog::SYSTEM_DATABASE || database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; - backupDatabase(database, {}, context, renaming_config, backup_entries); - } - } + ContextPtr context; + BackupMutablePtr backup; + BackupRenamingConfigPtr renaming_config; + std::map databases; + std::map tables; + }; } BackupEntries makeBackupEntries(const Elements & elements, const ContextPtr & context) { - BackupEntries backup_entries; - - auto elements2 = adjustElements(elements, context->getCurrentDatabase()); - auto renaming_config = std::make_shared(); - renaming_config->setFromBackupQueryElements(elements2); - - for (const auto & element : elements2) - { - switch (element.type) - { - case ElementType::TABLE: - { - const String & database_name = element.name.first; - const String & table_name = element.name.second; - auto [database, storage] = DatabaseCatalog::instance().getDatabaseAndTable({database_name, table_name}, context); - backupTable({database, storage}, table_name, element.partitions, context, renaming_config, backup_entries); - break; - } - - case ElementType::DATABASE: - { - const String & database_name = element.name.first; - auto database = DatabaseCatalog::instance().getDatabase(database_name, context); - backupDatabase(database, element.except_list, context, renaming_config, backup_entries); - break; - } - - case ElementType::ALL_DATABASES: - { - backupAllDatabases(element.except_list, context, renaming_config, backup_entries); - break; - } - - default: - throw Exception("Unexpected element type", ErrorCodes::LOGICAL_ERROR); /// other element types have been removed in deduplicateElements() - } - } - - /// A backup cannot be empty. - if (backup_entries.empty()) - throw Exception("Backup must not be empty", ErrorCodes::BACKUP_IS_EMPTY); - - /// Check that all backup entries are unique. - ::sort( - backup_entries.begin(), - backup_entries.end(), - [](const std::pair> & lhs, const std::pair> & rhs) - { - return lhs.first < rhs.first; - }); - auto adjacent = std::adjacent_find(backup_entries.begin(), backup_entries.end()); - if (adjacent != backup_entries.end()) - throw Exception("Cannot write multiple entries with the same name " + quoteString(adjacent->first), ErrorCodes::BACKUP_ELEMENT_DUPLICATE); - - return backup_entries; + BackupEntriesBuilder builder{context}; + builder.prepare(elements); + return builder.makeBackupEntries(); } -UInt64 estimateBackupSize(const BackupEntries & backup_entries, const BackupPtr & base_backup) -{ - UInt64 total_size = 0; - for (const auto & [name, entry] : backup_entries) - { - UInt64 data_size = entry->getSize(); - if (base_backup) - { - if (base_backup->fileExists(name) && (data_size == base_backup->getFileSize(name))) - { - auto checksum = entry->getChecksum(); - if (checksum && (*checksum == base_backup->getFileChecksum(name))) - continue; - } - } - total_size += data_size; - } - return total_size; -} void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads) { @@ -482,6 +386,7 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries backup->finalizeWriting(); } + String getDataPathInBackup(const DatabaseAndTableName & table_name) { if (table_name.first.empty() || table_name.second.empty()) diff --git a/src/Backups/BackupUtils.h b/src/Backups/BackupUtils.h index c2f56044d5d..51e849ae6f3 100644 --- a/src/Backups/BackupUtils.h +++ b/src/Backups/BackupUtils.h @@ -5,9 +5,7 @@ namespace DB { - class IBackup; -using BackupPtr = std::shared_ptr; using BackupMutablePtr = std::shared_ptr; class IBackupEntry; using BackupEntryPtr = std::unique_ptr; @@ -15,13 +13,9 @@ using BackupEntries = std::vector>; class Context; using ContextPtr = std::shared_ptr; - /// 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); - /// Write backup entries to an opened backup. void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads); diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 8bdae71b822..6af91bc8cf9 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -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) \ @@ -615,7 +615,8 @@ M(644, REMOTE_FS_OBJECT_CACHE_ERROR) \ M(645, NUMBER_OF_DIMENSIONS_MISMATHED) \ M(646, RBAC_VERSION_IS_TOO_NEW) \ - M(647, CANNOT_RESTORE_DATABASE) \ + M(647, CANNOT_BACKUP_DATABASE) \ + M(648, CANNOT_BACKUP_TABLE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ From 5a883c6b5a2cd4472d28bb379eb423e31abfc114 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 17:28:27 +0700 Subject: [PATCH 61/85] Fixed checking access rights while producing backup, add IStorage::hasHollowBackup(). --- src/Backups/BackupUtils.cpp | 49 ++++++++++++++++++-------- src/Backups/RestoreFromBackupUtils.cpp | 2 ++ src/Databases/IDatabase.h | 10 +++--- src/Storages/IStorage.h | 3 ++ 4 files changed, 44 insertions(+), 20 deletions(-) diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 820a7f86f7f..5d2b7362cb4 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -131,13 +131,15 @@ namespace for (auto & info : tables | boost::adaptors::map_values) { res.push_back(makeBackupEntryForMetadata(*info.create_query)); - - auto data_backup = info.storage->backup(info.partitions, context); - if (!data_backup.empty()) + if (info.has_data) { - String data_path = getDataPathInBackup(*info.create_query); - for (auto & [path_in_backup, backup_entry] : data_backup) - res.emplace_back(data_path + path_in_backup, std::move(backup_entry)); + auto data_backup = info.storage->backup(info.partitions, context); + if (!data_backup.empty()) + { + String data_path = getDataPathInBackup(*info.create_query); + for (auto & [path_in_backup, backup_entry] : data_backup) + res.emplace_back(data_path + path_in_backup, std::move(backup_entry)); + } } } @@ -158,7 +160,17 @@ namespace void prepareToBackupTable(const DatabaseAndTableName & table_name_, const DatabaseAndTable & table_, const ASTs & partitions_) { - context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); + context->checkAccess(AccessType::SHOW_TABLES, table_name_.first, table_name_.second); + + const auto & database = table_.first; + const auto & storage = table_.second; + + if (database->hasHollowBackup()) + throw Exception( + ErrorCodes::CANNOT_BACKUP_TABLE, + "Couldn't backup table {}.{} because of the database's engine {} is hollow", + backQuoteIfNeed(table_name_.first), backQuoteIfNeed(table_name_.second), + database->getEngineName()); /// Check that we are not trying to backup the same table again. DatabaseAndTableName new_table_name = renaming_config->getNewTableName(table_name_); @@ -173,13 +185,18 @@ namespace } /// Make a create query for this table. - auto create_query = renameInCreateQuery(table_.first->getCreateTableQuery(table_name_.second, context)); + auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); + + bool has_data = !storage->hasHollowBackup(); + if (has_data) + context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); CreateTableInfo info; info.create_query = create_query; - info.storage = table_.second; + info.storage = storage; info.name_in_backup = new_table_name; info.partitions = partitions_; + info.has_data = has_data; tables[new_table_name] = std::move(info); /// If it's not system or temporary database then probably we need to backup the database's definition too. @@ -188,7 +205,7 @@ namespace if (!databases.contains(new_table_name.first)) { /// Add a create query to backup the database if we haven't done it yet. - auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); + auto create_db_query = renameInCreateQuery(database->getCreateDatabaseQuery()); create_db_query->setDatabase(new_table_name.first); CreateDatabaseInfo info_db; @@ -243,11 +260,14 @@ namespace } /// Backup tables in this database. - for (auto it = database_->getTablesIteratorForBackup(context); it->isValid(); it->next()) + if (!database_->hasHollowBackup()) { - if (except_list_.contains(it->name())) - continue; - prepareToBackupTable({database_name_, it->name()}, {database_, it->table()}, {}); + for (auto it = database_->getTablesIterator(context); it->isValid(); it->next()) + { + if (except_list_.contains(it->name())) + continue; + prepareToBackupTable({database_name_, it->name()}, {database_, it->table()}, {}); + } } } @@ -289,6 +309,7 @@ namespace StoragePtr storage; DatabaseAndTableName name_in_backup; ASTs partitions; + bool has_data = false; }; /// Information which is used to make an instance of RestoreDatabaseFromBackupTask. diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreFromBackupUtils.cpp index af3cc926a7e..70543e50c58 100644 --- a/src/Backups/RestoreFromBackupUtils.cpp +++ b/src/Backups/RestoreFromBackupUtils.cpp @@ -165,6 +165,8 @@ namespace RestoreFromBackupTaskPtr insertDataIntoStorage(StoragePtr storage) { + if (storage->hasHollowBackup()) + return {}; context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); String data_path_in_backup = getDataPathInBackup(table_name_in_backup); return storage->restoreFromBackup(backup, data_path_in_backup, partitions, context); diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index f95653feb20..5d2a1289833 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -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 hasHollowBackup() const { return false; } + virtual ~IDatabase() = default; protected: diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index e77dd215ce9..3f08473680d 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -217,6 +217,9 @@ public: NameDependencies getDependentViewsByColumn(ContextPtr context) const; + /// Returns true if the backup is hollow, which means it doesn't contain any data. + virtual bool hasHollowBackup() const { return false; } + /// Prepares entries to backup data of the storage. virtual BackupEntries backup(const ASTs & partitions, ContextPtr context); From 54d554f55d415ac5566be0f25bd2a5635236d8b0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 18:45:22 +0700 Subject: [PATCH 62/85] Simplify elements of ASTBackupQuery. --- src/Backups/BackupRenamingConfig.cpp | 58 ++++---- src/Backups/BackupRenamingConfig.h | 8 +- src/Backups/BackupUtils.cpp | 53 +------- src/Backups/RestoreFromBackupUtils.cpp | 52 +------- src/Backups/renameInCreateQuery.cpp | 44 +++--- src/Parsers/ASTBackupQuery.cpp | 77 +++++++---- src/Parsers/ASTBackupQuery.h | 22 ++- src/Parsers/ParserBackupQuery.cpp | 177 +++++++++++++++++++------ src/Parsers/ParserBackupQuery.h | 14 +- 9 files changed, 274 insertions(+), 231 deletions(-) diff --git a/src/Backups/BackupRenamingConfig.cpp b/src/Backups/BackupRenamingConfig.cpp index ff510d82a32..e89cdfd51ea 100644 --- a/src/Backups/BackupRenamingConfig.cpp +++ b/src/Backups/BackupRenamingConfig.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB @@ -17,47 +18,52 @@ void BackupRenamingConfig::setNewDatabaseName(const String & old_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) +void BackupRenamingConfig::setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database) { - old_to_new_temporary_table_names[old_temporary_table_name] = new_temporary_table_name; + setFromBackupQueryElements(backup_query.elements, current_database); } -void BackupRenamingConfig::setFromBackupQuery(const ASTBackupQuery & backup_query) -{ - setFromBackupQueryElements(backup_query.elements); -} - -void BackupRenamingConfig::setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements) +void BackupRenamingConfig::setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database) { for (const auto & element : backup_query_elements) { switch (element.type) { - case ElementType::TABLE: [[fallthrough]]; - case ElementType::DICTIONARY: + case ElementType::TABLE: { - const auto & new_name = element.new_name.second.empty() ? element.name : element.new_name; - setNewTableName(element.name, new_name); + 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: { - const auto & new_name = element.new_name.first.empty() ? element.name.first : element.new_name.first; - setNewDatabaseName(element.name.first, new_name); - break; - } + String database_name = element.name.first; + if (element.name_is_in_temp_db) + database_name = DatabaseCatalog::TEMPORARY_DATABASE; - 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); + 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; - case ASTBackupQuery::ALL_TEMPORARY_TABLES: break; - case ASTBackupQuery::EVERYTHING: break; } } } @@ -78,12 +84,4 @@ const String & BackupRenamingConfig::getNewDatabaseName(const String & old_datab 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; -} - } diff --git a/src/Backups/BackupRenamingConfig.h b/src/Backups/BackupRenamingConfig.h index 740781c9c9f..17329dfaccd 100644 --- a/src/Backups/BackupRenamingConfig.h +++ b/src/Backups/BackupRenamingConfig.h @@ -19,19 +19,17 @@ public: 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); + + void setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database); + void setFromBackupQueryElements(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; - const String & getNewTemporaryTableName(const String & old_temporary_table_name) const; private: std::map old_to_new_table_names; std::unordered_map old_to_new_database_names; - std::unordered_map old_to_new_temporary_table_names; }; using BackupRenamingConfigPtr = std::shared_ptr; diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 5d2b7362cb4..800c6322f7f 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -28,23 +28,6 @@ namespace using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; - - /// Replaces an empty database with the current database. - void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database) - { - for (auto & element : elements) - { - if (element.type == ElementType::TABLE) - { - if (element.name.first.empty() && !element.name.second.empty()) - element.name.first = current_database; - if (element.new_name.first.empty() && !element.new_name.second.empty()) - element.new_name.first = current_database; - } - } - } - - /// Makes backup entries to backup databases and tables according to the elements of ASTBackupQuery. /// Keep this class consistent with RestoreTasksBuilder. class BackupEntriesBuilder @@ -55,30 +38,21 @@ namespace /// Prepares internal structures for making backup entries. void prepare(const ASTBackupQuery::Elements & elements) { - auto elements2 = elements; - replaceEmptyDatabaseWithCurrentDatabase(elements2, context->getCurrentDatabase()); - auto new_renaming_config = std::make_shared(); - new_renaming_config->setFromBackupQueryElements(elements2); + String current_database = context->getCurrentDatabase(); + new_renaming_config->setFromBackupQueryElements(elements, current_database); renaming_config = new_renaming_config; - for (const auto & element : elements2) + for (const auto & element : elements) { switch (element.type) { - case ElementType::TABLE: [[fallthrough]]; - case ElementType::DICTIONARY: + case ElementType::TABLE: { - const String & database_name = element.name.first; - const String & table_name = element.name.second; - prepareToBackupTable(DatabaseAndTableName{database_name, table_name}, element.partitions); - break; - } - - case ElementType::TEMPORARY_TABLE: - { - String database_name = DatabaseCatalog::TEMPORARY_DATABASE; const String & table_name = element.name.second; + String database_name = element.name.first; + if (database_name.empty()) + database_name = current_database; prepareToBackupTable(DatabaseAndTableName{database_name, table_name}, element.partitions); break; } @@ -90,24 +64,11 @@ namespace break; } - case ElementType::ALL_TEMPORARY_TABLES: - { - prepareToBackupDatabase(DatabaseCatalog::TEMPORARY_DATABASE, element.except_list); - break; - } - case ElementType::ALL_DATABASES: { prepareToBackupAllDatabases(element.except_list); break; } - - case ElementType::EVERYTHING: - { - prepareToBackupAllDatabases({}); - prepareToBackupDatabase(DatabaseCatalog::TEMPORARY_DATABASE, {}); - break; - } } } } diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreFromBackupUtils.cpp index 70543e50c58..2f4ed370a06 100644 --- a/src/Backups/RestoreFromBackupUtils.cpp +++ b/src/Backups/RestoreFromBackupUtils.cpp @@ -36,23 +36,6 @@ namespace using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; - - /// Replaces an empty database with the current database. - void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database) - { - for (auto & element : elements) - { - if (element.type == ElementType::TABLE) - { - if (element.name.first.empty() && !element.name.second.empty()) - element.name.first = current_database; - if (element.new_name.first.empty() && !element.new_name.second.empty()) - element.new_name.first = current_database; - } - } - } - - /// Restores a database (without tables inside), should be executed before executing /// RestoreTableFromBackupTask. class RestoreDatabaseFromBackupTask : public IRestoreFromBackupTask @@ -192,30 +175,21 @@ namespace /// Prepares internal structures for making tasks for restoring. void prepare(const ASTBackupQuery::Elements & elements) { - auto elements2 = elements; - replaceEmptyDatabaseWithCurrentDatabase(elements2, context->getCurrentDatabase()); - auto new_renaming_config = std::make_shared(); - new_renaming_config->setFromBackupQueryElements(elements2); + String current_database = context->getCurrentDatabase(); + new_renaming_config->setFromBackupQueryElements(elements, current_database); renaming_config = new_renaming_config; - for (const auto & element : elements2) + for (const auto & element : elements) { switch (element.type) { - case ElementType::TABLE: [[fallthrough]]; - case ElementType::DICTIONARY: + case ElementType::TABLE: { - const String & database_name = element.name.first; - const String & table_name = element.name.second; - prepareToRestoreTable(DatabaseAndTableName{database_name, table_name}, element.partitions); - break; - } - - case ElementType::TEMPORARY_TABLE: - { - String database_name = DatabaseCatalog::TEMPORARY_DATABASE; 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; } @@ -227,23 +201,11 @@ namespace break; } - case ElementType::ALL_TEMPORARY_TABLES: - { - prepareToRestoreDatabase(DatabaseCatalog::TEMPORARY_DATABASE, element.except_list); - break; - } - case ElementType::ALL_DATABASES: { prepareToRestoreAllDatabases(element.except_list); break; } - - case ElementType::EVERYTHING: - { - prepareToRestoreAllDatabases({}); - break; - } } } } diff --git a/src/Backups/renameInCreateQuery.cpp b/src/Backups/renameInCreateQuery.cpp index 5b2492b3be4..fe661868040 100644 --- a/src/Backups/renameInCreateQuery.cpp +++ b/src/Backups/renameInCreateQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -47,26 +48,39 @@ namespace /// 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) { - 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())); + 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_config->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.table) + else if (create.database) { - 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())); + String database_name = create.getDatabase(); + database_name = data.renaming_config->getNewDatabaseName(database_name); + create.setDatabase(database_name); } 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); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty"); create.uuid = UUIDHelpers::Nil; diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index adc6bb97985..f8fcbd98872 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -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 & 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 & elements, Kind kind, const IAST::FormatSettings & format) diff --git a/src/Parsers/ASTBackupQuery.h b/src/Parsers/ASTBackupQuery.h index 0042fca558f..648bcf27bce 100644 --- a/src/Parsers/ASTBackupQuery.h +++ b/src/Parsers/ASTBackupQuery.h @@ -11,22 +11,20 @@ using DatabaseAndTableName = std::pair; /** 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 except_list; }; diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 666600b58de..844a91fa515 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -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 & except_list) + { + if (!ParserKeyword{"EXCEPT"}.ignore(pos, expected)) + return false; + + std::set 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 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; }); } diff --git a/src/Parsers/ParserBackupQuery.h b/src/Parsers/ParserBackupQuery.h index e42326c2590..b01c149601c 100644 --- a/src/Parsers/ParserBackupQuery.h +++ b/src/Parsers/ParserBackupQuery.h @@ -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 From e8fc9cf476fb771a950d3954bd9f0916c0bb2518 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 20:05:13 +0700 Subject: [PATCH 63/85] Move renameInCreateQuery() to a better place. --- src/Backups/BackupRenamingConfig.cpp | 87 ---- src/Backups/BackupUtils.cpp | 15 +- src/Backups/DDLRenamingVisitor.cpp | 384 ++++++++++++++++++ ...pRenamingConfig.h => DDLRenamingVisitor.h} | 34 +- src/Backups/RestoreFromBackupUtils.cpp | 15 +- src/Backups/renameInCreateQuery.cpp | 296 -------------- src/Backups/renameInCreateQuery.h | 16 - src/Common/ErrorCodes.cpp | 1 + 8 files changed, 426 insertions(+), 422 deletions(-) delete mode 100644 src/Backups/BackupRenamingConfig.cpp create mode 100644 src/Backups/DDLRenamingVisitor.cpp rename src/Backups/{BackupRenamingConfig.h => DDLRenamingVisitor.h} (51%) delete mode 100644 src/Backups/renameInCreateQuery.cpp delete mode 100644 src/Backups/renameInCreateQuery.h diff --git a/src/Backups/BackupRenamingConfig.cpp b/src/Backups/BackupRenamingConfig.cpp deleted file mode 100644 index e89cdfd51ea..00000000000 --- a/src/Backups/BackupRenamingConfig.cpp +++ /dev/null @@ -1,87 +0,0 @@ -#include -#include -#include - - -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::setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database) -{ - setFromBackupQueryElements(backup_query.elements, current_database); -} - -void BackupRenamingConfig::setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database) -{ - 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 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; -} - -} diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 800c6322f7f..0ad5cf97393 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -1,8 +1,7 @@ #include #include -#include +#include #include -#include #include #include #include @@ -38,10 +37,8 @@ namespace /// Prepares internal structures for making backup entries. void prepare(const ASTBackupQuery::Elements & elements) { - auto new_renaming_config = std::make_shared(); String current_database = context->getCurrentDatabase(); - new_renaming_config->setFromBackupQueryElements(elements, current_database); - renaming_config = new_renaming_config; + renaming_settings.setFromBackupQuery(elements, current_database); for (const auto & element : elements) { @@ -134,7 +131,7 @@ namespace database->getEngineName()); /// Check that we are not trying to backup the same table again. - DatabaseAndTableName new_table_name = renaming_config->getNewTableName(table_name_); + DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_); if (tables.contains(new_table_name)) { String message; @@ -203,7 +200,7 @@ namespace context->checkAccess(AccessType::SHOW_DATABASES, database_name_); /// Check that we are not trying to restore the same database again. - String new_database_name = renaming_config->getNewDatabaseName(database_name_); + 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_BACKUP_DATABASE, "Couldn't backup database {} twice", backQuoteIfNeed(new_database_name)); @@ -248,7 +245,7 @@ namespace /// Do renaming in the create query according to the renaming config. std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const { - return typeid_cast>(::DB::renameInCreateQuery(ast, renaming_config, context)); + return typeid_cast>(::DB::renameInCreateQuery(ast, context, renaming_settings)); } static bool isSystemOrTemporaryDatabase(const String & database_name) @@ -292,7 +289,7 @@ namespace ContextPtr context; BackupMutablePtr backup; - BackupRenamingConfigPtr renaming_config; + DDLRenamingSettings renaming_settings; std::map databases; std::map tables; }; diff --git a/src/Backups/DDLRenamingVisitor.cpp b/src/Backups/DDLRenamingVisitor.cpp new file mode 100644 index 00000000000..9d87d18e142 --- /dev/null +++ b/src/Backups/DDLRenamingVisitor.cpp @@ -0,0 +1,384 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +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"); + + create.uuid = UUIDHelpers::Nil; + + 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(); + 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(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().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().value.safeGet(); + 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(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().children; + + const auto * second_arg_as_function = args[1]->as(); + 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().value.safeGet(); + + size_t table_name_index = static_cast(-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().value.safeGet(); + } + + 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(-1)) + { + if (new_db_name != db_name) + args[db_name_index] = std::make_shared(new_db_name); + if (new_table_name != table_name) + args[table_name_index] = std::make_shared(new_table_name); + } + else + { + args[db_name_index] = std::make_shared(new_db_name); + args.insert(args.begin() + db_name_index + 1, std::make_shared(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().children; + String db_name, table_name; + size_t db_name_index = static_cast(-1); + size_t table_name_index = static_cast(-1); + + for (size_t i = 0; i != elements.size(); ++i) + { + auto & pair = elements[i]->as(); + if (pair.first == "db") + { + if (db_name_index != static_cast(-1)) + return; + db_name = pair.second->as().value.safeGet(); + db_name_index = i; + } + else if (pair.first == "table") + { + if (table_name_index != static_cast(-1)) + return; + table_name = pair.second->as().value.safeGet(); + 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(); + pair.replace(pair.second, std::make_shared(new_db_name)); + } + if (new_table_name != table_name) + { + auto & pair = elements[table_name_index]->as(); + pair.replace(pair.second, std::make_shared(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()) + visitCreateQuery(*create, data); + else if (auto * expr = ast->as()) + visitTableExpression(*expr, data); + else if (auto * function = ast->as()) + visitFunction(*function, data); + else if (auto * dictionary = ast->as()) + visitDictionary(*dictionary, data); +} + +ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings) +{ + auto new_ast = ast->clone(); + try + { + DDLRenamingVisitor::Data data{renaming_settings, global_context}; + DDLRenamingVisitor::Visitor{data}.visit(new_ast); + return new_ast; + } + catch (...) + { + tryLogCurrentException("Backup", "Error while renaming in AST"); + return ast; + } +} + +} diff --git a/src/Backups/BackupRenamingConfig.h b/src/Backups/DDLRenamingVisitor.h similarity index 51% rename from src/Backups/BackupRenamingConfig.h rename to src/Backups/DDLRenamingVisitor.h index 17329dfaccd..e74f68a4033 100644 --- a/src/Backups/BackupRenamingConfig.h +++ b/src/Backups/DDLRenamingVisitor.h @@ -1,27 +1,33 @@ #pragma once -#include #include +#include +#include #include +#include #include namespace DB { using DatabaseAndTableName = std::pair; +class IAST; +using ASTPtr = std::shared_ptr; +class Context; +using ContextPtr = std::shared_ptr; /// 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 +class DDLRenamingSettings { public: - BackupRenamingConfig() = default; + 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 setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements, 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; @@ -32,6 +38,24 @@ private: std::unordered_map old_to_new_database_names; }; -using BackupRenamingConfigPtr = std::shared_ptr; + +/// Changes names in AST according to the renaming settings. +ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & config); + +/// 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; + + static bool needChildVisit(ASTPtr &, const ASTPtr &); + static void visit(ASTPtr & ast, const Data & data); +}; } diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreFromBackupUtils.cpp index 2f4ed370a06..121cd21f226 100644 --- a/src/Backups/RestoreFromBackupUtils.cpp +++ b/src/Backups/RestoreFromBackupUtils.cpp @@ -1,10 +1,9 @@ #include -#include +#include #include #include #include #include -#include #include #include #include @@ -175,10 +174,8 @@ namespace /// Prepares internal structures for making tasks for restoring. void prepare(const ASTBackupQuery::Elements & elements) { - auto new_renaming_config = std::make_shared(); String current_database = context->getCurrentDatabase(); - new_renaming_config->setFromBackupQueryElements(elements, current_database); - renaming_config = new_renaming_config; + renaming_settings.setFromBackupQuery(elements, current_database); for (const auto & element : elements) { @@ -238,7 +235,7 @@ namespace 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_config->getNewTableName(table_name_); + DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_); if (tables.contains(new_table_name)) { String message; @@ -310,7 +307,7 @@ namespace void prepareToRestoreDatabase(const String & database_name_, const std::set & except_list_) { /// Check that we are not trying to restore the same database again. - String new_database_name = renaming_config->getNewDatabaseName(database_name_); + 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, "Couldn't restore database {} twice", backQuoteIfNeed(new_database_name)); @@ -407,7 +404,7 @@ namespace /// Do renaming in the create query according to the renaming config. std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const { - return typeid_cast>(::DB::renameInCreateQuery(ast, renaming_config, context)); + return typeid_cast>(::DB::renameInCreateQuery(ast, context, renaming_settings)); } static bool isSystemOrTemporaryDatabase(const String & database_name) @@ -442,7 +439,7 @@ namespace ContextMutablePtr context; BackupPtr backup; - BackupRenamingConfigPtr renaming_config; + DDLRenamingSettings renaming_settings; std::map databases; std::map tables; }; diff --git a/src/Backups/renameInCreateQuery.cpp b/src/Backups/renameInCreateQuery.cpp deleted file mode 100644 index fe661868040..00000000000 --- a/src/Backups/renameInCreateQuery.cpp +++ /dev/null @@ -1,296 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -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()) - visitCreateQuery(*create, data); - else if (auto * expr = ast->as()) - visitTableExpression(*expr, data); - else if (auto * function = ast->as()) - visitFunction(*function, data); - else if (auto * dictionary = ast->as()) - 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.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_config->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_config->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"); - - 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(); - 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(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().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().value.safeGet(); - 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(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().children; - - const auto * second_arg_as_function = args[1]->as(); - 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().value.safeGet(); - - size_t table_name_index = static_cast(-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().value.safeGet(); - } - - 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(-1)) - { - if (new_db_name != db_name) - args[db_name_index] = std::make_shared(new_db_name); - if (new_table_name != table_name) - args[table_name_index] = std::make_shared(new_table_name); - } - else - { - args[db_name_index] = std::make_shared(new_db_name); - args.insert(args.begin() + db_name_index + 1, std::make_shared(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().children; - String db_name, table_name; - size_t db_name_index = static_cast(-1); - size_t table_name_index = static_cast(-1); - - for (size_t i = 0; i != elements.size(); ++i) - { - auto & pair = elements[i]->as(); - if (pair.first == "db") - { - if (db_name_index != static_cast(-1)) - return; - db_name = pair.second->as().value.safeGet(); - db_name_index = i; - } - else if (pair.first == "table") - { - if (table_name_index != static_cast(-1)) - return; - table_name = pair.second->as().value.safeGet(); - 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(); - pair.replace(pair.second, std::make_shared(new_db_name)); - } - if (new_table_name != table_name) - { - auto & pair = elements[table_name_index]->as(); - pair.replace(pair.second, std::make_shared(new_table_name)); - } - } - }; - - using RenameInCreateQueryTransformVisitor = InDepthNodeVisitor; -} - - -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; - } -} - -} diff --git a/src/Backups/renameInCreateQuery.h b/src/Backups/renameInCreateQuery.h deleted file mode 100644 index 9c62d07e5c6..00000000000 --- a/src/Backups/renameInCreateQuery.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#include - -namespace DB -{ -class IAST; -using ASTPtr = std::shared_ptr; -class Context; -using ContextPtr = std::shared_ptr; -class BackupRenamingConfig; -using BackupRenamingConfigPtr = std::shared_ptr; - -/// Changes names in AST according to the renaming settings. -ASTPtr renameInCreateQuery(const ASTPtr & ast, const BackupRenamingConfigPtr & renaming_config, const ContextPtr & context); -} diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 6af91bc8cf9..4626b9ffde3 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -617,6 +617,7 @@ M(646, RBAC_VERSION_IS_TOO_NEW) \ M(647, CANNOT_BACKUP_DATABASE) \ M(648, CANNOT_BACKUP_TABLE) \ + M(649, WRONG_DDL_RENAMING_SETTINGS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ From 258a4720011470ef67167bc40081cbc5471c6b4e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jan 2022 01:56:08 +0700 Subject: [PATCH 64/85] Shorter names: rename IRestoreFromBackupTask -> IRestoreTask. --- ...RestoreFromBackupTask.h => IRestoreTask.h} | 12 ++--- src/Backups/RestoreFromBackupUtils.h | 23 -------- ...reFromBackupUtils.cpp => RestoreUtils.cpp} | 53 ++++++++++--------- src/Backups/RestoreUtils.h | 23 ++++++++ src/Interpreters/InterpreterBackupQuery.cpp | 10 ++-- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 6 +-- src/Storages/MergeTree/MergeTreeData.cpp | 14 ++--- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageLog.cpp | 8 +-- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMergeTree.cpp | 4 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 8 +-- src/Storages/StorageStripeLog.h | 2 +- 15 files changed, 86 insertions(+), 85 deletions(-) rename src/Backups/{IRestoreFromBackupTask.h => IRestoreTask.h} (63%) delete mode 100644 src/Backups/RestoreFromBackupUtils.h rename src/Backups/{RestoreFromBackupUtils.cpp => RestoreUtils.cpp} (93%) create mode 100644 src/Backups/RestoreUtils.h diff --git a/src/Backups/IRestoreFromBackupTask.h b/src/Backups/IRestoreTask.h similarity index 63% rename from src/Backups/IRestoreFromBackupTask.h rename to src/Backups/IRestoreTask.h index 93e08b035ee..10046ff1807 100644 --- a/src/Backups/IRestoreFromBackupTask.h +++ b/src/Backups/IRestoreTask.h @@ -8,14 +8,14 @@ namespace DB { /// Represents a task of restoring something (database / table / table's part) from backup. -class IRestoreFromBackupTask +class IRestoreTask { public: - IRestoreFromBackupTask() = default; - virtual ~IRestoreFromBackupTask() = default; + 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> run() = 0; + virtual std::vector> run() = 0; /// Is it necessary to run this task sequentially? /// Sequential tasks are executed first and strictly in one thread. @@ -25,7 +25,7 @@ public: virtual void rollback() {} }; -using RestoreFromBackupTaskPtr = std::unique_ptr; -using RestoreFromBackupTasks = std::vector; +using RestoreTaskPtr = std::unique_ptr; +using RestoreTasks = std::vector; } diff --git a/src/Backups/RestoreFromBackupUtils.h b/src/Backups/RestoreFromBackupUtils.h deleted file mode 100644 index 75fe96495d6..00000000000 --- a/src/Backups/RestoreFromBackupUtils.h +++ /dev/null @@ -1,23 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class IBackup; -using BackupPtr = std::shared_ptr; -class IRestoreFromBackupTask; -using RestoreFromBackupTaskPtr = std::unique_ptr; -using RestoreFromBackupTasks = std::vector; -class Context; -using ContextMutablePtr = std::shared_ptr; - -/// Prepares restore tasks. -RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements); - -/// Executes restore tasks. -void executeRestoreTasks(RestoreFromBackupTasks && tasks, size_t num_threads); - -} diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreUtils.cpp similarity index 93% rename from src/Backups/RestoreFromBackupUtils.cpp rename to src/Backups/RestoreUtils.cpp index 121cd21f226..c2dac978caa 100644 --- a/src/Backups/RestoreFromBackupUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -1,8 +1,9 @@ +#include #include #include #include #include -#include +#include #include #include #include @@ -36,16 +37,16 @@ namespace using ElementType = ASTBackupQuery::ElementType; /// Restores a database (without tables inside), should be executed before executing - /// RestoreTableFromBackupTask. - class RestoreDatabaseFromBackupTask : public IRestoreFromBackupTask + /// RestoreTableTask. + class RestoreDatabaseTask : public IRestoreTask { public: - RestoreDatabaseFromBackupTask(ContextMutablePtr context_, const ASTPtr & create_query_) + RestoreDatabaseTask(ContextMutablePtr context_, const ASTPtr & create_query_) : context(context_), create_query(typeid_cast>(create_query_)) { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { createDatabase(); return {}; @@ -66,10 +67,10 @@ namespace /// Restores a table and fills it with data. - class RestoreTableFromBackupTask : public IRestoreFromBackupTask + class RestoreTableTask : public IRestoreTask { public: - RestoreTableFromBackupTask( + RestoreTableTask( ContextMutablePtr context_, const ASTPtr & create_query_, const ASTs & partitions_, @@ -83,11 +84,11 @@ namespace table_name.first = DatabaseCatalog::TEMPORARY_DATABASE; } - RestoreFromBackupTasks run() override + RestoreTasks run() override { createStorage(); auto storage = getStorage(); - RestoreFromBackupTasks tasks; + RestoreTasks tasks; if (auto task = insertDataIntoStorage(storage)) tasks.push_back(std::move(task)); return tasks; @@ -145,7 +146,7 @@ namespace throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); } - RestoreFromBackupTaskPtr insertDataIntoStorage(StoragePtr storage) + RestoreTaskPtr insertDataIntoStorage(StoragePtr storage) { if (storage->hasHollowBackup()) return {}; @@ -208,7 +209,7 @@ namespace } /// Makes tasks for restoring, should be called after prepare(). - RestoreFromBackupTasks makeTasks() const + RestoreTasks makeTasks() const { /// Check that there are not `different_create_query`. (If it's set it means error.) for (auto & info : databases | boost::adaptors::map_values) @@ -219,13 +220,13 @@ namespace serializeAST(*info.create_query), serializeAST(*info.different_create_query)); } - RestoreFromBackupTasks res; + RestoreTasks res; for (auto & info : databases | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query)); + res.push_back(std::make_unique(context, info.create_query)); /// TODO: We need to restore tables according to their dependencies. for (auto & info : tables | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup)); + res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup)); return res; } @@ -412,7 +413,7 @@ namespace return (database_name == DatabaseCatalog::SYSTEM_DATABASE) || (database_name == DatabaseCatalog::TEMPORARY_DATABASE); } - /// Information which is used to make an instance of RestoreTableFromBackupTask. + /// Information which is used to make an instance of RestoreTableTask. struct CreateTableInfo { ASTPtr create_query; @@ -420,7 +421,7 @@ namespace ASTs partitions; }; - /// Information which is used to make an instance of RestoreDatabaseFromBackupTask. + /// Information which is used to make an instance of RestoreDatabaseTask. struct CreateDatabaseInfo { ASTPtr create_query; @@ -446,7 +447,7 @@ namespace /// Reverts completed restore tasks (in reversed order). - void rollbackRestoreTasks(RestoreFromBackupTasks && restore_tasks) + void rollbackRestoreTasks(RestoreTasks && restore_tasks) { for (auto & restore_task : restore_tasks | boost::adaptors::reversed) { @@ -463,7 +464,7 @@ namespace } -RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements) +RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements) { RestoreTasksBuilder builder{context, backup}; builder.prepare(elements); @@ -471,12 +472,12 @@ RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupP } -void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_threads) +void executeRestoreTasks(RestoreTasks && restore_tasks, size_t num_threads) { if (!num_threads) num_threads = 1; - RestoreFromBackupTasks completed_tasks; + RestoreTasks completed_tasks; bool need_rollback_completed_tasks = true; SCOPE_EXIT({ @@ -484,8 +485,8 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr rollbackRestoreTasks(std::move(completed_tasks)); }); - std::deque> sequential_tasks; - std::deque> enqueued_tasks; + std::deque> sequential_tasks; + std::deque> enqueued_tasks; /// There are two kinds of restore tasks: sequential and non-sequential ones. /// Sequential tasks are executed first and always in one thread. @@ -503,7 +504,7 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr auto current_task = std::move(sequential_tasks.front()); sequential_tasks.pop_front(); - RestoreFromBackupTasks new_tasks = current_task->run(); + RestoreTasks new_tasks = current_task->run(); completed_tasks.push_back(std::move(current_task)); for (auto & task : new_tasks) @@ -516,7 +517,7 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr } /// Non-sequential tasks. - std::unordered_map> running_tasks; + std::unordered_map> running_tasks; std::vector threads; std::mutex mutex; std::condition_variable cond; @@ -524,7 +525,7 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr while (true) { - IRestoreFromBackupTask * current_task = nullptr; + IRestoreTask * current_task = nullptr; { std::unique_lock lock{mutex}; cond.wait(lock, [&] @@ -554,7 +555,7 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr return; } - RestoreFromBackupTasks new_tasks; + RestoreTasks new_tasks; std::exception_ptr new_exception; try { diff --git a/src/Backups/RestoreUtils.h b/src/Backups/RestoreUtils.h new file mode 100644 index 00000000000..df03dffa28c --- /dev/null +++ b/src/Backups/RestoreUtils.h @@ -0,0 +1,23 @@ +#pragma once + +#include + + +namespace DB +{ + +class IBackup; +using BackupPtr = std::shared_ptr; +class IRestoreTask; +using RestoreTaskPtr = std::unique_ptr; +using RestoreTasks = std::vector; +class Context; +using ContextMutablePtr = std::shared_ptr; + +/// Prepares restore tasks. +RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements); + +/// Executes restore tasks. +void executeRestoreTasks(RestoreTasks && tasks, size_t num_threads); + +} diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index de8a9c273cd..29cf77c7f52 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -1,13 +1,13 @@ #include +#include +#include +#include #include #include #include -#include -#include -#include -#include -#include +#include #include +#include namespace DB diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 4b174e9fd6d..43858dc080b 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -221,7 +221,7 @@ BackupEntries IStorage::backup(const ASTs &, ContextPtr) throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } -RestoreFromBackupTaskPtr IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) +RestoreTaskPtr IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) { throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 3f08473680d..367cb025724 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -72,8 +72,8 @@ class IBackup; using BackupPtr = std::shared_ptr; class IBackupEntry; using BackupEntries = std::vector>>; -class IRestoreFromBackupTask; -using RestoreFromBackupTaskPtr = std::unique_ptr; +class IRestoreTask; +using RestoreTaskPtr = std::unique_ptr; struct ColumnSize { @@ -224,7 +224,7 @@ public: virtual BackupEntries backup(const ASTs & partitions, ContextPtr context); /// Extract data from the backup and put it to the storage. - virtual RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); + virtual RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); /// Returns whether the column is virtual - by default all columns are real. /// Initially reserved virtual column name may be shadowed by real column. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6ca1f10b8f2..5a1ceca2401 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -3682,7 +3682,7 @@ BackupEntries MergeTreeData::backupDataParts(const DataPartsVector & data_parts) } -class MergeTreeDataRestoreTask : public IRestoreFromBackupTask +class MergeTreeDataRestoreTask : public IRestoreTask { public: MergeTreeDataRestoreTask( @@ -3699,9 +3699,9 @@ public: { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { - RestoreFromBackupTasks restore_part_tasks; + RestoreTasks restore_part_tasks; Strings part_names = backup->listFiles(data_path_in_backup); for (const String & part_name : part_names) { @@ -3725,7 +3725,7 @@ private: std::unordered_set partition_ids; SimpleIncrement * increment; - class RestorePartTask : public IRestoreFromBackupTask + class RestorePartTask : public IRestoreTask { public: RestorePartTask( @@ -3744,7 +3744,7 @@ private: { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { UInt64 total_size_of_part = 0; Strings filenames = backup->listFiles(data_path_in_backup + part_name + "/", ""); @@ -3789,7 +3789,7 @@ private: }; -RestoreFromBackupTaskPtr MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup, +RestoreTaskPtr MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const std::unordered_set & partition_ids, SimpleIncrement * increment) { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 9b3df89ac78..f3ae0e32f17 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -679,7 +679,7 @@ public: static BackupEntries backupDataParts(const DataPartsVector & data_parts); /// Extract data from the backup and put it to the storage. - RestoreFromBackupTaskPtr restoreDataPartsFromBackup( + RestoreTaskPtr restoreDataPartsFromBackup( const BackupPtr & backup, const String & data_path_in_backup, const std::unordered_set & partition_ids, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 6ac73eb01ee..bacf1fe49bd 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -28,7 +28,7 @@ #include #include #include -#include +#include #include #include @@ -953,7 +953,7 @@ BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context) return backup_entries; } -class LogRestoreTask : public IRestoreFromBackupTask +class LogRestoreTask : public IRestoreTask { using WriteLock = StorageLog::WriteLock; using Mark = StorageLog::Mark; @@ -965,7 +965,7 @@ public: { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { auto lock_timeout = getLockTimeout(context); WriteLock lock{storage->rwlock, lock_timeout}; @@ -1063,7 +1063,7 @@ private: ContextMutablePtr context; }; -RestoreFromBackupTaskPtr StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +RestoreTaskPtr StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 3c337bbd4fa..f974d983ff2 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -53,7 +53,7 @@ public: ColumnSizeByName getColumnSizes() const override; BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; protected: /** Attach the table with the appropriate name, along the appropriate path (with / at the end), diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 33893aa8a2d..71820d7e0a7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include #include @@ -1644,7 +1644,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ } -RestoreFromBackupTaskPtr StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) +RestoreTaskPtr StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) { return restoreDataPartsFromBackup(backup, data_path_in_backup, getPartitionIDsFromQuery(partitions, local_context), &increment); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 9a089d0074a..19977018695 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -97,7 +97,7 @@ public: CheckResults checkData(const ASTPtr & query, ContextPtr context) override; - RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index b4ad3cfc7c5..35b835122a6 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -38,7 +38,7 @@ #include #include #include -#include +#include #include #include @@ -553,7 +553,7 @@ BackupEntries StorageStripeLog::backup(const ASTs & partitions, ContextPtr conte return backup_entries; } -class StripeLogRestoreTask : public IRestoreFromBackupTask +class StripeLogRestoreTask : public IRestoreTask { using WriteLock = StorageStripeLog::WriteLock; @@ -567,7 +567,7 @@ public: { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { WriteLock lock{storage->rwlock, getLockTimeout(context)}; if (!lock) @@ -637,7 +637,7 @@ private: }; -RestoreFromBackupTaskPtr StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +RestoreTaskPtr StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 8fbd94f9492..cef03ef05a7 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -52,7 +52,7 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override; BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; protected: StorageStripeLog( From 7f89b98308ff172192388945e6704d8ed4f3f754 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jan 2022 05:13:16 +0700 Subject: [PATCH 65/85] Rework BackupSettings and RestoreSettings a little, pass StorageRestoreSettings to storages. --- src/Backups/BackupSettings.cpp | 16 +++++++- src/Backups/BackupSettings.h | 16 ++++---- src/Backups/BackupUtils.cpp | 9 ++--- src/Backups/BackupUtils.h | 3 +- src/Backups/RestoreSettings.cpp | 20 +++++++++ src/Backups/RestoreSettings.h | 22 ++++++++++ src/Backups/RestoreUtils.cpp | 25 +++++++----- src/Backups/RestoreUtils.h | 3 +- src/Interpreters/InterpreterBackupQuery.cpp | 45 +++++++++------------ src/Storages/IStorage.cpp | 4 +- src/Storages/IStorage.h | 5 ++- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++-- src/Storages/MergeTree/MergeTreeData.h | 4 +- src/Storages/StorageLog.cpp | 4 +- src/Storages/StorageLog.h | 4 +- src/Storages/StorageMergeTree.cpp | 4 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 4 +- src/Storages/StorageStripeLog.h | 4 +- 19 files changed, 130 insertions(+), 72 deletions(-) create mode 100644 src/Backups/RestoreSettings.cpp create mode 100644 src/Backups/RestoreSettings.h diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index f383330f246..a73b6974631 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -1,6 +1,20 @@ #include +#include +#include +#include + namespace DB { -IMPLEMENT_SETTINGS_TRAITS(BackupSettingsTraits, LIST_OF_BACKUP_SETTINGS) + +BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) +{ + BackupSettings res; + + if (query.base_backup_name) + res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + + return res; +} + } diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 05fbda2598e..81d74d2813b 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -1,16 +1,18 @@ #pragma once -#include +#include namespace DB { +class ASTBackupQuery; +struct BackupInfo; -#define LIST_OF_BACKUP_SETTINGS(M) \ - M(Bool, dummy, false, "", 0) \ - -DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(BackupSettingsTraits, LIST_OF_BACKUP_SETTINGS) - -struct BackupSettings : public BaseSettings {}; +/// Settings specified in the "SETTINGS" clause of a BACKUP query. +struct BackupSettings +{ + std::shared_ptr base_backup_info; + static BackupSettings fromBackupQuery(const ASTBackupQuery & query); +}; } diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 0ad5cf97393..5fff6eb3d37 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -32,7 +32,7 @@ namespace class BackupEntriesBuilder { public: - BackupEntriesBuilder(ContextPtr context_) : context(context_) { } + BackupEntriesBuilder(const ContextPtr & context_, const BackupSettings &) : context(context_) { } /// Prepares internal structures for making backup entries. void prepare(const ASTBackupQuery::Elements & elements) @@ -91,7 +91,7 @@ namespace res.push_back(makeBackupEntryForMetadata(*info.create_query)); if (info.has_data) { - auto data_backup = info.storage->backup(info.partitions, context); + auto data_backup = info.storage->backup(context, info.partitions); if (!data_backup.empty()) { String data_path = getDataPathInBackup(*info.create_query); @@ -288,7 +288,6 @@ namespace }; ContextPtr context; - BackupMutablePtr backup; DDLRenamingSettings renaming_settings; std::map databases; std::map tables; @@ -296,9 +295,9 @@ namespace } -BackupEntries makeBackupEntries(const Elements & elements, const ContextPtr & context) +BackupEntries makeBackupEntries(const ContextPtr & context, const Elements & elements, const BackupSettings & backup_settings) { - BackupEntriesBuilder builder{context}; + BackupEntriesBuilder builder{context, backup_settings}; builder.prepare(elements); return builder.makeBackupEntries(); } diff --git a/src/Backups/BackupUtils.h b/src/Backups/BackupUtils.h index 51e849ae6f3..d001d5a4bec 100644 --- a/src/Backups/BackupUtils.h +++ b/src/Backups/BackupUtils.h @@ -10,11 +10,12 @@ using BackupMutablePtr = std::shared_ptr; class IBackupEntry; using BackupEntryPtr = std::unique_ptr; using BackupEntries = std::vector>; +struct BackupSettings; class Context; using ContextPtr = std::shared_ptr; /// Prepares backup entries. -BackupEntries makeBackupEntries(const ASTBackupQuery::Elements & elements, const ContextPtr & context); +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); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp new file mode 100644 index 00000000000..2aede5faf88 --- /dev/null +++ b/src/Backups/RestoreSettings.cpp @@ -0,0 +1,20 @@ +#include +#include +#include +#include + + +namespace DB +{ + +RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) +{ + RestoreSettings res; + + if (query.base_backup_name) + res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + + return res; +} + +} diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h new file mode 100644 index 00000000000..9184a741066 --- /dev/null +++ b/src/Backups/RestoreSettings.h @@ -0,0 +1,22 @@ +#pragma once + +#include + + +namespace DB +{ +class ASTBackupQuery; +struct BackupInfo; + +struct StorageRestoreSettings +{ +}; + +/// Settings specified in the "SETTINGS" clause of a RESTORE query. +struct RestoreSettings : public StorageRestoreSettings +{ + std::shared_ptr base_backup_info; + static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); +}; + +} diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index c2dac978caa..f788742522b 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -41,7 +42,7 @@ namespace class RestoreDatabaseTask : public IRestoreTask { public: - RestoreDatabaseTask(ContextMutablePtr context_, const ASTPtr & create_query_) + RestoreDatabaseTask(ContextMutablePtr context_, const ASTPtr & create_query_, const RestoreSettings &) : context(context_), create_query(typeid_cast>(create_query_)) { } @@ -75,9 +76,11 @@ namespace const ASTPtr & create_query_, const ASTs & partitions_, const BackupPtr & backup_, - const DatabaseAndTableName & table_name_in_backup_) + const DatabaseAndTableName & table_name_in_backup_, + const RestoreSettings & restore_settings_) : context(context_), create_query(typeid_cast>(create_query_)), - partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_) + 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) @@ -152,7 +155,7 @@ namespace return {}; context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); String data_path_in_backup = getDataPathInBackup(table_name_in_backup); - return storage->restoreFromBackup(backup, data_path_in_backup, partitions, context); + return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, restore_settings); } ContextMutablePtr context; @@ -161,6 +164,7 @@ namespace ASTs partitions; BackupPtr backup; DatabaseAndTableName table_name_in_backup; + RestoreSettings restore_settings; }; @@ -169,8 +173,8 @@ namespace class RestoreTasksBuilder { public: - RestoreTasksBuilder(ContextMutablePtr context_, const BackupPtr & backup_) - : context(context_), backup(backup_) {} + 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) @@ -222,11 +226,11 @@ namespace RestoreTasks res; for (auto & info : databases | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query)); + res.push_back(std::make_unique(context, info.create_query, restore_settings)); /// TODO: We need to restore tables according to their dependencies. for (auto & info : tables | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup)); + res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings)); return res; } @@ -440,6 +444,7 @@ namespace ContextMutablePtr context; BackupPtr backup; + RestoreSettings restore_settings; DDLRenamingSettings renaming_settings; std::map databases; std::map tables; @@ -464,9 +469,9 @@ namespace } -RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements) +RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements, const RestoreSettings & restore_settings) { - RestoreTasksBuilder builder{context, backup}; + RestoreTasksBuilder builder{context, backup, restore_settings}; builder.prepare(elements); return builder.makeTasks(); } diff --git a/src/Backups/RestoreUtils.h b/src/Backups/RestoreUtils.h index df03dffa28c..33d2f7ff527 100644 --- a/src/Backups/RestoreUtils.h +++ b/src/Backups/RestoreUtils.h @@ -11,11 +11,12 @@ using BackupPtr = std::shared_ptr; class IRestoreTask; using RestoreTaskPtr = std::unique_ptr; using RestoreTasks = std::vector; +struct RestoreSettings; class Context; using ContextMutablePtr = std::shared_ptr; /// Prepares restore tasks. -RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements); +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); diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index 29cf77c7f52..ac930e1f6a7 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -5,49 +5,42 @@ #include #include #include +#include #include #include -#include namespace DB { namespace { - BackupMutablePtr createBackup(const ASTBackupQuery & query, const ContextPtr & context) + BackupMutablePtr createBackup( + const BackupInfo & backup_info, + IBackup::OpenMode open_mode, + const BackupInfo * base_backup_info, + const ContextPtr & context) { BackupFactory::CreateParams params; - params.open_mode = (query.kind == ASTBackupQuery::BACKUP) ? IBackup::OpenMode::WRITE : IBackup::OpenMode::READ; + params.open_mode = open_mode; 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 = base_backup_info ? *base_backup_info : std::optional{}; return BackupFactory::instance().createBackup(params); } -#if 0 - void getBackupSettings(const ASTBackupQuery & query, BackupSettings & settings, std::optional & base_backup) + void executeBackup(const ContextPtr & context, const ASTBackupQuery & query) { - settings = {}; - if (query.settings) - settings.applyChanges(query.settings->as().changes); - return settings; - } -#endif - - void executeBackup(const ASTBackupQuery & query, const ContextPtr & context) - { - 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), IBackup::OpenMode::WRITE, backup_settings.base_backup_info.get(), 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(context, backup, query.elements); + auto restore_settings = RestoreSettings::fromRestoreQuery(query); + BackupPtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), IBackup::OpenMode::READ, restore_settings.base_backup_info.get(), context); + auto restore_tasks = makeRestoreTasks(context, backup, query.elements, restore_settings); executeRestoreTasks(std::move(restore_tasks), context->getSettingsRef().max_backup_threads); } } @@ -56,9 +49,9 @@ BlockIO InterpreterBackupQuery::execute() { const auto & query = query_ptr->as(); if (query.kind == ASTBackupQuery::BACKUP) - executeBackup(query, context); + executeBackup(context, query); else if (query.kind == ASTBackupQuery::RESTORE) - executeRestore(query, context); + executeRestore(context, query); return {}; } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 43858dc080b..ab27f263b8e 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -216,12 +216,12 @@ bool IStorage::isStaticStorage() const return false; } -BackupEntries IStorage::backup(const ASTs &, ContextPtr) +BackupEntries IStorage::backup(ContextPtr, const ASTs &) { throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } -RestoreTaskPtr IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) +RestoreTaskPtr IStorage::restoreFromBackup(ContextMutablePtr, const ASTs &, const BackupPtr &, const String &, const StorageRestoreSettings &) { throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 367cb025724..2bfae021da1 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -74,6 +74,7 @@ class IBackupEntry; using BackupEntries = std::vector>>; class IRestoreTask; using RestoreTaskPtr = std::unique_ptr; +struct StorageRestoreSettings; struct ColumnSize { @@ -221,10 +222,10 @@ public: virtual bool hasHollowBackup() const { return false; } /// Prepares entries to backup data of the storage. - virtual BackupEntries backup(const ASTs & partitions, ContextPtr context); + virtual BackupEntries backup(ContextPtr context, const ASTs & partitions); /// Extract data from the backup and put it to the storage. - virtual RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); + virtual RestoreTaskPtr restoreFromBackup(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. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5a1ceca2401..8554426b073 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3630,7 +3630,7 @@ Pipe MergeTreeData::alterPartition( } -BackupEntries MergeTreeData::backup(const ASTs & partitions, ContextPtr local_context) +BackupEntries MergeTreeData::backup(ContextPtr local_context, const ASTs & partitions) { DataPartsVector data_parts; if (partitions.empty()) @@ -3789,9 +3789,9 @@ private: }; -RestoreTaskPtr MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup, - const std::unordered_set & partition_ids, - SimpleIncrement * increment) +RestoreTaskPtr MergeTreeData::restoreDataPartsFromBackup(const std::unordered_set & partition_ids, + const BackupPtr & backup, const String & data_path_in_backup, + SimpleIncrement * increment) { return std::make_unique( std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup, partition_ids, increment); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index f3ae0e32f17..fd9427d04fd 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -675,14 +675,14 @@ public: TableLockHolder & table_lock_holder); /// Prepares entries to backup data of the storage. - BackupEntries backup(const ASTs & partitions, ContextPtr context) override; + BackupEntries backup(ContextPtr context, const ASTs & partitions) override; static BackupEntries backupDataParts(const DataPartsVector & data_parts); /// Extract data from the backup and put it to the storage. RestoreTaskPtr restoreDataPartsFromBackup( + const std::unordered_set & partition_ids, const BackupPtr & backup, const String & data_path_in_backup, - const std::unordered_set & partition_ids, SimpleIncrement * increment); /// Moves partition to specified Disk diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index bacf1fe49bd..7cf453f0e43 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -888,7 +888,7 @@ IStorage::ColumnSizeByName StorageLog::getColumnSizes() const } -BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context) +BackupEntries StorageLog::backup(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); @@ -1063,7 +1063,7 @@ private: ContextMutablePtr context; }; -RestoreTaskPtr StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +RestoreTaskPtr StorageLog::restoreFromBackup(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()); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index f974d983ff2..6767d40583c 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -52,8 +52,8 @@ public: bool supportsSubcolumns() const override { return true; } ColumnSizeByName getColumnSizes() const override; - BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + BackupEntries backup(ContextPtr context, const ASTs & partitions) override; + RestoreTaskPtr restoreFromBackup(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), diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 71820d7e0a7..af4fa1d9b4e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1644,9 +1644,9 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ } -RestoreTaskPtr StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) +RestoreTaskPtr StorageMergeTree::restoreFromBackup(ContextMutablePtr local_context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) { - return restoreDataPartsFromBackup(backup, data_path_in_backup, getPartitionIDsFromQuery(partitions, local_context), &increment); + return restoreDataPartsFromBackup(getPartitionIDsFromQuery(partitions, local_context), backup, data_path_in_backup, &increment); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 19977018695..6acd19f73eb 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -97,7 +97,7 @@ public: CheckResults checkData(const ASTPtr & query, ContextPtr context) override; - RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 35b835122a6..374c861ae4c 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -491,7 +491,7 @@ void StorageStripeLog::saveFileSizes(const WriteLock & /* already locked for wri } -BackupEntries StorageStripeLog::backup(const ASTs & partitions, ContextPtr context) +BackupEntries StorageStripeLog::backup(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); @@ -637,7 +637,7 @@ private: }; -RestoreTaskPtr StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +RestoreTaskPtr StorageStripeLog::restoreFromBackup(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()); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index cef03ef05a7..a5d3b9a3da3 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -51,8 +51,8 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override; - BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + BackupEntries backup(ContextPtr context, const ASTs & partitions) override; + RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; protected: StorageStripeLog( From 6849f51c3b630b0af5c55a514a4d59e28cca5dce Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jan 2022 17:02:50 +0700 Subject: [PATCH 66/85] Add setting "structure_only." --- src/Backups/BackupSettings.cpp | 16 ++++++++++++++++ src/Backups/BackupSettings.h | 6 ++++++ src/Backups/BackupUtils.cpp | 9 +++++++-- src/Backups/RestoreSettings.cpp | 16 ++++++++++++++++ src/Backups/RestoreSettings.h | 8 ++++++++ src/Backups/RestoreUtils.cpp | 2 +- 6 files changed, 54 insertions(+), 3 deletions(-) diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index a73b6974631..602cf67cc2b 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -6,6 +6,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) { @@ -14,6 +18,18 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) if (query.base_backup_name) res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + if (query.settings) + { + const auto & settings = query.settings->as().changes; + for (const auto & setting : settings) + { + if (setting.name == "structure_only") + res.structure_only = setting.value.safeGet(); + else + throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); + } + } + return res; } diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 81d74d2813b..ce911382c78 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -11,7 +11,13 @@ struct BackupInfo; /// 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::shared_ptr base_backup_info; + + /// 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); }; diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 5fff6eb3d37..19287019a3e 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -32,7 +33,10 @@ namespace class BackupEntriesBuilder { public: - BackupEntriesBuilder(const ContextPtr & context_, const BackupSettings &) : context(context_) { } + BackupEntriesBuilder(const ContextPtr & context_, const BackupSettings & backup_settings_) + : context(context_), backup_settings(backup_settings_) + { + } /// Prepares internal structures for making backup entries. void prepare(const ASTBackupQuery::Elements & elements) @@ -145,7 +149,7 @@ namespace /// Make a create query for this table. auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); - bool has_data = !storage->hasHollowBackup(); + bool has_data = !storage->hasHollowBackup() && !backup_settings.structure_only; if (has_data) context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); @@ -288,6 +292,7 @@ namespace }; ContextPtr context; + BackupSettings backup_settings; DDLRenamingSettings renaming_settings; std::map databases; std::map tables; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 2aede5faf88..f7ff0a5ac05 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -6,6 +6,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) { @@ -14,6 +18,18 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) if (query.base_backup_name) res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + if (query.settings) + { + const auto & settings = query.settings->as().changes; + for (const auto & setting : settings) + { + if (setting.name == "structure_only") + res.structure_only = setting.value.safeGet(); + else + throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); + } + } + return res; } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 9184a741066..41d323d165d 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -15,7 +15,15 @@ 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::shared_ptr base_backup_info; + + /// If this is set to true then only create queries will be read from backup, + /// without the data of tables. + bool structure_only = false; + static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); }; diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index f788742522b..f7bac660944 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -151,7 +151,7 @@ namespace RestoreTaskPtr insertDataIntoStorage(StoragePtr storage) { - if (storage->hasHollowBackup()) + if (storage->hasHollowBackup() || restore_settings.structure_only) return {}; context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); String data_path_in_backup = getDataPathInBackup(table_name_in_backup); From fd1d24f62438c61c5f5b3052ccc90cc65cbb4cee Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jan 2022 20:10:54 +0700 Subject: [PATCH 67/85] Add settings "throw_if_database_exists" and "throw_if_table_exists". --- src/Backups/RestoreSettings.cpp | 4 ++++ src/Backups/RestoreSettings.h | 6 ++++++ src/Backups/RestoreUtils.cpp | 4 ++-- tests/integration/test_backup_restore_new/test.py | 4 ++-- 4 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index f7ff0a5ac05..f2752244601 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -25,6 +25,10 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) { if (setting.name == "structure_only") res.structure_only = setting.value.safeGet(); + else if (setting.name == "throw_if_database_exists") + res.throw_if_database_exists = setting.value.safeGet(); + else if (setting.name == "throw_if_table_exists") + res.throw_if_table_exists = setting.value.safeGet(); else throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 41d323d165d..8981ad0fac0 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -24,6 +24,12 @@ struct RestoreSettings : public StorageRestoreSettings /// without the data of tables. bool structure_only = false; + /// Whether RESTORE DATABASE will throw an exception if a destination database already exists. + bool throw_if_database_exists = true; + + /// Whether RESTORE TABLE will throw an exception if a destination table already exists. + bool throw_if_table_exists = true; + static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); }; diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index f7bac660944..5378666187c 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -253,7 +253,7 @@ namespace /// Make a create query for this table. auto create_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_)); - create_query->if_not_exists = true; + create_query->if_not_exists = !restore_settings.throw_if_table_exists; CreateTableInfo info; info.create_query = create_query; @@ -339,7 +339,7 @@ namespace db_name_in_backup.clear(); } - create_db_query->if_not_exists = true; + create_db_query->if_not_exists = !restore_settings.throw_if_database_exists; CreateDatabaseInfo info_db; info_db.create_query = create_db_query; diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index f9bfababadc..0d51a284e49 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -60,10 +60,10 @@ def test_restore_table_into_existing_table(engine): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") - instance.query(f"RESTORE TABLE test.table INTO test.table FROM {backup_name}") + instance.query(f"RESTORE TABLE test.table INTO test.table FROM {backup_name} SETTINGS throw_if_table_exists=0") assert instance.query("SELECT count(), sum(x) FROM test.table") == "200\t9900\n" - instance.query(f"RESTORE TABLE test.table INTO test.table FROM {backup_name}") + instance.query(f"RESTORE TABLE test.table INTO test.table FROM {backup_name} SETTINGS throw_if_table_exists=0") assert instance.query("SELECT count(), sum(x) FROM test.table") == "300\t14850\n" From ef57a87394b922505e03700d0c83f4509788ddab Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 23 Jan 2022 20:35:11 +0700 Subject: [PATCH 68/85] Add settings "throw_if_table_has_different_definition" and "throw_if_database_has_different_definition". --- src/Backups/BackupUtils.cpp | 24 +- src/Backups/DDLCompareUtils.cpp | 83 +++++++ src/Backups/DDLCompareUtils.h | 17 ++ src/Backups/RestoreSettings.cpp | 4 + src/Backups/RestoreSettings.h | 8 + src/Backups/RestoreUtils.cpp | 205 ++++++++++++------ .../formatTableNameOrTemporaryTableName.cpp | 17 ++ .../formatTableNameOrTemporaryTableName.h | 13 ++ .../hasCompatibleDataToRestoreTable.cpp | 26 --- src/Backups/hasCompatibleDataToRestoreTable.h | 11 - 10 files changed, 293 insertions(+), 115 deletions(-) create mode 100644 src/Backups/DDLCompareUtils.cpp create mode 100644 src/Backups/DDLCompareUtils.h create mode 100644 src/Backups/formatTableNameOrTemporaryTableName.cpp create mode 100644 src/Backups/formatTableNameOrTemporaryTableName.h delete mode 100644 src/Backups/hasCompatibleDataToRestoreTable.cpp delete mode 100644 src/Backups/hasCompatibleDataToRestoreTable.h diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 19287019a3e..5e40e1dbb69 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -1,8 +1,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -82,7 +84,7 @@ namespace { if (info.different_create_query) throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, - "Couldn't backup a database because two different create queries were generated for it: {} and {}", + "Cannot backup a database because two different create queries were generated for it: {} and {}", serializeAST(*info.create_query), serializeAST(*info.different_create_query)); } @@ -130,28 +132,24 @@ namespace if (database->hasHollowBackup()) throw Exception( ErrorCodes::CANNOT_BACKUP_TABLE, - "Couldn't backup table {}.{} because of the database's engine {} is hollow", - backQuoteIfNeed(table_name_.first), backQuoteIfNeed(table_name_.second), + "Cannot backup the {} because it's contained in a hollow database (engine: {})", + formatTableNameOrTemporaryTableName(table_name_), database->getEngineName()); /// Check that we are not trying to backup the same table again. DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_); if (tables.contains(new_table_name)) - { - String message; - if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - message = fmt::format("Couldn't backup temporary table {} twice", backQuoteIfNeed(new_table_name.second)); - else - message = fmt::format("Couldn't backup table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second)); - throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, message); - } + throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} twice", formatTableNameOrTemporaryTableName(new_table_name)); /// Make a create query for this table. auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); bool has_data = !storage->hasHollowBackup() && !backup_settings.structure_only; if (has_data) + { + /// We check for SELECT privilege only if we're going to read data from the table. context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); + } CreateTableInfo info; info.create_query = create_query; @@ -185,7 +183,7 @@ namespace { auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); create_db_query->setDatabase(new_table_name.first); - if (serializeAST(*info_db.create_query) != serializeAST(*create_db_query)) + if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query)) info_db.different_create_query = create_db_query; } } @@ -206,7 +204,7 @@ namespace /// 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_BACKUP_DATABASE, "Couldn't backup database {} twice", backQuoteIfNeed(new_database_name)); + throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Cannot backup the database {} twice", backQuoteIfNeed(new_database_name)); /// Of course we're not going to backup the definition of the system or the temporary database. if (!isSystemOrTemporaryDatabase(database_name_)) diff --git a/src/Backups/DDLCompareUtils.cpp b/src/Backups/DDLCompareUtils.cpp new file mode 100644 index 00000000000..7c857cff7d5 --- /dev/null +++ b/src/Backups/DDLCompareUtils.cpp @@ -0,0 +1,83 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + std::shared_ptr prepareDDLToCompare(const ASTCreateQuery & ast) + { + auto res = typeid_cast>(ast.shared_from_this()); + + std::shared_ptr clone; + auto get_clone = [&] + { + if (!clone) + { + clone = typeid_cast>(res->clone()); + res = clone; + } + return clone; + }; + + /// Remove UUID. + if (res->uuid != UUIDHelpers::Nil) + get_clone()->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>(table1.shared_from_this()); + if (!ast1 || !ast1->table) + return false; + + auto ast2 = typeid_cast>(table2.shared_from_this()); + if (!ast2 || !ast2->table) + 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 areDatabaseDefinitionsSame(const IAST & database1, const IAST & database2) +{ + auto ast1 = typeid_cast>(database1.shared_from_this()); + if (!ast1 || ast1->table || !ast1->database) + return false; + + auto ast2 = typeid_cast>(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); +} + +} diff --git a/src/Backups/DDLCompareUtils.h b/src/Backups/DDLCompareUtils.h new file mode 100644 index 00000000000..acb99c243ea --- /dev/null +++ b/src/Backups/DDLCompareUtils.h @@ -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); + +} diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index f2752244601..6d96c6f22eb 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -29,6 +29,10 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) res.throw_if_database_exists = setting.value.safeGet(); else if (setting.name == "throw_if_table_exists") res.throw_if_table_exists = setting.value.safeGet(); + else if (setting.name == "throw_if_database_has_different_definition") + res.throw_if_database_has_different_definition = setting.value.safeGet(); + else if (setting.name == "throw_if_table_has_different_definition") + res.throw_if_table_has_different_definition = setting.value.safeGet(); else throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 8981ad0fac0..ad046dfa1cf 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -30,6 +30,14 @@ struct RestoreSettings : public StorageRestoreSettings /// Whether RESTORE TABLE will throw an exception if a destination table already exists. bool throw_if_table_exists = true; + /// Whether RESTORE DATABASE will throw an exception if a destination database has + /// a different definition comparing with the definition read from backup. + bool throw_if_database_has_different_definition = true; + + /// Whether RESTORE TABLE will throw an exception if a destination table has + /// a different definition comparing with the definition read from backup. + bool throw_if_table_has_different_definition = true; + static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); }; diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index 5378666187c..3c8c1dcba09 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -1,11 +1,12 @@ #include #include +#include #include #include #include #include #include -#include +#include #include #include #include @@ -37,19 +38,29 @@ namespace using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; + /// 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 RestoreSettings &) - : context(context_), create_query(typeid_cast>(create_query_)) + RestoreDatabaseTask( + ContextMutablePtr context_, + const ASTPtr & create_query_, + const RestoreSettings & restore_settings_, + bool skip_same_definition_check_) + : context(context_) + , create_query(typeid_cast>(create_query_)) + , restore_settings(restore_settings_) + , skip_same_definition_check(skip_same_definition_check_) { } RestoreTasks run() override { createDatabase(); + getDatabase(); + checkDatabaseCreateQuery(); return {}; } @@ -58,12 +69,50 @@ namespace private: void createDatabase() { - InterpreterCreateQuery create_interpreter{create_query, context}; + /// 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 (skip_same_definition_check || !restore_settings.throw_if_database_has_different_definition) + 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 create_query; + RestoreSettings restore_settings; + bool skip_same_definition_check = false; + DatabasePtr database; + ASTPtr database_create_query; }; @@ -90,9 +139,10 @@ namespace RestoreTasks run() override { createStorage(); - auto storage = getStorage(); + getStorage(); + checkStorageCreateQuery(); RestoreTasks tasks; - if (auto task = insertDataIntoStorage(storage)) + if (auto task = insertData()) tasks.push_back(std::move(task)); return tasks; } @@ -102,59 +152,84 @@ namespace private: void createStorage() { - InterpreterCreateQuery create_interpreter{create_query, context}; + /// 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 tryGetStorage() - { - if (!DatabaseCatalog::instance().isTableExist({table_name.first, table_name.second}, context)) - return nullptr; - - DatabasePtr existing_database; - StoragePtr existing_storage; - std::tie(existing_database, existing_storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable({table_name.first, table_name.second}, context); - if (!existing_storage) - return nullptr; - - auto existing_table_create_query = existing_database->tryGetCreateTableQuery(table_name.second, context); - if (!existing_table_create_query) - return nullptr; - - if (!hasCompatibleDataToRestoreTable(*create_query, existing_table_create_query->as())) - throw Exception( - ErrorCodes::CANNOT_RESTORE_TABLE, - "Table {}.{} from backup is incompatible with existing table {}.{}. " - "The create query of the table from backup: {}." - "The create query of the existing table: {}", - backQuoteIfNeed(table_name_in_backup.first), - backQuoteIfNeed(table_name_in_backup.second), - backQuoteIfNeed(table_name.first), - backQuoteIfNeed(table_name.second), - serializeAST(*create_query), - serializeAST(*existing_table_create_query)); - - return existing_storage; - } - StoragePtr getStorage() { - if (auto storage = tryGetStorage()) - return storage; - - String error_message = (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - ? ("Could not create temporary table " + backQuoteIfNeed(table_name.second) + " for restoring") - : ("Could not create table " + backQuoteIfNeed(table_name.first) + "." + backQuoteIfNeed(table_name.second) - + " for restoring"); - throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); + if (!storage) + std::tie(database, storage) = DatabaseCatalog::instance().getDatabaseAndTable({table_name.first, table_name.second}, context); + return storage; } - RestoreTaskPtr insertDataIntoStorage(StoragePtr storage) + ASTPtr getStorageCreateQuery() { - if (storage->hasHollowBackup() || restore_settings.structure_only) - return {}; + 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_has_different_definition) + 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); - String data_path_in_backup = getDataPathInBackup(table_name_in_backup); + + has_data = true; + return true; + } + + RestoreTaskPtr insertData() + { + if (!hasData()) + return {}; return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, restore_settings); } @@ -165,6 +240,11 @@ namespace BackupPtr backup; DatabaseAndTableName table_name_in_backup; RestoreSettings restore_settings; + DatabasePtr database; + StoragePtr storage; + ASTPtr storage_create_query; + std::optional has_data; + String data_path_in_backup; }; @@ -220,13 +300,14 @@ namespace { if (info.different_create_query) throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, - "Couldn't restore a database because two different create queries were generated for it: {} and {}", + "Cannot restore a database because two different create queries were generated for it: {} and {}", serializeAST(*info.create_query), serializeAST(*info.different_create_query)); } RestoreTasks res; for (auto & info : databases | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, restore_settings)); + res.push_back(std::make_unique(context, info.create_query, restore_settings, + /* skip_same_definition_check = */ !info.is_explicit)); /// TODO: We need to restore tables according to their dependencies. for (auto & info : tables | boost::adaptors::map_values) @@ -242,14 +323,7 @@ namespace /// 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)) - { - String message; - if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - message = fmt::format("Couldn't restore temporary table {} twice", backQuoteIfNeed(new_table_name.second)); - else - message = fmt::format("Couldn't restore table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second)); - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, message); - } + 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_)); @@ -301,7 +375,7 @@ namespace create_db_query = std::make_shared(); create_db_query->setDatabase(new_table_name.first); create_db_query->if_not_exists = true; - if (serializeAST(*info_db.create_query) != serializeAST(*create_db_query)) + if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query)) info_db.different_create_query = create_db_query; } } @@ -314,13 +388,13 @@ namespace /// 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, "Couldn't restore database {} twice", backQuoteIfNeed(new_database_name)); + 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, "Could not restore database {} because there is no such database in the backup", backQuoteIfNeed(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)) @@ -376,7 +450,8 @@ namespace { String create_query_path = getMetadataPathInBackup(table_name); if (!backup->fileExists(create_query_path)) - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Could not restore table {}.{} because there is no such table in the backup", backQuoteIfNeed(table_name.first), backQuoteIfNeed(table_name.second)); + 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); @@ -390,7 +465,7 @@ namespace { String create_query_path = getMetadataPathInBackup(database_name); if (!backup->fileExists(create_query_path)) - throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Could not restore database {} because there is no such database in the backup", backQuoteIfNeed(database_name)); + 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); diff --git a/src/Backups/formatTableNameOrTemporaryTableName.cpp b/src/Backups/formatTableNameOrTemporaryTableName.cpp new file mode 100644 index 00000000000..7338e1dab23 --- /dev/null +++ b/src/Backups/formatTableNameOrTemporaryTableName.cpp @@ -0,0 +1,17 @@ +#include +#include +#include + + +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); +} + +} diff --git a/src/Backups/formatTableNameOrTemporaryTableName.h b/src/Backups/formatTableNameOrTemporaryTableName.h new file mode 100644 index 00000000000..a6b94cd4077 --- /dev/null +++ b/src/Backups/formatTableNameOrTemporaryTableName.h @@ -0,0 +1,13 @@ +#pragma once + +#include + + +namespace DB +{ +using DatabaseAndTableName = std::pair; + +/// Outputs either "table db_name.table_name" or "temporary table table_name". +String formatTableNameOrTemporaryTableName(const DatabaseAndTableName & table_name); + +} diff --git a/src/Backups/hasCompatibleDataToRestoreTable.cpp b/src/Backups/hasCompatibleDataToRestoreTable.cpp deleted file mode 100644 index 8ac2fda93d8..00000000000 --- a/src/Backups/hasCompatibleDataToRestoreTable.cpp +++ /dev/null @@ -1,26 +0,0 @@ -#include -#include -#include - - -namespace DB -{ - -bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCreateQuery & query2) -{ - /// TODO: Write more subtle condition here. - auto q1 = typeid_cast>(query1.clone()); - auto q2 = typeid_cast>(query2.clone()); - - /// Remove UUIDs. - q1->uuid = UUIDHelpers::Nil; - q2->uuid = UUIDHelpers::Nil; - - /// Clear IF NOT EXISTS flag. - q1->if_not_exists = false; - q2->if_not_exists = false; - - return serializeAST(*q1) == serializeAST(*q2); -} - -} diff --git a/src/Backups/hasCompatibleDataToRestoreTable.h b/src/Backups/hasCompatibleDataToRestoreTable.h deleted file mode 100644 index 92aab8b4579..00000000000 --- a/src/Backups/hasCompatibleDataToRestoreTable.h +++ /dev/null @@ -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); - -} From e72a343994f80c4cf89ab01cc25916980792e29c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 25 Jan 2022 00:41:13 +0700 Subject: [PATCH 69/85] Implement BACKUP & RESTORE for the Memory table engine. --- src/Common/FileChecker.cpp | 46 +++-- src/Common/FileChecker.h | 8 +- src/Storages/StorageMemory.cpp | 157 ++++++++++++++++++ src/Storages/StorageMemory.h | 4 + src/Storages/StorageStripeLog.h | 1 + .../test_backup_restore_new/test.py | 4 +- 6 files changed, 204 insertions(+), 16 deletions(-) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 4de5a92a1b8..ef90e31982c 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -1,13 +1,16 @@ -#include +#include +#include +#include #include #include #include #include #include -#include +#include +#include -#include +namespace fs = std::filesystem; namespace DB { @@ -19,6 +22,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 +52,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 +81,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 +106,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 +126,7 @@ void FileChecker::save() const std::string tmp_files_info_path = parentPath(files_info_path) + "tmp_" + fileName(files_info_path); { - std::unique_ptr out = disk->writeFile(tmp_files_info_path); + std::unique_ptr out = disk ? disk->writeFile(tmp_files_info_path) : std::make_unique(tmp_files_info_path); /// So complex JSON structure - for compatibility with the old format. writeCString("{\"clickhouse\":{", *out); @@ -141,17 +148,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 in = disk->readFile(files_info_path); + std::unique_ptr in = disk ? disk->readFile(files_info_path) : createReadBufferFromFileBase(files_info_path, {}); WriteBufferFromOwnString out; /// The JSON library does not support whitespace. We delete them. Inefficient. @@ -169,4 +179,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_); +} + } diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index a0ea449393e..98e7837db75 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -2,16 +2,19 @@ #include #include -#include namespace DB { +class IDisk; +using DiskPtr = std::shared_ptr; + /// 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"); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index c3601b33a04..88a9969fe1a 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -17,6 +17,17 @@ #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + namespace DB { @@ -364,6 +375,152 @@ void StorageMemory::truncate( total_size_rows.store(0, std::memory_order_relaxed); } + +BackupEntries StorageMemory::backup(ContextPtr context, const ASTs & partitions) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + auto blocks = data.get(); + + /// We store our data in the StripeLog format. + BackupEntries backup_entries; + auto temp_dir_owner = std::make_shared(); + 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(data_file_path); + CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), context->getSettingsRef().max_compress_block_size}; + NativeWriter block_out{data_out, 0, getInMemoryMetadataPtr()->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(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(); + + /// Prepare backup entries. + backup_entries.emplace_back( + data_file_name, + std::make_unique( + data_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); + + backup_entries.emplace_back( + index_file_name, + std::make_unique( + index_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); + + backup_entries.emplace_back( + sizes_file_name, + std::make_unique( + sizes_file_path, std::nullopt, std::nullopt, temp_dir_owner)); + + return backup_entries; +} + + +class MemoryRestoreTask : public IRestoreTask +{ +public: + MemoryRestoreTask( + std::shared_ptr 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 in = backup_entry->getReadBuffer(); + std::optional temp_data_copy; + if (!typeid_cast(in.get())) + { + temp_data_copy.emplace(); + auto temp_data_copy_out = std::make_unique(temp_data_copy->path()); + copyData(*in, *temp_data_copy_out); + temp_data_copy_out.reset(); + in = createReadBufferFromFileBase(temp_data_copy->path(), {}); + } + std::unique_ptr in_from_file{static_cast(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(std::move(old_and_new_blocks))); + storage->total_size_bytes += new_bytes; + storage->total_size_rows += new_rows; + + return {}; + } + +private: + std::shared_ptr storage; + BackupPtr backup; + String data_path_in_backup; + ContextMutablePtr context; +}; + + +RestoreTaskPtr StorageMemory::restoreFromBackup(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( + typeid_cast>(shared_from_this()), backup, data_path_in_backup, context); +} + + std::optional StorageMemory::totalRows(const Settings &) const { /// All modifications of these counters are done under mutex which automatically guarantees synchronization/consistency diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 1c4421e51a6..c530cbaa31b 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -22,6 +22,7 @@ namespace DB class StorageMemory final : public shared_ptr_helper, public IStorage { friend class MemorySink; +friend class MemoryRestoreTask; friend struct shared_ptr_helper; public: @@ -65,6 +66,9 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; + BackupEntries backup(ContextPtr context, const ASTs & partitions) override; + RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; + std::optional totalRows(const Settings &) const override; std::optional totalBytes(const Settings &) const override; diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index a5d3b9a3da3..9cd9bc2c165 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 0d51a284e49..ad1190baac0 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -37,7 +37,7 @@ def new_backup_name(): return f"Disk('backups', '{backup_id_counter}/')" -@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog"]) +@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"]) def test_restore_table(engine): backup_name = new_backup_name() create_and_fill_table(engine=engine) @@ -52,7 +52,7 @@ def test_restore_table(engine): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" -@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog"]) +@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"]) def test_restore_table_into_existing_table(engine): backup_name = new_backup_name() create_and_fill_table(engine=engine) From a1b364626af1788690bc8a92ae0295a46bf29c50 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 22 Feb 2022 07:59:41 +0300 Subject: [PATCH 70/85] Improve zip archive utilities. --- src/IO/Archives/IArchiveReader.h | 1 - src/IO/Archives/IArchiveWriter.h | 2 +- src/IO/Archives/ZipArchiveReader.cpp | 23 +++++- src/IO/Archives/ZipArchiveReader.h | 13 ++-- src/IO/Archives/ZipArchiveWriter.cpp | 75 +++++++++++-------- src/IO/Archives/ZipArchiveWriter.h | 25 +++---- .../hasRegisteredArchiveFileExtension.cpp | 12 +++ .../hasRegisteredArchiveFileExtension.h | 12 +++ 8 files changed, 105 insertions(+), 58 deletions(-) create mode 100644 src/IO/Archives/hasRegisteredArchiveFileExtension.cpp create mode 100644 src/IO/Archives/hasRegisteredArchiveFileExtension.h diff --git a/src/IO/Archives/IArchiveReader.h b/src/IO/Archives/IArchiveReader.h index 584e80a7d09..b5c474977bf 100644 --- a/src/IO/Archives/IArchiveReader.h +++ b/src/IO/Archives/IArchiveReader.h @@ -23,7 +23,6 @@ public: { UInt64 uncompressed_size; UInt64 compressed_size; - int compression_method; bool is_encrypted; }; diff --git a/src/IO/Archives/IArchiveWriter.h b/src/IO/Archives/IArchiveWriter.h index 3856d16fb89..c6330509f54 100644 --- a/src/IO/Archives/IArchiveWriter.h +++ b/src/IO/Archives/IArchiveWriter.h @@ -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 */) {} diff --git a/src/IO/Archives/ZipArchiveReader.cpp b/src/IO/Archives/ZipArchiveReader.cpp index 4e83234615c..68726248dc4 100644 --- a/src/IO/Archives/ZipArchiveReader.cpp +++ b/src/IO/Archives/ZipArchiveReader.cpp @@ -1,6 +1,7 @@ #include #if USE_MINIZIP +#include #include #include #include @@ -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 reader; RawHandle raw_handle = nullptr; mutable std::optional file_name; - mutable std::optional file_info; + mutable std::optional file_info; }; @@ -174,7 +189,7 @@ public: , handle(std::move(handle_)) { const auto & file_info = handle.getFileInfo(); - checkCompressionMethodIsEnabled(static_cast(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(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(CompressionMethod::kStore)) + if (file_info.compression_method == MZ_COMPRESS_METHOD_STORE) { /// unzSeek64() works only for non-compressed files. checkResult(unzSeek64(raw_handle, off, whence)); diff --git a/src/IO/Archives/ZipArchiveReader.h b/src/IO/Archives/ZipArchiveReader.h index 6932a93e23f..7236b0b660c 100644 --- a/src/IO/Archives/ZipArchiveReader.h +++ b/src/IO/Archives/ZipArchiveReader.h @@ -4,7 +4,6 @@ #if USE_MINIZIP #include -#include #include #include #include @@ -20,8 +19,6 @@ class SeekableReadBuffer; class ZipArchiveReader : public shared_ptr_helper, 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_); diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 79192223657..dbfd66a6293 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -80,7 +80,7 @@ public: { auto compress_method = handle.getWriter()->compression_method; auto compress_level = handle.getWriter()->compression_level; - checkCompressionMethodIsEnabled(static_cast(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 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(method)); + throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", compression_method_); } /// Checks that encryption is enabled. diff --git a/src/IO/Archives/ZipArchiveWriter.h b/src/IO/Archives/ZipArchiveWriter.h index 76f8dd8e9e5..58df4902434 100644 --- a/src/IO/Archives/ZipArchiveWriter.h +++ b/src/IO/Archives/ZipArchiveWriter.h @@ -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(CompressionMethod::kDeflate); + int compression_method; /// By default the compression method is "deflate". int compression_level = kDefaultCompressionLevel; String password; RawHandle handle = nullptr; diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp new file mode 100644 index 00000000000..6b2ef29d054 --- /dev/null +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp @@ -0,0 +1,12 @@ +#include + + +namespace DB +{ + +bool hasRegisteredArchiveFileExtension(const String & path) +{ + return path.ends_with(".zip") || path.ends_with(".zipx"); +} + +} diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.h b/src/IO/Archives/hasRegisteredArchiveFileExtension.h new file mode 100644 index 00000000000..cff2a0a8bec --- /dev/null +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.h @@ -0,0 +1,12 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Returns true if a specified path has one of the registed file extensions for an archive. +bool hasRegisteredArchiveFileExtension(const String & path); + +} From 5f195d6eadad26a1ba895ceb1c2f788d82ce18ac Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 29 Jan 2022 00:33:35 +0700 Subject: [PATCH 71/85] Implement writing backups as zip archive. --- src/Backups/ArchiveBackup.cpp | 105 +++++++++++ src/Backups/ArchiveBackup.h | 52 ++++++ src/Backups/BackupFactory.cpp | 11 +- src/Backups/BackupFactory.h | 3 + src/Backups/BackupImpl.cpp | 80 ++++++--- src/Backups/BackupImpl.h | 37 ++-- src/Backups/BackupInDirectory.cpp | 160 ----------------- src/Backups/BackupSettings.cpp | 11 +- src/Backups/BackupSettings.h | 8 + src/Backups/DirectoryBackup.cpp | 82 +++++++++ ...{BackupInDirectory.h => DirectoryBackup.h} | 11 +- src/Backups/IBackup.h | 16 +- src/Backups/RestoreSettings.cpp | 15 +- src/Backups/RestoreSettings.h | 4 + src/Backups/registerBackupEngines.cpp | 14 -- .../registerBackupEnginesFileAndDisk.cpp | 168 ++++++++++++++++++ src/Interpreters/InterpreterBackupQuery.cpp | 28 ++- .../test_backup_restore_new/test.py | 28 +++ 18 files changed, 584 insertions(+), 249 deletions(-) create mode 100644 src/Backups/ArchiveBackup.cpp create mode 100644 src/Backups/ArchiveBackup.h delete mode 100644 src/Backups/BackupInDirectory.cpp create mode 100644 src/Backups/DirectoryBackup.cpp rename src/Backups/{BackupInDirectory.h => DirectoryBackup.h} (80%) delete mode 100644 src/Backups/registerBackupEngines.cpp create mode 100644 src/Backups/registerBackupEnginesFileAndDisk.cpp diff --git a/src/Backups/ArchiveBackup.cpp b/src/Backups/ArchiveBackup.cpp new file mode 100644 index 00000000000..36148430f68 --- /dev/null +++ b/src/Backups/ArchiveBackup.cpp @@ -0,0 +1,105 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +ArchiveBackup::ArchiveBackup( + const String & backup_name_, + const DiskPtr & disk_, + const String & path_, + const ContextPtr & context_, + const std::optional & 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 { 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 ArchiveBackup::readFileImpl(const String & file_name) const +{ + /// mutex is already locked + return reader->readFile(file_name); +} + +std::unique_ptr ArchiveBackup::addFileImpl(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); +} + +} diff --git a/src/Backups/ArchiveBackup.h b/src/Backups/ArchiveBackup.h new file mode 100644 index 00000000000..8c76830977a --- /dev/null +++ b/src/Backups/ArchiveBackup.h @@ -0,0 +1,52 @@ +#pragma once + +#include + + +namespace DB +{ +class IDisk; +using DiskPtr = std::shared_ptr; +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 & 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 readFileImpl(const String & file_name) const override; + std::unique_ptr addFileImpl(const String & file_name) override; + + const DiskPtr disk; + const String path; + std::shared_ptr reader; + std::shared_ptr writer; + String compression_method; + int compression_level = kDefaultCompressionLevel; + String password; +}; + +} diff --git a/src/Backups/BackupFactory.cpp b/src/Backups/BackupFactory.cpp index 490bfb002db..d64c2bd0318 100644 --- a/src/Backups/BackupFactory.cpp +++ b/src/Backups/BackupFactory.cpp @@ -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() { diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index 51d70c61f54..d3ebcfe2369 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -26,6 +26,9 @@ public: OpenMode open_mode = OpenMode::WRITE; BackupInfo backup_info; std::optional base_backup_info; + String compression_method; + int compression_level = -1; + String password; ContextPtr context; }; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index d8ed247d20e..9de434e1e72 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -47,34 +47,44 @@ namespace } } -BackupImpl::BackupImpl(const String & backup_name_, OpenMode open_mode_, const ContextPtr & context_, const std::optional & base_backup_info_) - : backup_name(backup_name_), open_mode(open_mode_), context(context_), base_backup_info(base_backup_info_) +BackupImpl::BackupImpl(const String & backup_name_, const ContextPtr & context_, const std::optional & 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 +93,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() @@ -244,6 +272,9 @@ UInt128 BackupImpl::getFileChecksum(const String & file_name) const 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( @@ -329,7 +360,7 @@ void BackupImpl::addFile(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( @@ -467,8 +498,13 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry) 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; } diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 5a0269cd662..ac2771cfc6c 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -20,14 +20,15 @@ class BackupImpl : public IBackup public: BackupImpl( const String & backup_name_, - OpenMode open_mode_, const ContextPtr & context_, const std::optional & 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; @@ -38,11 +39,13 @@ public: 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. @@ -52,16 +55,7 @@ protected: /// Low level: the function doesn't check base backup or checksums. virtual std::unique_ptr addFileImpl(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(); @@ -78,17 +72,16 @@ private: }; const String backup_name; - const OpenMode open_mode; - UUID uuid; - time_t timestamp = 0; ContextPtr context; + const std::optional base_backup_info_param; + OpenMode open_mode = OpenMode::NONE; + UUID uuid = {}; + time_t timestamp = 0; std::optional base_backup_info; std::shared_ptr base_backup; std::optional base_backup_uuid; std::map file_infos; - bool writing_started = false; bool writing_finalized = false; - mutable std::mutex mutex; }; } diff --git a/src/Backups/BackupInDirectory.cpp b/src/Backups/BackupInDirectory.cpp deleted file mode 100644 index f071a21618c..00000000000 --- a/src/Backups/BackupInDirectory.cpp +++ /dev/null @@ -1,160 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -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 & 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 & 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(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 BackupInDirectory::readFileImpl(const String & file_name) const -{ - String file_path = path + file_name; - return disk->readFile(file_path); -} - -std::unique_ptr 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(); - - 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(); - disk = params.context->getDisk(disk_name); - path = args[1].safeGet(); - - if (!isDiskAllowed(disk_name, params.context->getConfigRef())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} is not allowed for backups", disk_name); - } - - return std::make_shared(backup_name, params.open_mode, disk, path, params.context, params.base_backup_info); - }; - - factory.registerBackupEngine("File", creator_fn); - factory.registerBackupEngine("Disk", creator_fn); -} - -} diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 602cf67cc2b..2189ac11a28 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -23,8 +24,14 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) const auto & settings = query.settings->as().changes; for (const auto & setting : settings) { - if (setting.name == "structure_only") - res.structure_only = setting.value.safeGet(); + 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); } diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index ce911382c78..f46e65353fb 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -14,6 +15,13 @@ struct BackupSettings /// Base backup, if it's set an incremental backup will be built. std::shared_ptr base_backup_info; + /// Compression method and level for writing the backup (when applicable). + String compression_method; /// "" means default method + int compression_level = -1; /// -1 means default level + + /// 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; diff --git a/src/Backups/DirectoryBackup.cpp b/src/Backups/DirectoryBackup.cpp new file mode 100644 index 00000000000..2adee160180 --- /dev/null +++ b/src/Backups/DirectoryBackup.cpp @@ -0,0 +1,82 @@ +#include +#include +#include +#include + + +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 & 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(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 DirectoryBackup::readFileImpl(const String & file_name) const +{ + String file_path = path + file_name; + return disk->readFile(file_path); +} + +std::unique_ptr DirectoryBackup::addFileImpl(const String & file_name) +{ + String file_path = path + file_name; + disk->createDirectories(fs::path(file_path).parent_path()); + return disk->writeFile(file_path); +} + +} diff --git a/src/Backups/BackupInDirectory.h b/src/Backups/DirectoryBackup.h similarity index 80% rename from src/Backups/BackupInDirectory.h rename to src/Backups/DirectoryBackup.h index dd6a39d019a..0b6dd1361d1 100644 --- a/src/Backups/BackupInDirectory.h +++ b/src/Backups/DirectoryBackup.h @@ -10,23 +10,22 @@ using DiskPtr = std::shared_ptr; /// 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 & 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 readFileImpl(const String & file_name) const override; std::unique_ptr addFileImpl(const String & file_name) override; diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index 0fd94a6d53c..7e9503a7754 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include @@ -13,7 +12,7 @@ using BackupEntryPtr = std::unique_ptr; /// 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, public TypePromotion +class IBackup : public std::enable_shared_from_this { public: IBackup() = default; @@ -24,13 +23,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; @@ -63,11 +67,11 @@ public: /// 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; } - /// 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; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 6d96c6f22eb..546f3007df5 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -23,16 +24,18 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) const auto & settings = query.settings->as().changes; for (const auto & setting : settings) { - if (setting.name == "structure_only") - res.structure_only = setting.value.safeGet(); + 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 = setting.value.safeGet(); + res.throw_if_database_exists = SettingFieldBool{setting.value}; else if (setting.name == "throw_if_table_exists") - res.throw_if_table_exists = setting.value.safeGet(); + res.throw_if_table_exists = SettingFieldBool{setting.value}; else if (setting.name == "throw_if_database_has_different_definition") - res.throw_if_database_has_different_definition = setting.value.safeGet(); + res.throw_if_database_has_different_definition = SettingFieldBool{setting.value}; else if (setting.name == "throw_if_table_has_different_definition") - res.throw_if_table_has_different_definition = setting.value.safeGet(); + res.throw_if_table_has_different_definition = SettingFieldBool{setting.value}; else throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index ad046dfa1cf..dcb55973d85 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -20,6 +21,9 @@ struct RestoreSettings : public StorageRestoreSettings /// so using this setting is optional. std::shared_ptr 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; diff --git a/src/Backups/registerBackupEngines.cpp b/src/Backups/registerBackupEngines.cpp deleted file mode 100644 index 33bc2c8fdd9..00000000000 --- a/src/Backups/registerBackupEngines.cpp +++ /dev/null @@ -1,14 +0,0 @@ - - -namespace DB -{ -class BackupFactory; - -void registerBackupEngineFile(BackupFactory &); - -void registerBackupEngines(BackupFactory & factory) -{ - registerBackupEngineFile(factory); -} - -} diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp new file mode 100644 index 00000000000..6a34d67115d --- /dev/null +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -0,0 +1,168 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +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 + { + 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(); + 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(); + path = args[1].safeGet(); + checkDiskNameAndPath(disk_name, path, params.context->getConfigRef()); + disk = params.context->getDisk(disk_name); + } + + std::unique_ptr 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(backup_name, disk, path, params.context, params.base_backup_info); + } + else if (hasRegisteredArchiveFileExtension(path)) + { + auto archive_backup = std::make_unique(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); +} + +} diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index ac930e1f6a7..390399e0470 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -14,24 +14,34 @@ namespace DB { namespace { - BackupMutablePtr createBackup( - const BackupInfo & backup_info, - IBackup::OpenMode open_mode, - const BackupInfo * base_backup_info, - const ContextPtr & context) + BackupMutablePtr createBackup(const BackupInfo & backup_info, const BackupSettings & backup_settings, const ContextPtr & context) { BackupFactory::CreateParams params; - params.open_mode = open_mode; + params.open_mode = IBackup::OpenMode::WRITE; params.context = context; params.backup_info = backup_info; - params.base_backup_info = base_backup_info ? *base_backup_info : std::optional{}; + params.base_backup_info = backup_settings.base_backup_info ? *backup_settings.base_backup_info : std::optional{}; + params.compression_method = backup_settings.compression_method; + params.compression_level = backup_settings.compression_level; + params.password = backup_settings.password; + return BackupFactory::instance().createBackup(params); + } + + BackupMutablePtr openBackup(const BackupInfo & backup_info, const RestoreSettings & restore_settings, const ContextPtr & context) + { + 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 ? *restore_settings.base_backup_info : std::optional{}; + params.password = restore_settings.password; return BackupFactory::instance().createBackup(params); } void executeBackup(const ContextPtr & context, const ASTBackupQuery & query) { auto backup_settings = BackupSettings::fromBackupQuery(query); - BackupMutablePtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), IBackup::OpenMode::WRITE, backup_settings.base_backup_info.get(), context); + 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); } @@ -39,7 +49,7 @@ namespace void executeRestore(ContextMutablePtr context, const ASTBackupQuery & query) { auto restore_settings = RestoreSettings::fromRestoreQuery(query); - BackupPtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), IBackup::OpenMode::READ, restore_settings.base_backup_info.get(), context); + 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); } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index ad1190baac0..dcd6c241801 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -147,3 +147,31 @@ def test_database(): instance.query(f"RESTORE DATABASE test FROM {backup_name}") assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + +def test_zip_archive(): + backup_name = f"File('/backups/archive.zip')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + +def test_zip_archive_with_settings(): + backup_name = f"File('/backups/archive_with_settings.zip')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name} SETTINGS compression_method='lzma', compression_level=3, password='qwerty'") + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name} SETTINGS password='qwerty'") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" From c884cf901f730c4da256475c52564f5191e645c2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 31 Jan 2022 13:35:07 +0700 Subject: [PATCH 72/85] Use BackupEntryFromAppendOnlyFile for Log engine, and use a BackupEntriesBatch for Memort engine to improve performance. A lot of minor corrections. --- src/Backups/BackupEntryConcat.cpp | 28 ---- src/Backups/BackupEntryConcat.h | 30 ---- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 2 +- src/Backups/BackupEntryFromCallback.h | 31 ---- src/Backups/BackupImpl.cpp | 53 ++++-- src/Backups/BackupImpl.h | 2 + src/Backups/BackupSettings.cpp | 2 +- src/Backups/BackupSettings.h | 7 +- src/Backups/IBackupEntriesBatch.cpp | 37 +++++ src/Backups/IBackupEntriesBatch.h | 29 ++++ src/Backups/RestoreSettings.cpp | 10 +- src/Backups/RestoreSettings.h | 19 ++- src/Backups/RestoreUtils.cpp | 31 ++-- src/IO/ConcatReadBuffer.h | 6 + src/Interpreters/InterpreterBackupQuery.cpp | 4 +- src/Storages/StorageLog.cpp | 18 +-- src/Storages/StorageMemory.cpp | 153 +++++++++++------- src/Storages/StorageStripeLog.cpp | 18 +-- 18 files changed, 271 insertions(+), 209 deletions(-) delete mode 100644 src/Backups/BackupEntryConcat.cpp delete mode 100644 src/Backups/BackupEntryConcat.h delete mode 100644 src/Backups/BackupEntryFromCallback.h create mode 100644 src/Backups/IBackupEntriesBatch.cpp create mode 100644 src/Backups/IBackupEntriesBatch.h diff --git a/src/Backups/BackupEntryConcat.cpp b/src/Backups/BackupEntryConcat.cpp deleted file mode 100644 index 1075b8be2ec..00000000000 --- a/src/Backups/BackupEntryConcat.cpp +++ /dev/null @@ -1,28 +0,0 @@ -#include -#include - - -namespace DB -{ -BackupEntryConcat::BackupEntryConcat( - BackupEntryPtr first_source_, - BackupEntryPtr second_source_, - const std::optional & 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 BackupEntryConcat::getReadBuffer() const -{ - return std::make_unique(*first_source->getReadBuffer(), *second_source->getReadBuffer()); -} -} diff --git a/src/Backups/BackupEntryConcat.h b/src/Backups/BackupEntryConcat.h deleted file mode 100644 index cb38fb9b163..00000000000 --- a/src/Backups/BackupEntryConcat.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include - - -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 & checksum_ = {}); - - UInt64 getSize() const override; - std::optional getChecksum() const override { return checksum; } - std::unique_ptr getReadBuffer() const override; - -private: - BackupEntryPtr first_source; - BackupEntryPtr second_source; - mutable std::optional size; - std::optional checksum; -}; - -} diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index 32d5713952f..d7f9d5624c8 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -29,7 +29,7 @@ BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile( std::unique_ptr BackupEntryFromAppendOnlyFile::getReadBuffer() const { auto buf = BackupEntryFromImmutableFile::getReadBuffer(); - return std::make_unique(std::move(buf), limit, true); + return std::make_unique(std::move(buf), limit, false); } } diff --git a/src/Backups/BackupEntryFromCallback.h b/src/Backups/BackupEntryFromCallback.h deleted file mode 100644 index e8df2d99f7c..00000000000 --- a/src/Backups/BackupEntryFromCallback.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/// Represents small preloaded data to be included in a backup. -class BackupEntryFromCallback : public IBackupEntry -{ -public: - using ReadBufferCreator = std::function()>; - - /// 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 & checksum_ = {}) - : callback(callback_), size(size_), checksum(checksum_) - { - } - - UInt64 getSize() const override { return size; } - std::optional getChecksum() const override { return checksum; } - std::unique_ptr getReadBuffer() const override { return callback(); } - -private: - const ReadBufferCreator callback; - const size_t size; - const std::optional checksum; -}; - -} diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 9de434e1e72..d146f0394d0 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -1,13 +1,11 @@ #include #include -#include -#include #include #include #include #include -#include #include +#include #include #include #include @@ -47,6 +45,44 @@ namespace } } + +class BackupImpl::BackupEntryFromBackupImpl : public IBackupEntry +{ +public: + BackupEntryFromBackupImpl( + const std::shared_ptr & backup_, + const String & file_name_, + UInt64 size_, + const std::optional 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 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(std::move(base_backup_read_buffer), std::move(read_buffer)); + } + return read_buffer; + } + + UInt64 getSize() const override { return size; } + std::optional getChecksum() const override { return checksum; } + +private: + const std::shared_ptr backup; + const String file_name; + const UInt64 size; + const std::optional checksum; + BackupEntryPtr base_backup_entry; +}; + + BackupImpl::BackupImpl(const String & backup_name_, const ContextPtr & context_, const std::optional & base_backup_info_) : backup_name(backup_name_), context(context_), base_backup_info_param(base_backup_info_) { @@ -295,7 +331,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(read_callback, info.size, info.checksum); + return std::make_unique( + std::static_pointer_cast(shared_from_this()), file_name, info.size, info.checksum); } if (info.size < info.base_size) @@ -349,10 +386,8 @@ 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( - std::move(base_entry), - std::make_unique(read_callback, info.size - info.base_size), - info.checksum); + return std::make_unique( + static_pointer_cast(shared_from_this()), file_name, info.size, info.checksum, std::move(base_entry)); } @@ -406,7 +441,7 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry) checksum = hashing_read_buffer.getHash(); } if (checksum == base_checksum) - use_base = true; /// The data has not been changed. + use_base = true; /// The data have not been changed. } else if (size > base_size) { diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index ac2771cfc6c..af31b71f460 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -71,6 +71,8 @@ private: UInt128 base_checksum{0, 0}; }; + class BackupEntryFromBackupImpl; + const String backup_name; ContextPtr context; const std::optional base_backup_info_param; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 2189ac11a28..817e0e7e1a7 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -17,7 +17,7 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) BackupSettings res; if (query.base_backup_name) - res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + res.base_backup_info = BackupInfo::fromAST(*query.base_backup_name); if (query.settings) { diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index f46e65353fb..ca95a08da8f 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -1,19 +1,18 @@ #pragma once -#include -#include +#include +#include namespace DB { class ASTBackupQuery; -struct BackupInfo; /// 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::shared_ptr base_backup_info; + std::optional base_backup_info; /// Compression method and level for writing the backup (when applicable). String compression_method; /// "" means default method diff --git a/src/Backups/IBackupEntriesBatch.cpp b/src/Backups/IBackupEntriesBatch.cpp new file mode 100644 index 00000000000..bf6bc6cce83 --- /dev/null +++ b/src/Backups/IBackupEntriesBatch.cpp @@ -0,0 +1,37 @@ +#include +#include + + +namespace DB +{ + +class IBackupEntriesBatch::BackupEntryFromBatch : public IBackupEntry +{ +public: + BackupEntryFromBatch(const std::shared_ptr & generator_, size_t index_) : batch(generator_), index(index_) + { + assert(batch); + } + + UInt64 getSize() const override { return batch->getSize(index); } + std::optional getChecksum() const override { return batch->getChecksum(index); } + std::unique_ptr getReadBuffer() const override { return batch->getReadBuffer(index); } + +private: + const std::shared_ptr 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(shared_from_this(), i)); + } + return res; +} + +} diff --git a/src/Backups/IBackupEntriesBatch.h b/src/Backups/IBackupEntriesBatch.h new file mode 100644 index 00000000000..0d8c8d5aa26 --- /dev/null +++ b/src/Backups/IBackupEntriesBatch.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Helper class designed to generate multiple backup entries from one source. +class IBackupEntriesBatch : public std::enable_shared_from_this +{ +public: + BackupEntries getBackupEntries(); + + virtual ~IBackupEntriesBatch() = default; + +protected: + IBackupEntriesBatch(const Strings & entry_names_) : entry_names(entry_names_) {} + + virtual std::unique_ptr getReadBuffer(size_t index) = 0; + virtual UInt64 getSize(size_t index) = 0; + virtual std::optional getChecksum(size_t) { return {}; } + +private: + class BackupEntryFromBatch; + const Strings entry_names; +}; + +} diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 546f3007df5..485650e39f0 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -17,7 +17,7 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) RestoreSettings res; if (query.base_backup_name) - res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + res.base_backup_info = BackupInfo::fromAST(*query.base_backup_name); if (query.settings) { @@ -32,10 +32,10 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) 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_has_different_definition") - res.throw_if_database_has_different_definition = SettingFieldBool{setting.value}; - else if (setting.name == "throw_if_table_has_different_definition") - res.throw_if_table_has_different_definition = 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); } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index dcb55973d85..b129224943b 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -1,13 +1,12 @@ #pragma once -#include -#include +#include +#include namespace DB { class ASTBackupQuery; -struct BackupInfo; struct StorageRestoreSettings { @@ -19,7 +18,7 @@ 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::shared_ptr base_backup_info; + std::optional base_backup_info; /// Password used to decrypt the backup. String password; @@ -28,19 +27,19 @@ struct RestoreSettings : public StorageRestoreSettings /// without the data of tables. bool structure_only = false; - /// Whether RESTORE DATABASE will throw an exception if a destination database already exists. + /// Whether RESTORE DATABASE must throw an exception if a destination database already exists. bool throw_if_database_exists = true; - /// Whether RESTORE TABLE will throw an exception if a destination table already exists. + /// Whether RESTORE TABLE must throw an exception if a destination table already exists. bool throw_if_table_exists = true; - /// Whether RESTORE DATABASE will throw an exception if a destination database has + /// 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_has_different_definition = true; + bool throw_if_database_def_differs = true; - /// Whether RESTORE TABLE will throw an exception if a destination table has + /// 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_has_different_definition = true; + bool throw_if_table_def_differs = true; static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); }; diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index 3c8c1dcba09..a94690ea881 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -37,6 +37,7 @@ namespace using Element = ASTBackupQuery::Element; using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; + using RestoreSettingsPtr = std::shared_ptr; /// Restores a database (without tables inside), should be executed before executing @@ -47,12 +48,12 @@ namespace RestoreDatabaseTask( ContextMutablePtr context_, const ASTPtr & create_query_, - const RestoreSettings & restore_settings_, - bool skip_same_definition_check_) + const RestoreSettingsPtr & restore_settings_, + bool ignore_if_database_def_differs_) : context(context_) , create_query(typeid_cast>(create_query_)) , restore_settings(restore_settings_) - , skip_same_definition_check(skip_same_definition_check_) + , ignore_if_database_def_differs(ignore_if_database_def_differs_) { } @@ -91,7 +92,7 @@ namespace void checkDatabaseCreateQuery() { - if (skip_same_definition_check || !restore_settings.throw_if_database_has_different_definition) + if (ignore_if_database_def_differs || !restore_settings->throw_if_database_def_differs) return; getDatabaseCreateQuery(); @@ -109,8 +110,8 @@ namespace ContextMutablePtr context; std::shared_ptr create_query; - RestoreSettings restore_settings; - bool skip_same_definition_check = false; + RestoreSettingsPtr restore_settings; + bool ignore_if_database_def_differs = false; DatabasePtr database; ASTPtr database_create_query; }; @@ -126,7 +127,7 @@ namespace const ASTs & partitions_, const BackupPtr & backup_, const DatabaseAndTableName & table_name_in_backup_, - const RestoreSettings & restore_settings_) + const RestoreSettingsPtr & restore_settings_) : context(context_), create_query(typeid_cast>(create_query_)), partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_), restore_settings(restore_settings_) @@ -177,7 +178,7 @@ namespace void checkStorageCreateQuery() { - if (!restore_settings.throw_if_table_has_different_definition) + if (!restore_settings->throw_if_table_def_differs) return; getStorageCreateQuery(); @@ -199,7 +200,7 @@ namespace return *has_data; has_data = false; - if (restore_settings.structure_only) + if (restore_settings->structure_only) return false; data_path_in_backup = getDataPathInBackup(table_name_in_backup); @@ -230,7 +231,7 @@ namespace { if (!hasData()) return {}; - return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, restore_settings); + return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, *restore_settings); } ContextMutablePtr context; @@ -239,7 +240,7 @@ namespace ASTs partitions; BackupPtr backup; DatabaseAndTableName table_name_in_backup; - RestoreSettings restore_settings; + RestoreSettingsPtr restore_settings; DatabasePtr database; StoragePtr storage; ASTPtr storage_create_query; @@ -304,14 +305,16 @@ namespace serializeAST(*info.create_query), serializeAST(*info.different_create_query)); } + auto restore_settings_ptr = std::make_shared(restore_settings); + RestoreTasks res; for (auto & info : databases | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, restore_settings, - /* skip_same_definition_check = */ !info.is_explicit)); + res.push_back(std::make_unique(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 (auto & info : tables | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings)); + res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings_ptr)); return res; } diff --git a/src/IO/ConcatReadBuffer.h b/src/IO/ConcatReadBuffer.h index 4ef8d04d4c9..3f44181a6e9 100644 --- a/src/IO/ConcatReadBuffer.h +++ b/src/IO/ConcatReadBuffer.h @@ -23,6 +23,12 @@ public: assert(!buffers.empty()); } + ConcatReadBuffer(std::unique_ptr buf1, std::unique_ptr buf2) : ConcatReadBuffer() + { + appendBuffer(std::move(buf1)); + appendBuffer(std::move(buf2)); + } + ConcatReadBuffer(ReadBuffer & buf1, ReadBuffer & buf2) : ConcatReadBuffer() { appendBuffer(wrapReadBufferReference(buf1)); diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index 390399e0470..01970bc5cc2 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -20,7 +20,7 @@ namespace params.open_mode = IBackup::OpenMode::WRITE; params.context = context; params.backup_info = backup_info; - params.base_backup_info = backup_settings.base_backup_info ? *backup_settings.base_backup_info : std::optional{}; + 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; @@ -33,7 +33,7 @@ namespace params.open_mode = IBackup::OpenMode::READ; params.context = context; params.backup_info = backup_info; - params.base_backup_info = restore_settings.base_backup_info ? *restore_settings.base_backup_info : std::optional{}; + params.base_backup_info = restore_settings.base_backup_info; params.password = restore_settings.password; return BackupFactory::instance().createBackup(params); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 7cf453f0e43..f22fa9cdb55 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -25,7 +25,7 @@ #include #include -#include +#include #include #include #include @@ -914,12 +914,12 @@ BackupEntries StorageLog::backup(ContextPtr context, const ASTs & partitions) { /// 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( - disk, temp_file_path, file_checker.getFileSize(data_file.path), std::nullopt, temp_dir_owner)); + std::make_unique( + disk, hardlink_file_path, file_checker.getFileSize(data_file.path), std::nullopt, temp_dir_owner)); } /// __marks.mrk @@ -927,12 +927,12 @@ BackupEntries StorageLog::backup(ContextPtr context, const ASTs & partitions) { /// 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( - disk, temp_file_path, file_checker.getFileSize(marks_file_path), std::nullopt, temp_dir_owner)); + std::make_unique( + disk, hardlink_file_path, file_checker.getFileSize(marks_file_path), std::nullopt, temp_dir_owner)); } /// sizes.json diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 88a9969fe1a..7b28149342e 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -21,8 +21,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -376,65 +376,106 @@ void StorageMemory::truncate( } +class MemoryBackupEntriesBatch : public shared_ptr_helper, public IBackupEntriesBatch +{ +private: + friend struct shared_ptr_helper; + + MemoryBackupEntriesBatch( + const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr 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(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(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 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 blocks; + UInt64 max_compress_block_size; + std::once_flag initialized_flag; + std::optional temp_dir_owner; + std::array file_paths; + std::array file_sizes; +}; + + BackupEntries StorageMemory::backup(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); - auto blocks = data.get(); - - /// We store our data in the StripeLog format. - BackupEntries backup_entries; - auto temp_dir_owner = std::make_shared(); - 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(data_file_path); - CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), context->getSettingsRef().max_compress_block_size}; - NativeWriter block_out{data_out, 0, getInMemoryMetadataPtr()->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(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(); - - /// Prepare backup entries. - backup_entries.emplace_back( - data_file_name, - std::make_unique( - data_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); - - backup_entries.emplace_back( - index_file_name, - std::make_unique( - index_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); - - backup_entries.emplace_back( - sizes_file_name, - std::make_unique( - sizes_file_path, std::nullopt, std::nullopt, temp_dir_owner)); - - return backup_entries; + return MemoryBackupEntriesBatch::create(getInMemoryMetadataPtr(), data.get(), context->getSettingsRef().max_compress_block_size) + ->getBackupEntries(); } @@ -470,7 +511,7 @@ public: auto backup_entry = backup->readFile(data_file_path); std::unique_ptr in = backup_entry->getReadBuffer(); std::optional temp_data_copy; - if (!typeid_cast(in.get())) + if (!dynamic_cast(in.get())) { temp_data_copy.emplace(); auto temp_data_copy_out = std::make_unique(temp_data_copy->path()); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 374c861ae4c..5c6af9fadc1 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -35,7 +35,7 @@ #include #include -#include +#include #include #include #include @@ -516,24 +516,24 @@ BackupEntries StorageStripeLog::backup(ContextPtr context, const ASTs & partitio { /// 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( - disk, temp_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); + std::make_unique( + disk, hardlink_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); } /// index.mrk { /// We make a copy of the data file because it can be changed later in write() or in truncate(). String index_file_name = fileName(index_file_path); - String temp_file_path = temp_dir + "/" + index_file_name; - disk->copy(index_file_path, disk, temp_file_path); + String hardlink_file_path = temp_dir + "/" + index_file_name; + disk->createHardLink(index_file_path, hardlink_file_path); backup_entries.emplace_back( index_file_name, - std::make_unique( - disk, temp_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); + std::make_unique( + disk, hardlink_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); } /// sizes.json From 18173d71fb461694b06e121d44fbf67e33285b70 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 22 Feb 2022 15:39:11 +0300 Subject: [PATCH 73/85] A corresponding file in base backup can be searched by checksum too, not only by name. --- src/Backups/ArchiveBackup.cpp | 2 +- src/Backups/ArchiveBackup.h | 2 +- src/Backups/BackupImpl.cpp | 225 +++++++++--------- src/Backups/BackupImpl.h | 9 +- src/Backups/BackupUtils.cpp | 2 +- src/Backups/DirectoryBackup.cpp | 2 +- src/Backups/DirectoryBackup.h | 2 +- src/Backups/IBackup.h | 5 +- .../test_backup_restore_new/test.py | 28 +++ 9 files changed, 162 insertions(+), 115 deletions(-) diff --git a/src/Backups/ArchiveBackup.cpp b/src/Backups/ArchiveBackup.cpp index 36148430f68..2a100867872 100644 --- a/src/Backups/ArchiveBackup.cpp +++ b/src/Backups/ArchiveBackup.cpp @@ -77,7 +77,7 @@ std::unique_ptr ArchiveBackup::readFileImpl(const String & file_name return reader->readFile(file_name); } -std::unique_ptr ArchiveBackup::addFileImpl(const String & file_name) +std::unique_ptr ArchiveBackup::writeFileImpl(const String & file_name) { /// mutex is already locked return writer->writeFile(file_name); diff --git a/src/Backups/ArchiveBackup.h b/src/Backups/ArchiveBackup.h index 8c76830977a..9649c0c1843 100644 --- a/src/Backups/ArchiveBackup.h +++ b/src/Backups/ArchiveBackup.h @@ -38,7 +38,7 @@ private: void closeImpl(bool writing_finalized_) override; bool supportsWritingInMultipleThreads() const override { return false; } std::unique_ptr readFileImpl(const String & file_name) const override; - std::unique_ptr addFileImpl(const String & file_name) override; + std::unique_ptr writeFileImpl(const String & file_name) override; const DiskPtr disk; const String path; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index d146f0394d0..c2129149934 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -176,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) @@ -192,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)); } } @@ -202,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()); } @@ -225,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(config->getString("base_backup_uuid")); file_infos.clear(); @@ -237,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); } } } @@ -304,6 +317,15 @@ UInt128 BackupImpl::getFileChecksum(const String & file_name) const return it->second.checksum; } +std::optional 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 { @@ -351,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, @@ -359,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) { @@ -369,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). @@ -391,7 +405,7 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const } -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) @@ -422,44 +436,61 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry) } std::unique_ptr read_buffer; /// We'll set that later. - UInt64 read_pos = 0; /// Current position in read_buffer. + std::optional 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 have 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; + base_exists = true; + use_base = true; + } } } @@ -467,70 +498,52 @@ 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(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(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 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}; diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index af31b71f460..d1fc3c3248c 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -34,8 +35,9 @@ public: 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 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: @@ -53,7 +55,7 @@ protected: /// Add a file to the backup. /// Low level: the function doesn't check base backup or checksums. - virtual std::unique_ptr addFileImpl(const String & file_name) = 0; + virtual std::unique_ptr writeFileImpl(const String & file_name) = 0; mutable std::mutex mutex; @@ -82,7 +84,8 @@ private: std::optional base_backup_info; std::shared_ptr base_backup; std::optional base_backup_uuid; - std::map file_infos; + std::map file_infos; /// Should be ordered alphabetically, see listFiles(). + std::unordered_map file_checksums; bool writing_finalized = false; }; diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 5e40e1dbb69..f9f9722c8d5 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -335,7 +335,7 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries { try { - backup->addFile(name, std::move(entry)); + backup->writeFile(name, std::move(entry)); } catch (...) { diff --git a/src/Backups/DirectoryBackup.cpp b/src/Backups/DirectoryBackup.cpp index 2adee160180..dc4d098dbe9 100644 --- a/src/Backups/DirectoryBackup.cpp +++ b/src/Backups/DirectoryBackup.cpp @@ -72,7 +72,7 @@ std::unique_ptr DirectoryBackup::readFileImpl(const String & file_na return disk->readFile(file_path); } -std::unique_ptr DirectoryBackup::addFileImpl(const String & file_name) +std::unique_ptr DirectoryBackup::writeFileImpl(const String & file_name) { String file_path = path + file_name; disk->createDirectories(fs::path(file_path).parent_path()); diff --git a/src/Backups/DirectoryBackup.h b/src/Backups/DirectoryBackup.h index 0b6dd1361d1..7d9b5cc4557 100644 --- a/src/Backups/DirectoryBackup.h +++ b/src/Backups/DirectoryBackup.h @@ -27,7 +27,7 @@ private: void openImpl(OpenMode open_mode_) override; void closeImpl(bool writing_finalized_) override; std::unique_ptr readFileImpl(const String & file_name) const override; - std::unique_ptr addFileImpl(const String & file_name) override; + std::unique_ptr writeFileImpl(const String & file_name) override; DiskPtr disk; String path; diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index 7e9503a7754..249f4414388 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -61,11 +61,14 @@ 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 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; + 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; diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index dcd6c241801..e3f0ab178cc 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1,5 +1,6 @@ import pytest import re +import os.path from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -37,6 +38,11 @@ def new_backup_name(): return f"Disk('backups', '{backup_id_counter}/')" +def get_backup_dir(backup_name): + counter = int(backup_name.split(',')[1].strip("')/ ")) + return os.path.join(instance.path, f'backups/{counter}') + + @pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"]) def test_restore_table(engine): backup_name = new_backup_name() @@ -110,6 +116,27 @@ def test_incremental_backup(): assert instance.query("SELECT count(), sum(x) FROM test.table2") == "102\t5081\n" +def test_incremental_backup_after_renaming_table(): + backup_name = new_backup_name() + incremental_backup_name = new_backup_name() + create_and_fill_table() + + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + instance.query("RENAME TABLE test.table TO test.table2") + instance.query(f"BACKUP TABLE test.table2 TO {incremental_backup_name} SETTINGS base_backup = {backup_name}") + + # Files in a base backup can be searched by checksum, so an incremental backup with a renamed table actually + # contains only its changed metadata. + assert os.path.isdir(os.path.join(get_backup_dir(backup_name), 'metadata')) == True + assert os.path.isdir(os.path.join(get_backup_dir(backup_name), 'data')) == True + assert os.path.isdir(os.path.join(get_backup_dir(incremental_backup_name), 'metadata')) == True + assert os.path.isdir(os.path.join(get_backup_dir(incremental_backup_name), 'data')) == False + + instance.query("DROP TABLE test.table2") + instance.query(f"RESTORE TABLE test.table2 FROM {incremental_backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table2") == "100\t4950\n" + + def test_backup_not_found_or_already_exists(): backup_name = new_backup_name() @@ -155,6 +182,7 @@ def test_zip_archive(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") + assert os.path.isfile(os.path.join(os.path.join(instance.path, 'backups/archive.zip'))) instance.query("DROP TABLE test.table") assert instance.query("EXISTS test.table") == "0\n" From ce25afb2e940d4619bb9b7972753cfd7cf9bba99 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 22 Feb 2022 16:31:50 +0300 Subject: [PATCH 74/85] Storages and databases are hollow by default now. --- src/Backups/BackupUtils.cpp | 8 ++++---- src/Backups/RestoreUtils.cpp | 2 +- src/Databases/DatabaseMemory.h | 3 +++ src/Databases/DatabaseOrdinary.h | 3 +++ src/Databases/IDatabase.h | 2 +- src/Storages/IStorage.cpp | 6 +++--- src/Storages/IStorage.h | 6 +++--- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeData.h | 7 +++++-- src/Storages/StorageLog.cpp | 4 ++-- src/Storages/StorageLog.h | 5 +++-- src/Storages/StorageMemory.cpp | 4 ++-- src/Storages/StorageMemory.h | 5 +++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 4 ++-- src/Storages/StorageStripeLog.h | 5 +++-- 17 files changed, 45 insertions(+), 33 deletions(-) diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index f9f9722c8d5..1d78be53ac6 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -97,7 +97,7 @@ namespace res.push_back(makeBackupEntryForMetadata(*info.create_query)); if (info.has_data) { - auto data_backup = info.storage->backup(context, info.partitions); + auto data_backup = info.storage->backupData(context, info.partitions); if (!data_backup.empty()) { String data_path = getDataPathInBackup(*info.create_query); @@ -129,7 +129,7 @@ namespace const auto & database = table_.first; const auto & storage = table_.second; - if (database->hasHollowBackup()) + if (!database->hasTablesToBackup()) throw Exception( ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} because it's contained in a hollow database (engine: {})", @@ -144,7 +144,7 @@ namespace /// Make a create query for this table. auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); - bool has_data = !storage->hasHollowBackup() && !backup_settings.structure_only; + bool has_data = storage->hasDataToBackup() && !backup_settings.structure_only; if (has_data) { /// We check for SELECT privilege only if we're going to read data from the table. @@ -220,7 +220,7 @@ namespace } /// Backup tables in this database. - if (!database_->hasHollowBackup()) + if (database_->hasTablesToBackup()) { for (auto it = database_->getTablesIterator(context); it->isValid(); it->next()) { diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index a94690ea881..0878fbbf233 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -231,7 +231,7 @@ namespace { if (!hasData()) return {}; - return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, *restore_settings); + return storage->restoreData(context, partitions, backup, data_path_in_backup, *restore_settings); } ContextMutablePtr context; diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index b854d9be1f3..87fae115b59 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -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; diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index 982be2024ce..2144f874b03 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -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, diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 5d2a1289833..51d4b8bb6b1 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -331,7 +331,7 @@ public: /// 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 hasHollowBackup() const { return false; } + virtual bool hasTablesToBackup() const { return false; } virtual ~IDatabase() = default; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index ab27f263b8e..88ddde32d83 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -216,14 +216,14 @@ bool IStorage::isStaticStorage() const return false; } -BackupEntries IStorage::backup(ContextPtr, const ASTs &) +BackupEntries IStorage::backupData(ContextPtr, const ASTs &) { throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } -RestoreTaskPtr IStorage::restoreFromBackup(ContextMutablePtr, const ASTs &, const BackupPtr &, const String &, const StorageRestoreSettings &) +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 diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 2bfae021da1..17e9e55455c 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -219,13 +219,13 @@ public: NameDependencies getDependentViewsByColumn(ContextPtr context) const; /// Returns true if the backup is hollow, which means it doesn't contain any data. - virtual bool hasHollowBackup() const { return false; } + virtual bool hasDataToBackup() const { return false; } /// Prepares entries to backup data of the storage. - virtual BackupEntries backup(ContextPtr context, const ASTs & partitions); + virtual BackupEntries backupData(ContextPtr context, const ASTs & partitions); /// Extract data from the backup and put it to the storage. - virtual RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings); + 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. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8554426b073..f66586b121a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3630,7 +3630,7 @@ Pipe MergeTreeData::alterPartition( } -BackupEntries MergeTreeData::backup(ContextPtr local_context, const ASTs & partitions) +BackupEntries MergeTreeData::backupData(ContextPtr local_context, const ASTs & partitions) { DataPartsVector data_parts; if (partitions.empty()) @@ -3789,9 +3789,9 @@ private: }; -RestoreTaskPtr MergeTreeData::restoreDataPartsFromBackup(const std::unordered_set & partition_ids, - const BackupPtr & backup, const String & data_path_in_backup, - SimpleIncrement * increment) +RestoreTaskPtr MergeTreeData::restoreDataParts(const std::unordered_set & partition_ids, + const BackupPtr & backup, const String & data_path_in_backup, + SimpleIncrement * increment) { return std::make_unique( std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup, partition_ids, increment); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fd9427d04fd..fb839e5a0dd 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -674,12 +674,15 @@ 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(ContextPtr context, const ASTs & partitions) 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. - RestoreTaskPtr restoreDataPartsFromBackup( + RestoreTaskPtr restoreDataParts( const std::unordered_set & partition_ids, const BackupPtr & backup, const String & data_path_in_backup, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index f22fa9cdb55..11116780734 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -888,7 +888,7 @@ IStorage::ColumnSizeByName StorageLog::getColumnSizes() const } -BackupEntries StorageLog::backup(ContextPtr context, const ASTs & partitions) +BackupEntries StorageLog::backupData(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); @@ -1063,7 +1063,7 @@ private: ContextMutablePtr context; }; -RestoreTaskPtr StorageLog::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) +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()); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 6767d40583c..b9255c16f2b 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -52,8 +52,9 @@ public: bool supportsSubcolumns() const override { return true; } ColumnSizeByName getColumnSizes() const override; - BackupEntries backup(ContextPtr context, const ASTs & partitions) override; - RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) 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), diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 7b28149342e..2231ee5da1d 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -469,7 +469,7 @@ private: }; -BackupEntries StorageMemory::backup(ContextPtr context, const ASTs & partitions) +BackupEntries StorageMemory::backupData(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); @@ -552,7 +552,7 @@ private: }; -RestoreTaskPtr StorageMemory::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) +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()); diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index c530cbaa31b..20f47828846 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -66,8 +66,9 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; - BackupEntries backup(ContextPtr context, const ASTs & partitions) override; - RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) 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; std::optional totalRows(const Settings &) const override; std::optional totalBytes(const Settings &) const override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index af4fa1d9b4e..7f4c3deca37 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1644,9 +1644,9 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ } -RestoreTaskPtr StorageMergeTree::restoreFromBackup(ContextMutablePtr local_context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) +RestoreTaskPtr StorageMergeTree::restoreData(ContextMutablePtr local_context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) { - return restoreDataPartsFromBackup(getPartitionIDsFromQuery(partitions, local_context), backup, data_path_in_backup, &increment); + return restoreDataParts(getPartitionIDsFromQuery(partitions, local_context), backup, data_path_in_backup, &increment); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 6acd19f73eb..a1fc310d912 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -97,7 +97,7 @@ public: CheckResults checkData(const ASTPtr & query, ContextPtr context) override; - RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; + RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 5c6af9fadc1..274789f012b 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -491,7 +491,7 @@ void StorageStripeLog::saveFileSizes(const WriteLock & /* already locked for wri } -BackupEntries StorageStripeLog::backup(ContextPtr context, const ASTs & partitions) +BackupEntries StorageStripeLog::backupData(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); @@ -637,7 +637,7 @@ private: }; -RestoreTaskPtr StorageStripeLog::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) +RestoreTaskPtr StorageStripeLog::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()); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 9cd9bc2c165..223b662d13c 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -52,8 +52,9 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override; - BackupEntries backup(ContextPtr context, const ASTs & partitions) override; - RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) 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: StorageStripeLog( From 15271c9b132909d6ec38dfc29ce90172516da084 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 21 Mar 2022 10:45:53 +0100 Subject: [PATCH 75/85] mysqlxx PoolWithFailover style fix --- src/Common/mysqlxx/PoolWithFailover.cpp | 32 ++++++++++++++++--------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/src/Common/mysqlxx/PoolWithFailover.cpp b/src/Common/mysqlxx/PoolWithFailover.cpp index 208ad10b63b..9f23f824f55 100644 --- a/src/Common/mysqlxx/PoolWithFailover.cpp +++ b/src/Common/mysqlxx/PoolWithFailover.cpp @@ -127,7 +127,13 @@ 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> error_detail; + + struct ErrorDetail { + std::string description; + int code; + }; + + std::unordered_map replica_name_to_error_detail; for (size_t try_no = 0; try_no < max_tries; ++try_no) { @@ -161,15 +167,15 @@ 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())) + auto [it, inserted] = replica_name_to_error_detail.emplace(pool->getDescription(), ErrorDetail{e.displayText(), e.code()}); + + if (!inserted) { - error_detail[pool->getDescription()] = {e.displayText(), e.code()}; - } - else - { - error_detail.insert({pool->getDescription(), {e.displayText(), e.code()}}); + auto & error_detail = it->second; + error_detail.description = e.displayText(); + error_detail.code = e.code(); } + continue; } @@ -189,15 +195,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())) + + auto error_detail_it = replica_name_to_error_detail.find(((*jt)->getDescription())); + if (error_detail_it != replica_name_to_error_detail.end()) { - std::tuple error_and_code = error_detail[(*jt)->getDescription()]; - message << ", ERROR " << std::get<1>(error_and_code) << " : " << std::get<0>(error_and_code); + auto & error_detail = error_detail_it->second; + message << ", ERROR " << error_detail.code << " : " << error_detail.description; } } + } throw Poco::Exception(message.str()); } From 5dcbc07d22c3b6f3db88db1a70ed1fb47bde1483 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 21 Mar 2022 10:49:46 +0100 Subject: [PATCH 76/85] Fixed style check --- src/Common/mysqlxx/PoolWithFailover.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/mysqlxx/PoolWithFailover.cpp b/src/Common/mysqlxx/PoolWithFailover.cpp index 9f23f824f55..d7c381f327b 100644 --- a/src/Common/mysqlxx/PoolWithFailover.cpp +++ b/src/Common/mysqlxx/PoolWithFailover.cpp @@ -128,7 +128,8 @@ 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; - struct ErrorDetail { + struct ErrorDetail + { std::string description; int code; }; From 51fc556fcd8ae5285956098042f284eaf9c6914d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 22 Feb 2022 20:05:52 +0300 Subject: [PATCH 77/85] Implement BACKUP and RESTORE for MatViews. --- src/Backups/ArchiveBackup.cpp | 7 ++++++ src/Backups/BackupImpl.cpp | 3 +-- src/Backups/BackupUtils.cpp | 24 ++++++++++++------- src/Backups/DDLCompareUtils.cpp | 8 +++++-- src/Backups/DDLRenamingVisitor.cpp | 9 ++----- src/Backups/DDLRenamingVisitor.h | 2 +- src/Backups/IBackup.h | 1 + src/Backups/RestoreUtils.cpp | 11 +++++---- src/Common/FileChecker.cpp | 3 +-- .../hasRegisteredArchiveFileExtension.h | 2 +- src/Storages/StorageMaterializedView.cpp | 17 +++++++++++++ src/Storages/StorageMaterializedView.h | 4 ++++ src/Storages/StorageMemory.cpp | 3 ++- .../test_backup_restore_new/test.py | 12 ++++++++++ 14 files changed, 77 insertions(+), 29 deletions(-) diff --git a/src/Backups/ArchiveBackup.cpp b/src/Backups/ArchiveBackup.cpp index 2a100867872..5d4837fff2e 100644 --- a/src/Backups/ArchiveBackup.cpp +++ b/src/Backups/ArchiveBackup.cpp @@ -10,6 +10,13 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + ArchiveBackup::ArchiveBackup( const String & backup_name_, const DiskPtr & disk_, diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index c2129149934..e4fc894837a 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -488,7 +488,6 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) /// but the entry itself has been moved or renamed. base_size = size; base_checksum = *checksum; - base_exists = true; use_base = true; } } diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 1d78be53ac6..9e2da6f34f8 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -21,6 +21,7 @@ namespace ErrorCodes extern const int CANNOT_BACKUP_TABLE; extern const int CANNOT_BACKUP_DATABASE; extern const int BACKUP_IS_EMPTY; + extern const int LOGICAL_ERROR; } namespace @@ -80,7 +81,7 @@ namespace BackupEntries makeBackupEntries() const { /// Check that there are not `different_create_query`. (If it's set it means error.) - for (auto & info : databases | boost::adaptors::map_values) + for (const auto & info : databases | boost::adaptors::map_values) { if (info.different_create_query) throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, @@ -89,10 +90,10 @@ namespace } BackupEntries res; - for (auto & info : databases | boost::adaptors::map_values) + for (const auto & info : databases | boost::adaptors::map_values) res.push_back(makeBackupEntryForMetadata(*info.create_query)); - for (auto & info : tables | boost::adaptors::map_values) + for (const auto & info : tables | boost::adaptors::map_values) { res.push_back(makeBackupEntryForMetadata(*info.create_query)); if (info.has_data) @@ -142,7 +143,7 @@ namespace throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} twice", formatTableNameOrTemporaryTableName(new_table_name)); /// Make a create query for this table. - auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); + auto create_query = prepareCreateQueryForBackup(database->getCreateTableQuery(table_name_.second, context)); bool has_data = storage->hasDataToBackup() && !backup_settings.structure_only; if (has_data) @@ -165,7 +166,7 @@ namespace if (!databases.contains(new_table_name.first)) { /// Add a create query to backup the database if we haven't done it yet. - auto create_db_query = renameInCreateQuery(database->getCreateDatabaseQuery()); + auto create_db_query = prepareCreateQueryForBackup(database->getCreateDatabaseQuery()); create_db_query->setDatabase(new_table_name.first); CreateDatabaseInfo info_db; @@ -181,7 +182,7 @@ namespace auto & info_db = databases[new_table_name.first]; if (!info_db.is_explicit && (info_db.original_name != table_name_.first) && !info_db.different_create_query) { - auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); + auto create_db_query = prepareCreateQueryForBackup(table_.first->getCreateDatabaseQuery()); create_db_query->setDatabase(new_table_name.first); if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query)) info_db.different_create_query = create_db_query; @@ -210,7 +211,7 @@ namespace if (!isSystemOrTemporaryDatabase(database_name_)) { /// Make a create query for this database. - auto create_db_query = renameInCreateQuery(database_->getCreateDatabaseQuery()); + auto create_db_query = prepareCreateQueryForBackup(database_->getCreateDatabaseQuery()); CreateDatabaseInfo info_db; info_db.create_query = create_db_query; @@ -245,9 +246,14 @@ namespace } /// Do renaming in the create query according to the renaming config. - std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const + std::shared_ptr prepareCreateQueryForBackup(const ASTPtr & ast) const { - return typeid_cast>(::DB::renameInCreateQuery(ast, context, renaming_settings)); + ASTPtr query = ast; + ::DB::renameInCreateQuery(query, context, renaming_settings); + auto create_query = typeid_cast>(query); + create_query->uuid = UUIDHelpers::Nil; + create_query->to_inner_uuid = UUIDHelpers::Nil; + return create_query; } static bool isSystemOrTemporaryDatabase(const String & database_name) diff --git a/src/Backups/DDLCompareUtils.cpp b/src/Backups/DDLCompareUtils.cpp index 7c857cff7d5..625a0befe63 100644 --- a/src/Backups/DDLCompareUtils.cpp +++ b/src/Backups/DDLCompareUtils.cpp @@ -22,10 +22,13 @@ namespace return clone; }; - /// Remove UUID. + /// 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; @@ -45,7 +48,8 @@ bool areTableDefinitionsSame(const IAST & table1, const IAST & table2) if (!ast2 || !ast2->table) return false; - if ((ast1->uuid != ast2->uuid) || (ast1->if_not_exists != ast2->if_not_exists)) + 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); diff --git a/src/Backups/DDLRenamingVisitor.cpp b/src/Backups/DDLRenamingVisitor.cpp index 9d87d18e142..bcdfb32c29f 100644 --- a/src/Backups/DDLRenamingVisitor.cpp +++ b/src/Backups/DDLRenamingVisitor.cpp @@ -59,8 +59,6 @@ namespace else throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty"); - create.uuid = UUIDHelpers::Nil; - 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}); @@ -365,19 +363,16 @@ void DDLRenamingVisitor::visit(ASTPtr & ast, const Data & data) visitDictionary(*dictionary, data); } -ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings) +void renameInCreateQuery(ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings) { - auto new_ast = ast->clone(); try { DDLRenamingVisitor::Data data{renaming_settings, global_context}; - DDLRenamingVisitor::Visitor{data}.visit(new_ast); - return new_ast; + DDLRenamingVisitor::Visitor{data}.visit(ast); } catch (...) { tryLogCurrentException("Backup", "Error while renaming in AST"); - return ast; } } diff --git a/src/Backups/DDLRenamingVisitor.h b/src/Backups/DDLRenamingVisitor.h index e74f68a4033..b624fffc267 100644 --- a/src/Backups/DDLRenamingVisitor.h +++ b/src/Backups/DDLRenamingVisitor.h @@ -40,7 +40,7 @@ private: /// Changes names in AST according to the renaming settings. -ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & config); +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 diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index 249f4414388..5e13ff88575 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index 0878fbbf233..8073b6d0818 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -297,7 +297,7 @@ namespace RestoreTasks makeTasks() const { /// Check that there are not `different_create_query`. (If it's set it means error.) - for (auto & info : databases | boost::adaptors::map_values) + for (const auto & info : databases | boost::adaptors::map_values) { if (info.different_create_query) throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, @@ -308,12 +308,12 @@ namespace auto restore_settings_ptr = std::make_shared(restore_settings); RestoreTasks res; - for (auto & info : databases | boost::adaptors::map_values) + for (const auto & info : databases | boost::adaptors::map_values) res.push_back(std::make_unique(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 (auto & info : tables | boost::adaptors::map_values) + for (const auto & info : tables | boost::adaptors::map_values) res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings_ptr)); return res; @@ -487,7 +487,10 @@ namespace /// Do renaming in the create query according to the renaming config. std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const { - return typeid_cast>(::DB::renameInCreateQuery(ast, context, renaming_settings)); + ASTPtr query = ast; + ::DB::renameInCreateQuery(query, context, renaming_settings); + auto create_query = typeid_cast>(query); + return create_query; } static bool isSystemOrTemporaryDatabase(const String & database_name) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index ef90e31982c..cdc928ea11e 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include @@ -161,7 +160,7 @@ void FileChecker::load() if (!fileReallyExists(files_info_path)) return; - std::unique_ptr in = disk ? disk->readFile(files_info_path) : createReadBufferFromFileBase(files_info_path, {}); + std::unique_ptr in = disk ? disk->readFile(files_info_path) : std::make_unique(files_info_path); WriteBufferFromOwnString out; /// The JSON library does not support whitespace. We delete them. Inefficient. diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.h b/src/IO/Archives/hasRegisteredArchiveFileExtension.h index cff2a0a8bec..cab938aa0b4 100644 --- a/src/IO/Archives/hasRegisteredArchiveFileExtension.h +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.h @@ -6,7 +6,7 @@ namespace DB { -/// Returns true if a specified path has one of the registed file extensions for an archive. +/// Returns true if a specified path has one of the registered file extensions for an archive. bool hasRegisteredArchiveFileExtension(const String & path); } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 008b42e3299..0c79c31eb7a 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -25,6 +25,9 @@ #include #include +#include +#include + 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) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 838f5278aa9..41c97fbc4d8 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -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(); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 2231ee5da1d..30be297194a 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -24,8 +24,8 @@ #include #include #include +#include #include -#include #include @@ -35,6 +35,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int NOT_IMPLEMENTED; } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index e3f0ab178cc..274c2639a9b 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -99,6 +99,18 @@ def test_backup_table_under_another_name(): assert instance.query("SELECT count(), sum(x) FROM test.table2") == "100\t4950\n" +def test_materialized_view(): + backup_name = new_backup_name() + instance.query("CREATE MATERIALIZED VIEW mv_1(x UInt8) ENGINE=MergeTree ORDER BY tuple() POPULATE AS SELECT 1 AS x") + + instance.query(f"BACKUP TABLE mv_1 TO {backup_name}") + instance.query("DROP TABLE mv_1") + instance.query(f"RESTORE TABLE mv_1 FROM {backup_name}") + + assert instance.query("SELECT * FROM mv_1") == "1\n" + instance.query("DROP TABLE mv_1") + + def test_incremental_backup(): backup_name = new_backup_name() incremental_backup_name = new_backup_name() From c379b9e0cf788065571246b139cea6a0c4b5f9b9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 21 Mar 2022 19:37:54 +0800 Subject: [PATCH 78/85] revert client nit fix and add comment --- src/Client/ClientBase.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index fdcc98f016f..56070267114 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1977,10 +1977,11 @@ void ClientBase::readArguments( in_external_group = true; external_tables_arguments.emplace_back(Arguments{""}); } - /// Options with value after equal sign. + /// Options with value after equal sign. It means startsWith. else if (in_external_group - && (arg == "--file="sv || arg == "--name="sv || arg == "--format="sv - || arg == "--structure="sv || arg == "--types="sv)) + && (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=")))) { external_tables_arguments.back().emplace_back(arg); } From 623ae212e42dcdf30be1d32806c3c15469dfa53b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 21 Mar 2022 19:54:30 +0800 Subject: [PATCH 79/85] Use string view instead of C-str --- src/Client/ClientBase.cpp | 54 +++++++++++++++++++-------------------- 1 file changed, 26 insertions(+), 28 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 56070267114..6bffc6be8ff 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1970,25 +1970,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. It means startsWith. - 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=")))) + /// Options with value after equal sign. + 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 +2003,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 +2017,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 +2057,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 +2086,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); From 8adf4ca4993d8bc5f1e0a1379da5964bad4a7a7b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 21 Mar 2022 13:32:18 +0100 Subject: [PATCH 80/85] Explicitly set allow_introspection_functions to 0 I spent dozen of time trying to understand why this test doesn't work locally --- tests/queries/0_stateless/02161_addressToLineWithInlines.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02161_addressToLineWithInlines.sql b/tests/queries/0_stateless/02161_addressToLineWithInlines.sql index 12cae6af189..b6b497b4b55 100644 --- a/tests/queries/0_stateless/02161_addressToLineWithInlines.sql +++ b/tests/queries/0_stateless/02161_addressToLineWithInlines.sql @@ -1,6 +1,6 @@ -- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-cpu-aarch64 - +SET allow_introspection_functions = 0; SELECT addressToLineWithInlines(1); -- { serverError 446 } SET allow_introspection_functions = 1; From 1cb8faf36c6aacf9969ac41a3e88001b229f2d16 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 21 Mar 2022 16:35:18 +0100 Subject: [PATCH 81/85] Fixed style --- src/Common/mysqlxx/PoolWithFailover.cpp | 19 ++++++------------- 1 file changed, 6 insertions(+), 13 deletions(-) diff --git a/src/Common/mysqlxx/PoolWithFailover.cpp b/src/Common/mysqlxx/PoolWithFailover.cpp index d7c381f327b..36dd713d454 100644 --- a/src/Common/mysqlxx/PoolWithFailover.cpp +++ b/src/Common/mysqlxx/PoolWithFailover.cpp @@ -130,8 +130,8 @@ PoolWithFailover::Entry PoolWithFailover::get() struct ErrorDetail { - std::string description; int code; + std::string description; }; std::unordered_map replica_name_to_error_detail; @@ -168,14 +168,7 @@ PoolWithFailover::Entry PoolWithFailover::get() } app.logger().warning("Connection to " + pool->getDescription() + " failed: " + e.displayText()); - auto [it, inserted] = replica_name_to_error_detail.emplace(pool->getDescription(), ErrorDetail{e.displayText(), e.code()}); - - if (!inserted) - { - auto & error_detail = it->second; - error_detail.description = e.displayText(); - error_detail.code = e.code(); - } + replica_name_to_error_detail.insert_or_assign(pool->getDescription(), ErrorDetail{e.code(), e.displayText()}); continue; } @@ -201,11 +194,11 @@ PoolWithFailover::Entry PoolWithFailover::get() { message << (it == replicas_by_priority.begin() && jt == it->second.begin() ? "" : ", ") << (*jt)->getDescription(); - auto error_detail_it = replica_name_to_error_detail.find(((*jt)->getDescription())); - if (error_detail_it != replica_name_to_error_detail.end()) + if (auto error_detail_it = replica_name_to_error_detail.find(((*jt)->getDescription())); + error_detail_it != replica_name_to_error_detail.end()) { - auto & error_detail = error_detail_it->second; - message << ", ERROR " << error_detail.code << " : " << error_detail.description; + const auto & [code, description] = error_detail_it->second; + message << ", ERROR " << code << " : " << description; } } } From 64e0b0cf8ffcd42e0ebead46229f24d9b838da54 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 21 Mar 2022 18:43:48 +0300 Subject: [PATCH 82/85] update issue templates --- .github/ISSUE_TEMPLATE/10_question.md | 2 +- .github/ISSUE_TEMPLATE/50_build-issue.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/10_question.md b/.github/ISSUE_TEMPLATE/10_question.md index a112b9599d5..5b3d00a3180 100644 --- a/.github/ISSUE_TEMPLATE/10_question.md +++ b/.github/ISSUE_TEMPLATE/10_question.md @@ -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. diff --git a/.github/ISSUE_TEMPLATE/50_build-issue.md b/.github/ISSUE_TEMPLATE/50_build-issue.md index a358575cd7c..9b05fbbdd13 100644 --- a/.github/ISSUE_TEMPLATE/50_build-issue.md +++ b/.github/ISSUE_TEMPLATE/50_build-issue.md @@ -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** From dba82a773227f0c1d8025a092fa80e14cb32cfc6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 21 Mar 2022 17:01:39 +0100 Subject: [PATCH 83/85] fix show create for information_schema --- src/Storages/System/attachInformationSchemaTables.cpp | 2 ++ .../02206_information_schema_show_database.reference | 3 +++ .../0_stateless/02206_information_schema_show_database.sql | 2 ++ 3 files changed, 7 insertions(+) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 68a1eac305e..61a91685324 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -32,6 +32,8 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d auto & ast_create = ast->as(); assert(view_name == ast_create.getTable()); + ast_create.attach = false; + ast_create.setDatabase(database.getDatabaseName()); if (is_uppercase) ast_create.setTable(Poco::toUpper(view_name)); diff --git a/tests/queries/0_stateless/02206_information_schema_show_database.reference b/tests/queries/0_stateless/02206_information_schema_show_database.reference index 551186fa0ab..af437aca989 100644 --- a/tests/queries/0_stateless/02206_information_schema_show_database.reference +++ b/tests/queries/0_stateless/02206_information_schema_show_database.reference @@ -1 +1,4 @@ CREATE DATABASE INFORMATION_SCHEMA\nENGINE = Memory +CREATE VIEW INFORMATION_SCHEMA.COLUMNS\n(\n `table_catalog` String,\n `table_schema` String,\n `table_name` String,\n `column_name` String,\n `ordinal_position` UInt64,\n `column_default` String,\n `is_nullable` UInt8,\n `data_type` String,\n `character_maximum_length` Nullable(UInt64),\n `character_octet_length` Nullable(UInt64),\n `numeric_precision` Nullable(UInt64),\n `numeric_precision_radix` Nullable(UInt64),\n `numeric_scale` Nullable(UInt64),\n `datetime_precision` Nullable(UInt64),\n `character_set_catalog` Nullable(String),\n `character_set_schema` Nullable(String),\n `character_set_name` Nullable(String),\n `collation_catalog` Nullable(String),\n `collation_schema` Nullable(String),\n `collation_name` Nullable(String),\n `domain_catalog` Nullable(String),\n `domain_schema` Nullable(String),\n `domain_name` Nullable(String),\n `column_comment` String,\n `column_type` String,\n `TABLE_CATALOG` String ALIAS table_catalog,\n `TABLE_SCHEMA` String ALIAS table_schema,\n `TABLE_NAME` String ALIAS table_name,\n `COLUMN_NAME` String ALIAS column_name,\n `ORDINAL_POSITION` UInt64 ALIAS ordinal_position,\n `COLUMN_DEFAULT` String ALIAS column_default,\n `IS_NULLABLE` UInt8 ALIAS is_nullable,\n `DATA_TYPE` String ALIAS data_type,\n `CHARACTER_MAXIMUM_LENGTH` Nullable(UInt64) ALIAS character_maximum_length,\n `CHARACTER_OCTET_LENGTH` Nullable(UInt64) ALIAS character_octet_length,\n `NUMERIC_PRECISION` Nullable(UInt64) ALIAS numeric_precision,\n `NUMERIC_PRECISION_RADIX` Nullable(UInt64) ALIAS numeric_precision_radix,\n `NUMERIC_SCALE` Nullable(UInt64) ALIAS numeric_scale,\n `DATETIME_PRECISION` Nullable(UInt64) ALIAS datetime_precision,\n `CHARACTER_SET_CATALOG` Nullable(String) ALIAS character_set_catalog,\n `CHARACTER_SET_SCHEMA` Nullable(String) ALIAS character_set_schema,\n `CHARACTER_SET_NAME` Nullable(String) ALIAS character_set_name,\n `COLLATION_CATALOG` Nullable(String) ALIAS collation_catalog,\n `COLLATION_SCHEMA` Nullable(String) ALIAS collation_schema,\n `COLLATION_NAME` Nullable(String) ALIAS collation_name,\n `DOMAIN_CATALOG` Nullable(String) ALIAS domain_catalog,\n `DOMAIN_SCHEMA` Nullable(String) ALIAS domain_schema,\n `DOMAIN_NAME` Nullable(String) ALIAS domain_name,\n `COLUMN_COMMENT` String ALIAS column_comment,\n `COLUMN_TYPE` String ALIAS column_type\n) AS\nSELECT\n database AS table_catalog,\n database AS table_schema,\n table AS table_name,\n name AS column_name,\n position AS ordinal_position,\n default_expression AS column_default,\n type LIKE \'Nullable(%)\' AS is_nullable,\n type AS data_type,\n character_octet_length AS character_maximum_length,\n character_octet_length,\n numeric_precision,\n numeric_precision_radix,\n numeric_scale,\n datetime_precision,\n NULL AS character_set_catalog,\n NULL AS character_set_schema,\n NULL AS character_set_name,\n NULL AS collation_catalog,\n NULL AS collation_schema,\n NULL AS collation_name,\n NULL AS domain_catalog,\n NULL AS domain_schema,\n NULL AS domain_name,\n comment AS column_comment,\n type AS column_type\nFROM system.columns +CREATE VIEW INFORMATION_SCHEMA.TABLES (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables +CREATE VIEW information_schema.tables (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables diff --git a/tests/queries/0_stateless/02206_information_schema_show_database.sql b/tests/queries/0_stateless/02206_information_schema_show_database.sql index de5ca495e2e..eced525d58e 100644 --- a/tests/queries/0_stateless/02206_information_schema_show_database.sql +++ b/tests/queries/0_stateless/02206_information_schema_show_database.sql @@ -1 +1,3 @@ SHOW CREATE DATABASE INFORMATION_SCHEMA; +SHOW CREATE INFORMATION_SCHEMA.COLUMNS; +SELECT create_table_query FROM system.tables WHERE database ILIKE 'INFORMATION_SCHEMA' AND table ILIKE 'TABLES'; From ef5062ec43eb67bd04776ebf929a5e0df0da8bdc Mon Sep 17 00:00:00 2001 From: Sergio Tulentsev Date: Mon, 21 Mar 2022 17:52:30 +0000 Subject: [PATCH 84/85] docs: Replace some usages of "rows" which are meant to be "strings" The wrong term was confusing in the description of the CSV format. --- docs/en/interfaces/formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 84865221711..a7066fca087 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -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. From 552106f2e7044f862e8759d62f7b85e74dbc2c8b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 21 Mar 2022 20:09:08 +0100 Subject: [PATCH 85/85] fix style check --- .../0_stateless/02206_information_schema_show_database.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02206_information_schema_show_database.sql b/tests/queries/0_stateless/02206_information_schema_show_database.sql index eced525d58e..91a8a0d1dea 100644 --- a/tests/queries/0_stateless/02206_information_schema_show_database.sql +++ b/tests/queries/0_stateless/02206_information_schema_show_database.sql @@ -1,3 +1,3 @@ SHOW CREATE DATABASE INFORMATION_SCHEMA; SHOW CREATE INFORMATION_SCHEMA.COLUMNS; -SELECT create_table_query FROM system.tables WHERE database ILIKE 'INFORMATION_SCHEMA' AND table ILIKE 'TABLES'; +SELECT create_table_query FROM system.tables WHERE database ILIKE 'INFORMATION_SCHEMA' AND table ILIKE 'TABLES'; -- supress style check: database = currentDatabase()