From 1d83c596f86d98738e676f58f4e63f675f54a9a0 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Fri, 5 Mar 2021 09:50:26 +0000 Subject: [PATCH 001/584] 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 002/584] 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 003/584] 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 004/584] 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 005/584] 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 006/584] 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 77c22dce3c63436a2b241cf5e144b4f6cafebcd8 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Mon, 18 Oct 2021 18:53:42 +0800 Subject: [PATCH 007/584] Zookeeper load balancing settings --- src/Common/ZooKeeper/ZooKeeper.cpp | 95 +++++++++++++++++++++++++---- src/Common/ZooKeeper/ZooKeeper.h | 37 ++++++++++- src/Core/SettingsEnums.cpp | 8 +++ src/Core/SettingsEnums.h | 19 ++++++ tests/config/config.d/zookeeper.xml | 2 + 5 files changed, 145 insertions(+), 16 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 3d505c088db..65295a96d3e 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -5,15 +5,16 @@ #include #include -#include #include #include -#include +#include #include #include +#include #include +#include #define ZOOKEEPER_CONNECTION_TIMEOUT_MS 1000 @@ -47,7 +48,7 @@ static void check(Coordination::Error code, const std::string & path) void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, - int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_) + int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, ZooKeeperLoadBalancing zookeeper_load_balancing_) { log = &Poco::Logger::get("ZooKeeper"); hosts = hosts_; @@ -56,6 +57,7 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ operation_timeout_ms = operation_timeout_ms_; chroot = chroot_; implementation = implementation_; + zookeeper_load_balancing = zookeeper_load_balancing_; if (implementation == "zookeeper") { @@ -65,14 +67,13 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ Coordination::ZooKeeper::Nodes nodes; nodes.reserve(hosts.size()); - Strings shuffled_hosts = hosts; /// Shuffle the hosts to distribute the load among ZooKeeper nodes. - pcg64 generator(randomSeed()); - std::shuffle(shuffled_hosts.begin(), shuffled_hosts.end(), generator); + std::vector shuffled_hosts = shuffleHosts(); bool dns_error = false; - for (auto & host_string : shuffled_hosts) + for (auto & host : shuffled_hosts) { + auto & host_string = host.host; try { bool secure = bool(startsWith(host_string, "secure://")); @@ -153,23 +154,85 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ } } +std::vector ZooKeeper::shuffleHosts() const +{ + std::vector hostname_differences; + hostname_differences.resize(hosts.size()); + String local_hostname = getFQDNOrHostName(); + for (size_t i = 0; i < hosts.size(); ++i) + { + String ip_or_hostname = hosts[i].substr(0, hosts[i].find_last_of(":")); + hostname_differences[i] = DB::getHostNameDifference(local_hostname, Poco::Net::DNS::resolve(ip_or_hostname).name()); + } + + size_t offset = 0; + std::function get_priority; + switch (ZooKeeperLoadBalancing(zookeeper_load_balancing)) + { + case ZooKeeperLoadBalancing::NEAREST_HOSTNAME: + get_priority = [&](size_t i) { return hostname_differences[i]; }; + break; + case ZooKeeperLoadBalancing::IN_ORDER: + get_priority = [](size_t i) { return i; }; + break; + case ZooKeeperLoadBalancing::RANDOM: + break; + case ZooKeeperLoadBalancing::FIRST_OR_RANDOM: + get_priority = [offset](size_t i) -> size_t { return i != offset; }; + break; + case ZooKeeperLoadBalancing::ROUND_ROBIN: + static size_t last_used = 0; + if (last_used >= hosts.size()) + last_used = 0; + ++last_used; + /* Consider hosts.size() equals to 5 + * last_used = 1 -> get_priority: 0 1 2 3 4 + * last_used = 2 -> get_priority: 4 0 1 2 3 + * last_used = 3 -> get_priority: 4 3 0 1 2 + * ... + * */ + get_priority = [&](size_t i) { ++i; return i < last_used ? hosts.size() - i : i - last_used; }; + break; + } + + std::vector shuffle_hosts; + for (size_t i = 0; i < hosts.size(); ++i) + { + ShuffleHost shuffle_host; + shuffle_host.host = hosts[i]; + if (get_priority) + shuffle_host.priority = get_priority(i); + shuffle_host.randomize(); + shuffle_hosts.emplace_back(shuffle_host); + } + + std::sort( + shuffle_hosts.begin(), shuffle_hosts.end(), + [](const ShuffleHost & lhs, const ShuffleHost & rhs) + { + return ShuffleHost::compare(lhs, rhs); + }); + + return shuffle_hosts; +} + ZooKeeper::ZooKeeper(const std::string & hosts_string, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_, - std::shared_ptr zk_log_) + std::shared_ptr zk_log_, ZooKeeperLoadBalancing zookeeper_load_balancing_) { zk_log = std::move(zk_log_); Strings hosts_strings; splitInto<','>(hosts_strings, hosts_string); - init(implementation_, hosts_strings, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_); + init(implementation_, hosts_strings, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_, zookeeper_load_balancing_); } ZooKeeper::ZooKeeper(const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_, - std::shared_ptr zk_log_) + std::shared_ptr zk_log_, ZooKeeperLoadBalancing zookeeper_load_balancing_) { zk_log = std::move(zk_log_); - init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_); + init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_, zookeeper_load_balancing_); } struct ZooKeeperArgs @@ -182,6 +245,7 @@ struct ZooKeeperArgs session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; implementation = "zookeeper"; + zookeeper_load_balancing = ZooKeeperLoadBalancing::RANDOM; for (const auto & key : keys) { if (startsWith(key, "node")) @@ -212,6 +276,10 @@ struct ZooKeeperArgs { implementation = config.getString(config_name + "." + key); } + else if (key == "zookeeper_load_balancing") + { + zookeeper_load_balancing = DB::SettingFieldZooKeeperLoadBalancingTraits::fromString(config.getString(config_name + "." + key)); + } else throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS); } @@ -231,13 +299,14 @@ struct ZooKeeperArgs int operation_timeout_ms; std::string chroot; std::string implementation; + ZooKeeperLoadBalancing zookeeper_load_balancing; }; ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, std::shared_ptr zk_log_) : zk_log(std::move(zk_log_)) { ZooKeeperArgs args(config, config_name); - init(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot); + init(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot, args.zookeeper_load_balancing); } bool ZooKeeper::configChanged(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) const @@ -752,7 +821,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & ZooKeeperPtr ZooKeeper::startNewSession() const { - return std::make_shared(hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, implementation, zk_log); + return std::make_shared(hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, implementation, zk_log, zookeeper_load_balancing); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 8e015b1f331..4ad19eb3a4c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -13,7 +13,10 @@ #include #include #include +#include +#include #include +#include namespace ProfileEvents @@ -37,6 +40,29 @@ namespace zkutil /// Preferred size of multi() command (in number of ops) constexpr size_t MULTI_BATCH_SIZE = 100; +struct ShuffleHost +{ + String host; + /// Priority from the GetPriorityFunc. + Int64 priority = 0; + UInt32 random = 0; + + void randomize() + { + random = rng(); + } + + static bool compare(const ShuffleHost & lhs, const ShuffleHost & rhs) + { + return std::forward_as_tuple(lhs.priority, lhs.random) + < std::forward_as_tuple(rhs.priority, rhs.random); + } + +private: + std::minstd_rand rng = std::minstd_rand(randomSeed()); +}; + +using ZooKeeperLoadBalancing = DB::ZooKeeperLoadBalancing; /// ZooKeeper session. The interface is substantially different from the usual libzookeeper API. /// @@ -58,14 +84,16 @@ public: int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS, const std::string & chroot_ = "", const std::string & implementation_ = "zookeeper", - std::shared_ptr zk_log_ = nullptr); + std::shared_ptr zk_log_ = nullptr, + ZooKeeperLoadBalancing zookeeper_load_balancing_ = ZooKeeperLoadBalancing::RANDOM); ZooKeeper(const Strings & hosts_, const std::string & identity_ = "", int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS, int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS, const std::string & chroot_ = "", const std::string & implementation_ = "zookeeper", - std::shared_ptr zk_log_ = nullptr); + std::shared_ptr zk_log_ = nullptr, + ZooKeeperLoadBalancing zookeeper_load_balancing_ = ZooKeeperLoadBalancing::RANDOM); /** Config of the form: @@ -91,6 +119,8 @@ public: */ ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, std::shared_ptr zk_log_); + std::vector shuffleHosts() const; + /// Creates a new session with the same parameters. This method can be used for reconnecting /// after the session has expired. /// This object remains unchanged, and the new session is returned. @@ -284,7 +314,7 @@ private: friend class EphemeralNodeHolder; void init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, - int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_); + int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, ZooKeeperLoadBalancing zookeeper_load_balancing_); /// The following methods don't any throw exceptions but return error codes. Coordination::Error createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created); @@ -310,6 +340,7 @@ private: Poco::Logger * log = nullptr; std::shared_ptr zk_log; + ZooKeeperLoadBalancing zookeeper_load_balancing; AtomicStopwatch session_uptime; }; diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 8e588b62326..2aa296533fe 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -116,4 +116,12 @@ IMPLEMENT_SETTING_ENUM(ShortCircuitFunctionEvaluation, ErrorCodes::BAD_ARGUMENTS {{"enable", ShortCircuitFunctionEvaluation::ENABLE}, {"force_enable", ShortCircuitFunctionEvaluation::FORCE_ENABLE}, {"disable", ShortCircuitFunctionEvaluation::DISABLE}}) + +IMPLEMENT_SETTING_ENUM(ZooKeeperLoadBalancing, ErrorCodes::UNKNOWN_LOAD_BALANCING, + {{"random", ZooKeeperLoadBalancing::RANDOM}, + {"nearest_hostname", ZooKeeperLoadBalancing::NEAREST_HOSTNAME}, + {"in_order", ZooKeeperLoadBalancing::IN_ORDER}, + {"first_or_random", ZooKeeperLoadBalancing::FIRST_OR_RANDOM}, + {"round_robin", ZooKeeperLoadBalancing::ROUND_ROBIN}}) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 33c5a6d8645..a308fa1745b 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -168,4 +168,23 @@ enum class ShortCircuitFunctionEvaluation DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) +enum class ZooKeeperLoadBalancing +{ + /// among replicas with a minimum number of errors selected randomly + RANDOM = 0, + /// a replica is selected among the replicas with the minimum number of errors + /// with the minimum number of distinguished characters in the replica name and local hostname + NEAREST_HOSTNAME, + // replicas with the same number of errors are accessed in the same order + // as they are specified in the configuration. + IN_ORDER, + /// if first replica one has higher number of errors, + /// pick a random one from replicas with minimum number of errors + FIRST_OR_RANDOM, + // round robin across replicas with the same number of errors. + ROUND_ROBIN, +}; + +DECLARE_SETTING_ENUM(ZooKeeperLoadBalancing) + } diff --git a/tests/config/config.d/zookeeper.xml b/tests/config/config.d/zookeeper.xml index 4fa529a6180..63057224ef9 100644 --- a/tests/config/config.d/zookeeper.xml +++ b/tests/config/config.d/zookeeper.xml @@ -1,5 +1,7 @@ + + random localhost 9181 From fed7bb594179257373aaf1b2109bcd1da6dd3bb8 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Mon, 18 Oct 2021 19:10:53 +0800 Subject: [PATCH 008/584] Update comments. --- src/Core/SettingsEnums.h | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index a308fa1745b..b7506e52176 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -170,18 +170,17 @@ DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) enum class ZooKeeperLoadBalancing { - /// among replicas with a minimum number of errors selected randomly + /// Randomly select one from the zookeeper nodes. RANDOM = 0, - /// a replica is selected among the replicas with the minimum number of errors - /// with the minimum number of distinguished characters in the replica name and local hostname + /// Choose one from the zookeeper node that has the least + /// number of characters different from the hostname of the local host NEAREST_HOSTNAME, - // replicas with the same number of errors are accessed in the same order - // as they are specified in the configuration. + /// Select one from the zookeeper node configuration in order. IN_ORDER, - /// if first replica one has higher number of errors, - /// pick a random one from replicas with minimum number of errors + /// If the first node cannot be connected, + /// one will be randomly selected from other nodes. FIRST_OR_RANDOM, - // round robin across replicas with the same number of errors. + /// Round robin from the node configured by zookeeper. ROUND_ROBIN, }; From 62a15c1c1a7077539d9faafd4615b9f3a755af75 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Tue, 19 Oct 2021 12:43:54 +0800 Subject: [PATCH 009/584] Fix some build error and try fix undefined symbol: DB::SettingFieldZooKeeperLoadBalancingTraits::fromString build error. --- src/Common/ZooKeeper/ZooKeeper.cpp | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 65295a96d3e..3bff19fc2d9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -158,10 +158,10 @@ std::vector ZooKeeper::shuffleHosts() const { std::vector hostname_differences; hostname_differences.resize(hosts.size()); - String local_hostname = getFQDNOrHostName(); + const String & local_hostname = getFQDNOrHostName(); for (size_t i = 0; i < hosts.size(); ++i) { - String ip_or_hostname = hosts[i].substr(0, hosts[i].find_last_of(":")); + const String & ip_or_hostname = hosts[i].substr(0, hosts[i].find_last_of(':')); hostname_differences[i] = DB::getHostNameDifference(local_hostname, Poco::Net::DNS::resolve(ip_or_hostname).name()); } @@ -278,7 +278,9 @@ struct ZooKeeperArgs } else if (key == "zookeeper_load_balancing") { - zookeeper_load_balancing = DB::SettingFieldZooKeeperLoadBalancingTraits::fromString(config.getString(config_name + "." + key)); + DB::SettingFieldZooKeeperLoadBalancing setting_field; + setting_field.parseFromString(config.getString(config_name + "." + key)); + zookeeper_load_balancing = setting_field.value; } else throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS); @@ -317,8 +319,8 @@ bool ZooKeeper::configChanged(const Poco::Util::AbstractConfiguration & config, if (args.implementation == implementation && implementation == "testkeeper") return false; - return std::tie(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot) - != std::tie(implementation, hosts, identity, session_timeout_ms, operation_timeout_ms, chroot); + return std::tie(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot, args.zookeeper_load_balancing) + != std::tie(implementation, hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, zookeeper_load_balancing); } From a6ae846f02c32a2c91728ade3b4ff58c86ca85c2 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Tue, 19 Oct 2021 17:39:03 +0800 Subject: [PATCH 010/584] fix biuld and PVS check --- src/Common/ZooKeeper/ZooKeeper.cpp | 40 ++++++++++--- src/Common/ZooKeeper/ZooKeeper.h | 18 +++++- src/Core/SettingsEnums.cpp | 7 --- src/Core/SettingsEnums.h | 18 ------ .../__init__.py | 0 .../configs/remote_servers.xml | 23 ++++++++ .../configs/zookeeper_config_in_order.xml | 20 +++++++ .../test.py | 57 +++++++++++++++++++ 8 files changed, 149 insertions(+), 34 deletions(-) create mode 100644 tests/integration/test_zookeeper_config_load_balancing/__init__.py create mode 100644 tests/integration/test_zookeeper_config_load_balancing/configs/remote_servers.xml create mode 100644 tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_in_order.xml create mode 100644 tests/integration/test_zookeeper_config_load_balancing/test.py diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 3bff19fc2d9..2f33888846b 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -6,7 +6,6 @@ #include #include -#include #include #include #include @@ -27,6 +26,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; + extern const int UNKNOWN_LOAD_BALANCING; } } @@ -46,6 +46,35 @@ static void check(Coordination::Error code, const std::string & path) throw KeeperException(code, path); } +static ZooKeeperLoadBalancing fromString(const std::string_view & str) +{ + static const std::unordered_map map = [] { + std::unordered_map res; + constexpr std::pair pairs[] + = {{"random", ZooKeeperLoadBalancing::RANDOM}, + {"nearest_hostname", ZooKeeperLoadBalancing::NEAREST_HOSTNAME}, + {"in_order", ZooKeeperLoadBalancing::IN_ORDER}, + {"first_or_random", ZooKeeperLoadBalancing::FIRST_OR_RANDOM}, + {"round_robin", ZooKeeperLoadBalancing::ROUND_ROBIN}}; + for (const auto & [name, val] : pairs) + res.emplace(name, val); + return res; + }(); + auto it = map.find(str); + if (it != map.end()) + return it->second; + String msg = "Unexpected value of ZooKeeperLoadBalancing: '" + String{str} + "'. Must be one of ["; + bool need_comma = false; + for (auto & name : map | boost::adaptors::map_keys) + { + if (std::exchange(need_comma, true)) + msg += ", "; + msg += "'" + String{name} + "'"; + } + msg += "]"; + throw DB::Exception(msg, DB::ErrorCodes::UNKNOWN_LOAD_BALANCING); +} + void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, ZooKeeperLoadBalancing zookeeper_load_balancing_) @@ -165,7 +194,6 @@ std::vector ZooKeeper::shuffleHosts() const hostname_differences[i] = DB::getHostNameDifference(local_hostname, Poco::Net::DNS::resolve(ip_or_hostname).name()); } - size_t offset = 0; std::function get_priority; switch (ZooKeeperLoadBalancing(zookeeper_load_balancing)) { @@ -178,7 +206,7 @@ std::vector ZooKeeper::shuffleHosts() const case ZooKeeperLoadBalancing::RANDOM: break; case ZooKeeperLoadBalancing::FIRST_OR_RANDOM: - get_priority = [offset](size_t i) -> size_t { return i != offset; }; + get_priority = [](size_t i) -> size_t { return i != 0; }; break; case ZooKeeperLoadBalancing::ROUND_ROBIN: static size_t last_used = 0; @@ -191,7 +219,7 @@ std::vector ZooKeeper::shuffleHosts() const * last_used = 3 -> get_priority: 4 3 0 1 2 * ... * */ - get_priority = [&](size_t i) { ++i; return i < last_used ? hosts.size() - i : i - last_used; }; + get_priority = [this, last_used_value = last_used](size_t i) { ++i; return i < last_used_value ? hosts.size() - i : i - last_used_value; }; break; } @@ -278,9 +306,7 @@ struct ZooKeeperArgs } else if (key == "zookeeper_load_balancing") { - DB::SettingFieldZooKeeperLoadBalancing setting_field; - setting_field.parseFromString(config.getString(config_name + "." + key)); - zookeeper_load_balancing = setting_field.value; + zookeeper_load_balancing = fromString(config.getString(config_name + "." + key)); } else throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS); diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 4ad19eb3a4c..c992ffe3a43 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -14,7 +14,6 @@ #include #include #include -#include #include #include @@ -62,7 +61,22 @@ private: std::minstd_rand rng = std::minstd_rand(randomSeed()); }; -using ZooKeeperLoadBalancing = DB::ZooKeeperLoadBalancing; +enum class ZooKeeperLoadBalancing +{ + /// Randomly select one from the zookeeper nodes. + RANDOM = 0, + /// Choose one from the zookeeper node that has the least + /// number of characters different from the hostname of the local host + NEAREST_HOSTNAME, + /// Select one from the zookeeper node configuration in order. + IN_ORDER, + /// If the first node cannot be connected, + /// one will be randomly selected from other nodes. + FIRST_OR_RANDOM, + /// Round robin from the node configured by zookeeper. + ROUND_ROBIN, +}; + /// ZooKeeper session. The interface is substantially different from the usual libzookeeper API. /// diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 2aa296533fe..6a5d8136227 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -117,11 +117,4 @@ IMPLEMENT_SETTING_ENUM(ShortCircuitFunctionEvaluation, ErrorCodes::BAD_ARGUMENTS {"force_enable", ShortCircuitFunctionEvaluation::FORCE_ENABLE}, {"disable", ShortCircuitFunctionEvaluation::DISABLE}}) -IMPLEMENT_SETTING_ENUM(ZooKeeperLoadBalancing, ErrorCodes::UNKNOWN_LOAD_BALANCING, - {{"random", ZooKeeperLoadBalancing::RANDOM}, - {"nearest_hostname", ZooKeeperLoadBalancing::NEAREST_HOSTNAME}, - {"in_order", ZooKeeperLoadBalancing::IN_ORDER}, - {"first_or_random", ZooKeeperLoadBalancing::FIRST_OR_RANDOM}, - {"round_robin", ZooKeeperLoadBalancing::ROUND_ROBIN}}) - } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index b7506e52176..33c5a6d8645 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -168,22 +168,4 @@ enum class ShortCircuitFunctionEvaluation DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) -enum class ZooKeeperLoadBalancing -{ - /// Randomly select one from the zookeeper nodes. - RANDOM = 0, - /// Choose one from the zookeeper node that has the least - /// number of characters different from the hostname of the local host - NEAREST_HOSTNAME, - /// Select one from the zookeeper node configuration in order. - IN_ORDER, - /// If the first node cannot be connected, - /// one will be randomly selected from other nodes. - FIRST_OR_RANDOM, - /// Round robin from the node configured by zookeeper. - ROUND_ROBIN, -}; - -DECLARE_SETTING_ENUM(ZooKeeperLoadBalancing) - } diff --git a/tests/integration/test_zookeeper_config_load_balancing/__init__.py b/tests/integration/test_zookeeper_config_load_balancing/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_zookeeper_config_load_balancing/configs/remote_servers.xml b/tests/integration/test_zookeeper_config_load_balancing/configs/remote_servers.xml new file mode 100644 index 00000000000..63fdcea5dab --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/configs/remote_servers.xml @@ -0,0 +1,23 @@ + + + + + + node1 + 9000 + + + + node2 + 9000 + + + + node3 + 9000 + + + + + + diff --git a/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_in_order.xml b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_in_order.xml new file mode 100644 index 00000000000..bbed71532aa --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_in_order.xml @@ -0,0 +1,20 @@ + + + + random + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 3000 + /root_a + + diff --git a/tests/integration/test_zookeeper_config_load_balancing/test.py b/tests/integration/test_zookeeper_config_load_balancing/test.py new file mode 100644 index 00000000000..95d9db27a7d --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/test.py @@ -0,0 +1,57 @@ +import time +import pytest +import logging +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') + +node1 = cluster.add_instance('node1', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"]) +node2 = cluster.add_instance('node2', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"]) +node3 = cluster.add_instance('node3', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_b.xml"]) + +def create_zk_roots(zk): + zk.ensure_path('/root_a') + zk.ensure_path('/root_b') + logging.debug(f"Create ZK roots:{zk.get_children('/')}") + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.add_zookeeper_startup_command(create_zk_roots) + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def test_chroot_with_same_root(started_cluster): + for i, node in enumerate([node1, node2]): + node.query('DROP TABLE IF EXISTS simple SYNC') + node.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + for j in range(2): # Second insert to test deduplication + node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) + + time.sleep(1) + + assert node1.query('select count() from simple').strip() == '2' + assert node2.query('select count() from simple').strip() == '2' + +def test_chroot_with_different_root(started_cluster): + for i, node in [(1, node1), (3, node3)]: + node.query('DROP TABLE IF EXISTS simple_different SYNC') + node.query(''' + CREATE TABLE simple_different (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple_different', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + for j in range(2): # Second insert to test deduplication + node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) + + assert node1.query('select count() from simple_different').strip() == '1' + assert node3.query('select count() from simple_different').strip() == '1' From 8480ae631acda0caa381141c4204583a6e1a9150 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Wed, 20 Oct 2021 16:35:37 +0800 Subject: [PATCH 011/584] Refactor and add test. --- src/Client/ConnectionPoolWithFailover.cpp | 69 ++-------- src/Client/ConnectionPoolWithFailover.h | 8 +- src/Common/GetPriorityForLoadBalancing.cpp | 41 ++++++ src/Common/GetPriorityForLoadBalancing.h | 32 +++++ src/Common/ZooKeeper/ZooKeeper.cpp | 114 +++++---------- src/Common/ZooKeeper/ZooKeeper.h | 27 +--- .../configs/zookeeper_config_in_order.xml | 3 +- .../configs/zookeeper_config_round_robin.xml | 19 +++ .../configs/zookeeper_log.xml | 7 + .../test.py | 130 ++++++++++++++---- .../test_round_robin.py | 100 ++++++++++++++ 11 files changed, 357 insertions(+), 193 deletions(-) create mode 100644 src/Common/GetPriorityForLoadBalancing.cpp create mode 100644 src/Common/GetPriorityForLoadBalancing.h create mode 100644 tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_round_robin.xml create mode 100644 tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_log.xml create mode 100644 tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index aaffe85ae2e..ecfc6bd5c08 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -29,15 +29,15 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover( time_t decrease_error_period_, size_t max_error_cap_) : Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Poco::Logger::get("ConnectionPoolWithFailover")) - , default_load_balancing(load_balancing) + , get_priority_load_balancing(load_balancing) { const std::string & local_hostname = getFQDNOrHostName(); - hostname_differences.resize(nested_pools.size()); + get_priority_load_balancing.hostname_differences.resize(nested_pools.size()); for (size_t i = 0; i < nested_pools.size(); ++i) { ConnectionPool & connection_pool = dynamic_cast(*nested_pools[i]); - hostname_differences[i] = getHostNameDifference(local_hostname, connection_pool.getHost()); + get_priority_load_balancing.hostname_differences[i] = getHostNameDifference(local_hostname, connection_pool.getHost()); } } @@ -50,36 +50,12 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts return tryGetEntry(pool, timeouts, fail_message, settings); }; - size_t offset = 0; if (settings) - offset = settings->load_balancing_first_offset % nested_pools.size(); - GetPriorityFunc get_priority; - switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing) { - case LoadBalancing::NEAREST_HOSTNAME: - get_priority = [&](size_t i) { return hostname_differences[i]; }; - break; - case LoadBalancing::IN_ORDER: - get_priority = [](size_t i) { return i; }; - break; - case LoadBalancing::RANDOM: - break; - case LoadBalancing::FIRST_OR_RANDOM: - get_priority = [offset](size_t i) -> size_t { return i != offset; }; - break; - case LoadBalancing::ROUND_ROBIN: - if (last_used >= nested_pools.size()) - last_used = 0; - ++last_used; - /* Consider nested_pools.size() equals to 5 - * last_used = 1 -> get_priority: 0 1 2 3 4 - * last_used = 2 -> get_priority: 4 0 1 2 3 - * last_used = 3 -> get_priority: 4 3 0 1 2 - * ... - * */ - get_priority = [&](size_t i) { ++i; return i < last_used ? nested_pools.size() - i : i - last_used; }; - break; + get_priority_load_balancing.offset = settings->load_balancing_first_offset % nested_pools.size(); + get_priority_load_balancing.load_balancing = settings->load_balancing; } + GetPriorityFunc get_priority = get_priority_load_balancing.getPriorityFunc(); UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0; bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true; @@ -172,39 +148,12 @@ std::vector ConnectionPoolWithFailover::g ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings * settings) { - size_t offset = 0; if (settings) - offset = settings->load_balancing_first_offset % nested_pools.size(); - - GetPriorityFunc get_priority; - switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing) { - case LoadBalancing::NEAREST_HOSTNAME: - get_priority = [&](size_t i) { return hostname_differences[i]; }; - break; - case LoadBalancing::IN_ORDER: - get_priority = [](size_t i) { return i; }; - break; - case LoadBalancing::RANDOM: - break; - case LoadBalancing::FIRST_OR_RANDOM: - get_priority = [offset](size_t i) -> size_t { return i != offset; }; - break; - case LoadBalancing::ROUND_ROBIN: - if (last_used >= nested_pools.size()) - last_used = 0; - ++last_used; - /* Consider nested_pools.size() equals to 5 - * last_used = 1 -> get_priority: 0 1 2 3 4 - * last_used = 2 -> get_priority: 5 0 1 2 3 - * last_used = 3 -> get_priority: 5 4 0 1 2 - * ... - * */ - get_priority = [&](size_t i) { ++i; return i < last_used ? nested_pools.size() - i : i - last_used; }; - break; + get_priority_load_balancing.offset = settings->load_balancing_first_offset % nested_pools.size(); + get_priority_load_balancing.load_balancing = settings->load_balancing; } - - return get_priority; + return get_priority_load_balancing.getPriorityFunc(); } std::vector ConnectionPoolWithFailover::getManyImpl( diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index ce70c27838b..3c838459733 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -110,9 +111,10 @@ private: GetPriorityFunc makeGetPriorityFunc(const Settings * settings); private: - std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. - size_t last_used = 0; /// Last used for round_robin policy. - LoadBalancing default_load_balancing; + GetPriorityForLoadBalancing get_priority_load_balancing; +// std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. +// size_t last_used = 0; /// Last used for round_robin policy. +// LoadBalancing default_load_balancing; }; using ConnectionPoolWithFailoverPtr = std::shared_ptr; diff --git a/src/Common/GetPriorityForLoadBalancing.cpp b/src/Common/GetPriorityForLoadBalancing.cpp new file mode 100644 index 00000000000..ae621d9e75c --- /dev/null +++ b/src/Common/GetPriorityForLoadBalancing.cpp @@ -0,0 +1,41 @@ +#include + +namespace DB +{ + +std::function GetPriorityForLoadBalancing::getPriorityFunc() const +{ + std::function get_priority; + switch (load_balancing) + { + case LoadBalancing::NEAREST_HOSTNAME: + get_priority = [&](size_t i) { return hostname_differences[i]; }; + break; + case LoadBalancing::IN_ORDER: + get_priority = [](size_t i) { return i; }; + break; + case LoadBalancing::RANDOM: + break; + case LoadBalancing::FIRST_OR_RANDOM: + get_priority = [&](size_t i) -> size_t { return i != offset; }; + break; + case LoadBalancing::ROUND_ROBIN: + if (last_used >= pool_size) + last_used = 0; + ++last_used; + /* Consider pool_size equals to 5 + * last_used = 1 -> get_priority: 0 1 2 3 4 + * last_used = 2 -> get_priority: 4 0 1 2 3 + * last_used = 3 -> get_priority: 4 3 0 1 2 + * ... + * */ + get_priority = [&](size_t i) { + ++i; + return i < last_used ? pool_size - i : i - last_used; + }; + break; + } + return get_priority; +} + +} diff --git a/src/Common/GetPriorityForLoadBalancing.h b/src/Common/GetPriorityForLoadBalancing.h new file mode 100644 index 00000000000..b845c2e7616 --- /dev/null +++ b/src/Common/GetPriorityForLoadBalancing.h @@ -0,0 +1,32 @@ +#pragma once + +#include + +namespace DB +{ + +class GetPriorityForLoadBalancing +{ +public: + GetPriorityForLoadBalancing(LoadBalancing load_balancing_) : load_balancing(load_balancing_) {} + GetPriorityForLoadBalancing(){} + + bool operator!=(const GetPriorityForLoadBalancing & other) + { + return offset != other.offset || pool_size != other.pool_size || load_balancing != other.load_balancing + || hostname_differences != other.hostname_differences; + } + + std::function getPriorityFunc() const; + + std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. + size_t offset = 0; /// for first_or_random policy. + size_t pool_size; /// for round_robin policy. + + LoadBalancing load_balancing = LoadBalancing::RANDOM; + +private: + mutable size_t last_used = 0; /// Last used for round_robin policy. +}; + +} diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 2f33888846b..b1f6269d128 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -46,38 +46,9 @@ static void check(Coordination::Error code, const std::string & path) throw KeeperException(code, path); } -static ZooKeeperLoadBalancing fromString(const std::string_view & str) -{ - static const std::unordered_map map = [] { - std::unordered_map res; - constexpr std::pair pairs[] - = {{"random", ZooKeeperLoadBalancing::RANDOM}, - {"nearest_hostname", ZooKeeperLoadBalancing::NEAREST_HOSTNAME}, - {"in_order", ZooKeeperLoadBalancing::IN_ORDER}, - {"first_or_random", ZooKeeperLoadBalancing::FIRST_OR_RANDOM}, - {"round_robin", ZooKeeperLoadBalancing::ROUND_ROBIN}}; - for (const auto & [name, val] : pairs) - res.emplace(name, val); - return res; - }(); - auto it = map.find(str); - if (it != map.end()) - return it->second; - String msg = "Unexpected value of ZooKeeperLoadBalancing: '" + String{str} + "'. Must be one of ["; - bool need_comma = false; - for (auto & name : map | boost::adaptors::map_keys) - { - if (std::exchange(need_comma, true)) - msg += ", "; - msg += "'" + String{name} + "'"; - } - msg += "]"; - throw DB::Exception(msg, DB::ErrorCodes::UNKNOWN_LOAD_BALANCING); -} - void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, - int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, ZooKeeperLoadBalancing zookeeper_load_balancing_) + int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const GetPriorityForLoadBalancing & get_priority_load_balancing_) { log = &Poco::Logger::get("ZooKeeper"); hosts = hosts_; @@ -86,7 +57,7 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ operation_timeout_ms = operation_timeout_ms_; chroot = chroot_; implementation = implementation_; - zookeeper_load_balancing = zookeeper_load_balancing_; + get_priority_load_balancing = get_priority_load_balancing_; if (implementation == "zookeeper") { @@ -185,44 +156,7 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ std::vector ZooKeeper::shuffleHosts() const { - std::vector hostname_differences; - hostname_differences.resize(hosts.size()); - const String & local_hostname = getFQDNOrHostName(); - for (size_t i = 0; i < hosts.size(); ++i) - { - const String & ip_or_hostname = hosts[i].substr(0, hosts[i].find_last_of(':')); - hostname_differences[i] = DB::getHostNameDifference(local_hostname, Poco::Net::DNS::resolve(ip_or_hostname).name()); - } - - std::function get_priority; - switch (ZooKeeperLoadBalancing(zookeeper_load_balancing)) - { - case ZooKeeperLoadBalancing::NEAREST_HOSTNAME: - get_priority = [&](size_t i) { return hostname_differences[i]; }; - break; - case ZooKeeperLoadBalancing::IN_ORDER: - get_priority = [](size_t i) { return i; }; - break; - case ZooKeeperLoadBalancing::RANDOM: - break; - case ZooKeeperLoadBalancing::FIRST_OR_RANDOM: - get_priority = [](size_t i) -> size_t { return i != 0; }; - break; - case ZooKeeperLoadBalancing::ROUND_ROBIN: - static size_t last_used = 0; - if (last_used >= hosts.size()) - last_used = 0; - ++last_used; - /* Consider hosts.size() equals to 5 - * last_used = 1 -> get_priority: 0 1 2 3 4 - * last_used = 2 -> get_priority: 4 0 1 2 3 - * last_used = 3 -> get_priority: 4 3 0 1 2 - * ... - * */ - get_priority = [this, last_used_value = last_used](size_t i) { ++i; return i < last_used_value ? hosts.size() - i : i - last_used_value; }; - break; - } - + std::function get_priority = get_priority_load_balancing.getPriorityFunc(); std::vector shuffle_hosts; for (size_t i = 0; i < hosts.size(); ++i) { @@ -246,21 +180,21 @@ std::vector ZooKeeper::shuffleHosts() const ZooKeeper::ZooKeeper(const std::string & hosts_string, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_, - std::shared_ptr zk_log_, ZooKeeperLoadBalancing zookeeper_load_balancing_) + std::shared_ptr zk_log_, const GetPriorityForLoadBalancing & get_priority_load_balancing_) { zk_log = std::move(zk_log_); Strings hosts_strings; splitInto<','>(hosts_strings, hosts_string); - init(implementation_, hosts_strings, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_, zookeeper_load_balancing_); + init(implementation_, hosts_strings, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_, get_priority_load_balancing_); } ZooKeeper::ZooKeeper(const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_, - std::shared_ptr zk_log_, ZooKeeperLoadBalancing zookeeper_load_balancing_) + std::shared_ptr zk_log_, const GetPriorityForLoadBalancing & get_priority_load_balancing_) { zk_log = std::move(zk_log_); - init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_, zookeeper_load_balancing_); + init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_, get_priority_load_balancing_); } struct ZooKeeperArgs @@ -273,7 +207,6 @@ struct ZooKeeperArgs session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; implementation = "zookeeper"; - zookeeper_load_balancing = ZooKeeperLoadBalancing::RANDOM; for (const auto & key : keys) { if (startsWith(key, "node")) @@ -306,7 +239,7 @@ struct ZooKeeperArgs } else if (key == "zookeeper_load_balancing") { - zookeeper_load_balancing = fromString(config.getString(config_name + "." + key)); + get_priority_load_balancing.load_balancing = DB::SettingFieldLoadBalancingTraits::fromString(config.getString(config_name + "." + key)); } else throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS); @@ -319,6 +252,24 @@ struct ZooKeeperArgs if (chroot.back() == '/') chroot.pop_back(); } + + /// init get_priority_load_balancing + get_priority_load_balancing.hostname_differences.resize(hosts.size()); + const String & local_hostname = getFQDNOrHostName(); + for (size_t i = 0; i < hosts.size(); ++i) + { + const String & ip_or_hostname = hosts[i].substr(0, hosts[i].find_last_of(':')); + try + { + get_priority_load_balancing.hostname_differences[i] = DB::getHostNameDifference(local_hostname, Poco::Net::DNS::resolve(ip_or_hostname).name()); + } + catch (...) + { + /// There may be HostNotFoundException or DNSException, these exceptions will be processed later. + LOG_ERROR(&Poco::Logger::get("ZooKeeperArgs"), "Cannot use ZooKeeper host {}, hostname differences will be set to the maximum value", hosts[i]); + } + } + get_priority_load_balancing.pool_size = hosts.size(); } Strings hosts; @@ -327,14 +278,14 @@ struct ZooKeeperArgs int operation_timeout_ms; std::string chroot; std::string implementation; - ZooKeeperLoadBalancing zookeeper_load_balancing; + GetPriorityForLoadBalancing get_priority_load_balancing; }; ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, std::shared_ptr zk_log_) : zk_log(std::move(zk_log_)) { ZooKeeperArgs args(config, config_name); - init(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot, args.zookeeper_load_balancing); + init(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot, args.get_priority_load_balancing); } bool ZooKeeper::configChanged(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) const @@ -345,8 +296,11 @@ bool ZooKeeper::configChanged(const Poco::Util::AbstractConfiguration & config, if (args.implementation == implementation && implementation == "testkeeper") return false; - return std::tie(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot, args.zookeeper_load_balancing) - != std::tie(implementation, hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, zookeeper_load_balancing); + if (args.get_priority_load_balancing != get_priority_load_balancing) + return true; + + return std::tie(args.implementation, args.hosts, args.identity, args.session_timeout_ms, args.operation_timeout_ms, args.chroot) + != std::tie(implementation, hosts, identity, session_timeout_ms, operation_timeout_ms, chroot); } @@ -849,7 +803,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & ZooKeeperPtr ZooKeeper::startNewSession() const { - return std::make_shared(hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, implementation, zk_log, zookeeper_load_balancing); + return std::make_shared(hosts, identity, session_timeout_ms, operation_timeout_ms, chroot, implementation, zk_log, get_priority_load_balancing); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index c992ffe3a43..94232aeac86 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -61,22 +62,7 @@ private: std::minstd_rand rng = std::minstd_rand(randomSeed()); }; -enum class ZooKeeperLoadBalancing -{ - /// Randomly select one from the zookeeper nodes. - RANDOM = 0, - /// Choose one from the zookeeper node that has the least - /// number of characters different from the hostname of the local host - NEAREST_HOSTNAME, - /// Select one from the zookeeper node configuration in order. - IN_ORDER, - /// If the first node cannot be connected, - /// one will be randomly selected from other nodes. - FIRST_OR_RANDOM, - /// Round robin from the node configured by zookeeper. - ROUND_ROBIN, -}; - +using GetPriorityForLoadBalancing = DB::GetPriorityForLoadBalancing; /// ZooKeeper session. The interface is substantially different from the usual libzookeeper API. /// @@ -99,7 +85,7 @@ public: const std::string & chroot_ = "", const std::string & implementation_ = "zookeeper", std::shared_ptr zk_log_ = nullptr, - ZooKeeperLoadBalancing zookeeper_load_balancing_ = ZooKeeperLoadBalancing::RANDOM); + const GetPriorityForLoadBalancing & get_priority_load_balancing_ = {}); ZooKeeper(const Strings & hosts_, const std::string & identity_ = "", int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS, @@ -107,7 +93,7 @@ public: const std::string & chroot_ = "", const std::string & implementation_ = "zookeeper", std::shared_ptr zk_log_ = nullptr, - ZooKeeperLoadBalancing zookeeper_load_balancing_ = ZooKeeperLoadBalancing::RANDOM); + const GetPriorityForLoadBalancing & get_priority_load_balancing_ = {}); /** Config of the form: @@ -328,7 +314,7 @@ private: friend class EphemeralNodeHolder; void init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, - int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, ZooKeeperLoadBalancing zookeeper_load_balancing_); + int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const GetPriorityForLoadBalancing & get_priority_load_balancing_); /// The following methods don't any throw exceptions but return error codes. Coordination::Error createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created); @@ -354,7 +340,8 @@ private: Poco::Logger * log = nullptr; std::shared_ptr zk_log; - ZooKeeperLoadBalancing zookeeper_load_balancing; + + GetPriorityForLoadBalancing get_priority_load_balancing; AtomicStopwatch session_uptime; }; diff --git a/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_in_order.xml b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_in_order.xml index bbed71532aa..ebd266d80b0 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_in_order.xml +++ b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_in_order.xml @@ -1,7 +1,7 @@ - random + in_order zoo1 2181 @@ -15,6 +15,5 @@ 2181 3000 - /root_a diff --git a/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_round_robin.xml b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_round_robin.xml new file mode 100644 index 00000000000..3b64d629e6e --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_round_robin.xml @@ -0,0 +1,19 @@ + + + + round_robin + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 3000 + + diff --git a/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_log.xml b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_log.xml new file mode 100644 index 00000000000..a70cbc3ecc2 --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_log.xml @@ -0,0 +1,7 @@ + + + system + zookeeper_log
+ 7500 +
+
diff --git a/tests/integration/test_zookeeper_config_load_balancing/test.py b/tests/integration/test_zookeeper_config_load_balancing/test.py index 95d9db27a7d..951dd7f12b4 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test.py @@ -3,24 +3,19 @@ import pytest import logging from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') +cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_in_order.xml') node1 = cluster.add_instance('node1', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) node2 = cluster.add_instance('node2', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) node3 = cluster.add_instance('node3', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_b.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) -def create_zk_roots(zk): - zk.ensure_path('/root_a') - zk.ensure_path('/root_b') - logging.debug(f"Create ZK roots:{zk.get_children('/')}") @pytest.fixture(scope="module", autouse=True) def started_cluster(): try: - cluster.add_zookeeper_startup_command(create_zk_roots) cluster.start() yield cluster @@ -28,30 +23,109 @@ def started_cluster(): finally: cluster.shutdown() -def test_chroot_with_same_root(started_cluster): - for i, node in enumerate([node1, node2]): +def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + for instance in zk_nodes: + conn = started_cluster.get_kazoo_client(instance) + conn.get_children('/') + print("All instances of ZooKeeper started") + return + except Exception as ex: + print(("Can't connect to ZooKeeper " + str(ex))) + time.sleep(0.5) + +def test_in_order(started_cluster): + zoo1_ip = started_cluster.get_instance_ip("zoo1") + for i, node in enumerate([node1, node3]): node.query('DROP TABLE IF EXISTS simple SYNC') node.query(''' CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); '''.format(replica=node.name)) - for j in range(2): # Second insert to test deduplication - node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) - time.sleep(1) + time.sleep(5) + assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo1_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo1_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert node1.query('select count() from simple').strip() == '2' - assert node2.query('select count() from simple').strip() == '2' -def test_chroot_with_different_root(started_cluster): - for i, node in [(1, node1), (3, node3)]: - node.query('DROP TABLE IF EXISTS simple_different SYNC') - node.query(''' - CREATE TABLE simple_different (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple_different', '{replica}', date, id, 8192); - '''.format(replica=node.name)) - for j in range(2): # Second insert to test deduplication - node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) - - assert node1.query('select count() from simple_different').strip() == '1' - assert node3.query('select count() from simple_different').strip() == '1' +# def test_round_robin(started_cluster): +# new_config = """ +# +# +# round_robin +# +# zoo1 +# 2181 +# +# +# zoo2 +# 2181 +# +# +# zoo3 +# 2181 +# +# 3000 +# +# +# """ +# for i, node in enumerate([node1, node3]): +# node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) +# node.query("SYSTEM RELOAD CONFIG") +# +# started_cluster.stop_zookeeper_nodes(["zoo1"]) +# zoo2_ip = started_cluster.get_instance_ip("zoo2") +# for i, node in enumerate([node1, node3]): +# node.query('DROP TABLE IF EXISTS simple SYNC') +# node.query(''' +# CREATE TABLE simple (date Date, id UInt32) +# ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); +# '''.format(replica=node.name)) +# assert '::ffff:' + str(zoo2_ip) + '\n' == node.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') +# ## start zoo2, zoo3, table will be readonly too, because it only connect to zoo1 +# started_cluster.start_zookeeper_nodes(["zoo1"]) +# wait_zookeeper_node_to_start(started_cluster, ["zoo1"]) +# +# +# def test_nearest_hostname(started_cluster): +# new_config = """ +# +# +# nearest_hostname +# +# zoo1 +# 2181 +# +# +# zoo2 +# 2181 +# +# +# zoo3 +# 2181 +# +# 3000 +# +# +# """ +# for i, node in enumerate([node1, node3]): +# node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) +# node.query("SYSTEM RELOAD CONFIG") +# +# zoo1_ip = started_cluster.get_instance_ip("zoo1") +# zoo2_ip = started_cluster.get_instance_ip("zoo2") +# zoo3_ip = started_cluster.get_instance_ip("zoo3") +# +# for i, node in enumerate([node1, node3]): +# node.query('DROP TABLE IF EXISTS simple SYNC') +# node.query(''' +# CREATE TABLE simple (date Date, id UInt32) +# ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); +# '''.format(replica=node.name)) +# +# assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') +# assert '::ffff:' + str(zoo2_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') +# assert '::ffff:' + str(zoo3_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py b/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py new file mode 100644 index 00000000000..bee32205499 --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py @@ -0,0 +1,100 @@ +import time +import pytest +import logging +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_in_order.xml') + +node1 = cluster.add_instance('node1', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml", "configs/zookeeper_log.xml"]) +node2 = cluster.add_instance('node2', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml", "configs/zookeeper_log.xml"]) +node3 = cluster.add_instance('node3', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml", "configs/zookeeper_log.xml"]) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + for instance in zk_nodes: + conn = started_cluster.get_kazoo_client(instance) + conn.get_children('/') + print("All instances of ZooKeeper started") + return + except Exception as ex: + print(("Can't connect to ZooKeeper " + str(ex))) + time.sleep(0.5) + + + +def test_round_robin(started_cluster): + + started_cluster.stop_zookeeper_nodes(["zoo1"]) + zoo2_ip = started_cluster.get_instance_ip("zoo2") + for i, node in enumerate([node1, node3]): + node.query('DROP TABLE IF EXISTS simple SYNC') + node.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + + time.sleep(5) + assert '::ffff:' + str(zoo2_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo2_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo2_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + + ## start zoo2, zoo3, table will be readonly too, because it only connect to zoo1 + started_cluster.start_zookeeper_nodes(["zoo1"]) + wait_zookeeper_node_to_start(started_cluster, ["zoo1"]) + + +# def test_nearest_hostname(started_cluster): +# new_config = """ +# +# +# nearest_hostname +# +# zoo1 +# 2181 +# +# +# zoo2 +# 2181 +# +# +# zoo3 +# 2181 +# +# 3000 +# +# +# """ +# for i, node in enumerate([node1, node3]): +# node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) +# node.query("SYSTEM RELOAD CONFIG") +# +# zoo1_ip = started_cluster.get_instance_ip("zoo1") +# zoo2_ip = started_cluster.get_instance_ip("zoo2") +# zoo3_ip = started_cluster.get_instance_ip("zoo3") +# +# for i, node in enumerate([node1, node3]): +# node.query('DROP TABLE IF EXISTS simple SYNC') +# node.query(''' +# CREATE TABLE simple (date Date, id UInt32) +# ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); +# '''.format(replica=node.name)) +# +# assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') +# assert '::ffff:' + str(zoo2_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') +# assert '::ffff:' + str(zoo3_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') From 2110a0fbd1a16cf4fa5f96cb9d7fff16b3aec034 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 08:43:06 +0300 Subject: [PATCH 012/584] 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 e6cf9605a5b88345b8387647dc893bfdbb8d5a3a Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Thu, 21 Oct 2021 15:46:34 +0800 Subject: [PATCH 013/584] Refactor and add test. --- src/Client/ConnectionPoolWithFailover.h | 3 - src/Common/ZooKeeper/ZooKeeper.cpp | 12 +- .../remote_servers_nearest_hostname.xml | 23 +++ .../zookeeper_config_first_or_random.xml | 19 +++ .../zookeeper_config_nearest_hostname.xml | 19 +++ .../test.py | 131 ------------------ .../test_first_or_random.py | 53 +++++++ .../test_in_order.py | 53 +++++++ .../test_nearest_hostname.py | 56 ++++++++ .../test_round_robin.py | 47 +------ 10 files changed, 229 insertions(+), 187 deletions(-) create mode 100644 tests/integration/test_zookeeper_config_load_balancing/configs/remote_servers_nearest_hostname.xml create mode 100644 tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_first_or_random.xml create mode 100644 tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_nearest_hostname.xml delete mode 100644 tests/integration/test_zookeeper_config_load_balancing/test.py create mode 100644 tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py create mode 100644 tests/integration/test_zookeeper_config_load_balancing/test_in_order.py create mode 100644 tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 3c838459733..0ec02cc48de 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -112,9 +112,6 @@ private: private: GetPriorityForLoadBalancing get_priority_load_balancing; -// std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. -// size_t last_used = 0; /// Last used for round_robin policy. -// LoadBalancing default_load_balancing; }; using ConnectionPoolWithFailoverPtr = std::shared_ptr; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index b1f6269d128..30e13c982ec 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -258,16 +258,8 @@ struct ZooKeeperArgs const String & local_hostname = getFQDNOrHostName(); for (size_t i = 0; i < hosts.size(); ++i) { - const String & ip_or_hostname = hosts[i].substr(0, hosts[i].find_last_of(':')); - try - { - get_priority_load_balancing.hostname_differences[i] = DB::getHostNameDifference(local_hostname, Poco::Net::DNS::resolve(ip_or_hostname).name()); - } - catch (...) - { - /// There may be HostNotFoundException or DNSException, these exceptions will be processed later. - LOG_ERROR(&Poco::Logger::get("ZooKeeperArgs"), "Cannot use ZooKeeper host {}, hostname differences will be set to the maximum value", hosts[i]); - } + const String & node_host = hosts[i].substr(0, hosts[i].find_last_of(':')); + get_priority_load_balancing.hostname_differences[i] = DB::getHostNameDifference(local_hostname, node_host); } get_priority_load_balancing.pool_size = hosts.size(); } diff --git a/tests/integration/test_zookeeper_config_load_balancing/configs/remote_servers_nearest_hostname.xml b/tests/integration/test_zookeeper_config_load_balancing/configs/remote_servers_nearest_hostname.xml new file mode 100644 index 00000000000..62f361049c9 --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/configs/remote_servers_nearest_hostname.xml @@ -0,0 +1,23 @@ + + + + + + nod1 + 9000 + + + + nod2 + 9000 + + + + nod3 + 9000 + + + + + + diff --git a/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_first_or_random.xml b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_first_or_random.xml new file mode 100644 index 00000000000..9688480fa90 --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_first_or_random.xml @@ -0,0 +1,19 @@ + + + + first_or_random + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 3000 + + diff --git a/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_nearest_hostname.xml b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_nearest_hostname.xml new file mode 100644 index 00000000000..265ebe05fab --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_config_nearest_hostname.xml @@ -0,0 +1,19 @@ + + + + nearest_hostname + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 3000 + + diff --git a/tests/integration/test_zookeeper_config_load_balancing/test.py b/tests/integration/test_zookeeper_config_load_balancing/test.py deleted file mode 100644 index 951dd7f12b4..00000000000 --- a/tests/integration/test_zookeeper_config_load_balancing/test.py +++ /dev/null @@ -1,131 +0,0 @@ -import time -import pytest -import logging -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_in_order.xml') - -node1 = cluster.add_instance('node1', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) -node2 = cluster.add_instance('node2', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) -node3 = cluster.add_instance('node3', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) - - -@pytest.fixture(scope="module", autouse=True) -def started_cluster(): - try: - cluster.start() - - yield cluster - - finally: - cluster.shutdown() - -def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): - start = time.time() - while time.time() - start < timeout: - try: - for instance in zk_nodes: - conn = started_cluster.get_kazoo_client(instance) - conn.get_children('/') - print("All instances of ZooKeeper started") - return - except Exception as ex: - print(("Can't connect to ZooKeeper " + str(ex))) - time.sleep(0.5) - -def test_in_order(started_cluster): - zoo1_ip = started_cluster.get_instance_ip("zoo1") - for i, node in enumerate([node1, node3]): - node.query('DROP TABLE IF EXISTS simple SYNC') - node.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node.name)) - - time.sleep(5) - assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo1_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo1_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - - -# def test_round_robin(started_cluster): -# new_config = """ -# -# -# round_robin -# -# zoo1 -# 2181 -# -# -# zoo2 -# 2181 -# -# -# zoo3 -# 2181 -# -# 3000 -# -# -# """ -# for i, node in enumerate([node1, node3]): -# node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) -# node.query("SYSTEM RELOAD CONFIG") -# -# started_cluster.stop_zookeeper_nodes(["zoo1"]) -# zoo2_ip = started_cluster.get_instance_ip("zoo2") -# for i, node in enumerate([node1, node3]): -# node.query('DROP TABLE IF EXISTS simple SYNC') -# node.query(''' -# CREATE TABLE simple (date Date, id UInt32) -# ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); -# '''.format(replica=node.name)) -# assert '::ffff:' + str(zoo2_ip) + '\n' == node.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') -# ## start zoo2, zoo3, table will be readonly too, because it only connect to zoo1 -# started_cluster.start_zookeeper_nodes(["zoo1"]) -# wait_zookeeper_node_to_start(started_cluster, ["zoo1"]) -# -# -# def test_nearest_hostname(started_cluster): -# new_config = """ -# -# -# nearest_hostname -# -# zoo1 -# 2181 -# -# -# zoo2 -# 2181 -# -# -# zoo3 -# 2181 -# -# 3000 -# -# -# """ -# for i, node in enumerate([node1, node3]): -# node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) -# node.query("SYSTEM RELOAD CONFIG") -# -# zoo1_ip = started_cluster.get_instance_ip("zoo1") -# zoo2_ip = started_cluster.get_instance_ip("zoo2") -# zoo3_ip = started_cluster.get_instance_ip("zoo3") -# -# for i, node in enumerate([node1, node3]): -# node.query('DROP TABLE IF EXISTS simple SYNC') -# node.query(''' -# CREATE TABLE simple (date Date, id UInt32) -# ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); -# '''.format(replica=node.name)) -# -# assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') -# assert '::ffff:' + str(zoo2_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') -# assert '::ffff:' + str(zoo3_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py b/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py new file mode 100644 index 00000000000..5d510ae3da4 --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py @@ -0,0 +1,53 @@ +import time +import pytest +import logging +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_first_or_random.xml') + +node1 = cluster.add_instance('node1', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml", "configs/zookeeper_log.xml"]) +node2 = cluster.add_instance('node2', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml", "configs/zookeeper_log.xml"]) +node3 = cluster.add_instance('node3', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml", "configs/zookeeper_log.xml"]) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + for instance in zk_nodes: + conn = started_cluster.get_kazoo_client(instance) + conn.get_children('/') + print("All instances of ZooKeeper started") + return + except Exception as ex: + print(("Can't connect to ZooKeeper " + str(ex))) + time.sleep(0.5) + +def test_first_or_random(started_cluster): + wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) + time.sleep(2) + zoo1_ip = started_cluster.get_instance_ip("zoo1") + for i, node in enumerate([node1, node3]): + node.query('DROP TABLE IF EXISTS simple SYNC') + node.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + + time.sleep(5) + assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo1_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo1_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py b/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py new file mode 100644 index 00000000000..88143116170 --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py @@ -0,0 +1,53 @@ +import time +import pytest +import logging +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_in_order.xml') + +node1 = cluster.add_instance('node1', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) +node2 = cluster.add_instance('node2', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) +node3 = cluster.add_instance('node3', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + for instance in zk_nodes: + conn = started_cluster.get_kazoo_client(instance) + conn.get_children('/') + print("All instances of ZooKeeper started") + return + except Exception as ex: + print(("Can't connect to ZooKeeper " + str(ex))) + time.sleep(0.5) + +def test_in_order(started_cluster): + wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) + time.sleep(2) + zoo1_ip = started_cluster.get_instance_ip("zoo1") + for i, node in enumerate([node1, node3]): + node.query('DROP TABLE IF EXISTS simple SYNC') + node.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + + time.sleep(5) + assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo1_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo1_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py b/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py new file mode 100644 index 00000000000..79fa61c0104 --- /dev/null +++ b/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py @@ -0,0 +1,56 @@ +import time +import pytest +import logging +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_nearest_hostname.xml') + +node1 = cluster.add_instance('nod1', with_zookeeper=True, + main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml", "configs/zookeeper_log.xml"]) +node2 = cluster.add_instance('nod2', with_zookeeper=True, + main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml", "configs/zookeeper_log.xml"]) +node3 = cluster.add_instance('nod3', with_zookeeper=True, + main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml", "configs/zookeeper_log.xml"]) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + for instance in zk_nodes: + conn = started_cluster.get_kazoo_client(instance) + conn.get_children('/') + print("All instances of ZooKeeper started") + return + except Exception as ex: + print(("Can't connect to ZooKeeper " + str(ex))) + time.sleep(0.5) + +def test_nearest_hostname(started_cluster): + wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) + time.sleep(2) + zoo1_ip = started_cluster.get_instance_ip("zoo1") + zoo2_ip = started_cluster.get_instance_ip("zoo2") + zoo3_ip = started_cluster.get_instance_ip("zoo3") + + for i, node in enumerate([node1, node3]): + node.query('DROP TABLE IF EXISTS simple SYNC') + node.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + + time.sleep(5) + assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo2_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '::ffff:' + str(zoo3_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py b/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py index bee32205499..f447f929d38 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py @@ -3,7 +3,7 @@ import pytest import logging from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_in_order.xml') +cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_round_robin.xml') node1 = cluster.add_instance('node1', with_zookeeper=True, main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml", "configs/zookeeper_log.xml"]) @@ -39,8 +39,9 @@ def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): def test_round_robin(started_cluster): - + wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) started_cluster.stop_zookeeper_nodes(["zoo1"]) + time.sleep(10) zoo2_ip = started_cluster.get_instance_ip("zoo2") for i, node in enumerate([node1, node3]): node.query('DROP TABLE IF EXISTS simple SYNC') @@ -50,6 +51,7 @@ def test_round_robin(started_cluster): '''.format(replica=node.name)) time.sleep(5) + print("zoo2", zoo2_ip) assert '::ffff:' + str(zoo2_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') assert '::ffff:' + str(zoo2_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') assert '::ffff:' + str(zoo2_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') @@ -57,44 +59,3 @@ def test_round_robin(started_cluster): ## start zoo2, zoo3, table will be readonly too, because it only connect to zoo1 started_cluster.start_zookeeper_nodes(["zoo1"]) wait_zookeeper_node_to_start(started_cluster, ["zoo1"]) - - -# def test_nearest_hostname(started_cluster): -# new_config = """ -# -# -# nearest_hostname -# -# zoo1 -# 2181 -# -# -# zoo2 -# 2181 -# -# -# zoo3 -# 2181 -# -# 3000 -# -# -# """ -# for i, node in enumerate([node1, node3]): -# node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) -# node.query("SYSTEM RELOAD CONFIG") -# -# zoo1_ip = started_cluster.get_instance_ip("zoo1") -# zoo2_ip = started_cluster.get_instance_ip("zoo2") -# zoo3_ip = started_cluster.get_instance_ip("zoo3") -# -# for i, node in enumerate([node1, node3]): -# node.query('DROP TABLE IF EXISTS simple SYNC') -# node.query(''' -# CREATE TABLE simple (date Date, id UInt32) -# ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); -# '''.format(replica=node.name)) -# -# assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') -# assert '::ffff:' + str(zoo2_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') -# assert '::ffff:' + str(zoo3_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') From 5c34e24f4948cab5a49ab9fb7c7a87c29435a9ef Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Thu, 21 Oct 2021 15:57:21 +0800 Subject: [PATCH 014/584] Fix code style --- src/Common/GetPriorityForLoadBalancing.cpp | 3 ++- src/Common/ZooKeeper/ZooKeeper.cpp | 1 - src/Common/ZooKeeper/ZooKeeper.h | 1 - 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Common/GetPriorityForLoadBalancing.cpp b/src/Common/GetPriorityForLoadBalancing.cpp index ae621d9e75c..15ba4e2534c 100644 --- a/src/Common/GetPriorityForLoadBalancing.cpp +++ b/src/Common/GetPriorityForLoadBalancing.cpp @@ -29,7 +29,8 @@ std::function GetPriorityForLoadBalancing::getPriorityFunc * last_used = 3 -> get_priority: 4 3 0 1 2 * ... * */ - get_priority = [&](size_t i) { + get_priority = [&](size_t i) + { ++i; return i < last_used ? pool_size - i : i - last_used; }; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 30e13c982ec..2a964ceba89 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -26,7 +26,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; - extern const int UNKNOWN_LOAD_BALANCING; } } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 94232aeac86..392c0427545 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -43,7 +43,6 @@ constexpr size_t MULTI_BATCH_SIZE = 100; struct ShuffleHost { String host; - /// Priority from the GetPriorityFunc. Int64 priority = 0; UInt32 random = 0; From 6cd0f18bfda86578232d0ace9cd521bb02b65481 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Thu, 21 Oct 2021 17:59:24 +0800 Subject: [PATCH 015/584] Fix PVS check --- src/Client/ConnectionPoolWithFailover.cpp | 2 +- src/Common/GetPriorityForLoadBalancing.h | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index ecfc6bd5c08..0e213dc6700 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -29,7 +29,7 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover( time_t decrease_error_period_, size_t max_error_cap_) : Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Poco::Logger::get("ConnectionPoolWithFailover")) - , get_priority_load_balancing(load_balancing) + , get_priority_load_balancing(load_balancing, nested_pools.size()) { const std::string & local_hostname = getFQDNOrHostName(); diff --git a/src/Common/GetPriorityForLoadBalancing.h b/src/Common/GetPriorityForLoadBalancing.h index b845c2e7616..940ece2b0bc 100644 --- a/src/Common/GetPriorityForLoadBalancing.h +++ b/src/Common/GetPriorityForLoadBalancing.h @@ -8,7 +8,7 @@ namespace DB class GetPriorityForLoadBalancing { public: - GetPriorityForLoadBalancing(LoadBalancing load_balancing_) : load_balancing(load_balancing_) {} + GetPriorityForLoadBalancing(LoadBalancing load_balancing_, size_t pool_size_) : load_balancing(load_balancing_), pool_size(pool_size_) {} GetPriorityForLoadBalancing(){} bool operator!=(const GetPriorityForLoadBalancing & other) @@ -19,11 +19,11 @@ public: std::function getPriorityFunc() const; + LoadBalancing load_balancing = LoadBalancing::RANDOM; + std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. size_t offset = 0; /// for first_or_random policy. - size_t pool_size; /// for round_robin policy. - - LoadBalancing load_balancing = LoadBalancing::RANDOM; + size_t pool_size = 0; /// for round_robin policy. private: mutable size_t last_used = 0; /// Last used for round_robin policy. From ca02f9757c74d22e1bea333def29b29de6e37af1 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 13:19:21 +0300 Subject: [PATCH 016/584] 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 d1891c2527258d6bbfd3b699ae5e836ed687b03c Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Fri, 22 Oct 2021 14:27:50 +0800 Subject: [PATCH 017/584] Fix build --- src/Common/ZooKeeper/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index d29fba53277..7510cd0755c 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -4,7 +4,7 @@ add_headers_and_sources(clickhouse_common_zookeeper .) add_library(clickhouse_common_zookeeper ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) -target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_common_io common PRIVATE string_utils) +target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_common_io common dbms PRIVATE string_utils) if (ENABLE_EXAMPLES) add_subdirectory(examples) From 4e49eba087c8b5fc0db7c7f7dd819e0b5a80701e Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Fri, 22 Oct 2021 20:23:25 +0800 Subject: [PATCH 018/584] Fix data race --- src/Client/ConnectionPoolWithFailover.cpp | 21 ++++++++++++++------- src/Common/GetPriorityForLoadBalancing.cpp | 6 +++--- src/Common/GetPriorityForLoadBalancing.h | 13 +++++-------- src/Common/ZooKeeper/ZooKeeper.cpp | 3 +-- 4 files changed, 23 insertions(+), 20 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 0e213dc6700..accde6b5475 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -29,7 +29,7 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover( time_t decrease_error_period_, size_t max_error_cap_) : Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Poco::Logger::get("ConnectionPoolWithFailover")) - , get_priority_load_balancing(load_balancing, nested_pools.size()) + , get_priority_load_balancing(load_balancing) { const std::string & local_hostname = getFQDNOrHostName(); @@ -50,12 +50,16 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts return tryGetEntry(pool, timeouts, fail_message, settings); }; + GetPriorityForLoadBalancing get_priority_local(get_priority_load_balancing); + size_t offset = 0; + LoadBalancing load_balancing = get_priority_load_balancing.load_balancing; if (settings) { - get_priority_load_balancing.offset = settings->load_balancing_first_offset % nested_pools.size(); - get_priority_load_balancing.load_balancing = settings->load_balancing; + offset = settings->load_balancing_first_offset % nested_pools.size(); + load_balancing = LoadBalancing(settings->load_balancing); } - GetPriorityFunc get_priority = get_priority_load_balancing.getPriorityFunc(); + + GetPriorityFunc get_priority = get_priority_local.getPriorityFunc(load_balancing, offset, nested_pools.size()); UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0; bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true; @@ -148,12 +152,15 @@ std::vector ConnectionPoolWithFailover::g ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings * settings) { + size_t offset = 0; + LoadBalancing load_balancing = get_priority_load_balancing.load_balancing; if (settings) { - get_priority_load_balancing.offset = settings->load_balancing_first_offset % nested_pools.size(); - get_priority_load_balancing.load_balancing = settings->load_balancing; + offset = settings->load_balancing_first_offset % nested_pools.size(); + load_balancing = LoadBalancing(settings->load_balancing); } - return get_priority_load_balancing.getPriorityFunc(); + + return get_priority_load_balancing.getPriorityFunc(load_balancing, offset, nested_pools.size()); } std::vector ConnectionPoolWithFailover::getManyImpl( diff --git a/src/Common/GetPriorityForLoadBalancing.cpp b/src/Common/GetPriorityForLoadBalancing.cpp index 15ba4e2534c..fa0eeb14bed 100644 --- a/src/Common/GetPriorityForLoadBalancing.cpp +++ b/src/Common/GetPriorityForLoadBalancing.cpp @@ -3,10 +3,10 @@ namespace DB { -std::function GetPriorityForLoadBalancing::getPriorityFunc() const +std::function GetPriorityForLoadBalancing::getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const { std::function get_priority; - switch (load_balancing) + switch (load_balance) { case LoadBalancing::NEAREST_HOSTNAME: get_priority = [&](size_t i) { return hostname_differences[i]; }; @@ -17,7 +17,7 @@ std::function GetPriorityForLoadBalancing::getPriorityFunc case LoadBalancing::RANDOM: break; case LoadBalancing::FIRST_OR_RANDOM: - get_priority = [&](size_t i) -> size_t { return i != offset; }; + get_priority = [offset](size_t i) -> size_t { return i != offset; }; break; case LoadBalancing::ROUND_ROBIN: if (last_used >= pool_size) diff --git a/src/Common/GetPriorityForLoadBalancing.h b/src/Common/GetPriorityForLoadBalancing.h index 940ece2b0bc..a6b8c88bb73 100644 --- a/src/Common/GetPriorityForLoadBalancing.h +++ b/src/Common/GetPriorityForLoadBalancing.h @@ -8,22 +8,19 @@ namespace DB class GetPriorityForLoadBalancing { public: - GetPriorityForLoadBalancing(LoadBalancing load_balancing_, size_t pool_size_) : load_balancing(load_balancing_), pool_size(pool_size_) {} + GetPriorityForLoadBalancing(LoadBalancing load_balancing_) : load_balancing(load_balancing_) {} GetPriorityForLoadBalancing(){} bool operator!=(const GetPriorityForLoadBalancing & other) { - return offset != other.offset || pool_size != other.pool_size || load_balancing != other.load_balancing - || hostname_differences != other.hostname_differences; + return load_balancing != other.load_balancing || hostname_differences != other.hostname_differences; } - std::function getPriorityFunc() const; - - LoadBalancing load_balancing = LoadBalancing::RANDOM; + std::function getPriorityFunc(LoadBalancing load_balancing, size_t offset, size_t pool_size) const; std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. - size_t offset = 0; /// for first_or_random policy. - size_t pool_size = 0; /// for round_robin policy. + + LoadBalancing load_balancing = LoadBalancing::RANDOM; private: mutable size_t last_used = 0; /// Last used for round_robin policy. diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 2a964ceba89..5e43eda636c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -155,7 +155,7 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ std::vector ZooKeeper::shuffleHosts() const { - std::function get_priority = get_priority_load_balancing.getPriorityFunc(); + std::function get_priority = get_priority_load_balancing.getPriorityFunc(get_priority_load_balancing.load_balancing, 0, hosts.size()); std::vector shuffle_hosts; for (size_t i = 0; i < hosts.size(); ++i) { @@ -260,7 +260,6 @@ struct ZooKeeperArgs const String & node_host = hosts[i].substr(0, hosts[i].find_last_of(':')); get_priority_load_balancing.hostname_differences[i] = DB::getHostNameDifference(local_hostname, node_host); } - get_priority_load_balancing.pool_size = hosts.size(); } Strings hosts; From 1710e5ea5989cbe9103df7551a98e8bbf20e1a7b Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Mon, 25 Oct 2021 19:10:54 +0800 Subject: [PATCH 019/584] Fix build --- src/CMakeLists.txt | 2 ++ src/Common/ZooKeeper/CMakeLists.txt | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 09aaa85c394..95f2051399d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -183,6 +183,8 @@ endmacro() add_object_library(clickhouse_access Access) add_object_library(clickhouse_backups Backups) add_object_library(clickhouse_core Core) +add_library (clickhouse_core_settings_enums Core/SettingsEnums.cpp) +target_link_libraries(clickhouse_core_settings_enums PRIVATE common clickhouse_common_io) add_object_library(clickhouse_core_mysql Core/MySQL) add_object_library(clickhouse_compression Compression) add_object_library(clickhouse_querypipeline QueryPipeline) diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index 7510cd0755c..5797a0a5e21 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -4,7 +4,7 @@ add_headers_and_sources(clickhouse_common_zookeeper .) add_library(clickhouse_common_zookeeper ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) -target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_common_io common dbms PRIVATE string_utils) +target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_core_settings_enums clickhouse_common_io common PRIVATE string_utils) if (ENABLE_EXAMPLES) add_subdirectory(examples) From bf9aebac90be71566b8cf8650dd16c56b478bb27 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Tue, 26 Oct 2021 12:45:09 +0800 Subject: [PATCH 020/584] Fix test and build --- src/Client/ConnectionPoolWithFailover.cpp | 3 +-- src/Common/GetPriorityForLoadBalancing.h | 2 +- .../test_first_or_random.py | 2 +- .../test_zookeeper_config_load_balancing/test_in_order.py | 2 +- .../test_nearest_hostname.py | 2 +- 5 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index accde6b5475..13d39980e1c 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -50,7 +50,6 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts return tryGetEntry(pool, timeouts, fail_message, settings); }; - GetPriorityForLoadBalancing get_priority_local(get_priority_load_balancing); size_t offset = 0; LoadBalancing load_balancing = get_priority_load_balancing.load_balancing; if (settings) @@ -59,7 +58,7 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts load_balancing = LoadBalancing(settings->load_balancing); } - GetPriorityFunc get_priority = get_priority_local.getPriorityFunc(load_balancing, offset, nested_pools.size()); + GetPriorityFunc get_priority = get_priority_load_balancing.getPriorityFunc(load_balancing, offset, nested_pools.size()); UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0; bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true; diff --git a/src/Common/GetPriorityForLoadBalancing.h b/src/Common/GetPriorityForLoadBalancing.h index a6b8c88bb73..4ec686188e4 100644 --- a/src/Common/GetPriorityForLoadBalancing.h +++ b/src/Common/GetPriorityForLoadBalancing.h @@ -16,7 +16,7 @@ public: return load_balancing != other.load_balancing || hostname_differences != other.hostname_differences; } - std::function getPriorityFunc(LoadBalancing load_balancing, size_t offset, size_t pool_size) const; + std::function getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const; std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py b/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py index 5d510ae3da4..71084492b44 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py @@ -38,7 +38,7 @@ def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): def test_first_or_random(started_cluster): wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) - time.sleep(2) + time.sleep(10) zoo1_ip = started_cluster.get_instance_ip("zoo1") for i, node in enumerate([node1, node3]): node.query('DROP TABLE IF EXISTS simple SYNC') diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py b/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py index 88143116170..92ba927860c 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py @@ -38,7 +38,7 @@ def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): def test_in_order(started_cluster): wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) - time.sleep(2) + time.sleep(10) zoo1_ip = started_cluster.get_instance_ip("zoo1") for i, node in enumerate([node1, node3]): node.query('DROP TABLE IF EXISTS simple SYNC') diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py b/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py index 79fa61c0104..832af32bbae 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py @@ -38,7 +38,7 @@ def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): def test_nearest_hostname(started_cluster): wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) - time.sleep(2) + time.sleep(10) zoo1_ip = started_cluster.get_instance_ip("zoo1") zoo2_ip = started_cluster.get_instance_ip("zoo2") zoo3_ip = started_cluster.get_instance_ip("zoo3") From 4e53eb0e4e65f8d1da865db971f78955a39f5920 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Wed, 27 Oct 2021 11:29:31 +0800 Subject: [PATCH 021/584] empty commit 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 022/584] 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 49e03025dde6513921787d0ab9b56df27fb557b7 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Mon, 1 Nov 2021 18:26:50 +0800 Subject: [PATCH 023/584] better test --- .../configs/zookeeper_log.xml | 7 ---- .../test_first_or_random.py | 34 +++------------ .../test_in_order.py | 35 +++------------- .../test_nearest_hostname.py | 38 +++-------------- .../test_round_robin.py | 41 ++++--------------- 5 files changed, 26 insertions(+), 129 deletions(-) delete mode 100644 tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_log.xml diff --git a/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_log.xml b/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_log.xml deleted file mode 100644 index a70cbc3ecc2..00000000000 --- a/tests/integration/test_zookeeper_config_load_balancing/configs/zookeeper_log.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - system - zookeeper_log
- 7500 -
-
diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py b/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py index 71084492b44..38361016512 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py @@ -6,11 +6,11 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_first_or_random.xml') node1 = cluster.add_instance('node1', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml"]) node2 = cluster.add_instance('node2', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml"]) node3 = cluster.add_instance('node3', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml"]) @pytest.fixture(scope="module", autouse=True) @@ -23,31 +23,9 @@ def started_cluster(): finally: cluster.shutdown() -def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): - start = time.time() - while time.time() - start < timeout: - try: - for instance in zk_nodes: - conn = started_cluster.get_kazoo_client(instance) - conn.get_children('/') - print("All instances of ZooKeeper started") - return - except Exception as ex: - print(("Can't connect to ZooKeeper " + str(ex))) - time.sleep(0.5) def test_first_or_random(started_cluster): - wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) - time.sleep(10) - zoo1_ip = started_cluster.get_instance_ip("zoo1") - for i, node in enumerate([node1, node3]): - node.query('DROP TABLE IF EXISTS simple SYNC') - node.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node.name)) - time.sleep(5) - assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo1_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo1_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '1' == str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + assert '1' == str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + assert '1' == str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py b/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py index 92ba927860c..c11d05f0a75 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py @@ -6,11 +6,11 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_in_order.xml') node1 = cluster.add_instance('node1', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml"]) node2 = cluster.add_instance('node2', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml"]) node3 = cluster.add_instance('node3', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml"]) @pytest.fixture(scope="module", autouse=True) @@ -23,31 +23,8 @@ def started_cluster(): finally: cluster.shutdown() -def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): - start = time.time() - while time.time() - start < timeout: - try: - for instance in zk_nodes: - conn = started_cluster.get_kazoo_client(instance) - conn.get_children('/') - print("All instances of ZooKeeper started") - return - except Exception as ex: - print(("Can't connect to ZooKeeper " + str(ex))) - time.sleep(0.5) - def test_in_order(started_cluster): - wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) - time.sleep(10) - zoo1_ip = started_cluster.get_instance_ip("zoo1") - for i, node in enumerate([node1, node3]): - node.query('DROP TABLE IF EXISTS simple SYNC') - node.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node.name)) - time.sleep(5) - assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo1_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo1_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '1' == str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + assert '1' == str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + assert '1' == str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py b/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py index 832af32bbae..30fca5c5395 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py @@ -6,11 +6,11 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_nearest_hostname.xml') node1 = cluster.add_instance('nod1', with_zookeeper=True, - main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml"]) node2 = cluster.add_instance('nod2', with_zookeeper=True, - main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml"]) node3 = cluster.add_instance('nod3', with_zookeeper=True, - main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml"]) @pytest.fixture(scope="module", autouse=True) @@ -23,34 +23,8 @@ def started_cluster(): finally: cluster.shutdown() -def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): - start = time.time() - while time.time() - start < timeout: - try: - for instance in zk_nodes: - conn = started_cluster.get_kazoo_client(instance) - conn.get_children('/') - print("All instances of ZooKeeper started") - return - except Exception as ex: - print(("Can't connect to ZooKeeper " + str(ex))) - time.sleep(0.5) - def test_nearest_hostname(started_cluster): - wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) - time.sleep(10) - zoo1_ip = started_cluster.get_instance_ip("zoo1") - zoo2_ip = started_cluster.get_instance_ip("zoo2") - zoo3_ip = started_cluster.get_instance_ip("zoo3") - for i, node in enumerate([node1, node3]): - node.query('DROP TABLE IF EXISTS simple SYNC') - node.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node.name)) - - time.sleep(5) - assert '::ffff:' + str(zoo1_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo2_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo3_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '1' == str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + assert '1' == str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo2_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + assert '1' == str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo3_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py b/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py index f447f929d38..98d751f075b 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py @@ -6,11 +6,11 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_round_robin.xml') node1 = cluster.add_instance('node1', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml"]) node2 = cluster.add_instance('node2', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml"]) node3 = cluster.add_instance('node3', with_zookeeper=True, - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml", "configs/zookeeper_log.xml"]) + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml"]) @pytest.fixture(scope="module", autouse=True) @@ -23,39 +23,14 @@ def started_cluster(): finally: cluster.shutdown() -def wait_zookeeper_node_to_start(started_cluster, zk_nodes, timeout=60): - start = time.time() - while time.time() - start < timeout: - try: - for instance in zk_nodes: - conn = started_cluster.get_kazoo_client(instance) - conn.get_children('/') - print("All instances of ZooKeeper started") - return - except Exception as ex: - print(("Can't connect to ZooKeeper " + str(ex))) - time.sleep(0.5) - - def test_round_robin(started_cluster): - wait_zookeeper_node_to_start(started_cluster, ["zoo1", "zoo2", "zoo3"]) + started_cluster.stop_zookeeper_nodes(["zoo1"]) - time.sleep(10) - zoo2_ip = started_cluster.get_instance_ip("zoo2") - for i, node in enumerate([node1, node3]): - node.query('DROP TABLE IF EXISTS simple SYNC') - node.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node.name)) + time.sleep(1) - time.sleep(5) - print("zoo2", zoo2_ip) - assert '::ffff:' + str(zoo2_ip) + '\n' == node1.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo2_ip) + '\n' == node2.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') - assert '::ffff:' + str(zoo2_ip) + '\n' == node3.query('SELECT IPv6NumToString(address) FROM system.zookeeper_log ORDER BY event_time DESC LIMIT 1') + assert '1' == str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo2_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + assert '1' == str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo2_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + assert '1' == str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo2_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() - ## start zoo2, zoo3, table will be readonly too, because it only connect to zoo1 started_cluster.start_zookeeper_nodes(["zoo1"]) - wait_zookeeper_node_to_start(started_cluster, ["zoo1"]) From 821ad7cb2ae2c0c66afe3279cf0ebbf6c1c09bc2 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Tue, 2 Nov 2021 15:40:05 +0800 Subject: [PATCH 024/584] try fix test --- .../{test_first_or_random.py => test.py} | 8 ++++++-- .../test_zookeeper_config_load_balancing/test_in_order.py | 7 ++++++- .../test_nearest_hostname.py | 7 ++++++- .../test_round_robin.py | 7 ++++++- 4 files changed, 24 insertions(+), 5 deletions(-) rename tests/integration/test_zookeeper_config_load_balancing/{test_first_or_random.py => test.py} (78%) diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py b/tests/integration/test_zookeeper_config_load_balancing/test.py similarity index 78% rename from tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py rename to tests/integration/test_zookeeper_config_load_balancing/test.py index 38361016512..144ba14ce40 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_first_or_random.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test.py @@ -13,7 +13,7 @@ node3 = cluster.add_instance('node3', with_zookeeper=True, main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_first_or_random.xml"]) -@pytest.fixture(scope="module", autouse=True) +@pytest.fixture(scope="module") def started_cluster(): try: cluster.start() @@ -25,7 +25,11 @@ def started_cluster(): def test_first_or_random(started_cluster): - + print(str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + + print(str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + + print(str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py b/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py index c11d05f0a75..095aba72217 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_in_order.py @@ -13,7 +13,7 @@ node3 = cluster.add_instance('node3', with_zookeeper=True, main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_in_order.xml"]) -@pytest.fixture(scope="module", autouse=True) +@pytest.fixture(scope="module") def started_cluster(): try: cluster.start() @@ -25,6 +25,11 @@ def started_cluster(): def test_in_order(started_cluster): + print(str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + + print(str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + + print(str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py b/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py index 30fca5c5395..23c0386b1d2 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_nearest_hostname.py @@ -13,7 +13,7 @@ node3 = cluster.add_instance('nod3', with_zookeeper=True, main_configs=["configs/remote_servers_nearest_hostname.xml", "configs/zookeeper_config_nearest_hostname.xml"]) -@pytest.fixture(scope="module", autouse=True) +@pytest.fixture(scope="module") def started_cluster(): try: cluster.start() @@ -25,6 +25,11 @@ def started_cluster(): def test_nearest_hostname(started_cluster): + print(str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo1_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + + print(str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo2_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + + print(str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo3_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() diff --git a/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py b/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py index 98d751f075b..3623371c244 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test_round_robin.py @@ -13,7 +13,7 @@ node3 = cluster.add_instance('node3', with_zookeeper=True, main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_round_robin.xml"]) -@pytest.fixture(scope="module", autouse=True) +@pytest.fixture(scope="module") def started_cluster(): try: cluster.start() @@ -29,8 +29,13 @@ def test_round_robin(started_cluster): started_cluster.stop_zookeeper_nodes(["zoo1"]) time.sleep(1) + print(str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node1.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo2_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + + print(str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node2.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo2_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() + + print(str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED"], privileged=True, user='root'))) assert '1' == str(node3.exec_in_container(['bash', '-c', "lsof -a -i4 -i6 -itcp -w | grep 'roottestzookeeperconfigloadbalancing_zoo2_1.roottestzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l"], privileged=True, user='root')).strip() started_cluster.start_zookeeper_nodes(["zoo1"]) 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 025/584] 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 026/584] 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 027/584] 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 028/584] 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 029/584] 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 030/584] 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 031/584] 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 032/584] 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 033/584] 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 034/584] 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 035/584] 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 036/584] 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 37ba8004ff1ece618aee91d446a4c7cc23ccfdd7 Mon Sep 17 00:00:00 2001 From: liyang Date: Wed, 8 Dec 2021 02:40:59 +0000 Subject: [PATCH 037/584] Speep up mergetree starting up process --- programs/server/Server.cpp | 9 + programs/server/config.xml | 3 + src/CMakeLists.txt | 3 + src/Common/CurrentMetrics.cpp | 1 + src/Common/ProfileEvents.cpp | 8 + src/Functions/checkPartMetaCache.cpp | 157 ++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + src/Interpreters/Context.cpp | 92 +++++ src/Interpreters/Context.h | 29 +- src/Processors/CMakeLists.txt | 4 +- src/Processors/examples/CMakeLists.txt | 2 + .../examples/merge_tree_meta_cache.cpp | 51 +++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 336 ++++++++++++++++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 58 ++- src/Storages/MergeTree/KeyCondition.cpp | 2 +- src/Storages/MergeTree/KeyCondition.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 8 + src/Storages/MergeTree/MergeTreeData.h | 2 + .../MergeTree/MergeTreeDataPartCompact.cpp | 13 + .../MergeTree/MergeTreeDataPartCompact.h | 4 + .../MergeTree/MergeTreeDataPartInMemory.cpp | 11 + .../MergeTree/MergeTreeDataPartInMemory.h | 4 + .../MergeTree/MergeTreeDataPartWide.cpp | 15 +- .../MergeTree/MergeTreeDataPartWide.h | 5 + src/Storages/MergeTree/MergeTreePartition.cpp | 22 +- src/Storages/MergeTree/MergeTreePartition.h | 5 +- src/Storages/MergeTree/PartMetaCache.cpp | 134 +++++++ src/Storages/MergeTree/PartMetaCache.h | 45 +++ .../StorageSystemMergeTreeMetaCache.cpp | 139 ++++++++ .../System/StorageSystemMergeTreeMetaCache.h | 29 ++ src/Storages/System/attachSystemTables.cpp | 2 + .../01233_check_part_meta_cache.reference | 28 ++ .../01233_check_part_meta_cache.sql | 123 +++++++ ..._check_part_meta_cache_in_atomic.reference | 28 ++ .../01233_check_part_meta_cache_in_atomic.sql | 124 +++++++ ...check_part_meta_cache_replicated.reference | 28 ++ ...01233_check_part_meta_cache_replicated.sql | 125 +++++++ ..._meta_cache_replicated_in_atomic.reference | 28 ++ ...k_part_meta_cache_replicated_in_atomic.sql | 125 +++++++ 39 files changed, 1767 insertions(+), 39 deletions(-) create mode 100644 src/Functions/checkPartMetaCache.cpp create mode 100644 src/Processors/examples/merge_tree_meta_cache.cpp create mode 100644 src/Storages/MergeTree/PartMetaCache.cpp create mode 100644 src/Storages/MergeTree/PartMetaCache.h create mode 100644 src/Storages/System/StorageSystemMergeTreeMetaCache.cpp create mode 100644 src/Storages/System/StorageSystemMergeTreeMetaCache.h create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache.reference create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache.sql create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 14075f9fbf2..fd2dc851ae7 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -747,6 +747,15 @@ if (ThreadFuzzer::instance().isEffective()) /// Directory with metadata of tables, which was marked as dropped by Atomic database fs::create_directories(path / "metadata_dropped/"); + + fs::create_directories(path / "rocksdb/"); + } + + + /// initialize meta file cache + { + size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); + global_context->initializeMergeTreeMetaCache(path_str + "/" + "rocksdb", size); } if (config().has("interserver_http_port") && config().has("interserver_https_port")) diff --git a/programs/server/config.xml b/programs/server/config.xml index d88773a3fc4..470c4dfa35f 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1291,4 +1291,7 @@ --> + + + 268435456
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 7124961821e..42cd4f65f60 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -500,6 +500,9 @@ endif () if (USE_ROCKSDB) dbms_target_link_libraries(PUBLIC ${ROCKSDB_LIBRARY}) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) + + target_link_libraries (clickhouse_common_io PUBLIC ${ROCKSDB_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) endif() if (USE_LIBPQXX) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 5c9ba177b78..d214952deae 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -78,6 +78,7 @@ M(SyncDrainedConnections, "Number of connections drained synchronously.") \ M(ActiveSyncDrainedConnections, "Number of active connections drained synchronously.") \ M(AsynchronousReadWait, "Number of threads waiting for asynchronous read.") \ + M(ServerStartupSeconds, "Server start seconds") \ namespace CurrentMetrics { diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 982523a3ef2..8b0144be842 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -274,6 +274,14 @@ M(ThreadPoolReaderPageCacheMissElapsedMicroseconds, "Time spent reading data inside the asynchronous job in ThreadPoolReader - when read was not done from page cache.") \ \ M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ + \ + M(RocksdbGet, "Number of rocksdb reads(used for file meta cache)") \ + M(RocksdbPut, "Number of rocksdb puts(used for file meta cache)") \ + M(RocksdbDelete, "Number of rocksdb deletes(used for file meta cache)") \ + M(RocksdbSeek, "Number of rocksdb seeks(used for file meta cache)") \ + M(MergeTreeMetaCacheHit, "Number of times the read of meta file was done from MergeTree meta cache") \ + M(MergeTreeMetaCacheMiss, "Number of times the read of meta file was not done from MergeTree meta cache") \ + \ namespace ProfileEvents diff --git a/src/Functions/checkPartMetaCache.cpp b/src/Functions/checkPartMetaCache.cpp new file mode 100644 index 00000000000..e0479a5fdcc --- /dev/null +++ b/src/Functions/checkPartMetaCache.cpp @@ -0,0 +1,157 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; +} + +class FunctionCheckPartMetaCache: public IFunction, WithContext +{ +public: + using uint128 = IMergeTreeDataPart::uint128; + using DataPartPtr = MergeTreeData::DataPartPtr; + using DataPartState = MergeTreeData::DataPartState; + using DataPartStates = MergeTreeData::DataPartStates; + + + static constexpr auto name = "checkPartMetaCache"; + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + static constexpr DataPartStates part_states = { + DataPartState::Committed, + DataPartState::Temporary, + DataPartState::PreCommitted, + DataPartState::Outdated, + DataPartState::Deleting, + DataPartState::DeleteOnDestroy + }; + + explicit FunctionCheckPartMetaCache(ContextPtr context_): WithContext(context_) {} + + String getName() const override { return name; } + + bool isDeterministic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + bool isDeterministicInScopeOfQuery() const override { return false; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + for (const auto & argument : arguments) + { + if (!isString(argument)) + throw Exception("The argument of function " + getName() + " must have String type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + DataTypePtr key_type = std::make_unique(); + DataTypePtr state_type = std::make_unique(); + DataTypePtr cache_checksum_type = std::make_unique(32); + DataTypePtr disk_checksum_type = std::make_unique(32); + DataTypePtr match_type = std::make_unique(); + DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, state_type, cache_checksum_type, disk_checksum_type, match_type}); + return std::make_shared(tuple_type); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + // get database name + const auto * arg_database = arguments[0].column.get(); + const ColumnString * column_database = checkAndGetColumnConstData(arg_database); + if (! column_database) + throw Exception("The first argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); + String database_name = column_database->getDataAt(0).toString(); + + // get table name + const auto * arg_table = arguments[1].column.get(); + const ColumnString * column_table = checkAndGetColumnConstData(arg_table); + if (! column_table) + throw Exception("The second argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); + String table_name = column_table->getDataAt(0).toString(); + + // get storage + StorageID storage_id(database_name, table_name); + auto storage = DatabaseCatalog::instance().getTable(storage_id, getContext()); + auto data = std::dynamic_pointer_cast(storage); + if (! data) + throw Exception("The table in function " + getName() + " must be in MergeTree Family", ErrorCodes::ILLEGAL_COLUMN); + + // fill in result + auto col_result = result_type->createColumn(); + auto& col_arr = assert_cast(*col_result); + col_arr.reserve(1); + auto& col_tuple = assert_cast(col_arr.getData()); + col_tuple.reserve(data->fileNumberOfDataParts(part_states)); + auto& col_key = assert_cast(col_tuple.getColumn(0)); + auto& col_state = assert_cast(col_tuple.getColumn(1)); + auto& col_cache_checksum = assert_cast(col_tuple.getColumn(2)); + auto& col_disk_checksum = assert_cast(col_tuple.getColumn(3)); + auto& col_match = assert_cast(col_tuple.getColumn(4)); + auto parts = data->getDataParts(part_states); + for (const auto & part : parts) + executePart(part, col_key, col_state, col_cache_checksum, col_disk_checksum, col_match); + col_arr.getOffsets().push_back(col_tuple.size()); + return result_type->createColumnConst(input_rows_count, col_arr[0]); + } + + static void executePart(const DataPartPtr& part, ColumnString& col_key, ColumnString& col_state, + ColumnFixedString& col_cache_checksum, ColumnFixedString& col_disk_checksum, ColumnUInt8& col_match) + { + Strings keys; + auto state_view = part->stateString(); + String state(state_view.data(), state_view.size()); + std::vector cache_checksums; + std::vector disk_checksums; + uint8_t match = 0; + size_t file_number = part->fileNumberOfColumnsChecksumsIndexes(); + keys.reserve(file_number); + cache_checksums.reserve(file_number); + disk_checksums.reserve(file_number); + + part->checkMetaCache(keys, cache_checksums, disk_checksums); + for (size_t i = 0; i < keys.size(); ++i) + { + col_key.insert(keys[i]); + col_state.insert(state); + col_cache_checksum.insert(getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); + col_disk_checksum.insert(getHexUIntUppercase(disk_checksums[i].first) + getHexUIntUppercase(disk_checksums[i].second)); + + match = cache_checksums[i] == disk_checksums[i] ? 1 : 0; + col_match.insertValue(match); + } + } +}; + +void registerFunctionCheckPartMetaCache(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 76d61ce509a..d613d7c85bd 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -80,6 +80,7 @@ void registerFunctionInitialQueryID(FunctionFactory & factory); void registerFunctionServerUUID(FunctionFactory &); void registerFunctionZooKeeperSessionUptime(FunctionFactory &); void registerFunctionGetOSKernelVersion(FunctionFactory &); +void registerFunctionCheckPartMetaCache(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -166,6 +167,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionServerUUID(factory); registerFunctionZooKeeperSessionUptime(factory); registerFunctionGetOSKernelVersion(factory); + registerFunctionCheckPartMetaCache(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index db1d6a37877..81db45c6461 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -94,6 +94,11 @@ namespace fs = std::filesystem; namespace ProfileEvents { extern const Event ContextLock; + extern const Event CompiledCacheSizeBytes; + extern const Event RocksdbPut; + extern const Event RocksdbGet; + extern const Event RocksdbDelete; + extern const Event RocksdbSeek; } namespace CurrentMetrics @@ -126,6 +131,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int INVALID_SETTING_VALUE; extern const int UNKNOWN_READ_METHOD; + extern const int SYSTEM_ERROR; } @@ -272,6 +278,9 @@ struct ContextSharedPart Context::ConfigReloadCallback config_reload_callback; + /// MergeTree metadata cache stored in rocksdb. + MergeTreeMetaCachePtr merge_tree_meta_cache; + ContextSharedPart() : access_control(std::make_unique()), macros(std::make_unique()) { @@ -382,6 +391,13 @@ struct ContextSharedPart trace_collector.reset(); /// Stop zookeeper connection zookeeper.reset(); + + /// Shutdown meta file cache + if (merge_tree_meta_cache) + { + merge_tree_meta_cache->shutdown(); + merge_tree_meta_cache.reset(); + } } /// Can be removed w/o context lock @@ -425,6 +441,57 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr shar void SharedContextHolder::reset() { shared.reset(); } +MergeTreeMetaCache::Status MergeTreeMetaCache::put(const String & key, const String & value) +{ + auto options = rocksdb::WriteOptions(); + auto status = rocksdb->Put(options, key, value); + ProfileEvents::increment(ProfileEvents::RocksdbPut); + return status; +} + +MergeTreeMetaCache::Status MergeTreeMetaCache::del(const String & key) +{ + auto options = rocksdb::WriteOptions(); + auto status = rocksdb->Delete(options, key); + ProfileEvents::increment(ProfileEvents::RocksdbDelete); + LOG_TRACE(log, "Delete key:{} from MergeTreeMetaCache status:{}", key, status.ToString()); + return status; +} + +MergeTreeMetaCache::Status MergeTreeMetaCache::get(const String & key, String & value) +{ + auto status = rocksdb->Get(rocksdb::ReadOptions(), key, &value); + ProfileEvents::increment(ProfileEvents::RocksdbGet); + LOG_TRACE(log, "Get key:{} from MergeTreeMetaCache status:{}", key, status.ToString()); + return status; +} + +void MergeTreeMetaCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) +{ + auto * it = rocksdb->NewIterator(rocksdb::ReadOptions()); + rocksdb::Slice target(prefix); + for (it->Seek(target); it->Valid(); it->Next()) + { + const auto key = it->key(); + if (!key.starts_with(target)) + break; + + const auto value = it->value(); + keys.emplace_back(key.data(), key.size()); + values.emplace_back(value.data(), value.size()); + } + LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetaCache items:{}", prefix, keys.size()); + ProfileEvents::increment(ProfileEvents::RocksdbSeek); +} + +void MergeTreeMetaCache::shutdown() +{ + if (rocksdb) + { + rocksdb->Close(); + } +} + ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) { auto res = std::shared_ptr(new Context); @@ -2005,6 +2072,11 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const return zookeeper->second; } +MergeTreeMetaCachePtr Context::getMergeTreeMetaCache() const +{ + return shared->merge_tree_meta_cache; +} + void Context::resetZooKeeper() const { std::lock_guard lock(shared->zookeeper_mutex); @@ -2237,6 +2309,26 @@ void Context::initializeTraceCollector() shared->initializeTraceCollector(getTraceLog()); } +void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) +{ + rocksdb::Options options; + rocksdb::BlockBasedTableOptions table_options; + rocksdb::DB * db; + + options.create_if_missing = true; + options.statistics = rocksdb::CreateDBStatistics(); + auto cache = rocksdb::NewLRUCache(size); + table_options.block_cache = cache; + options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); + rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); + if (status != rocksdb::Status::OK()) + { + String message = "Fail to open rocksdb path at: " + dir + " status:" + status.ToString(); + throw Exception(message, ErrorCodes::SYSTEM_ERROR); + } + shared->merge_tree_meta_cache = std::make_shared(db); +} + bool Context::hasTraceCollector() const { return shared->hasTraceCollector(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 823bc028c15..b39e06b0b0f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include "config_core.h" @@ -29,7 +31,6 @@ namespace Poco::Net { class IPAddress; } namespace zkutil { class ZooKeeper; } - namespace DB { @@ -178,6 +179,27 @@ private: std::unique_ptr shared; }; +class MergeTreeMetaCache +{ +public: + using Status = rocksdb::Status; + + explicit MergeTreeMetaCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } + MergeTreeMetaCache(const MergeTreeMetaCache &) = delete; + MergeTreeMetaCache & operator=(const MergeTreeMetaCache &) = delete; + + Status put(const String & key, const String & value); + Status del(const String & key); + Status get(const String & key, String & value); + void getByPrefix(const String & prefix, Strings & keys, Strings & values); + + void shutdown(); +private: + std::unique_ptr rocksdb; + Poco::Logger * log = &Poco::Logger::get("MergeTreeMetaCache"); +}; +using MergeTreeMetaCachePtr = std::shared_ptr; + /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). @@ -677,6 +699,9 @@ public: UInt32 getZooKeeperSessionUptime() const; + MergeTreeMetaCachePtr getMergeTreeMetaCache() const; + + #if USE_NURAFT std::shared_ptr & getKeeperDispatcher() const; #endif @@ -763,6 +788,8 @@ public: /// Call after initialization before using trace collector. void initializeTraceCollector(); + void initializeMergeTreeMetaCache(const String & dir, size_t size); + bool hasTraceCollector() const; /// Nullptr if the query log is not ready for this moment. diff --git a/src/Processors/CMakeLists.txt b/src/Processors/CMakeLists.txt index 7e965188b4c..7c9ad405432 100644 --- a/src/Processors/CMakeLists.txt +++ b/src/Processors/CMakeLists.txt @@ -1,4 +1,4 @@ -if (ENABLE_EXAMPLES) +#if (ENABLE_EXAMPLES) add_subdirectory(examples) -endif () +#endif () diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index e69de29bb2d..dcb640c383a 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) +target_link_libraries (merge_tree_meta_cache PRIVATE dbms) diff --git a/src/Processors/examples/merge_tree_meta_cache.cpp b/src/Processors/examples/merge_tree_meta_cache.cpp new file mode 100644 index 00000000000..a394323bcb7 --- /dev/null +++ b/src/Processors/examples/merge_tree_meta_cache.cpp @@ -0,0 +1,51 @@ +#include +#include + +int main() +{ + using namespace DB; + auto shared_context = Context::createShared(); + auto global_context = Context::createGlobal(shared_context.get()); + global_context->makeGlobalContext(); + global_context->initializeMergeTreeMetaCache("./db/", 256 << 20); + + auto cache = global_context->getMergeTreeMetaCache(); + + std::vector files + = {"columns.txt", "checksums.txt", "primary.idx", "count.txt", "partition.dat", "minmax_p.idx", "default_compression_codec.txt"}; + String prefix = "data/test_meta_cache/check_part_meta_cache/201806_1_1_0_4/"; + + for (const auto & file : files) + { + auto status = cache->put(prefix + file, prefix + file); + std::cout << "put " << file << " " << status.ToString() << std::endl; + } + + for (const auto & file : files) + { + String value; + auto status = cache->get(prefix + file, value); + std::cout << "get " << file << " " << status.ToString() << " " << value << std::endl; + } + + + for (const auto & file : files) + { + auto status = cache->del(prefix + file); + std::cout << "del " << file << " " << status.ToString() << std::endl; + } + + for (const auto & file : files) + { + String value; + auto status = cache->get(prefix + file, value); + std::cout << "get " << file << " " << status.ToString() << " " << value << std::endl; + } + + Strings keys; + Strings values; + cache->getByPrefix(prefix, keys, values); + for (size_t i=0; i +#include #include #include +#include #include #include #include @@ -61,7 +63,8 @@ static std::unique_ptr openForReading(const DiskPtr & di return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } -void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path) +void IMergeTreeDataPart::MinMaxIndex::load( + const MergeTreeData & data, const PartMetaCachePtr & cache, const DiskPtr & disk, const String & /*part_path*/) { auto metadata_snapshot = data.getInMemoryMetadataPtr(); const auto & partition_key = metadata_snapshot->getPartitionKey(); @@ -73,14 +76,15 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Dis hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { - String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); - auto file = openForReading(disk_, file_name); + String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + String value; + auto buf = cache->readOrSetMeta(disk, file_name, value); auto serialization = minmax_column_types[i]->getDefaultSerialization(); Field min_val; - serialization->deserializeBinary(min_val, *file); + serialization->deserializeBinary(min_val, *buf); Field max_val; - serialization->deserializeBinary(max_val, *file); + serialization->deserializeBinary(max_val, *buf); // NULL_LAST if (min_val.isNull()) @@ -181,6 +185,19 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other) } } +void IMergeTreeDataPart::MinMaxIndex::appendFiles(const MergeTreeData & data, Strings & files) +{ + auto metadata_snapshot = data.getInMemoryMetadataPtr(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); + size_t minmax_idx_size = minmax_column_names.size(); + for (size_t i = 0; i < minmax_idx_size; ++i) + { + String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + files.push_back(file_name); + } +} + static void incrementStateMetric(IMergeTreeDataPart::State state) { @@ -284,6 +301,9 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) + , meta_cache(std::make_shared( + storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) + { if (parent_part) state = State::Committed; @@ -309,6 +329,9 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) + , meta_cache(std::make_shared( + storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) + { if (parent_part) state = State::Committed; @@ -631,6 +654,41 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadDefaultCompressionCodec(); } +void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection) const +{ + if (isStoredOnDisk()) + { + appendFilesOfUUID(files); + appendFilesOfColumns(files); + appendFilesOfChecksums(files); + appendFilesOfIndexGranularity(files); + appendFilesofIndex(files); + appendFilesOfRowsCount(files); + appendFilesOfPartitionAndMinMaxIndex(files); + appendFilesOfTTLInfos(files); + appendFilesOfDefaultCompressionCodec(files); + } + + if (!parent_part && include_projection) + { + for (const auto & [projection_name, projection_part] : projection_parts) + { + Strings projection_files; + projection_part->appendFilesOfColumnsChecksumsIndexes(projection_files, true); + for (const auto & projection_file : projection_files) + files.push_back(fs::path(projection_part->relative_path) / projection_file); + } + } +} + +size_t IMergeTreeDataPart::fileNumberOfColumnsChecksumsIndexes() const +{ + Strings files; + files.reserve(16); + appendFilesOfColumnsChecksumsIndexes(files, true); + return files.size(); +} + void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); @@ -651,6 +709,11 @@ void IMergeTreeDataPart::loadIndexGranularity() throw Exception("Method 'loadIndexGranularity' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED); } +void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) const +{ + throw Exception("Method 'appendFilesOfIndexGranularity' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED); +} + void IMergeTreeDataPart::loadIndex() { /// It can be empty in case of mutations @@ -675,7 +738,8 @@ void IMergeTreeDataPart::loadIndex() } String index_path = fs::path(getFullRelativePath()) / "primary.idx"; - auto index_file = openForReading(volume->getDisk(), index_path); + String value; + auto index_buf = meta_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); size_t marks_count = index_granularity.getMarksCount(); @@ -685,7 +749,7 @@ void IMergeTreeDataPart::loadIndex() for (size_t i = 0; i < marks_count; ++i) //-V756 for (size_t j = 0; j < key_size; ++j) - key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file); + key_serializations[j]->deserializeBinary(*loaded_index[j], *index_buf); for (size_t i = 0; i < key_size; ++i) { @@ -696,13 +760,27 @@ void IMergeTreeDataPart::loadIndex() ErrorCodes::CANNOT_READ_ALL_DATA); } - if (!index_file->eof()) + if (!index_buf->eof()) throw Exception("Index file " + fullPath(volume->getDisk(), index_path) + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE); index.assign(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end())); } } +void IMergeTreeDataPart::appendFilesofIndex(Strings & files) const +{ + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + if (parent_part) + metadata_snapshot = metadata_snapshot->projections.has(name) ? metadata_snapshot->projections.get(name).metadata : nullptr; + + if (!metadata_snapshot) + return; + + size_t key_size = metadata_snapshot->getPrimaryKeyColumns().size(); + if (key_size) + files.push_back("primary.idx"); +} + NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const { if (!isStoredOnDisk()) @@ -726,23 +804,30 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() return; } + String v; String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; - if (!volume->getDisk()->exists(path)) + auto in_buf = meta_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); + if (!in_buf) { default_codec = detectDefaultCompressionCodec(); } else { - auto file_buf = openForReading(volume->getDisk(), path); String codec_line; - readEscapedStringUntilEOL(codec_line, *file_buf); + readEscapedStringUntilEOL(codec_line, *in_buf); ReadBufferFromString buf(codec_line); if (!checkString("CODEC", buf)) { - LOG_WARNING(storage.log, "Cannot parse default codec for part {} from file {}, content '{}'. Default compression codec will be deduced automatically, from data on disk", name, path, codec_line); + LOG_WARNING( + storage.log, + "Cannot parse default codec for part {} from file {}, content '{}'. Default compression codec will be deduced " + "automatically, from data on disk", + name, + path, + codec_line); default_codec = detectDefaultCompressionCodec(); } @@ -760,6 +845,11 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() } } +void IMergeTreeDataPart::appendFilesOfDefaultCompressionCodec(Strings & files) const +{ + files.push_back(DEFAULT_COMPRESSION_CODEC_FILE_NAME); +} + CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const { /// In memory parts doesn't have any compression @@ -822,7 +912,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() { String path = getFullRelativePath(); if (!parent_part) - partition.load(storage, volume->getDisk(), path); + partition.load(storage, meta_cache, volume->getDisk(), path); if (!isEmpty()) { @@ -830,7 +920,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() // projection parts don't have minmax_idx, and it's always initialized minmax_idx->initialized = true; else - minmax_idx->load(storage, volume->getDisk(), path); + minmax_idx->load(storage, meta_cache, volume->getDisk(), path); } if (parent_part) return; @@ -845,13 +935,27 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() ErrorCodes::CORRUPTED_DATA); } +void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) const +{ + if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING && !parent_part) + return; + + if (!parent_part) + partition.appendFiles(storage, files); + + if (!isEmpty()) + if (!parent_part) + minmax_idx->appendFiles(storage, files); +} + void IMergeTreeDataPart::loadChecksums(bool require) { const String path = fs::path(getFullRelativePath()) / "checksums.txt"; - if (volume->getDisk()->exists(path)) + String value; + auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); + if (buf) { - auto buf = openForReading(volume->getDisk(), path); if (checksums.read(*buf)) { assertEOF(*buf); @@ -882,6 +986,11 @@ void IMergeTreeDataPart::loadChecksums(bool require) } } +void IMergeTreeDataPart::appendFilesOfChecksums(Strings & files) const +{ + files.push_back("checksums.txt"); +} + void IMergeTreeDataPart::loadRowsCount() { String path = fs::path(getFullRelativePath()) / "count.txt"; @@ -899,10 +1008,13 @@ void IMergeTreeDataPart::loadRowsCount() } else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { - if (!volume->getDisk()->exists(path)) + String v; + auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); + if (!buf) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - read_rows_count(); + readIntText(rows_count, *buf); + assertEOF(*buf); #ifndef NDEBUG /// columns have to be loaded @@ -997,12 +1109,18 @@ void IMergeTreeDataPart::loadRowsCount() } } +void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) const +{ + files.push_back("count.txt"); +} + void IMergeTreeDataPart::loadTTLInfos() { String path = fs::path(getFullRelativePath()) / "ttl.txt"; - if (volume->getDisk()->exists(path)) + String v; + auto in = meta_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); + if (in) { - auto in = openForReading(volume->getDisk(), path); assertString("ttl format version: ", *in); size_t format_version; readText(format_version, *in); @@ -1024,19 +1142,30 @@ void IMergeTreeDataPart::loadTTLInfos() } } + +void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) const +{ + files.push_back("ttl.txt"); +} + void IMergeTreeDataPart::loadUUID() { + String v; String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; - - if (volume->getDisk()->exists(path)) + auto in = meta_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); + if (in) { - auto in = openForReading(volume->getDisk(), path); readText(uuid, *in); if (uuid == UUIDHelpers::Nil) throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR); } } +void IMergeTreeDataPart::appendFilesOfUUID(Strings & files) const +{ + files.push_back(UUID_FILE_NAME); +} + void IMergeTreeDataPart::loadColumns(bool require) { String path = fs::path(getFullRelativePath()) / "columns.txt"; @@ -1045,7 +1174,9 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; - if (!volume->getDisk()->exists(path)) + String v; + auto in = meta_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); + if (!in) { /// We can get list of columns only from columns.txt in compact parts. if (require || part_type == Type::COMPACT) @@ -1061,14 +1192,14 @@ void IMergeTreeDataPart::loadColumns(bool require) throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); { - auto buf = volume->getDisk()->writeFile(path + ".tmp", 4096); - loaded_columns.writeText(*buf); + auto out = volume->getDisk()->writeFile(path + ".tmp", 4096); + loaded_columns.writeText(*out); } volume->getDisk()->moveFile(path + ".tmp", path); } else { - loaded_columns.readText(*volume->getDisk()->readFile(path)); + loaded_columns.readText(*in); for (const auto & column : loaded_columns) { @@ -1092,6 +1223,11 @@ void IMergeTreeDataPart::loadColumns(bool require) setColumns(loaded_columns, infos); } +void IMergeTreeDataPart::appendFilesOfColumns(Strings & files) const +{ + files.push_back("columns.txt"); +} + bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const { /// `IMergeTreeDataPart::volume` describes space where current part belongs, and holds @@ -1142,9 +1278,14 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } } + modifyAllMetaCaches(ModifyCacheType::DROP, true); +// #ifndef NDEBUG + assertMetaCacheDropped(true); +// #endif volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); volume->getDisk()->moveDirectory(from, to); relative_path = new_relative_path; + modifyAllMetaCaches(ModifyCacheType::PUT, true); SyncGuardPtr sync_guard; if (storage.getSettings()->fsync_part_directory) @@ -1153,6 +1294,71 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ storage.lockSharedData(*this); } + +void IMergeTreeDataPart::modifyAllMetaCaches(ModifyCacheType type, bool include_projection) const +{ + Strings files; + files.reserve(16); + appendFilesOfColumnsChecksumsIndexes(files, include_projection); + LOG_TRACE( + storage.log, + "part name:{} path:{} {} keys:{}", + name, + getFullRelativePath(), + modifyCacheTypeToString(type), + boost::algorithm::join(files, ", ")); + + switch (type) + { + case ModifyCacheType::PUT: + meta_cache->setMetas(volume->getDisk(), files); + break; + case ModifyCacheType::DROP: + meta_cache->dropMetas(files); + break; + } +} + + +void IMergeTreeDataPart::assertMetaCacheDropped(bool include_projection) const +{ + Strings files; + std::vector _; + meta_cache->getFilesAndCheckSums(files, _); + if (files.empty()) + return; + + for (const auto & file : files) + { + String file_name = fs::path(file).filename(); + /// file belongs to current part + if (fs::path(getFullRelativePath()) / file_name == file) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Data part {} with type {} with meta file {} still in cache", name, getType().toString(), file); + } + + /// file belongs to projection part of current part + if (!parent_part && include_projection) + { + for (const auto & [projection_name, projection_part] : projection_parts) + { + if (fs::path(projection_part->getFullRelativePath()) / file_name == file) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Data part {} with type {} with meta file {} with projection name still in cache", + name, + getType().toString(), + file, + projection_name); + } + } + } + // LOG_WARNING(storage.log, "cache of file {} does't belong to any part", file); + } +} + std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const { /// NOTE: It's needed for zero-copy replication @@ -1190,6 +1396,11 @@ void IMergeTreeDataPart::remove() const return; } + modifyAllMetaCaches(ModifyCacheType::DROP); +// #ifndef NDEBUG + assertMetaCacheDropped(); +// #endif + /** Atomic directory removal: * - rename directory to temporary name; * - remove it recursive. @@ -1293,6 +1504,11 @@ void IMergeTreeDataPart::remove() const void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_shared_data) const { + modifyAllMetaCaches(ModifyCacheType::DROP); +// #ifndef NDEBUG + assertMetaCacheDropped(); +// #endif + String to = parent_to + "/" + relative_path; auto disk = volume->getDisk(); if (checksums.empty()) @@ -1380,6 +1596,7 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) return "detached/" + getRelativePathForPrefix(prefix, /* detached */ true); } + void IMergeTreeDataPart::renameToDetached(const String & prefix) const { renameTo(getRelativePathForDetachedPart(prefix), true); @@ -1632,6 +1849,71 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); } +IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const +{ + String file_name = std::filesystem::path(file_path).filename(); + const auto filenames_without_checksums = getFileNamesWithoutChecksums(); + auto it = checksums.files.find(file_name); + if (filenames_without_checksums.count(file_name) == 0 && it != checksums.files.end()) + { + return it->second.file_hash; + } + + if (!volume->getDisk()->exists(file_path)) + { + return {}; + } + std::unique_ptr in_file = volume->getDisk()->readFile(file_path); + HashingReadBuffer in_hash(*in_file); + + String value; + readStringUntilEOF(value, in_hash); + return in_hash.getHash(); +} + +void IMergeTreeDataPart::checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const +{ + /// checkMetaCache only applies for normal part + if (isProjectionPart()) + return; + + /// the directory of projection part is under the directory of its parent part + const auto filenames_without_checksums = getFileNamesWithoutChecksums(); + meta_cache->getFilesAndCheckSums(files, cache_checksums); + for (const auto & file : files) + { + // std::cout << "check key:" << file << std::endl; + String file_name = fs::path(file).filename(); + + /// file belongs to normal part + if (fs::path(getFullRelativePath()) / file_name == file) + { + auto disk_checksum = getActualChecksumByFile(file); + disk_checksums.push_back(disk_checksum); + continue; + } + + /// file belongs to projection part + String proj_dir_name = fs::path(file).parent_path().filename(); + auto pos = proj_dir_name.find_last_of('.'); + if (pos == String::npos) + { + disk_checksums.push_back({}); + continue; + } + String proj_name = proj_dir_name.substr(0, pos); + auto it = projection_parts.find(proj_name); + if (it == projection_parts.end()) + { + disk_checksums.push_back({}); + continue; + } + + auto disk_checksum = it->second->getActualChecksumByFile(file); + disk_checksums.push_back(disk_checksum); + } +} + bool isCompactPart(const MergeTreeDataPartPtr & data_part) { return (data_part && data_part->getType() == MergeTreeDataPartType::COMPACT); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ab08ca1c33a..ad072af10a5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -14,6 +14,7 @@ #include #include #include +#include #include @@ -44,6 +45,23 @@ class UncompressedCache; class IMergeTreeDataPart : public std::enable_shared_from_this { public: + enum ModifyCacheType + { + PUT, // override set + DROP, // remove keys + }; + + static String modifyCacheTypeToString(ModifyCacheType type) + { + switch (type) + { + case PUT: + return "PUT"; + case DROP: + return "DROP"; + } + } + static constexpr auto DATA_FILE_EXTENSION = ".bin"; using Checksums = MergeTreeDataPartChecksums; @@ -59,6 +77,7 @@ public: using IndexSizeByName = std::unordered_map; using Type = MergeTreeDataPartType; + using uint128 = PartMetaCache::uint128; IMergeTreeDataPart( @@ -138,6 +157,8 @@ public: /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; + void assertMetaCacheDropped(bool include_projection = false) const; + void remove() const; void projectionRemove(const String & parent_to, bool keep_shared_data = false) const; @@ -145,6 +166,8 @@ public: /// Initialize columns (from columns.txt if exists, or create from column files if not). /// Load checksums from checksums.txt if exists. Load index if required. void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency); + void appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection = false) const; + size_t fileNumberOfColumnsChecksumsIndexes() const; String getMarksFileExtension() const { return index_granularity_info.marks_file_extension; } @@ -239,7 +262,7 @@ public: using TTLInfo = MergeTreeDataPartTTLInfo; using TTLInfos = MergeTreeDataPartTTLInfos; - TTLInfos ttl_infos; + mutable TTLInfos ttl_infos; /// Current state of the part. If the part is in working set already, it should be accessed via data_parts mutex void setState(State new_state) const; @@ -296,12 +319,13 @@ public: { } - void load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path); - void store(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const; + void load(const MergeTreeData & data, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); + void store(const MergeTreeData & data, const DiskPtr & disk, const String & part_path, Checksums & checksums) const; void store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const; void update(const Block & block, const Names & column_names); void merge(const MinMaxIndex & other); + static void appendFiles(const MergeTreeData & data, Strings & files); }; using MinMaxIndexPtr = std::shared_ptr; @@ -351,6 +375,8 @@ public: /// storage and pass it to this method. virtual bool hasColumnFiles(const NameAndTypePair & /* column */) const { return false; } + virtual Strings getIndexGranularityFiles() const = 0; + /// Returns true if this part shall participate in merges according to /// settings of given storage policy. bool shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const; @@ -363,6 +389,8 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; + virtual void checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; + bool isProjectionPart() const { return parent_part != nullptr; } const IMergeTreeDataPart * getParentPart() const { return parent_part; } @@ -434,6 +462,8 @@ protected: std::map> projection_parts; + mutable PartMetaCachePtr meta_cache; + void removeIfNeeded(); virtual void checkConsistency(bool require_part_metadata) const; @@ -457,40 +487,62 @@ private: /// Reads part unique identifier (if exists) from uuid.txt void loadUUID(); + void appendFilesOfUUID(Strings & files) const; + /// Reads columns names and types from columns.txt void loadColumns(bool require); + void appendFilesOfColumns(Strings & files) const; + /// If checksums.txt exists, reads file's checksums (and sizes) from it void loadChecksums(bool require); + void appendFilesOfChecksums(Strings & files) const; + /// Loads marks index granularity into memory virtual void loadIndexGranularity(); + virtual void appendFilesOfIndexGranularity(Strings & files) const; + /// Loads index file. void loadIndex(); + void appendFilesofIndex(Strings & files) const; + /// Load rows count for this part from disk (for the newer storage format version). /// For the older format version calculates rows count from the size of a column with a fixed size. void loadRowsCount(); + void appendFilesOfRowsCount(Strings & files) const; + /// Loads ttl infos in json format from file ttl.txt. If file doesn't exists assigns ttl infos with all zeros void loadTTLInfos(); + void appendFilesOfTTLInfos(Strings & files) const; + void loadPartitionAndMinMaxIndex(); void calculateColumnsSizesOnDisk(); void calculateSecondaryIndicesSizesOnDisk(); + void appendFilesOfPartitionAndMinMaxIndex(Strings & files) const; + /// Load default compression codec from file default_compression_codec.txt /// if it not exists tries to deduce codec from compressed column without /// any specifial compression. void loadDefaultCompressionCodec(); + void appendFilesOfDefaultCompressionCodec(Strings & files) const; + + void modifyAllMetaCaches(ModifyCacheType type, bool include_projection = false) const; + /// Found column without specific compression and return codec /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; + IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; + mutable State state{State::Temporary}; }; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 3d4e909cf60..10530f25927 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -53,7 +53,7 @@ String Range::toString() const /// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. -static String extractFixedPrefixFromLikePattern(const String & like_pattern) +String extractFixedPrefixFromLikePattern(const String & like_pattern) { String fixed_prefix; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index dee46ae52ce..c8d9fda77c4 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -442,4 +442,6 @@ private: bool strict; }; +String extractFixedPrefixFromLikePattern(const String & like_pattern); + } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1b7be8ca98d..8d861e404f0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1330,6 +1330,14 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size()); } +size_t MergeTreeData::fileNumberOfDataParts(const DataPartStates & states) const +{ + size_t result = 0; + auto parts = getDataParts(states); + for (const auto & part : parts) + result += part->fileNumberOfColumnsChecksumsIndexes(); + return result; +} /// Is the part directory old. /// True if its modification time and the modification time of all files inside it is less then threshold. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 380c2f4f4c5..d349646ab88 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -427,6 +427,8 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); + size_t fileNumberOfDataParts(const DataPartStates & states) const; + String getLogName() const { return log_name; } Int64 getMaxBlockNumber() const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index f4da730b1f0..e51b64b3842 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -91,6 +91,7 @@ void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*eac total_size.marks += mrk_checksum->second.file_size; } +// load marks from meta cache void MergeTreeDataPartCompact::loadIndexGranularity() { String full_path = getFullRelativePath(); @@ -192,4 +193,16 @@ MergeTreeDataPartCompact::~MergeTreeDataPartCompact() removeIfNeeded(); } +// Do not cache mark file, because cache other meta files is enough to speed up loading. +void MergeTreeDataPartCompact::appendFilesOfIndexGranularity(Strings& /* files */) const +{ +} + +// find all connected file and do modification +Strings MergeTreeDataPartCompact::getIndexGranularityFiles() const +{ + auto marks_file = index_granularity_info.getMarksFilePath("data"); + return {marks_file}; +} + } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 38bfa11652a..87066ab2ff0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -72,6 +72,10 @@ private: /// Compact parts doesn't support per column size, only total size void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + + void appendFilesOfIndexGranularity(Strings& files) const override; + + Strings getIndexGranularityFiles() const override; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 4ec53d88339..5c21ead3208 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -166,6 +166,17 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() return checksum; } +// No mark files for part in memory +void MergeTreeDataPartInMemory::appendFilesOfIndexGranularity(Strings& /* files */) const +{ +} + +// No mark files for part in memory +Strings MergeTreeDataPartInMemory::getIndexGranularityFiles() const +{ + return {}; +} + DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part) { return std::dynamic_pointer_cast(part); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index c5ee9ebd01f..4f83b54d402 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -62,6 +62,10 @@ private: /// Calculates uncompressed sizes in memory. void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + + void appendFilesOfIndexGranularity(Strings & files) const override; + + Strings getIndexGranularityFiles() const override; }; using DataPartInMemoryPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index b279c1aba6a..5132177aa5c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -107,7 +107,6 @@ void MergeTreeDataPartWide::loadIndexGranularity() String full_path = getFullRelativePath(); index_granularity_info.changeGranularityIfRequired(volume->getDisk(), full_path); - if (columns.empty()) throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); @@ -268,4 +267,18 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col } } +// Do not cache mark files of part, because cache other meta files is enough to speed up loading. +void MergeTreeDataPartWide::appendFilesOfIndexGranularity(Strings& /* files */) const +{ +} + +Strings MergeTreeDataPartWide::getIndexGranularityFiles() const +{ + if (columns.empty()) + return {}; + + auto marks_file = getFileNameForColumn(columns.front()); + return {marks_file}; +} + } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 4796143e11d..bf73d16d758 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -66,9 +66,14 @@ private: /// Loads marks index granularity into memory void loadIndexGranularity() override; + void appendFilesOfIndexGranularity(Strings & files) const override; + + Strings getIndexGranularityFiles() const override; + ColumnSize getColumnSizeImpl(const NameAndTypePair & column, std::unordered_set * processed_substreams) const; void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + }; } diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 06fcb24f846..1d4e14c628b 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,11 +160,13 @@ namespace }; } +/* static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) { size_t file_size = disk->getFileSize(path); return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } +*/ String MergeTreePartition::getID(const MergeTreeData & storage) const { @@ -355,7 +357,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe } } -void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path) +void MergeTreePartition::load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) @@ -363,10 +365,12 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; auto partition_file_path = part_path + "partition.dat"; - auto file = openForReading(disk, partition_file_path); + + String v; + auto buf = meta_cache->readOrSetMeta(disk, "partition.dat", v); value.resize(partition_key_sample.columns()); for (size_t i = 0; i < partition_key_sample.columns(); ++i) - partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file); + partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *buf); } void MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const @@ -384,7 +388,9 @@ void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr auto out = disk->writeFile(part_path + "partition.dat"); HashingWriteBuffer out_hashing(*out); for (size_t i = 0; i < value.size(); ++i) + { partition_key_sample.getByPosition(i).type->getDefaultSerialization()->serializeBinary(value[i], out_hashing); + } out_hashing.next(); checksums.files["partition.dat"].file_size = out_hashing.count(); @@ -443,4 +449,14 @@ KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr & return partition_key; } + +void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files) const +{ + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + if (!metadata_snapshot->hasPartitionKey()) + return; + + files.push_back("partition.dat"); +} + } diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index d501d615621..b8b5b301219 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB @@ -37,7 +38,7 @@ public: void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; - void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path); + void load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; @@ -45,6 +46,8 @@ public: void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); + void appendFiles(const MergeTreeData & storage, Strings & files) const; + /// Adjust partition key and execute its expression on block. Return sample block according to used expression. static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context); diff --git a/src/Storages/MergeTree/PartMetaCache.cpp b/src/Storages/MergeTree/PartMetaCache.cpp new file mode 100644 index 00000000000..33c95d6963e --- /dev/null +++ b/src/Storages/MergeTree/PartMetaCache.cpp @@ -0,0 +1,134 @@ +#include "PartMetaCache.h" + +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event MergeTreeMetaCacheHit; + extern const Event MergeTreeMetaCacheMiss; +} + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB +{ + +std::unique_ptr +PartMetaCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value) +{ + String file_path = fs::path(getFullRelativePath()) / file_name; + auto status = cache->get(file_path, value); + if (!status.ok()) + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetaCacheMiss); + if (!disk->exists(file_path)) + { + return nullptr; + } + + auto in = disk->readFile(file_path); + if (in) + { + readStringUntilEOF(value, *in); + cache->put(file_path, value); + } + } + else + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetaCacheHit); + } + return std::make_unique(value); +} + +void PartMetaCache::setMetas(const DiskPtr & disk, const Strings & file_names) +{ + String text; + String read_value; + for (const auto & file_name : file_names) + { + const String file_path = fs::path(getFullRelativePath()) / file_name; + if (!disk->exists(file_path)) + continue; + + auto in = disk->readFile(file_path); + if (!in) + continue; + + readStringUntilEOF(text, *in); + auto status = cache->put(file_path, text); + if (!status.ok()) + { + status = cache->get(file_path, read_value); + if (status.IsNotFound() || read_value == text) + continue; + throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); + } + } +} + +void PartMetaCache::dropMetas(const Strings & file_names) +{ + for (const auto & file_name : file_names) + { + String file_path = fs::path(getFullRelativePath()) / file_name; + auto status = cache->del(file_path); + if (!status.ok()) + { + String read_value; + status = cache->get(file_path, read_value); + if (status.IsNotFound()) + continue; + throw Exception(ErrorCodes::LOGICAL_ERROR, "drop meta failed status:{}, file_path:{}", status.ToString(), file_path); + } + } +} + +void PartMetaCache::setMeta(const String & file_name, const String & value) +{ + String file_path = fs::path(getFullRelativePath()) / file_name; + String read_value; + auto status = cache->get(file_path, read_value); + if (status == rocksdb::Status::OK() && value == read_value) + return; + + status = cache->put(file_path, value); + if (!status.ok()) + { + status = cache->get(file_path, read_value); + if (status.IsNotFound() || read_value == value) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); + } +} + +void PartMetaCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const +{ + String prefix = fs::path(getFullRelativePath()) / ""; + Strings values; + values.reserve(files.capacity()); + cache->getByPrefix(prefix, files, values); + size_t size = files.size(); + for (size_t i = 0; i < size; ++i) + { + ReadBufferFromString rbuf(values[i]); + HashingReadBuffer hbuf(rbuf); + checksums.push_back(hbuf.getHash()); + } +} + +String PartMetaCache::getFullRelativePath() const +{ + return fs::path(relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; +} + +} diff --git a/src/Storages/MergeTree/PartMetaCache.h b/src/Storages/MergeTree/PartMetaCache.h new file mode 100644 index 00000000000..d6a86d86ba1 --- /dev/null +++ b/src/Storages/MergeTree/PartMetaCache.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class SeekableReadBuffer; +class IMergeTreeDataPart; +class PartMetaCache; +using PartMetaCachePtr = std::shared_ptr; + +class PartMetaCache +{ +public: + using uint128 = CityHash_v1_0_2::uint128; + + PartMetaCache(const MergeTreeMetaCachePtr & cache_, const String & relative_data_path_, const String & relative_path_, const IMergeTreeDataPart * parent_part_) + : cache(cache_) + , relative_data_path(relative_data_path_) + , relative_path(relative_path_) + , parent_part(parent_part_) + { + } + + std::unique_ptr + readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value); + void setMetas(const DiskPtr & disk, const Strings & file_names); + void dropMetas(const Strings & file_names); + void setMeta(const String & file_name, const String & value); + void getFilesAndCheckSums(Strings & file_names, std::vector & checksums) const; + +private: + std::string getFullRelativePath() const; + + MergeTreeMetaCachePtr cache; + const String & relative_data_path; // relative path of table to disk + const String & relative_path; // relative path of part to table + const IMergeTreeDataPart * parent_part; +}; + +} diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp new file mode 100644 index 00000000000..45cf45edb31 --- /dev/null +++ b/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp @@ -0,0 +1,139 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +NamesAndTypesList StorageSystemMergeTreeMetaCache::getNamesAndTypes() +{ + return { + {"key", std::make_shared()}, + {"value", std::make_shared()}, + }; +} + +static bool extractKeyImpl(const IAST & elem, String & res, bool & precise) +{ + const auto * function = elem.as(); + if (!function) + return false; + + if (function->name == "and") + { + for (const auto & child : function->arguments->children) + { + bool tmp_precise = false; + if (extractKeyImpl(*child, res, tmp_precise)) + { + precise = tmp_precise; + return true; + } + } + return false; + } + + if (function->name == "equals" || function->name == "like") + { + const auto & args = function->arguments->as(); + const IAST * value; + + if (args.children.size() != 2) + return false; + + const ASTIdentifier * ident; + if ((ident = args.children.at(0)->as())) + value = args.children.at(1).get(); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0).get(); + else + return false; + + if (ident->name() != "key") + return false; + + const auto * literal = value->as(); + if (!literal) + return false; + + if (literal->value.getType() != Field::Types::String) + return false; + + res = literal->value.safeGet(); + precise = function->name == "equals"; + return true; + } + return false; +} + + +/** Retrieve from the query a condition of the form `key= 'key'`, from conjunctions in the WHERE clause. + */ +static String extractKey(const ASTPtr & query, bool& precise) +{ + const auto & select = query->as(); + if (!select.where()) + return ""; + + String res; + return extractKeyImpl(*select.where(), res, precise) ? res : ""; +} + + +void StorageSystemMergeTreeMetaCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +{ + bool precise = false; + String key = extractKey(query_info.query, precise); + if (key.empty()) + throw Exception( + "SELECT from system.merge_tree_meta_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); + + auto cache = context->getMergeTreeMetaCache(); + if (precise) + { + String value; + if (cache->get(key, value) != MergeTreeMetaCache::Status::OK()) + return; + + size_t col_num = 0; + res_columns[col_num++]->insert(key); + res_columns[col_num++]->insert(value); + } + else + { + String target = extractFixedPrefixFromLikePattern(key); + if (target.empty()) + throw Exception( + "SELECT from system.merge_tree_meta_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); + + Strings keys; + Strings values; + keys.reserve(4096); + values.reserve(4096); + cache->getByPrefix(target, keys, values); + if (keys.empty()) + return; + + assert(keys.size() == values.size()); + for (size_t i = 0; i < keys.size(); ++i) + { + size_t col_num = 0; + res_columns[col_num++]->insert(keys[i]); + res_columns[col_num++]->insert(values[i]); + } + } +} + +} diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.h b/src/Storages/System/StorageSystemMergeTreeMetaCache.h new file mode 100644 index 00000000000..a5f65862243 --- /dev/null +++ b/src/Storages/System/StorageSystemMergeTreeMetaCache.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class Context; + + +/** Implements `merge_tree_meta_cache` system table, which allows you to view the metacache data in rocksdb for debugging purposes. + */ +class StorageSystemMergeTreeMetaCache : public shared_ptr_helper, public IStorageSystemOneBlock +{ + friend struct shared_ptr_helper; + +public: + std::string getName() const override { return "SystemMergeTreeMetaCache"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 023ced35a6b..0e7d4b624c5 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -68,6 +68,7 @@ #include #include #include +#include #ifdef OS_LINUX #include @@ -129,6 +130,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) #endif #if USE_ROCKSDB attach(context, system_database, "rocksdb"); + attach(context, system_database, "merge_tree_meta_cache"); #endif } diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.reference b/tests/queries/0_stateless/01233_check_part_meta_cache.reference new file mode 100644 index 00000000000..914add905ce --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.reference @@ -0,0 +1,28 @@ +0 0 +7 0 +14 0 +28 0 +42 0 +56 0 +70 0 +77 0 +63 0 +77 0 +84 0 +98 0 +122 0 +154 0 +122 0 +12 0 +24 0 +48 0 +72 0 +96 0 +120 0 +132 0 +108 0 +132 0 +144 0 +183 0 +235 0 +183 0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql new file mode 100644 index 00000000000..f0ca3b608d1 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -0,0 +1,123 @@ +-- Create table under database with engine ordinary. +set mutations_sync = 1; +DROP DATABASE IF EXISTS test_meta_cache; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache; +CREATE DATABASE test_meta_cache ENGINE = Ordinary; +CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert third batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete column. +alter table test_meta_cache.check_part_meta_cache drop column v1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Recreate table with projection. +drop table if exists test_meta_cache.check_part_meta_cache; +CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert third batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference new file mode 100644 index 00000000000..95de1ef56a9 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference @@ -0,0 +1,28 @@ +0 0 +7 0 +14 0 +28 0 +42 0 +56 0 +70 0 +77 0 +63 0 +77 0 +84 0 +98 0 +124 0 +150 0 +124 0 +12 0 +24 0 +48 0 +72 0 +96 0 +120 0 +132 0 +108 0 +132 0 +144 0 +183 0 +235 0 +183 0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql new file mode 100644 index 00000000000..b57caf55cb8 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -0,0 +1,124 @@ +-- Create table under database with engine atomic. +set mutations_sync = 1; +DROP DATABASE IF EXISTS test_meta_cache; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache SYNC; +CREATE DATABASE test_meta_cache ENGINE = Atomic; +CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete column. +alter table test_meta_cache.check_part_meta_cache drop column v1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + 30; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + 60; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Recreate table with projection. +drop table if exists test_meta_cache.check_part_meta_cache SYNC; +CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- nsert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference new file mode 100644 index 00000000000..2275537d212 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference @@ -0,0 +1,28 @@ +0 0 +7 0 +14 0 +28 0 +42 0 +56 0 +70 0 +77 0 +7 0 +14 0 +21 0 +35 0 +51 0 +67 0 +0 0 +12 0 +24 0 +48 0 +72 0 +96 0 +120 0 +132 0 +108 0 +132 0 +144 0 +183 0 +235 0 +183 0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql new file mode 100644 index 00000000000..6d08bb146a5 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -0,0 +1,125 @@ +-- Create table under database with engine ordinary. +set mutations_sync = 1; +set replication_alter_partitions_sync = 2; +DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; +CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Ordinary; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/test_meta_cache/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +--Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 drop column v1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Recreate table with projection. +drop table if exists test_meta_cache.check_part_meta_cache on cluster preonline_hk5; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +--Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference new file mode 100644 index 00000000000..2275537d212 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference @@ -0,0 +1,28 @@ +0 0 +7 0 +14 0 +28 0 +42 0 +56 0 +70 0 +77 0 +7 0 +14 0 +21 0 +35 0 +51 0 +67 0 +0 0 +12 0 +24 0 +48 0 +72 0 +96 0 +120 0 +132 0 +108 0 +132 0 +144 0 +183 0 +235 0 +183 0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql new file mode 100644 index 00000000000..c41d036cef1 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -0,0 +1,125 @@ +-- Create table under database with engine ordinary. +set mutations_sync = 1; +set replication_alter_partitions_sync = 2; +DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; +CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Atomic; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/test_meta_cache/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +--Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 drop column v1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Recreate table with projection. +drop table if exists test_meta_cache.check_part_meta_cache on cluster preonline_hk5; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +--Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + From e7401d2a5ee3a5a765e50af64af1375c2af15344 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 27 Dec 2021 11:50:59 +0800 Subject: [PATCH 038/584] wrap rocksdb metacache related code with USE_ROCKSDB --- programs/server/Server.cpp | 2 ++ src/Common/CurrentMetrics.cpp | 1 - src/Common/ProfileEvents.cpp | 8 ++++---- src/Functions/checkPartMetaCache.cpp | 9 +++++---- src/Functions/registerFunctionsMiscellaneous.cpp | 6 ++++++ src/Interpreters/Context.cpp | 14 ++++++++++++-- src/Interpreters/Context.h | 10 ++++++++++ src/Processors/CMakeLists.txt | 4 ++-- src/Processors/examples/CMakeLists.txt | 6 ++++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 1 - src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 1 + src/Storages/MergeTree/MergeTreePartition.cpp | 7 ------- src/Storages/MergeTree/PartMetaCache.cpp | 2 ++ src/Storages/MergeTree/PartMetaCache.h | 4 ++++ .../System/StorageSystemMergeTreeMetaCache.cpp | 6 +++++- .../System/StorageSystemMergeTreeMetaCache.h | 4 ++++ src/Storages/System/attachSystemTables.cpp | 2 +- .../01233_check_part_meta_cache_replicated.sql | 2 +- ..._check_part_meta_cache_replicated_in_atomic.sql | 2 +- 20 files changed, 65 insertions(+), 28 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fd2dc851ae7..7228608c9f1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -748,7 +748,9 @@ if (ThreadFuzzer::instance().isEffective()) /// Directory with metadata of tables, which was marked as dropped by Atomic database fs::create_directories(path / "metadata_dropped/"); +#if USE_ROCKSDB fs::create_directories(path / "rocksdb/"); +#endif } diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index d214952deae..5c9ba177b78 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -78,7 +78,6 @@ M(SyncDrainedConnections, "Number of connections drained synchronously.") \ M(ActiveSyncDrainedConnections, "Number of active connections drained synchronously.") \ M(AsynchronousReadWait, "Number of threads waiting for asynchronous read.") \ - M(ServerStartupSeconds, "Server start seconds") \ namespace CurrentMetrics { diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index af87ba10a31..cb9c6f594a6 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -277,10 +277,10 @@ \ M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ \ - M(RocksdbGet, "Number of rocksdb reads(used for file meta cache)") \ - M(RocksdbPut, "Number of rocksdb puts(used for file meta cache)") \ - M(RocksdbDelete, "Number of rocksdb deletes(used for file meta cache)") \ - M(RocksdbSeek, "Number of rocksdb seeks(used for file meta cache)") \ + M(RocksdbGet, "Number of rocksdb reads(used for merge tree metadata cache)") \ + M(RocksdbPut, "Number of rocksdb puts(used for merge tree metadata cache)") \ + M(RocksdbDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \ + M(RocksdbSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \ M(MergeTreeMetaCacheHit, "Number of times the read of meta file was done from MergeTree meta cache") \ M(MergeTreeMetaCacheMiss, "Number of times the read of meta file was not done from MergeTree meta cache") \ \ diff --git a/src/Functions/checkPartMetaCache.cpp b/src/Functions/checkPartMetaCache.cpp index e0479a5fdcc..e77b8ca0d50 100644 --- a/src/Functions/checkPartMetaCache.cpp +++ b/src/Functions/checkPartMetaCache.cpp @@ -1,6 +1,8 @@ +#include "config_core.h" + +#if USE_ROCKSDB + #include -#include -#include #include #include #include @@ -14,8 +16,6 @@ #include #include #include -#include -#include #include #include @@ -155,3 +155,4 @@ void registerFunctionCheckPartMetaCache(FunctionFactory & factory) } } +#endif diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index d613d7c85bd..77e3e109081 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -80,7 +80,10 @@ void registerFunctionInitialQueryID(FunctionFactory & factory); void registerFunctionServerUUID(FunctionFactory &); void registerFunctionZooKeeperSessionUptime(FunctionFactory &); void registerFunctionGetOSKernelVersion(FunctionFactory &); + +#if USE_ROCKSDB void registerFunctionCheckPartMetaCache(FunctionFactory &); +#endif #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -167,7 +170,10 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionServerUUID(factory); registerFunctionZooKeeperSessionUptime(factory); registerFunctionGetOSKernelVersion(factory); + +#if USE_ROCKSDB registerFunctionCheckPartMetaCache(factory); +#endif #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 81db45c6461..c5a5b3d1d49 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -94,11 +94,13 @@ namespace fs = std::filesystem; namespace ProfileEvents { extern const Event ContextLock; - extern const Event CompiledCacheSizeBytes; + +#if USE_ROCKSDB extern const Event RocksdbPut; extern const Event RocksdbGet; extern const Event RocksdbDelete; extern const Event RocksdbSeek; +#endif } namespace CurrentMetrics @@ -278,8 +280,10 @@ struct ContextSharedPart Context::ConfigReloadCallback config_reload_callback; +#if USE_ROCKSDB /// MergeTree metadata cache stored in rocksdb. MergeTreeMetaCachePtr merge_tree_meta_cache; +#endif ContextSharedPart() : access_control(std::make_unique()), macros(std::make_unique()) @@ -392,12 +396,14 @@ struct ContextSharedPart /// Stop zookeeper connection zookeeper.reset(); - /// Shutdown meta file cache +#if USE_ROCKSDB + /// Shutdown meta file cache if (merge_tree_meta_cache) { merge_tree_meta_cache->shutdown(); merge_tree_meta_cache.reset(); } +#endif } /// Can be removed w/o context lock @@ -441,6 +447,7 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr shar void SharedContextHolder::reset() { shared.reset(); } +#if USE_ROCKSDB MergeTreeMetaCache::Status MergeTreeMetaCache::put(const String & key, const String & value) { auto options = rocksdb::WriteOptions(); @@ -491,6 +498,7 @@ void MergeTreeMetaCache::shutdown() rocksdb->Close(); } } +#endif ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) { @@ -2309,6 +2317,7 @@ void Context::initializeTraceCollector() shared->initializeTraceCollector(getTraceLog()); } +#if USE_ROCKSDB void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) { rocksdb::Options options; @@ -2328,6 +2337,7 @@ void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) } shared->merge_tree_meta_cache = std::make_shared(db); } +#endif bool Context::hasTraceCollector() const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b39e06b0b0f..bc191e80c9a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,5 +1,6 @@ #pragma once +#include "config_core.h" #include #include #include @@ -15,8 +16,11 @@ #include #include #include + +#if USE_ROCKSDB #include #include +#endif #include "config_core.h" @@ -179,6 +183,7 @@ private: std::unique_ptr shared; }; +#if USE_ROCKSDB class MergeTreeMetaCache { public: @@ -199,6 +204,7 @@ private: Poco::Logger * log = &Poco::Logger::get("MergeTreeMetaCache"); }; using MergeTreeMetaCachePtr = std::shared_ptr; +#endif /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) @@ -699,7 +705,9 @@ public: UInt32 getZooKeeperSessionUptime() const; +#if USE_ROCKSDB MergeTreeMetaCachePtr getMergeTreeMetaCache() const; +#endif #if USE_NURAFT @@ -788,7 +796,9 @@ public: /// Call after initialization before using trace collector. void initializeTraceCollector(); +#if USE_ROCKSDB void initializeMergeTreeMetaCache(const String & dir, size_t size); +#endif bool hasTraceCollector() const; diff --git a/src/Processors/CMakeLists.txt b/src/Processors/CMakeLists.txt index 7c9ad405432..7e965188b4c 100644 --- a/src/Processors/CMakeLists.txt +++ b/src/Processors/CMakeLists.txt @@ -1,4 +1,4 @@ -#if (ENABLE_EXAMPLES) +if (ENABLE_EXAMPLES) add_subdirectory(examples) -#endif () +endif () diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index dcb640c383a..ceb022432a1 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,2 +1,4 @@ -add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) -target_link_libraries (merge_tree_meta_cache PRIVATE dbms) +if (USE_ROCKSDB) + add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) + target_link_libraries (merge_tree_meta_cache PRIVATE dbms) +endif() diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ad072af10a5..0d2b2f57fd0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -50,7 +50,7 @@ public: PUT, // override set DROP, // remove keys }; - + static String modifyCacheTypeToString(ModifyCacheType type) { switch (type) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index e51b64b3842..e5fbdd5cd19 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -91,7 +91,6 @@ void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*eac total_size.marks += mrk_checksum->second.file_size; } -// load marks from meta cache void MergeTreeDataPartCompact::loadIndexGranularity() { String full_path = getFullRelativePath(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 5132177aa5c..f6efdc5f05c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -107,6 +107,7 @@ void MergeTreeDataPartWide::loadIndexGranularity() String full_path = getFullRelativePath(); index_granularity_info.changeGranularityIfRequired(volume->getDisk(), full_path); + if (columns.empty()) throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 1d4e14c628b..6b5bebd81f6 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,13 +160,6 @@ namespace }; } -/* -static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) -{ - size_t file_size = disk->getFileSize(path); - return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); -} -*/ String MergeTreePartition::getID(const MergeTreeData & storage) const { diff --git a/src/Storages/MergeTree/PartMetaCache.cpp b/src/Storages/MergeTree/PartMetaCache.cpp index 33c95d6963e..fe8475d0dda 100644 --- a/src/Storages/MergeTree/PartMetaCache.cpp +++ b/src/Storages/MergeTree/PartMetaCache.cpp @@ -1,5 +1,6 @@ #include "PartMetaCache.h" +#if USE_ROCKSDB #include #include #include @@ -132,3 +133,4 @@ String PartMetaCache::getFullRelativePath() const } } +#endif diff --git a/src/Storages/MergeTree/PartMetaCache.h b/src/Storages/MergeTree/PartMetaCache.h index d6a86d86ba1..5ffd0413c4b 100644 --- a/src/Storages/MergeTree/PartMetaCache.h +++ b/src/Storages/MergeTree/PartMetaCache.h @@ -1,5 +1,8 @@ #pragma once +#include "config_core.h" + +#if USE_ROCKSDB #include #include #include @@ -43,3 +46,4 @@ private: }; } +#endif diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp index 45cf45edb31..f53c32e5a42 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp @@ -1,3 +1,6 @@ +#include + +#if USE_ROCKSDB #include #include #include @@ -7,9 +10,9 @@ #include #include #include -#include #include #include + namespace DB { namespace ErrorCodes @@ -137,3 +140,4 @@ void StorageSystemMergeTreeMetaCache::fillData(MutableColumns & res_columns, Con } } +#endif diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.h b/src/Storages/System/StorageSystemMergeTreeMetaCache.h index a5f65862243..c8e0f475cd8 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetaCache.h +++ b/src/Storages/System/StorageSystemMergeTreeMetaCache.h @@ -1,5 +1,8 @@ #pragma once +#include "config_core.h" + +#if USE_ROCKSDB #include #include @@ -27,3 +30,4 @@ protected: }; } +#endif diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 0e7d4b624c5..5f5a17069f3 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -68,7 +68,6 @@ #include #include #include -#include #ifdef OS_LINUX #include @@ -76,6 +75,7 @@ #if USE_ROCKSDB #include +#include #endif diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 6d08bb146a5..cb028e77d54 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -4,7 +4,7 @@ set replication_alter_partitions_sync = 2; DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Ordinary; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/test_meta_cache/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index c41d036cef1..c56e2cb0a99 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -4,7 +4,7 @@ set replication_alter_partitions_sync = 2; DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Atomic; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/test_meta_cache/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. From aa092eeffb8f09d9d65294bfa2a62cacc258e562 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 27 Dec 2021 16:42:06 +0300 Subject: [PATCH 039/584] proper handle of 'max_rows_to_read' in case of reading in order of sorting key and limit --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 14 ++++++++++-- .../MergeTree/MergeTreeSelectProcessor.cpp | 5 ----- ...5_read_in_order_max_rows_to_read.reference | 6 +++++ .../02155_read_in_order_max_rows_to_read.sql | 22 +++++++++++++++++++ 4 files changed, 40 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference create mode 100644 tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index cdedd37e14a..07ac6f5764b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -875,12 +875,22 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd { std::atomic total_rows{0}; + /// Do not check number of read rows if we have reading + /// in order of sorting key with limit. + /// In general case, when there exists WHERE clause + /// it's impossible to estimate number of rows precisely, + /// because we can stop reading at any time. + SizeLimits limits; - if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read) + if (settings.read_overflow_mode == OverflowMode::THROW + && settings.max_rows_to_read + && !query_info.input_order_info) limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode); SizeLimits leaf_limits; - if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf) + if (settings.read_overflow_mode_leaf == OverflowMode::THROW + && settings.max_rows_to_read_leaf + && !query_info.input_order_info) leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); auto mark_cache = context->getIndexMarkCache(); diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 2d4d3617cee..332eb27094a 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -37,11 +37,6 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( has_limit_below_one_block(has_limit_below_one_block_), total_rows(data_part->index_granularity.getRowsCountInRanges(all_mark_ranges)) { - /// Actually it means that parallel reading from replicas enabled - /// and we have to collaborate with initiator. - /// In this case we won't set approximate rows, because it will be accounted multiple times - if (!extension_.has_value()) - addTotalRowsApprox(total_rows); ordered_names = header_without_virtual_columns.getNames(); } diff --git a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference new file mode 100644 index 00000000000..b73ab43cabb --- /dev/null +++ b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference @@ -0,0 +1,6 @@ +10 +0 +1 +2 +3 +4 diff --git a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql new file mode 100644 index 00000000000..e82c78b5e42 --- /dev/null +++ b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS t_max_rows_to_read; + +CREATE TABLE t_max_rows_to_read (a UInt64) +ENGINE = MergeTree ORDER BY a +SETTINGS index_granularity = 4; + +INSERT INTO t_max_rows_to_read SELECT number FROM numbers(100); + +SET max_threads = 1; + +SELECT a FROM t_max_rows_to_read WHERE a = 10 SETTINGS max_rows_to_read = 4; + +SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 12; + +-- This should work, but actually it doesn't. Need to investigate. +-- SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 20; + +SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 20 FORMAT Null SETTINGS max_rows_to_read = 12; -- { serverError 158 } +SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 FORMAT Null SETTINGS max_rows_to_read = 12; -- { serverError 158 } +SELECT a FROM t_max_rows_to_read WHERE a = 10 OR a = 20 FORMAT Null SETTINGS max_rows_to_read = 4; -- { serverError 158 } + +DROP TABLE t_max_rows_to_read; From 30ffa57bd511f18252bea31e69b5cb6a08c134e0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 10:17:16 +0800 Subject: [PATCH 040/584] remove unused log --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 80d9d869772..5bc125c6787 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1355,7 +1355,6 @@ void IMergeTreeDataPart::assertMetaCacheDropped(bool include_projection) const } } } - // LOG_WARNING(storage.log, "cache of file {} does't belong to any part", file); } } From 7dab7caa9d44e21a5747c997d95252059f75a665 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 11:57:43 +0800 Subject: [PATCH 041/584] wrap cache related code with USE_ROCKSDB --- programs/server/Server.cpp | 2 + src/Interpreters/Context.cpp | 2 + src/Processors/examples/CMakeLists.txt | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 126 ++++++++++++++---- src/Storages/MergeTree/IMergeTreeDataPart.h | 22 ++- .../MergeTree/MergeTreeDataPartCompact.cpp | 4 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 +- src/Storages/MergeTree/MergeTreePartition.cpp | 20 ++- src/Storages/MergeTree/MergeTreePartition.h | 10 +- 9 files changed, 156 insertions(+), 36 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 7228608c9f1..45f7834c96c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -754,11 +754,13 @@ if (ThreadFuzzer::instance().isEffective()) } +#if USE_ROCKSDB /// initialize meta file cache { size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); global_context->initializeMergeTreeMetaCache(path_str + "/" + "rocksdb", size); } +#endif if (config().has("interserver_http_port") && config().has("interserver_https_port")) throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c5a5b3d1d49..8b8c8f982fd 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2080,10 +2080,12 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const return zookeeper->second; } +#if USE_ROCKSDB MergeTreeMetaCachePtr Context::getMergeTreeMetaCache() const { return shared->merge_tree_meta_cache; } +#endif void Context::resetZooKeeper() const { diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index ceb022432a1..a07224d4462 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,4 +1,4 @@ if (USE_ROCKSDB) add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) target_link_libraries (merge_tree_meta_cache PRIVATE dbms) -endif() +endif() \ No newline at end of file diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 5bc125c6787..0b5f9f19782 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -63,8 +63,13 @@ static std::unique_ptr openForReading(const DiskPtr & di return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } +#if USE_ROCKSDB void IMergeTreeDataPart::MinMaxIndex::load( const MergeTreeData & data, const PartMetaCachePtr & cache, const DiskPtr & disk, const String & /*part_path*/) +#else +void IMergeTreeDataPart::MinMaxIndex::load( + const MergeTreeData & data, const DiskPtr & disk, const String & part_path) +#endif { auto metadata_snapshot = data.getInMemoryMetadataPtr(); const auto & partition_key = metadata_snapshot->getPartitionKey(); @@ -76,15 +81,20 @@ void IMergeTreeDataPart::MinMaxIndex::load( hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { +#if USE_ROCKSDB String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; String value; - auto buf = cache->readOrSetMeta(disk, file_name, value); + auto file = cache->readOrSetMeta(disk, file_name, value); +#else + String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); + auto file = openForReading(disk, file_name); +#endif auto serialization = minmax_column_types[i]->getDefaultSerialization(); Field min_val; - serialization->deserializeBinary(min_val, *buf); + serialization->deserializeBinary(min_val, *file); Field max_val; - serialization->deserializeBinary(max_val, *buf); + serialization->deserializeBinary(max_val, *file); // NULL_LAST if (min_val.isNull()) @@ -301,9 +311,10 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) +#if USE_ROCKSDB , meta_cache(std::make_shared( storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) - +#endif { if (parent_part) state = State::Committed; @@ -329,9 +340,10 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) +#if USE_ROCKSDB , meta_cache(std::make_shared( storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) - +#endif { if (parent_part) state = State::Committed; @@ -738,8 +750,13 @@ void IMergeTreeDataPart::loadIndex() } String index_path = fs::path(getFullRelativePath()) / "primary.idx"; + +#if USE_ROCKSDB String value; - auto index_buf = meta_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); + auto index_file = meta_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); +#else + auto index_file = openForReading(volume->getDisk(), index_path); +#endif size_t marks_count = index_granularity.getMarksCount(); @@ -749,7 +766,7 @@ void IMergeTreeDataPart::loadIndex() for (size_t i = 0; i < marks_count; ++i) //-V756 for (size_t j = 0; j < key_size; ++j) - key_serializations[j]->deserializeBinary(*loaded_index[j], *index_buf); + key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file); for (size_t i = 0; i < key_size; ++i) { @@ -760,7 +777,7 @@ void IMergeTreeDataPart::loadIndex() ErrorCodes::CANNOT_READ_ALL_DATA); } - if (!index_buf->eof()) + if (!index_file->eof()) throw Exception("Index file " + fullPath(volume->getDisk(), index_path) + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE); index.assign(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end())); @@ -804,18 +821,25 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() return; } - String v; String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; - auto in_buf = meta_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); - if (!in_buf) + +#if USE_ROCKSDB + String v; + auto file_buf = meta_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); + if (!file_buf) +#else + if (!volume->getDisk()->exists(path)) +#endif { default_codec = detectDefaultCompressionCodec(); } else { - +#if !USE_ROCKSDB + auto file_buf = openForReading(volume->getDisk(), path); +#endif String codec_line; - readEscapedStringUntilEOL(codec_line, *in_buf); + readEscapedStringUntilEOL(codec_line, *file_buf); ReadBufferFromString buf(codec_line); @@ -912,7 +936,11 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() { String path = getFullRelativePath(); if (!parent_part) +#if USE_ROCKSDB partition.load(storage, meta_cache, volume->getDisk(), path); +#else + partition.load(storage, volume->getDisk(), path); +#endif if (!isEmpty()) { @@ -920,7 +948,11 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() // projection parts don't have minmax_idx, and it's always initialized minmax_idx->initialized = true; else +#if USE_ROCKSDB minmax_idx->load(storage, meta_cache, volume->getDisk(), path); +#else + minmax_idx->load(storage, volume->getDisk(), path); +#endif } if (parent_part) return; @@ -950,12 +982,20 @@ void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) c void IMergeTreeDataPart::loadChecksums(bool require) { - const String path = fs::path(getFullRelativePath()) / "checksums.txt"; +#if USE_ROCKSDB String value; auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); if (buf) +#else + const String path = fs::path(getFullRelativePath()) / "checksums.txt"; + if (volume->getDisk()->exists(path)) +#endif { + +#if !USE_ROCKSDB + auto buf = openForReading(volume->getDisk(), path); +#endif if (checksums.read(*buf)) { assertEOF(*buf); @@ -1008,6 +1048,7 @@ void IMergeTreeDataPart::loadRowsCount() } else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { +#if USE_ROCKSDB String v; auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); if (!buf) @@ -1015,6 +1056,12 @@ void IMergeTreeDataPart::loadRowsCount() readIntText(rows_count, *buf); assertEOF(*buf); +#else + if (!volume->getDisk()->exists(path)) + throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + + read_rows_count(); +#endif #ifndef NDEBUG /// columns have to be loaded @@ -1117,10 +1164,18 @@ void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) const void IMergeTreeDataPart::loadTTLInfos() { String path = fs::path(getFullRelativePath()) / "ttl.txt"; +#if USE_ROCKSDB String v; auto in = meta_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); if (in) +#else + if (volume->getDisk()->exists(path)) +#endif { + +#if !USE_ROCKSDB + auto in = openForReading(volume->getDisk(), path); +#endif assertString("ttl format version: ", *in); size_t format_version; readText(format_version, *in); @@ -1150,11 +1205,18 @@ void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) const void IMergeTreeDataPart::loadUUID() { +#if USE_ROCKSDB String v; - String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; auto in = meta_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); if (in) +#else + String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; + if (volume->getDisk()->exists(path)) +#endif { +#if !USE_ROCKSDB + auto in = openForReading(volume->getDisk(), path); +#endif readText(uuid, *in); if (uuid == UUIDHelpers::Nil) throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR); @@ -1174,9 +1236,13 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; +#if USE_ROCKSDB String v; auto in = meta_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); if (!in) +#else + if (!volume->getDisk()->exists(path)) +#endif { /// We can get list of columns only from columns.txt in compact parts. if (require || part_type == Type::COMPACT) @@ -1192,14 +1258,18 @@ void IMergeTreeDataPart::loadColumns(bool require) throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); { - auto out = volume->getDisk()->writeFile(path + ".tmp", 4096); - loaded_columns.writeText(*out); + auto buf = volume->getDisk()->writeFile(path + ".tmp", 4096); + loaded_columns.writeText(*buf); } volume->getDisk()->moveFile(path + ".tmp", path); } else { +#if USE_ROCKSDB loaded_columns.readText(*in); +#else + loaded_columns.readText(*volume->getDisk()->readFile(path)); +#endif for (const auto & column : loaded_columns) { @@ -1278,14 +1348,18 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } } +#if USE_ROCKSDB modifyAllMetaCaches(ModifyCacheType::DROP, true); -// #ifndef NDEBUG assertMetaCacheDropped(true); -// #endif +#endif + volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); volume->getDisk()->moveDirectory(from, to); relative_path = new_relative_path; + +#if USE_ROCKSDB modifyAllMetaCaches(ModifyCacheType::PUT, true); +#endif SyncGuardPtr sync_guard; if (storage.getSettings()->fsync_part_directory) @@ -1294,7 +1368,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ storage.lockSharedData(*this); } - +#if USE_ROCKSDB void IMergeTreeDataPart::modifyAllMetaCaches(ModifyCacheType type, bool include_projection) const { Strings files; @@ -1357,6 +1431,7 @@ void IMergeTreeDataPart::assertMetaCacheDropped(bool include_projection) const } } } +#endif std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const { @@ -1395,10 +1470,10 @@ void IMergeTreeDataPart::remove() const return; } +#if USE_ROCKSDB modifyAllMetaCaches(ModifyCacheType::DROP); -// #ifndef NDEBUG assertMetaCacheDropped(); -// #endif +#endif /** Atomic directory removal: * - rename directory to temporary name; @@ -1503,10 +1578,10 @@ void IMergeTreeDataPart::remove() const void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_shared_data) const { +#if USE_ROCKSDB modifyAllMetaCaches(ModifyCacheType::DROP); -// #ifndef NDEBUG assertMetaCacheDropped(); -// #endif +#endif String to = parent_to + "/" + relative_path; auto disk = volume->getDisk(); @@ -1595,7 +1670,6 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) return "detached/" + getRelativePathForPrefix(prefix, /* detached */ true); } - void IMergeTreeDataPart::renameToDetached(const String & prefix) const { renameTo(getRelativePathForDetachedPart(prefix), true); @@ -1848,6 +1922,7 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); } +#if USE_ROCKSDB IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const { String file_name = std::filesystem::path(file_path).filename(); @@ -1912,6 +1987,7 @@ void IMergeTreeDataPart::checkMetaCache(Strings & files, std::vector & disk_checksums.push_back(disk_checksum); } } +#endif bool isCompactPart(const MergeTreeDataPartPtr & data_part) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 0d2b2f57fd0..0364012bfec 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -45,6 +45,8 @@ class UncompressedCache; class IMergeTreeDataPart : public std::enable_shared_from_this { public: + +#if USE_ROCKSDB enum ModifyCacheType { PUT, // override set @@ -62,6 +64,9 @@ public: } } + using uint128 = PartMetaCache::uint128; +#endif + static constexpr auto DATA_FILE_EXTENSION = ".bin"; using Checksums = MergeTreeDataPartChecksums; @@ -77,7 +82,6 @@ public: using IndexSizeByName = std::unordered_map; using Type = MergeTreeDataPartType; - using uint128 = PartMetaCache::uint128; IMergeTreeDataPart( @@ -157,7 +161,9 @@ public: /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; +#if USE_ROCKSDB void assertMetaCacheDropped(bool include_projection = false) const; +#endif void remove() const; @@ -319,7 +325,12 @@ public: { } +#if USE_ROCKSDB void load(const MergeTreeData & data, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); +#else + void load(const MergeTreeData & data, const DiskPtr & disk, const String & part_path); +#endif + void store(const MergeTreeData & data, const DiskPtr & disk, const String & part_path, Checksums & checksums) const; void store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const; @@ -389,7 +400,9 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; +#if USE_ROCKSDB virtual void checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; +#endif bool isProjectionPart() const { return parent_part != nullptr; } @@ -462,7 +475,9 @@ protected: std::map> projection_parts; +#if USE_ROCKSDB mutable PartMetaCachePtr meta_cache; +#endif void removeIfNeeded(); @@ -535,13 +550,14 @@ private: void appendFilesOfDefaultCompressionCodec(Strings & files) const; - void modifyAllMetaCaches(ModifyCacheType type, bool include_projection = false) const; - /// Found column without specific compression and return codec /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; +#if USE_ROCKSDB + void modifyAllMetaCaches(ModifyCacheType type, bool include_projection = false) const; IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; +#endif mutable State state{State::Temporary}; }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index e5fbdd5cd19..fa9996b3382 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -192,12 +192,12 @@ MergeTreeDataPartCompact::~MergeTreeDataPartCompact() removeIfNeeded(); } -// Do not cache mark file, because cache other meta files is enough to speed up loading. +/// Do not cache mark file, because cache other meta files is enough to speed up loading. void MergeTreeDataPartCompact::appendFilesOfIndexGranularity(Strings& /* files */) const { } -// find all connected file and do modification +/// find all connected file and do modification Strings MergeTreeDataPartCompact::getIndexGranularityFiles() const { auto marks_file = index_granularity_info.getMarksFilePath("data"); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 5c21ead3208..f3c4b613078 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -166,12 +166,12 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() return checksum; } -// No mark files for part in memory +/// No mark files for part in memory void MergeTreeDataPartInMemory::appendFilesOfIndexGranularity(Strings& /* files */) const { } -// No mark files for part in memory +/// No mark files for part in memory Strings MergeTreeDataPartInMemory::getIndexGranularityFiles() const { return {}; diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 6b5bebd81f6..88a418b1129 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,6 +160,13 @@ namespace }; } +#if !USE_ROCKSDB +static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) +{ + size_t file_size = disk->getFileSize(path); + return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); +} +#endif String MergeTreePartition::getID(const MergeTreeData & storage) const { @@ -350,7 +357,11 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe } } +#if USE_ROCKSDB void MergeTreePartition::load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path) +#else +void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path) +#endif { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) @@ -359,11 +370,16 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetaCache const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; auto partition_file_path = part_path + "partition.dat"; +#if USE_ROCKSDB String v; - auto buf = meta_cache->readOrSetMeta(disk, "partition.dat", v); + auto file = meta_cache->readOrSetMeta(disk, "partition.dat", v); +#else + auto file = openForReading(disk, partition_file_path); +#endif + value.resize(partition_key_sample.columns()); for (size_t i = 0; i < partition_key_sample.columns(); ++i) - partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *buf); + partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file); } void MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index b8b5b301219..4920e658163 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -4,9 +4,12 @@ #include #include #include -#include #include +#if USE_ROCKSDB +#include +#endif + namespace DB { @@ -38,7 +41,12 @@ public: void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; +#if USE_ROCKSDB void load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); +#else + void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path); +#endif + void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; From 22b9f9d3e5062380335043b6fa2197cc797e7773 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 15:45:48 +0800 Subject: [PATCH 042/584] fix ut --- src/Interpreters/Context.h | 5 ++-- ...01233_check_part_meta_cache_replicated.sql | 28 ++++++++++--------- ...k_part_meta_cache_replicated_in_atomic.sql | 28 ++++++++++--------- 3 files changed, 32 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index bc191e80c9a..dcc24999944 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,6 +1,5 @@ #pragma once -#include "config_core.h" #include #include #include @@ -17,13 +16,13 @@ #include #include +#include "config_core.h" + #if USE_ROCKSDB #include #include #endif -#include "config_core.h" - #include #include #include diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index cb028e77d54..7722bba71dd 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -1,10 +1,12 @@ +-- Tags: zookeeper + -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; -CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Ordinary; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +DROP DATABASE IF EXISTS test_meta_cache; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache; +CREATE DATABASE test_meta_cache ENGINE = Ordinary; +CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. @@ -43,21 +45,21 @@ optimize table test_meta_cache.check_part_meta_cache FINAL; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 drop column v1; +alter table test_meta_cache.check_part_meta_cache drop column v1; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); @@ -66,8 +68,8 @@ truncate table test_meta_cache.check_part_meta_cache; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache on cluster preonline_hk5; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_meta_cache.check_part_meta_cache ; +CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; -- Insert first batch of data. INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -105,17 +107,17 @@ optimize table test_meta_cache.check_part_meta_cache FINAL; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index c56e2cb0a99..4dcc4478139 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -1,10 +1,12 @@ +-- Tags: zookeeper + -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; -CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Atomic; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +DROP DATABASE IF EXISTS test_meta_cache ; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache ; +CREATE DATABASE test_meta_cache ENGINE = Atomic; +CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. @@ -43,21 +45,21 @@ optimize table test_meta_cache.check_part_meta_cache FINAL; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 drop column v1; +alter table test_meta_cache.check_part_meta_cache drop column v1; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); @@ -66,8 +68,8 @@ truncate table test_meta_cache.check_part_meta_cache; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache on cluster preonline_hk5; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_meta_cache.check_part_meta_cache ; +CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; -- Insert first batch of data. INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -105,17 +107,17 @@ optimize table test_meta_cache.check_part_meta_cache FINAL; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); From 2a1fe52b9f2136405c3a5b2497e36be3528b4e74 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 18:06:13 +0800 Subject: [PATCH 043/584] rename symbols && fix uts && add setting use_metadata_cache --- programs/server/Server.cpp | 2 +- src/Common/ProfileEvents.cpp | 14 +- ...taCache.cpp => checkPartMetadataCache.cpp} | 79 +++-- .../registerFunctionsMiscellaneous.cpp | 4 +- src/Interpreters/Context.cpp | 48 +-- src/Interpreters/Context.h | 16 +- src/Processors/examples/CMakeLists.txt | 4 +- ...ache.cpp => merge_tree_metadata_cache.cpp} | 6 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 286 ++++++++++++------ src/Storages/MergeTree/IMergeTreeDataPart.h | 17 +- src/Storages/MergeTree/MergeTreeData.cpp | 11 + src/Storages/MergeTree/MergeTreePartition.cpp | 18 +- src/Storages/MergeTree/MergeTreePartition.h | 4 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + ...artMetaCache.cpp => PartMetadataCache.cpp} | 22 +- .../{PartMetaCache.h => PartMetadataCache.h} | 10 +- ...> StorageSystemMergeTreeMetadataCache.cpp} | 14 +- ... => StorageSystemMergeTreeMetadataCache.h} | 8 +- src/Storages/System/attachSystemTables.cpp | 4 +- .../01233_check_part_meta_cache.sql | 140 ++++----- .../01233_check_part_meta_cache_in_atomic.sql | 140 ++++----- ...01233_check_part_meta_cache_replicated.sql | 139 ++++----- ...k_part_meta_cache_replicated_in_atomic.sql | 139 ++++----- 23 files changed, 634 insertions(+), 492 deletions(-) rename src/Functions/{checkPartMetaCache.cpp => checkPartMetadataCache.cpp} (68%) rename src/Processors/examples/{merge_tree_meta_cache.cpp => merge_tree_metadata_cache.cpp} (86%) rename src/Storages/MergeTree/{PartMetaCache.cpp => PartMetadataCache.cpp} (80%) rename src/Storages/MergeTree/{PartMetaCache.h => PartMetadataCache.h} (76%) rename src/Storages/System/{StorageSystemMergeTreeMetaCache.cpp => StorageSystemMergeTreeMetadataCache.cpp} (84%) rename src/Storages/System/{StorageSystemMergeTreeMetaCache.h => StorageSystemMergeTreeMetadataCache.h} (55%) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6cfdc2376d6..1e10af9dc07 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -823,7 +823,7 @@ if (ThreadFuzzer::instance().isEffective()) /// initialize meta file cache { size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); - global_context->initializeMergeTreeMetaCache(path_str + "/" + "rocksdb", size); + global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); } #endif diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index e0383da29f4..45089312ed1 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -277,15 +277,15 @@ \ M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ \ - M(RocksdbGet, "Number of rocksdb reads(used for merge tree metadata cache)") \ - M(RocksdbPut, "Number of rocksdb puts(used for merge tree metadata cache)") \ - M(RocksdbDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \ - M(RocksdbSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \ - M(MergeTreeMetaCacheHit, "Number of times the read of meta file was done from MergeTree meta cache") \ - M(MergeTreeMetaCacheMiss, "Number of times the read of meta file was not done from MergeTree meta cache") \ - \ M(MainConfigLoads, "Number of times the main configuration was reloaded.") \ \ + M(MergeTreeMetadataCacheGet, "Number of rocksdb reads(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCachePut, "Number of rocksdb puts(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheHit, "Number of times the read of meta file was done from MergeTree metadata cache") \ + M(MergeTreeMetadataCacheMiss, "Number of times the read of meta file was not done from MergeTree metadata cache") \ + \ namespace ProfileEvents { diff --git a/src/Functions/checkPartMetaCache.cpp b/src/Functions/checkPartMetadataCache.cpp similarity index 68% rename from src/Functions/checkPartMetaCache.cpp rename to src/Functions/checkPartMetadataCache.cpp index e77b8ca0d50..2883424f996 100644 --- a/src/Functions/checkPartMetaCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -21,14 +21,13 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; } -class FunctionCheckPartMetaCache: public IFunction, WithContext +class FunctionCheckPartMetadataCache : public IFunction, WithContext { public: using uint128 = IMergeTreeDataPart::uint128; @@ -37,22 +36,18 @@ public: using DataPartStates = MergeTreeData::DataPartStates; - static constexpr auto name = "checkPartMetaCache"; - static FunctionPtr create(ContextPtr context_) - { - return std::make_shared(context_); - } + static constexpr auto name = "checkPartMetadataCache"; + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - static constexpr DataPartStates part_states = { - DataPartState::Committed, - DataPartState::Temporary, - DataPartState::PreCommitted, - DataPartState::Outdated, - DataPartState::Deleting, - DataPartState::DeleteOnDestroy - }; + static constexpr DataPartStates part_states + = {DataPartState::Committed, + DataPartState::Temporary, + DataPartState::PreCommitted, + DataPartState::Outdated, + DataPartState::Deleting, + DataPartState::DeleteOnDestroy}; - explicit FunctionCheckPartMetaCache(ContextPtr context_): WithContext(context_) {} + explicit FunctionCheckPartMetadataCache(ContextPtr context_) : WithContext(context_) { } String getName() const override { return name; } @@ -76,44 +71,44 @@ public: DataTypePtr cache_checksum_type = std::make_unique(32); DataTypePtr disk_checksum_type = std::make_unique(32); DataTypePtr match_type = std::make_unique(); - DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, state_type, cache_checksum_type, disk_checksum_type, match_type}); + DataTypePtr tuple_type + = std::make_unique(DataTypes{key_type, state_type, cache_checksum_type, disk_checksum_type, match_type}); return std::make_shared(tuple_type); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { - // get database name + /// Get database name const auto * arg_database = arguments[0].column.get(); - const ColumnString * column_database = checkAndGetColumnConstData(arg_database); - if (! column_database) + const ColumnString * column_database = checkAndGetColumnConstData(arg_database); + if (!column_database) throw Exception("The first argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); String database_name = column_database->getDataAt(0).toString(); - // get table name + /// Get table name const auto * arg_table = arguments[1].column.get(); const ColumnString * column_table = checkAndGetColumnConstData(arg_table); - if (! column_table) + if (!column_table) throw Exception("The second argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); String table_name = column_table->getDataAt(0).toString(); - // get storage + /// Get storage StorageID storage_id(database_name, table_name); auto storage = DatabaseCatalog::instance().getTable(storage_id, getContext()); auto data = std::dynamic_pointer_cast(storage); - if (! data) + if (!data || !data->getSettings()->use_metadata_cache) throw Exception("The table in function " + getName() + " must be in MergeTree Family", ErrorCodes::ILLEGAL_COLUMN); - // fill in result + /// Fill in result auto col_result = result_type->createColumn(); - auto& col_arr = assert_cast(*col_result); - col_arr.reserve(1); - auto& col_tuple = assert_cast(col_arr.getData()); + auto & col_arr = assert_cast(*col_result); + auto & col_tuple = assert_cast(col_arr.getData()); col_tuple.reserve(data->fileNumberOfDataParts(part_states)); - auto& col_key = assert_cast(col_tuple.getColumn(0)); - auto& col_state = assert_cast(col_tuple.getColumn(1)); - auto& col_cache_checksum = assert_cast(col_tuple.getColumn(2)); - auto& col_disk_checksum = assert_cast(col_tuple.getColumn(3)); - auto& col_match = assert_cast(col_tuple.getColumn(4)); + auto & col_key = assert_cast(col_tuple.getColumn(0)); + auto & col_state = assert_cast(col_tuple.getColumn(1)); + auto & col_cache_checksum = assert_cast(col_tuple.getColumn(2)); + auto & col_disk_checksum = assert_cast(col_tuple.getColumn(3)); + auto & col_match = assert_cast(col_tuple.getColumn(4)); auto parts = data->getDataParts(part_states); for (const auto & part : parts) executePart(part, col_key, col_state, col_cache_checksum, col_disk_checksum, col_match); @@ -121,21 +116,26 @@ public: return result_type->createColumnConst(input_rows_count, col_arr[0]); } - static void executePart(const DataPartPtr& part, ColumnString& col_key, ColumnString& col_state, - ColumnFixedString& col_cache_checksum, ColumnFixedString& col_disk_checksum, ColumnUInt8& col_match) + static void executePart( + const DataPartPtr & part, + ColumnString & col_key, + ColumnString & col_state, + ColumnFixedString & col_cache_checksum, + ColumnFixedString & col_disk_checksum, + ColumnUInt8 & col_match) { Strings keys; auto state_view = part->stateString(); String state(state_view.data(), state_view.size()); std::vector cache_checksums; std::vector disk_checksums; - uint8_t match = 0; + uint8_t match = 0; size_t file_number = part->fileNumberOfColumnsChecksumsIndexes(); keys.reserve(file_number); cache_checksums.reserve(file_number); disk_checksums.reserve(file_number); - part->checkMetaCache(keys, cache_checksums, disk_checksums); + part->checkMetadataCache(keys, cache_checksums, disk_checksums); for (size_t i = 0; i < keys.size(); ++i) { col_key.insert(keys[i]); @@ -149,10 +149,9 @@ public: } }; -void registerFunctionCheckPartMetaCache(FunctionFactory & factory) +void registerFunctionCheckPartMetadataCache(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } - } #endif diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 77e3e109081..297e6dfb452 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -82,7 +82,7 @@ void registerFunctionZooKeeperSessionUptime(FunctionFactory &); void registerFunctionGetOSKernelVersion(FunctionFactory &); #if USE_ROCKSDB -void registerFunctionCheckPartMetaCache(FunctionFactory &); +void registerFunctionCheckPartMetadataCache(FunctionFactory &); #endif #if USE_ICU @@ -172,7 +172,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionGetOSKernelVersion(factory); #if USE_ROCKSDB - registerFunctionCheckPartMetaCache(factory); + registerFunctionCheckPartMetadataCache(factory); #endif #if USE_ICU diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4d9c571232e..4897cdccac9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -97,10 +97,10 @@ namespace ProfileEvents extern const Event ContextLock; #if USE_ROCKSDB - extern const Event RocksdbPut; - extern const Event RocksdbGet; - extern const Event RocksdbDelete; - extern const Event RocksdbSeek; + extern const Event MergeTreeMetadataCachePut; + extern const Event MergeTreeMetadataCacheGet; + extern const Event MergeTreeMetadataCacheDelete; + extern const Event MergeTreeMetadataCacheSeek; #endif } @@ -284,7 +284,7 @@ struct ContextSharedPart #if USE_ROCKSDB /// MergeTree metadata cache stored in rocksdb. - MergeTreeMetaCachePtr merge_tree_meta_cache; + MergeTreeMetadataCachePtr merge_tree_metadata_cache; #endif ContextSharedPart() @@ -400,10 +400,10 @@ struct ContextSharedPart #if USE_ROCKSDB /// Shutdown meta file cache - if (merge_tree_meta_cache) + if (merge_tree_metadata_cache) { - merge_tree_meta_cache->shutdown(); - merge_tree_meta_cache.reset(); + merge_tree_metadata_cache->shutdown(); + merge_tree_metadata_cache.reset(); } #endif } @@ -450,32 +450,32 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr shar void SharedContextHolder::reset() { shared.reset(); } #if USE_ROCKSDB -MergeTreeMetaCache::Status MergeTreeMetaCache::put(const String & key, const String & value) +MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) { auto options = rocksdb::WriteOptions(); auto status = rocksdb->Put(options, key, value); - ProfileEvents::increment(ProfileEvents::RocksdbPut); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCachePut); return status; } -MergeTreeMetaCache::Status MergeTreeMetaCache::del(const String & key) +MergeTreeMetadataCache::Status MergeTreeMetadataCache::del(const String & key) { auto options = rocksdb::WriteOptions(); auto status = rocksdb->Delete(options, key); - ProfileEvents::increment(ProfileEvents::RocksdbDelete); - LOG_TRACE(log, "Delete key:{} from MergeTreeMetaCache status:{}", key, status.ToString()); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheDelete); + LOG_TRACE(log, "Delete key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); return status; } -MergeTreeMetaCache::Status MergeTreeMetaCache::get(const String & key, String & value) +MergeTreeMetadataCache::Status MergeTreeMetadataCache::get(const String & key, String & value) { auto status = rocksdb->Get(rocksdb::ReadOptions(), key, &value); - ProfileEvents::increment(ProfileEvents::RocksdbGet); - LOG_TRACE(log, "Get key:{} from MergeTreeMetaCache status:{}", key, status.ToString()); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheGet); + LOG_TRACE(log, "Get key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); return status; } -void MergeTreeMetaCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) +void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) { auto * it = rocksdb->NewIterator(rocksdb::ReadOptions()); rocksdb::Slice target(prefix); @@ -489,11 +489,11 @@ void MergeTreeMetaCache::getByPrefix(const String & prefix, Strings & keys, Stri keys.emplace_back(key.data(), key.size()); values.emplace_back(value.data(), value.size()); } - LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetaCache items:{}", prefix, keys.size()); - ProfileEvents::increment(ProfileEvents::RocksdbSeek); + LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetadataCache items:{}", prefix, keys.size()); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); } -void MergeTreeMetaCache::shutdown() +void MergeTreeMetadataCache::shutdown() { if (rocksdb) { @@ -2083,9 +2083,9 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const } #if USE_ROCKSDB -MergeTreeMetaCachePtr Context::getMergeTreeMetaCache() const +MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const { - return shared->merge_tree_meta_cache; + return shared->merge_tree_metadata_cache; } #endif @@ -2333,7 +2333,7 @@ void Context::initializeTraceCollector() } #if USE_ROCKSDB -void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) +void Context::initializeMergeTreeMetadataCache(const String & dir, size_t size) { rocksdb::Options options; rocksdb::BlockBasedTableOptions table_options; @@ -2350,7 +2350,7 @@ void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) String message = "Fail to open rocksdb path at: " + dir + " status:" + status.ToString(); throw Exception(message, ErrorCodes::SYSTEM_ERROR); } - shared->merge_tree_meta_cache = std::make_shared(db); + shared->merge_tree_metadata_cache = std::make_shared(db); } #endif diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e579fc9c4bf..ea606fdea4e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -183,14 +183,14 @@ private: }; #if USE_ROCKSDB -class MergeTreeMetaCache +class MergeTreeMetadataCache { public: using Status = rocksdb::Status; - explicit MergeTreeMetaCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } - MergeTreeMetaCache(const MergeTreeMetaCache &) = delete; - MergeTreeMetaCache & operator=(const MergeTreeMetaCache &) = delete; + explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } + MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; + MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; Status put(const String & key, const String & value); Status del(const String & key); @@ -200,9 +200,9 @@ public: void shutdown(); private: std::unique_ptr rocksdb; - Poco::Logger * log = &Poco::Logger::get("MergeTreeMetaCache"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeMetadataCache"); }; -using MergeTreeMetaCachePtr = std::shared_ptr; +using MergeTreeMetadataCachePtr = std::shared_ptr; #endif /** A set of known objects that can be used in the query. @@ -705,7 +705,7 @@ public: UInt32 getZooKeeperSessionUptime() const; #if USE_ROCKSDB - MergeTreeMetaCachePtr getMergeTreeMetaCache() const; + MergeTreeMetadataCachePtr getMergeTreeMetadataCache() const; #endif @@ -799,7 +799,7 @@ public: void initializeTraceCollector(); #if USE_ROCKSDB - void initializeMergeTreeMetaCache(const String & dir, size_t size); + void initializeMergeTreeMetadataCache(const String & dir, size_t size); #endif bool hasTraceCollector() const; diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index a07224d4462..2b6b9128e4c 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,4 +1,4 @@ if (USE_ROCKSDB) - add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) - target_link_libraries (merge_tree_meta_cache PRIVATE dbms) + add_executable (merge_tree_metadata_cache merge_tree_metadata_cache.cpp) + target_link_libraries (merge_tree_metadata_cache PRIVATE dbms) endif() \ No newline at end of file diff --git a/src/Processors/examples/merge_tree_meta_cache.cpp b/src/Processors/examples/merge_tree_metadata_cache.cpp similarity index 86% rename from src/Processors/examples/merge_tree_meta_cache.cpp rename to src/Processors/examples/merge_tree_metadata_cache.cpp index a394323bcb7..c726eb7ce5a 100644 --- a/src/Processors/examples/merge_tree_meta_cache.cpp +++ b/src/Processors/examples/merge_tree_metadata_cache.cpp @@ -7,13 +7,13 @@ int main() auto shared_context = Context::createShared(); auto global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); - global_context->initializeMergeTreeMetaCache("./db/", 256 << 20); + global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); - auto cache = global_context->getMergeTreeMetaCache(); + auto cache = global_context->getMergeTreeMetadataCache(); std::vector files = {"columns.txt", "checksums.txt", "primary.idx", "count.txt", "partition.dat", "minmax_p.idx", "default_compression_codec.txt"}; - String prefix = "data/test_meta_cache/check_part_meta_cache/201806_1_1_0_4/"; + String prefix = "data/test_metadata_cache/check_part_metadata_cache/201806_1_1_0_4/"; for (const auto & file : files) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 0b5f9f19782..b29886265f6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -65,7 +65,7 @@ static std::unique_ptr openForReading(const DiskPtr & di #if USE_ROCKSDB void IMergeTreeDataPart::MinMaxIndex::load( - const MergeTreeData & data, const PartMetaCachePtr & cache, const DiskPtr & disk, const String & /*part_path*/) + const MergeTreeData & data, const PartMetadataCachePtr & cache, const DiskPtr & disk, const String & part_path) #else void IMergeTreeDataPart::MinMaxIndex::load( const MergeTreeData & data, const DiskPtr & disk, const String & part_path) @@ -78,16 +78,30 @@ void IMergeTreeDataPart::MinMaxIndex::load( auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); size_t minmax_idx_size = minmax_column_types.size(); + auto read_min_max_index = [&](size_t i) + { + String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); + auto file = openForReading(disk, file_name); + return file; + }; + hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { + std::unique_ptr file; #if USE_ROCKSDB - String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; - String value; - auto file = cache->readOrSetMeta(disk, file_name, value); + if (cache) + { + String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + String value; + file = cache->readOrSetMeta(disk, file_name, value); + } + else + { + file = read_min_max_index(i); + } #else - String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); - auto file = openForReading(disk, file_name); + file = read_min_max_index(i); #endif auto serialization = minmax_column_types[i]->getDefaultSerialization(); @@ -311,10 +325,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) -#if USE_ROCKSDB - , meta_cache(std::make_shared( - storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) -#endif + , use_metadata_cache(storage.getSettings()->use_metadata_cache) { if (parent_part) state = State::Committed; @@ -322,6 +333,12 @@ IMergeTreeDataPart::IMergeTreeDataPart( incrementTypeMetric(part_type); minmax_idx = std::make_shared(); + +#if USE_ROCKSDB + if (use_metadata_cache) + metadata_cache = std::make_shared( + storage.getContext()->getMergeTreeMetadataCache(), storage.relative_data_path, relative_path, parent_part); +#endif } IMergeTreeDataPart::IMergeTreeDataPart( @@ -340,10 +357,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) -#if USE_ROCKSDB - , meta_cache(std::make_shared( - storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) -#endif + , use_metadata_cache(storage.getSettings()->use_metadata_cache) { if (parent_part) state = State::Committed; @@ -351,6 +365,12 @@ IMergeTreeDataPart::IMergeTreeDataPart( incrementTypeMetric(part_type); minmax_idx = std::make_shared(); + +#if USE_ROCKSDB + if (use_metadata_cache) + metadata_cache = std::make_shared( + storage.getContext()->getMergeTreeMetadataCache(), storage.relative_data_path, relative_path, parent_part); +#endif } IMergeTreeDataPart::~IMergeTreeDataPart() @@ -751,11 +771,19 @@ void IMergeTreeDataPart::loadIndex() String index_path = fs::path(getFullRelativePath()) / "primary.idx"; + std::unique_ptr index_file; #if USE_ROCKSDB - String value; - auto index_file = meta_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); + if (use_metadata_cache) + { + String value; + index_file = metadata_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); + } + else + { + index_file = openForReading(volume->getDisk(), index_path); + } #else - auto index_file = openForReading(volume->getDisk(), index_path); + index_file = openForReading(volume->getDisk(), index_path); #endif size_t marks_count = index_granularity.getMarksCount(); @@ -823,21 +851,33 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; + bool exists = false; + std::unique_ptr file_buf; #if USE_ROCKSDB - String v; - auto file_buf = meta_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); - if (!file_buf) + if (use_metadata_cache) + { + String v; + file_buf = metadata_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); + exists = file_buf != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + file_buf = openForReading(volume->getDisk(), path); + } #else - if (!volume->getDisk()->exists(path)) + exists = volume->getDisk()->exists(path); + if (exists) + file_buf = openForReading(volume->getDisk(), path); #endif + + if (!exists) { default_codec = detectDefaultCompressionCodec(); } else { -#if !USE_ROCKSDB - auto file_buf = openForReading(volume->getDisk(), path); -#endif String codec_line; readEscapedStringUntilEOL(codec_line, *file_buf); @@ -937,7 +977,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() String path = getFullRelativePath(); if (!parent_part) #if USE_ROCKSDB - partition.load(storage, meta_cache, volume->getDisk(), path); + partition.load(storage, metadata_cache, volume->getDisk(), path); #else partition.load(storage, volume->getDisk(), path); #endif @@ -949,7 +989,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() minmax_idx->initialized = true; else #if USE_ROCKSDB - minmax_idx->load(storage, meta_cache, volume->getDisk(), path); + minmax_idx->load(storage, metadata_cache, volume->getDisk(), path); #else minmax_idx->load(storage, volume->getDisk(), path); #endif @@ -982,20 +1022,31 @@ void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) c void IMergeTreeDataPart::loadChecksums(bool require) { + const String path = fs::path(getFullRelativePath()) / "checksums.txt"; + bool exists = false; + std::unique_ptr buf; #if USE_ROCKSDB - String value; - auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); - if (buf) -#else - const String path = fs::path(getFullRelativePath()) / "checksums.txt"; - if (volume->getDisk()->exists(path)) -#endif + if (use_metadata_cache) { - -#if !USE_ROCKSDB - auto buf = openForReading(volume->getDisk(), path); + String value; + buf = metadata_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); + exists = buf != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + buf = openForReading(volume->getDisk(), path); + } +#else + exists = volume->getDisk()->exists(path); + if (exists) + buf = openForReading(volume->getDisk(), path); #endif + + if (exists) + { if (checksums.read(*buf)) { assertEOF(*buf); @@ -1049,13 +1100,23 @@ void IMergeTreeDataPart::loadRowsCount() else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { #if USE_ROCKSDB - String v; - auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); - if (!buf) - throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + if (use_metadata_cache) + { + String v; + auto buf = metadata_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); + if (!buf) + throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - readIntText(rows_count, *buf); - assertEOF(*buf); + readIntText(rows_count, *buf); + assertEOF(*buf); + } + else + { + if (!volume->getDisk()->exists(path)) + throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + + read_rows_count(); + } #else if (!volume->getDisk()->exists(path)) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); @@ -1164,18 +1225,30 @@ void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) const void IMergeTreeDataPart::loadTTLInfos() { String path = fs::path(getFullRelativePath()) / "ttl.txt"; -#if USE_ROCKSDB - String v; - auto in = meta_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); - if (in) -#else - if (volume->getDisk()->exists(path)) -#endif - { + bool exists = false; + std::unique_ptr in; -#if !USE_ROCKSDB - auto in = openForReading(volume->getDisk(), path); +#if USE_ROCKSDB + if (use_metadata_cache) + { + String v; + in = metadata_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); + exists = in != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); + } +#else + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); #endif + + if (exists) + { assertString("ttl format version: ", *in); size_t format_version; readText(format_version, *in); @@ -1205,18 +1278,31 @@ void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) const void IMergeTreeDataPart::loadUUID() { -#if USE_ROCKSDB - String v; - auto in = meta_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); - if (in) -#else String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; - if (volume->getDisk()->exists(path)) -#endif + bool exists = false; + std::unique_ptr in; + +#if USE_ROCKSDB + if (use_metadata_cache) { -#if !USE_ROCKSDB - auto in = openForReading(volume->getDisk(), path); + String v; + in = metadata_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); + exists = in != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); + } +#else + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); #endif + + if (exists) + { readText(uuid, *in); if (uuid == UUIDHelpers::Nil) throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR); @@ -1236,13 +1322,28 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; + bool exists = false; + std::unique_ptr in; #if USE_ROCKSDB - String v; - auto in = meta_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); - if (!in) + if (use_metadata_cache) + { + String v; + in = metadata_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); + exists = in != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); + } #else - if (!volume->getDisk()->exists(path)) + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); #endif + + if (!exists) { /// We can get list of columns only from columns.txt in compact parts. if (require || part_type == Type::COMPACT) @@ -1265,11 +1366,7 @@ void IMergeTreeDataPart::loadColumns(bool require) } else { -#if USE_ROCKSDB loaded_columns.readText(*in); -#else - loaded_columns.readText(*volume->getDisk()->readFile(path)); -#endif for (const auto & column : loaded_columns) { @@ -1349,8 +1446,11 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } #if USE_ROCKSDB - modifyAllMetaCaches(ModifyCacheType::DROP, true); - assertMetaCacheDropped(true); + if (use_metadata_cache) + { + modifyAllMetadataCaches(ModifyCacheType::DROP, true); + assertMetadataCacheDropped(true); + } #endif volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); @@ -1358,7 +1458,10 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ relative_path = new_relative_path; #if USE_ROCKSDB - modifyAllMetaCaches(ModifyCacheType::PUT, true); + if (use_metadata_cache) + { + modifyAllMetadataCaches(ModifyCacheType::PUT, true); + } #endif SyncGuardPtr sync_guard; @@ -1369,8 +1472,10 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } #if USE_ROCKSDB -void IMergeTreeDataPart::modifyAllMetaCaches(ModifyCacheType type, bool include_projection) const +void IMergeTreeDataPart::modifyAllMetadataCaches(ModifyCacheType type, bool include_projection) const { + assert(use_metadata_cache); + Strings files; files.reserve(16); appendFilesOfColumnsChecksumsIndexes(files, include_projection); @@ -1385,20 +1490,21 @@ void IMergeTreeDataPart::modifyAllMetaCaches(ModifyCacheType type, bool include_ switch (type) { case ModifyCacheType::PUT: - meta_cache->setMetas(volume->getDisk(), files); + metadata_cache->setMetas(volume->getDisk(), files); break; case ModifyCacheType::DROP: - meta_cache->dropMetas(files); + metadata_cache->dropMetas(files); break; } } - -void IMergeTreeDataPart::assertMetaCacheDropped(bool include_projection) const +void IMergeTreeDataPart::assertMetadataCacheDropped(bool include_projection) const { + assert(use_metadata_cache); + Strings files; std::vector _; - meta_cache->getFilesAndCheckSums(files, _); + metadata_cache->getFilesAndCheckSums(files, _); if (files.empty()) return; @@ -1471,8 +1577,11 @@ void IMergeTreeDataPart::remove() const } #if USE_ROCKSDB - modifyAllMetaCaches(ModifyCacheType::DROP); - assertMetaCacheDropped(); + if (use_metadata_cache) + { + modifyAllMetadataCaches(ModifyCacheType::DROP); + assertMetadataCacheDropped(); + } #endif /** Atomic directory removal: @@ -1579,8 +1688,11 @@ void IMergeTreeDataPart::remove() const void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_shared_data) const { #if USE_ROCKSDB - modifyAllMetaCaches(ModifyCacheType::DROP); - assertMetaCacheDropped(); + if (use_metadata_cache) + { + modifyAllMetadataCaches(ModifyCacheType::DROP); + assertMetadataCacheDropped(); + } #endif String to = parent_to + "/" + relative_path; @@ -1925,6 +2037,8 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const #if USE_ROCKSDB IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const { + assert(use_metadata_cache); + String file_name = std::filesystem::path(file_path).filename(); const auto filenames_without_checksums = getFileNamesWithoutChecksums(); auto it = checksums.files.find(file_name); @@ -1945,15 +2059,17 @@ IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const St return in_hash.getHash(); } -void IMergeTreeDataPart::checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const +void IMergeTreeDataPart::checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const { - /// checkMetaCache only applies for normal part + assert(use_metadata_cache); + + /// Only applies for normal part if (isProjectionPart()) return; /// the directory of projection part is under the directory of its parent part const auto filenames_without_checksums = getFileNamesWithoutChecksums(); - meta_cache->getFilesAndCheckSums(files, cache_checksums); + metadata_cache->getFilesAndCheckSums(files, cache_checksums); for (const auto & file : files) { // std::cout << "check key:" << file << std::endl; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 0364012bfec..2798f42b6c4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include @@ -64,7 +64,7 @@ public: } } - using uint128 = PartMetaCache::uint128; + using uint128 = PartMetadataCache::uint128; #endif static constexpr auto DATA_FILE_EXTENSION = ".bin"; @@ -162,7 +162,7 @@ public: void assertOnDisk() const; #if USE_ROCKSDB - void assertMetaCacheDropped(bool include_projection = false) const; + void assertMetadataCacheDropped(bool include_projection = false) const; #endif void remove() const; @@ -326,7 +326,7 @@ public: } #if USE_ROCKSDB - void load(const MergeTreeData & data, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); + void load(const MergeTreeData & data, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path); #else void load(const MergeTreeData & data, const DiskPtr & disk, const String & part_path); #endif @@ -401,7 +401,7 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; #if USE_ROCKSDB - virtual void checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; + virtual void checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; #endif bool isProjectionPart() const { return parent_part != nullptr; } @@ -475,8 +475,11 @@ protected: std::map> projection_parts; + /// Disabled when USE_ROCKSDB is OFF, or use_metadata_cache is set true in merge tree settings + bool use_metadata_cache = false; + #if USE_ROCKSDB - mutable PartMetaCachePtr meta_cache; + mutable PartMetadataCachePtr metadata_cache; #endif void removeIfNeeded(); @@ -555,7 +558,7 @@ private: CompressionCodecPtr detectDefaultCompressionCodec() const; #if USE_ROCKSDB - void modifyAllMetaCaches(ModifyCacheType type, bool include_projection = false) const; + void modifyAllMetadataCaches(ModifyCacheType type, bool include_projection = false) const; IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; #endif diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8d861e404f0..6add439d1f9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -317,6 +317,17 @@ MergeTreeData::MergeTreeData( LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', " "'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason); +#if !USE_ROCKSDB + if (settings->use_metadata_cache) + { + LOG_WARNING( + log, + "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." + "set use_metadata_cache to false forcely"); + settings->use_metadata_cache = false; + } +#endif + common_assignee_trigger = [this] (bool delay) noexcept { if (delay) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 88a418b1129..933248c9a1e 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,13 +160,11 @@ namespace }; } -#if !USE_ROCKSDB static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) { size_t file_size = disk->getFileSize(path); return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } -#endif String MergeTreePartition::getID(const MergeTreeData & storage) const { @@ -358,7 +356,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe } #if USE_ROCKSDB -void MergeTreePartition::load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path) +void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path) #else void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path) #endif @@ -370,11 +368,19 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; auto partition_file_path = part_path + "partition.dat"; + std::unique_ptr file; #if USE_ROCKSDB - String v; - auto file = meta_cache->readOrSetMeta(disk, "partition.dat", v); + if (metadata_cache) + { + String v; + file = metadata_cache->readOrSetMeta(disk, "partition.dat", v); + } + else + { + file = openForReading(disk, partition_file_path); + } #else - auto file = openForReading(disk, partition_file_path); + file = openForReading(disk, partition_file_path); #endif value.resize(partition_key_sample.columns()); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 4920e658163..644ddd5ba88 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -7,7 +7,7 @@ #include #if USE_ROCKSDB -#include +#include #endif namespace DB @@ -42,7 +42,7 @@ public: void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; #if USE_ROCKSDB - void load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); + void load(const MergeTreeData & storage, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path); #else void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path); #endif diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index b991166b3b6..51831eecf84 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -137,6 +137,7 @@ struct Settings; /** Experimental/work in progress feature. Unsafe for production. */ \ M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ + M(Bool, use_metadata_cache, false, "Experimental feature to speed up parts loading process by using MergeTree metadata cache", 0) \ \ /** Obsolete settings. Kept for backward compatibility only. */ \ M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ diff --git a/src/Storages/MergeTree/PartMetaCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp similarity index 80% rename from src/Storages/MergeTree/PartMetaCache.cpp rename to src/Storages/MergeTree/PartMetadataCache.cpp index fe8475d0dda..a8ab2c0bf2d 100644 --- a/src/Storages/MergeTree/PartMetaCache.cpp +++ b/src/Storages/MergeTree/PartMetadataCache.cpp @@ -1,4 +1,4 @@ -#include "PartMetaCache.h" +#include "PartMetadataCache.h" #if USE_ROCKSDB #include @@ -11,8 +11,8 @@ namespace ProfileEvents { - extern const Event MergeTreeMetaCacheHit; - extern const Event MergeTreeMetaCacheMiss; + extern const Event MergeTreeMetadataCacheHit; + extern const Event MergeTreeMetadataCacheMiss; } namespace ErrorCodes @@ -24,13 +24,13 @@ namespace DB { std::unique_ptr -PartMetaCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value) +PartMetadataCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; auto status = cache->get(file_path, value); if (!status.ok()) { - ProfileEvents::increment(ProfileEvents::MergeTreeMetaCacheMiss); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); if (!disk->exists(file_path)) { return nullptr; @@ -45,12 +45,12 @@ PartMetaCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, Str } else { - ProfileEvents::increment(ProfileEvents::MergeTreeMetaCacheHit); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit); } return std::make_unique(value); } -void PartMetaCache::setMetas(const DiskPtr & disk, const Strings & file_names) +void PartMetadataCache::setMetas(const DiskPtr & disk, const Strings & file_names) { String text; String read_value; @@ -76,7 +76,7 @@ void PartMetaCache::setMetas(const DiskPtr & disk, const Strings & file_names) } } -void PartMetaCache::dropMetas(const Strings & file_names) +void PartMetadataCache::dropMetas(const Strings & file_names) { for (const auto & file_name : file_names) { @@ -93,7 +93,7 @@ void PartMetaCache::dropMetas(const Strings & file_names) } } -void PartMetaCache::setMeta(const String & file_name, const String & value) +void PartMetadataCache::setMeta(const String & file_name, const String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; String read_value; @@ -112,7 +112,7 @@ void PartMetaCache::setMeta(const String & file_name, const String & value) } } -void PartMetaCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const +void PartMetadataCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const { String prefix = fs::path(getFullRelativePath()) / ""; Strings values; @@ -127,7 +127,7 @@ void PartMetaCache::getFilesAndCheckSums(Strings & files, std::vector & } } -String PartMetaCache::getFullRelativePath() const +String PartMetadataCache::getFullRelativePath() const { return fs::path(relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; } diff --git a/src/Storages/MergeTree/PartMetaCache.h b/src/Storages/MergeTree/PartMetadataCache.h similarity index 76% rename from src/Storages/MergeTree/PartMetaCache.h rename to src/Storages/MergeTree/PartMetadataCache.h index 5ffd0413c4b..97880f41b6c 100644 --- a/src/Storages/MergeTree/PartMetaCache.h +++ b/src/Storages/MergeTree/PartMetadataCache.h @@ -13,15 +13,15 @@ namespace DB class SeekableReadBuffer; class IMergeTreeDataPart; -class PartMetaCache; -using PartMetaCachePtr = std::shared_ptr; +class PartMetadataCache; +using PartMetadataCachePtr = std::shared_ptr; -class PartMetaCache +class PartMetadataCache { public: using uint128 = CityHash_v1_0_2::uint128; - PartMetaCache(const MergeTreeMetaCachePtr & cache_, const String & relative_data_path_, const String & relative_path_, const IMergeTreeDataPart * parent_part_) + PartMetadataCache(const MergeTreeMetadataCachePtr & cache_, const String & relative_data_path_, const String & relative_path_, const IMergeTreeDataPart * parent_part_) : cache(cache_) , relative_data_path(relative_data_path_) , relative_path(relative_path_) @@ -39,7 +39,7 @@ public: private: std::string getFullRelativePath() const; - MergeTreeMetaCachePtr cache; + MergeTreeMetadataCachePtr cache; const String & relative_data_path; // relative path of table to disk const String & relative_path; // relative path of part to table const IMergeTreeDataPart * parent_part; diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp similarity index 84% rename from src/Storages/System/StorageSystemMergeTreeMetaCache.cpp rename to src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index f53c32e5a42..b62a7985c0c 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_ROCKSDB #include @@ -20,7 +20,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -NamesAndTypesList StorageSystemMergeTreeMetaCache::getNamesAndTypes() +NamesAndTypesList StorageSystemMergeTreeMetadataCache::getNamesAndTypes() { return { {"key", std::make_shared()}, @@ -95,19 +95,19 @@ static String extractKey(const ASTPtr & query, bool& precise) } -void StorageSystemMergeTreeMetaCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void StorageSystemMergeTreeMetadataCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const { bool precise = false; String key = extractKey(query_info.query, precise); if (key.empty()) throw Exception( - "SELECT from system.merge_tree_meta_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); + "SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); - auto cache = context->getMergeTreeMetaCache(); + auto cache = context->getMergeTreeMetadataCache(); if (precise) { String value; - if (cache->get(key, value) != MergeTreeMetaCache::Status::OK()) + if (cache->get(key, value) != MergeTreeMetadataCache::Status::OK()) return; size_t col_num = 0; @@ -119,7 +119,7 @@ void StorageSystemMergeTreeMetaCache::fillData(MutableColumns & res_columns, Con String target = extractFixedPrefixFromLikePattern(key); if (target.empty()) throw Exception( - "SELECT from system.merge_tree_meta_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); + "SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); Strings keys; Strings values; diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.h b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h similarity index 55% rename from src/Storages/System/StorageSystemMergeTreeMetaCache.h rename to src/Storages/System/StorageSystemMergeTreeMetadataCache.h index c8e0f475cd8..a61f996f4df 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetaCache.h +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h @@ -12,14 +12,14 @@ namespace DB class Context; -/** Implements `merge_tree_meta_cache` system table, which allows you to view the metacache data in rocksdb for debugging purposes. +/** Implements `merge_tree_metadata_cache` system table, which allows you to view the metadata cache data in rocksdb for debugging purposes. */ -class StorageSystemMergeTreeMetaCache : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemMergeTreeMetadataCache : public shared_ptr_helper, public IStorageSystemOneBlock { - friend struct shared_ptr_helper; + friend struct shared_ptr_helper; public: - std::string getName() const override { return "SystemMergeTreeMetaCache"; } + std::string getName() const override { return "SystemMergeTreeMetadataCache"; } static NamesAndTypesList getNamesAndTypes(); diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 5f5a17069f3..96ee000c0bf 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -75,7 +75,7 @@ #if USE_ROCKSDB #include -#include +#include #endif @@ -130,7 +130,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) #endif #if USE_ROCKSDB attach(context, system_database, "rocksdb"); - attach(context, system_database, "merge_tree_meta_cache"); + attach(context, system_database, "merge_tree_metadata_cache"); #endif } diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql index f0ca3b608d1..3c5d55a0069 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -1,123 +1,125 @@ +--Tags: no-fasttest + -- Create table under database with engine ordinary. set mutations_sync = 1; -DROP DATABASE IF EXISTS test_meta_cache; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache; -CREATE DATABASE test_meta_cache ENGINE = Ordinary; -CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +DROP DATABASE IF EXISTS test_metadata_cache; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; +CREATE DATABASE test_metadata_cache ENGINE = Ordinary; +CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert third batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache drop column v1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop column v1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache; -CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_metadata_cache.check_part_metadata_cache; +CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert third batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql index b57caf55cb8..7c78721f692 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -1,124 +1,126 @@ +-- Tags: no-fasttest + -- Create table under database with engine atomic. set mutations_sync = 1; -DROP DATABASE IF EXISTS test_meta_cache; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache SYNC; -CREATE DATABASE test_meta_cache ENGINE = Atomic; -CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +DROP DATABASE IF EXISTS test_metadata_cache; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; +CREATE DATABASE test_metadata_cache ENGINE = Atomic; +CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache drop column v1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop column v1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + 30; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 30; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + 60; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 60; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache SYNC; -CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; +CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- nsert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 7722bba71dd..6111f20b599 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -1,127 +1,128 @@ +-- Tags: no-fasttest -- Tags: zookeeper -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_meta_cache; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache; -CREATE DATABASE test_meta_cache ENGINE = Ordinary; -CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +DROP DATABASE IF EXISTS test_metadata_cache; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; +CREATE DATABASE test_metadata_cache ENGINE = Ordinary; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); --Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache drop column v1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop column v1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache ; -CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_metadata_cache.check_part_metadata_cache ; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); --Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index 4dcc4478139..ee8ad61c97c 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -1,127 +1,128 @@ +-- Tags: no-fasttest -- Tags: zookeeper -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_meta_cache ; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache ; -CREATE DATABASE test_meta_cache ENGINE = Atomic; -CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +DROP DATABASE IF EXISTS test_metadata_cache ; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache ; +CREATE DATABASE test_metadata_cache ENGINE = Atomic; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); --Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache drop column v1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop column v1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache ; -CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_metadata_cache.check_part_metadata_cache ; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); --Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); From 98f37afc80df33de1d3eb9c1676804c5285ab422 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 19:29:01 +0800 Subject: [PATCH 044/584] fix building --- programs/server/Server.cpp | 2 +- src/Functions/checkPartMetadataCache.cpp | 2 +- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 41 +++++++++---------- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++- src/Storages/MergeTree/MergeTreeData.h | 1 + .../MergeTree/MergeTreeDataPartCompact.cpp | 3 +- .../MergeTree/MergeTreeDataPartWide.cpp | 2 +- src/Storages/MergeTree/MergeTreePartition.cpp | 2 +- src/Storages/MergeTree/PartMetadataCache.cpp | 9 ++-- src/Storages/MergeTree/PartMetadataCache.h | 10 ++--- .../StorageSystemMergeTreeMetadataCache.cpp | 3 +- .../StorageSystemMergeTreeMetadataCache.h | 3 +- 14 files changed, 42 insertions(+), 46 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1e10af9dc07..dee8a3ffd4e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -820,7 +820,7 @@ if (ThreadFuzzer::instance().isEffective()) #if USE_ROCKSDB - /// initialize meta file cache + /// Initialize merge tree metadata cache { size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp index 2883424f996..87d84297226 100644 --- a/src/Functions/checkPartMetadataCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -99,7 +99,7 @@ public: if (!data || !data->getSettings()->use_metadata_cache) throw Exception("The table in function " + getName() + " must be in MergeTree Family", ErrorCodes::ILLEGAL_COLUMN); - /// Fill in result + /// Fill in checking results. auto col_result = result_type->createColumn(); auto & col_arr = assert_cast(*col_result); auto & col_tuple = assert_cast(col_arr.getData()); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4897cdccac9..76348a16710 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -283,7 +283,7 @@ struct ContextSharedPart Context::ConfigReloadCallback config_reload_callback; #if USE_ROCKSDB - /// MergeTree metadata cache stored in rocksdb. + /// Global merge tree metadata cache, stored in rocksdb. MergeTreeMetadataCachePtr merge_tree_metadata_cache; #endif @@ -399,7 +399,7 @@ struct ContextSharedPart zookeeper.reset(); #if USE_ROCKSDB - /// Shutdown meta file cache + /// Shutdown merge tree metadata cache if (merge_tree_metadata_cache) { merge_tree_metadata_cache->shutdown(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ea606fdea4e..fe8df22b8fb 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -708,7 +708,6 @@ public: MergeTreeMetadataCachePtr getMergeTreeMetadataCache() const; #endif - #if USE_NURAFT std::shared_ptr & getKeeperDispatcher() const; #endif diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b29886265f6..35125a7095f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -90,11 +90,11 @@ void IMergeTreeDataPart::MinMaxIndex::load( { std::unique_ptr file; #if USE_ROCKSDB + String _; if (cache) { String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; - String value; - file = cache->readOrSetMeta(disk, file_name, value); + file = cache->readOrSet(disk, file_name, _); } else { @@ -325,7 +325,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) - , use_metadata_cache(storage.getSettings()->use_metadata_cache) + , use_metadata_cache(storage.use_metadata_cache) { if (parent_part) state = State::Committed; @@ -357,7 +357,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) - , use_metadata_cache(storage.getSettings()->use_metadata_cache) + , use_metadata_cache(storage.use_metadata_cache) { if (parent_part) state = State::Committed; @@ -716,7 +716,6 @@ void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, b size_t IMergeTreeDataPart::fileNumberOfColumnsChecksumsIndexes() const { Strings files; - files.reserve(16); appendFilesOfColumnsChecksumsIndexes(files, true); return files.size(); } @@ -773,10 +772,10 @@ void IMergeTreeDataPart::loadIndex() std::unique_ptr index_file; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String value; - index_file = metadata_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); + index_file = metadata_cache->readOrSet(volume->getDisk(), "primary.idx", _); } else { @@ -854,10 +853,10 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() bool exists = false; std::unique_ptr file_buf; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - file_buf = metadata_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); + file_buf = metadata_cache->readOrSet(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, _); exists = file_buf != nullptr; } else @@ -1027,10 +1026,10 @@ void IMergeTreeDataPart::loadChecksums(bool require) std::unique_ptr buf; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String value; - buf = metadata_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); + buf = metadata_cache->readOrSet(volume->getDisk(), "checksums.txt", _); exists = buf != nullptr; } else @@ -1100,10 +1099,10 @@ void IMergeTreeDataPart::loadRowsCount() else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - auto buf = metadata_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); + auto buf = metadata_cache->readOrSet(volume->getDisk(), "count.txt", _); if (!buf) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); @@ -1229,10 +1228,10 @@ void IMergeTreeDataPart::loadTTLInfos() std::unique_ptr in; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - in = metadata_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); + in = metadata_cache->readOrSet(volume->getDisk(), "ttl.txt", _); exists = in != nullptr; } else @@ -1283,10 +1282,10 @@ void IMergeTreeDataPart::loadUUID() std::unique_ptr in; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - in = metadata_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); + in = metadata_cache->readOrSet(volume->getDisk(), UUID_FILE_NAME, _); exists = in != nullptr; } else @@ -1325,10 +1324,10 @@ void IMergeTreeDataPart::loadColumns(bool require) bool exists = false; std::unique_ptr in; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - in = metadata_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); + in = metadata_cache->readOrSet(volume->getDisk(), "columns.txt", _); exists = in != nullptr; } else @@ -1490,10 +1489,10 @@ void IMergeTreeDataPart::modifyAllMetadataCaches(ModifyCacheType type, bool incl switch (type) { case ModifyCacheType::PUT: - metadata_cache->setMetas(volume->getDisk(), files); + metadata_cache->batchSet(volume->getDisk(), files); break; case ModifyCacheType::DROP: - metadata_cache->dropMetas(files); + metadata_cache->batchDelete(files); break; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6add439d1f9..d03b1e287e5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -206,6 +206,7 @@ MergeTreeData::MergeTreeData( , parts_mover(this) , background_operations_assignee(*this, BackgroundJobsAssignee::Type::DataProcessing, getContext()) , background_moves_assignee(*this, BackgroundJobsAssignee::Type::Moving, getContext()) + , use_metadata_cache(getSettings()->use_metadata_cache) { context_->getGlobalContext()->initializeBackgroundExecutorsIfNeeded(); @@ -318,13 +319,13 @@ MergeTreeData::MergeTreeData( "'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason); #if !USE_ROCKSDB - if (settings->use_metadata_cache) + if (use_metadata_cache) { LOG_WARNING( log, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." "set use_metadata_cache to false forcely"); - settings->use_metadata_cache = false; + use_metadata_cache = false; } #endif diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d349646ab88..468a5ccf72b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -972,6 +972,7 @@ protected: /// And for ReplicatedMergeTree we don't have LogEntry type for this operation. BackgroundJobsAssignee background_operations_assignee; BackgroundJobsAssignee background_moves_assignee; + bool use_metadata_cache; /// Strongly connected with two fields above. /// Every task that is finished will ask to assign a new one into an executor. diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index fa9996b3382..d294b69f79f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -192,12 +192,11 @@ MergeTreeDataPartCompact::~MergeTreeDataPartCompact() removeIfNeeded(); } -/// Do not cache mark file, because cache other meta files is enough to speed up loading. +/// We don't cache mark file, because cache other metadata files is enough to speed up loading. void MergeTreeDataPartCompact::appendFilesOfIndexGranularity(Strings& /* files */) const { } -/// find all connected file and do modification Strings MergeTreeDataPartCompact::getIndexGranularityFiles() const { auto marks_file = index_granularity_info.getMarksFilePath("data"); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index f6efdc5f05c..f4aec7deb15 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -268,7 +268,7 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col } } -// Do not cache mark files of part, because cache other meta files is enough to speed up loading. +/// We don't cache mark files of part, because cache other meta files is enough to speed up loading. void MergeTreeDataPartWide::appendFilesOfIndexGranularity(Strings& /* files */) const { } diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 933248c9a1e..6a394fa5baa 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -373,7 +373,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis if (metadata_cache) { String v; - file = metadata_cache->readOrSetMeta(disk, "partition.dat", v); + file = metadata_cache->readOrSet(disk, "partition.dat", v); } else { diff --git a/src/Storages/MergeTree/PartMetadataCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp index a8ab2c0bf2d..dcb6ce54378 100644 --- a/src/Storages/MergeTree/PartMetadataCache.cpp +++ b/src/Storages/MergeTree/PartMetadataCache.cpp @@ -24,7 +24,7 @@ namespace DB { std::unique_ptr -PartMetadataCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value) +PartMetadataCache::readOrSet(const DiskPtr & disk, const String & file_name, String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; auto status = cache->get(file_path, value); @@ -50,7 +50,7 @@ PartMetadataCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, return std::make_unique(value); } -void PartMetadataCache::setMetas(const DiskPtr & disk, const Strings & file_names) +void PartMetadataCache::batchSet(const DiskPtr & disk, const Strings & file_names) { String text; String read_value; @@ -76,7 +76,7 @@ void PartMetadataCache::setMetas(const DiskPtr & disk, const Strings & file_name } } -void PartMetadataCache::dropMetas(const Strings & file_names) +void PartMetadataCache::batchDelete(const Strings & file_names) { for (const auto & file_name : file_names) { @@ -93,7 +93,7 @@ void PartMetadataCache::dropMetas(const Strings & file_names) } } -void PartMetadataCache::setMeta(const String & file_name, const String & value) +void PartMetadataCache::set(const String & file_name, const String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; String read_value; @@ -116,7 +116,6 @@ void PartMetadataCache::getFilesAndCheckSums(Strings & files, std::vectorgetByPrefix(prefix, files, values); size_t size = files.size(); for (size_t i = 0; i < size; ++i) diff --git a/src/Storages/MergeTree/PartMetadataCache.h b/src/Storages/MergeTree/PartMetadataCache.h index 97880f41b6c..91440da014c 100644 --- a/src/Storages/MergeTree/PartMetadataCache.h +++ b/src/Storages/MergeTree/PartMetadataCache.h @@ -30,11 +30,11 @@ public: } std::unique_ptr - readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value); - void setMetas(const DiskPtr & disk, const Strings & file_names); - void dropMetas(const Strings & file_names); - void setMeta(const String & file_name, const String & value); - void getFilesAndCheckSums(Strings & file_names, std::vector & checksums) const; + readOrSet(const DiskPtr & disk, const String & file_name, String & value); + void batchSet(const DiskPtr & disk, const Strings & file_names); + void batchDelete(const Strings & file_names); + void set(const String & file_name, const String & value); + void getFilesAndCheckSums(Strings & files, std::vector & checksums) const; private: std::string getFullRelativePath() const; diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index b62a7985c0c..bf74f15e822 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -82,8 +82,7 @@ static bool extractKeyImpl(const IAST & elem, String & res, bool & precise) } -/** Retrieve from the query a condition of the form `key= 'key'`, from conjunctions in the WHERE clause. - */ +/// Retrieve from the query a condition of the form `key= 'key'`, from conjunctions in the WHERE clause. static String extractKey(const ASTPtr & query, bool& precise) { const auto & select = query->as(); diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.h b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h index a61f996f4df..8169d1a83fb 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.h +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h @@ -12,8 +12,7 @@ namespace DB class Context; -/** Implements `merge_tree_metadata_cache` system table, which allows you to view the metadata cache data in rocksdb for debugging purposes. - */ +/// Implements `merge_tree_metadata_cache` system table, which allows you to view the metadata cache data in rocksdb for testing purposes. class StorageSystemMergeTreeMetadataCache : public shared_ptr_helper, public IStorageSystemOneBlock { friend struct shared_ptr_helper; From 63dc6821d2d16c7100687ab83e47a70faf0cc5ec Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 29 Dec 2021 12:31:54 +0800 Subject: [PATCH 045/584] fix ut and some bug --- src/Interpreters/Context.cpp | 64 ++--------------- src/Interpreters/Context.h | 34 ++------- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 - .../MergeTree/MergeTreeDataPartCompact.cpp | 6 -- .../MergeTree/MergeTreeDataPartCompact.h | 2 - .../MergeTree/MergeTreeDataPartInMemory.cpp | 6 -- .../MergeTree/MergeTreeDataPartInMemory.h | 2 - .../MergeTree/MergeTreeDataPartWide.cpp | 9 --- .../MergeTree/MergeTreeDataPartWide.h | 2 - .../MergeTree/MergeTreeMetadataCache.cpp | 70 +++++++++++++++++++ .../MergeTree/MergeTreeMetadataCache.h | 39 +++++++++++ src/Storages/MergeTree/MergeTreePartition.cpp | 4 +- src/Storages/MergeTree/PartMetadataCache.cpp | 1 + src/Storages/MergeTree/PartMetadataCache.h | 11 ++- .../StorageSystemMergeTreeMetadataCache.cpp | 1 + .../01233_check_part_meta_cache.reference | 56 +++++++-------- .../01233_check_part_meta_cache.sql | 56 +++++++-------- ..._check_part_meta_cache_in_atomic.reference | 56 +++++++-------- .../01233_check_part_meta_cache_in_atomic.sql | 56 +++++++-------- ...check_part_meta_cache_replicated.reference | 56 +++++++-------- ...01233_check_part_meta_cache_replicated.sql | 56 +++++++-------- ..._meta_cache_replicated_in_atomic.reference | 56 +++++++-------- ...k_part_meta_cache_replicated_in_atomic.sql | 62 ++++++++-------- 24 files changed, 360 insertions(+), 350 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeMetadataCache.cpp create mode 100644 src/Storages/MergeTree/MergeTreeMetadataCache.h diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 76348a16710..3986b5bf822 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -84,24 +84,21 @@ #include #include #include +#include #include #include #include #include +#if USE_ROCKSDB +#include +#endif namespace fs = std::filesystem; namespace ProfileEvents { extern const Event ContextLock; - -#if USE_ROCKSDB - extern const Event MergeTreeMetadataCachePut; - extern const Event MergeTreeMetadataCacheGet; - extern const Event MergeTreeMetadataCacheDelete; - extern const Event MergeTreeMetadataCacheSeek; -#endif } namespace CurrentMetrics @@ -449,59 +446,6 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr shar void SharedContextHolder::reset() { shared.reset(); } -#if USE_ROCKSDB -MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) -{ - auto options = rocksdb::WriteOptions(); - auto status = rocksdb->Put(options, key, value); - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCachePut); - return status; -} - -MergeTreeMetadataCache::Status MergeTreeMetadataCache::del(const String & key) -{ - auto options = rocksdb::WriteOptions(); - auto status = rocksdb->Delete(options, key); - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheDelete); - LOG_TRACE(log, "Delete key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); - return status; -} - -MergeTreeMetadataCache::Status MergeTreeMetadataCache::get(const String & key, String & value) -{ - auto status = rocksdb->Get(rocksdb::ReadOptions(), key, &value); - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheGet); - LOG_TRACE(log, "Get key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); - return status; -} - -void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) -{ - auto * it = rocksdb->NewIterator(rocksdb::ReadOptions()); - rocksdb::Slice target(prefix); - for (it->Seek(target); it->Valid(); it->Next()) - { - const auto key = it->key(); - if (!key.starts_with(target)) - break; - - const auto value = it->value(); - keys.emplace_back(key.data(), key.size()); - values.emplace_back(value.data(), value.size()); - } - LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetadataCache items:{}", prefix, keys.size()); - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); -} - -void MergeTreeMetadataCache::shutdown() -{ - if (rocksdb) - { - rocksdb->Close(); - } -} -#endif - ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) { auto res = std::shared_ptr(new Context); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fe8df22b8fb..b8616e8b634 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -16,12 +16,8 @@ #include #include -#include "config_core.h" -#if USE_ROCKSDB -#include -#include -#endif +#include "config_core.h" #include #include @@ -155,6 +151,12 @@ using ReadTaskCallback = std::function; using MergeTreeReadTaskCallback = std::function(PartitionReadRequest)>; + +#if USE_ROCKSDB +class MergeTreeMetadataCache; +using MergeTreeMetadataCachePtr = std::shared_ptr; +#endif + /// An empty interface for an arbitrary object that may be attached by a shared pointer /// to query context, when using ClickHouse as a library. struct IHostContext @@ -182,28 +184,6 @@ private: std::unique_ptr shared; }; -#if USE_ROCKSDB -class MergeTreeMetadataCache -{ -public: - using Status = rocksdb::Status; - - explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } - MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; - MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; - - Status put(const String & key, const String & value); - Status del(const String & key); - Status get(const String & key, String & value); - void getByPrefix(const String & prefix, Strings & keys, Strings & values); - - void shutdown(); -private: - std::unique_ptr rocksdb; - Poco::Logger * log = &Poco::Logger::get("MergeTreeMetadataCache"); -}; -using MergeTreeMetadataCachePtr = std::shared_ptr; -#endif /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 35125a7095f..abcc52dd295 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -820,8 +820,7 @@ void IMergeTreeDataPart::appendFilesofIndex(Strings & files) const if (!metadata_snapshot) return; - size_t key_size = metadata_snapshot->getPrimaryKeyColumns().size(); - if (key_size) + if (metadata_snapshot->hasPrimaryKey()) files.push_back("primary.idx"); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2798f42b6c4..a992d2dadd7 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -386,8 +386,6 @@ public: /// storage and pass it to this method. virtual bool hasColumnFiles(const NameAndTypePair & /* column */) const { return false; } - virtual Strings getIndexGranularityFiles() const = 0; - /// Returns true if this part shall participate in merges according to /// settings of given storage policy. bool shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index d294b69f79f..6a747960a40 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -197,10 +197,4 @@ void MergeTreeDataPartCompact::appendFilesOfIndexGranularity(Strings& /* files * { } -Strings MergeTreeDataPartCompact::getIndexGranularityFiles() const -{ - auto marks_file = index_granularity_info.getMarksFilePath("data"); - return {marks_file}; -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 87066ab2ff0..b96afc4b972 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -74,8 +74,6 @@ private: void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; void appendFilesOfIndexGranularity(Strings& files) const override; - - Strings getIndexGranularityFiles() const override; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index f3c4b613078..e482f867e7b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -171,12 +171,6 @@ void MergeTreeDataPartInMemory::appendFilesOfIndexGranularity(Strings& /* files { } -/// No mark files for part in memory -Strings MergeTreeDataPartInMemory::getIndexGranularityFiles() const -{ - return {}; -} - DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part) { return std::dynamic_pointer_cast(part); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 4f83b54d402..068b15d2bdc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -64,8 +64,6 @@ private: void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; void appendFilesOfIndexGranularity(Strings & files) const override; - - Strings getIndexGranularityFiles() const override; }; using DataPartInMemoryPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index f4aec7deb15..1582a7f3274 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -273,13 +273,4 @@ void MergeTreeDataPartWide::appendFilesOfIndexGranularity(Strings& /* files */) { } -Strings MergeTreeDataPartWide::getIndexGranularityFiles() const -{ - if (columns.empty()) - return {}; - - auto marks_file = getFileNameForColumn(columns.front()); - return {marks_file}; -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index bf73d16d758..2b11fc4eb02 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -68,8 +68,6 @@ private: void appendFilesOfIndexGranularity(Strings & files) const override; - Strings getIndexGranularityFiles() const override; - ColumnSize getColumnSizeImpl(const NameAndTypePair & column, std::unordered_set * processed_substreams) const; void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp new file mode 100644 index 00000000000..d9dacadcead --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -0,0 +1,70 @@ +#include "MergeTreeMetadataCache.h" + +#if USE_ROCKSDB +#include +#include + +namespace ProfileEvents +{ + extern const Event MergeTreeMetadataCachePut; + extern const Event MergeTreeMetadataCacheGet; + extern const Event MergeTreeMetadataCacheDelete; + extern const Event MergeTreeMetadataCacheSeek; +} + +namespace DB +{ +MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) +{ + auto options = rocksdb::WriteOptions(); + auto status = rocksdb->Put(options, key, value); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCachePut); + return status; +} + +MergeTreeMetadataCache::Status MergeTreeMetadataCache::del(const String & key) +{ + auto options = rocksdb::WriteOptions(); + auto status = rocksdb->Delete(options, key); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheDelete); + LOG_TRACE(log, "Delete key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); + return status; +} + +MergeTreeMetadataCache::Status MergeTreeMetadataCache::get(const String & key, String & value) +{ + auto status = rocksdb->Get(rocksdb::ReadOptions(), key, &value); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheGet); + LOG_TRACE(log, "Get key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); + return status; +} + +void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) +{ + auto * it = rocksdb->NewIterator(rocksdb::ReadOptions()); + rocksdb::Slice target(prefix); + for (it->Seek(target); it->Valid(); it->Next()) + { + const auto key = it->key(); + if (!key.starts_with(target)) + break; + + const auto value = it->value(); + keys.emplace_back(key.data(), key.size()); + values.emplace_back(value.data(), value.size()); + } + LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetadataCache items:{}", prefix, keys.size()); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); +} + +void MergeTreeMetadataCache::shutdown() +{ + if (rocksdb) + { + rocksdb->Close(); + } +} + +} + +#endif diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h new file mode 100644 index 00000000000..00c783d881d --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -0,0 +1,39 @@ +#pragma once + + +#include +#include + +#include +#include +#include + +#include "config_core.h" +#if USE_ROCKSDB + +namespace DB +{ +class MergeTreeMetadataCache +{ +public: + using Status = rocksdb::Status; + + explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } + MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; + MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; + + Status put(const String & key, const String & value); + Status del(const String & key); + Status get(const String & key, String & value); + void getByPrefix(const String & prefix, Strings & keys, Strings & values); + + void shutdown(); +private: + std::unique_ptr rocksdb; + Poco::Logger * log = &Poco::Logger::get("MergeTreeMetadataCache"); +}; + +using MergeTreeMetadataCachePtr = std::shared_ptr; +} + +#endif diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 6a394fa5baa..f1a669eeb2f 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -370,10 +370,10 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis std::unique_ptr file; #if USE_ROCKSDB + String _; if (metadata_cache) { - String v; - file = metadata_cache->readOrSet(disk, "partition.dat", v); + file = metadata_cache->readOrSet(disk, "partition.dat", _); } else { diff --git a/src/Storages/MergeTree/PartMetadataCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp index dcb6ce54378..eee04d24405 100644 --- a/src/Storages/MergeTree/PartMetadataCache.cpp +++ b/src/Storages/MergeTree/PartMetadataCache.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace ProfileEvents { diff --git a/src/Storages/MergeTree/PartMetadataCache.h b/src/Storages/MergeTree/PartMetadataCache.h index 91440da014c..427b9524afd 100644 --- a/src/Storages/MergeTree/PartMetadataCache.h +++ b/src/Storages/MergeTree/PartMetadataCache.h @@ -5,7 +5,6 @@ #if USE_ROCKSDB #include #include -#include namespace DB @@ -13,8 +12,12 @@ namespace DB class SeekableReadBuffer; class IMergeTreeDataPart; -class PartMetadataCache; -using PartMetadataCachePtr = std::shared_ptr; + +class MergeTreeMetadataCache; +using MergeTreeMetadataCachePtr = std::shared_ptr; + +class IDisk; +using DiskPtr = std::shared_ptr; class PartMetadataCache { @@ -45,5 +48,7 @@ private: const IMergeTreeDataPart * parent_part; }; +using PartMetadataCachePtr = std::shared_ptr; + } #endif diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index bf74f15e822..275365648f3 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include namespace DB diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.reference b/tests/queries/0_stateless/01233_check_part_meta_cache.reference index 914add905ce..2bd6025fea2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.reference +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.reference @@ -1,28 +1,28 @@ -0 0 -7 0 -14 0 -28 0 -42 0 -56 0 -70 0 -77 0 -63 0 -77 0 -84 0 -98 0 -122 0 -154 0 -122 0 -12 0 -24 0 -48 0 -72 0 -96 0 -120 0 -132 0 -108 0 -132 0 -144 0 -183 0 -235 0 -183 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql index 3c5d55a0069..c15c2883436 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -6,65 +6,65 @@ DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert third batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete column. alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache; @@ -72,54 +72,54 @@ CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert third batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference index 95de1ef56a9..2bd6025fea2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference @@ -1,28 +1,28 @@ -0 0 -7 0 -14 0 -28 0 -42 0 -56 0 -70 0 -77 0 -63 0 -77 0 -84 0 -98 0 -124 0 -150 0 -124 0 -12 0 -24 0 -48 0 -72 0 -96 0 -120 0 -132 0 -108 0 -132 0 -144 0 -183 0 -235 0 -183 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql index 7c78721f692..6bd8425a8ea 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -6,65 +6,65 @@ DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; CREATE DATABASE test_metadata_cache ENGINE = Atomic; CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete column. alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 30; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 60; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; @@ -72,55 +72,55 @@ CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- nsert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference index 2275537d212..2bd6025fea2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference @@ -1,28 +1,28 @@ -0 0 -7 0 -14 0 -28 0 -42 0 -56 0 -70 0 -77 0 -7 0 -14 0 -21 0 -35 0 -51 0 -67 0 -0 0 -12 0 -24 0 -48 0 -72 0 -96 0 -120 0 -132 0 -108 0 -132 0 -144 0 -183 0 -235 0 -183 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 6111f20b599..2c490c80d70 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -8,65 +8,65 @@ DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); --Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete column. alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache ; @@ -74,55 +74,55 @@ CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); --Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference index 2275537d212..2bd6025fea2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference @@ -1,28 +1,28 @@ -0 0 -7 0 -14 0 -28 0 -42 0 -56 0 -70 0 -77 0 -7 0 -14 0 -21 0 -35 0 -51 0 -67 0 -0 0 -12 0 -24 0 -48 0 -72 0 -96 0 -120 0 -132 0 -108 0 -132 0 -144 0 -183 0 -235 0 -183 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index ee8ad61c97c..c7cc2a09899 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -5,124 +5,124 @@ set mutations_sync = 1; set replication_alter_partitions_sync = 2; DROP DATABASE IF EXISTS test_metadata_cache ; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache ; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; CREATE DATABASE test_metadata_cache ENGINE = Atomic; CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); --Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete column. alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache ; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k TTL p + INTERVAL 15 YEAR settings use_metadata_cache = 1; -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); --Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); From e5ff05c50dc0fac888ae961804c0ace95be59c5f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 29 Dec 2021 16:20:29 +0800 Subject: [PATCH 046/584] fix building --- src/Functions/checkPartMetadataCache.cpp | 1 - src/Storages/MergeTree/MergeTreeMetadataCache.h | 10 ++++------ 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp index 87d84297226..a57607d23f6 100644 --- a/src/Functions/checkPartMetadataCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -1,7 +1,6 @@ #include "config_core.h" #if USE_ROCKSDB - #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index 00c783d881d..f8d7c52cb06 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -1,15 +1,13 @@ #pragma once +#include "config_core.h" -#include -#include - +#if USE_ROCKSDB #include #include #include - -#include "config_core.h" -#if USE_ROCKSDB +#include +#include namespace DB { From 05dfc4eb25be9d0b761c798968fffc4c15da14c0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 31 Dec 2021 11:13:38 +0800 Subject: [PATCH 047/584] fix building and stateless test --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 13 ++++++------- src/Storages/MergeTree/IMergeTreeDataPart.h | 12 ++++++------ .../0_stateless/01233_check_part_meta_cache.sql | 2 +- .../01233_check_part_meta_cache_in_atomic.sql | 2 +- .../01233_check_part_meta_cache_replicated.sql | 2 +- ...3_check_part_meta_cache_replicated_in_atomic.sql | 2 +- 6 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index abcc52dd295..83e91e7ad27 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -907,7 +907,7 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() } } -void IMergeTreeDataPart::appendFilesOfDefaultCompressionCodec(Strings & files) const +void IMergeTreeDataPart::appendFilesOfDefaultCompressionCodec(Strings & files) { files.push_back(DEFAULT_COMPRESSION_CODEC_FILE_NAME); } @@ -1075,7 +1075,7 @@ void IMergeTreeDataPart::loadChecksums(bool require) } } -void IMergeTreeDataPart::appendFilesOfChecksums(Strings & files) const +void IMergeTreeDataPart::appendFilesOfChecksums(Strings & files) { files.push_back("checksums.txt"); } @@ -1215,7 +1215,7 @@ void IMergeTreeDataPart::loadRowsCount() } } -void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) const +void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) { files.push_back("count.txt"); } @@ -1269,7 +1269,7 @@ void IMergeTreeDataPart::loadTTLInfos() } -void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) const +void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) { files.push_back("ttl.txt"); } @@ -1307,7 +1307,7 @@ void IMergeTreeDataPart::loadUUID() } } -void IMergeTreeDataPart::appendFilesOfUUID(Strings & files) const +void IMergeTreeDataPart::appendFilesOfUUID(Strings & files) { files.push_back(UUID_FILE_NAME); } @@ -1388,7 +1388,7 @@ void IMergeTreeDataPart::loadColumns(bool require) setColumns(loaded_columns, infos); } -void IMergeTreeDataPart::appendFilesOfColumns(Strings & files) const +void IMergeTreeDataPart::appendFilesOfColumns(Strings & files) { files.push_back("columns.txt"); } @@ -1475,7 +1475,6 @@ void IMergeTreeDataPart::modifyAllMetadataCaches(ModifyCacheType type, bool incl assert(use_metadata_cache); Strings files; - files.reserve(16); appendFilesOfColumnsChecksumsIndexes(files, include_projection); LOG_TRACE( storage.log, diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a992d2dadd7..1457a643b18 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -503,17 +503,17 @@ private: /// Reads part unique identifier (if exists) from uuid.txt void loadUUID(); - void appendFilesOfUUID(Strings & files) const; + static void appendFilesOfUUID(Strings & files); /// Reads columns names and types from columns.txt void loadColumns(bool require); - void appendFilesOfColumns(Strings & files) const; + static void appendFilesOfColumns(Strings & files); /// If checksums.txt exists, reads file's checksums (and sizes) from it void loadChecksums(bool require); - void appendFilesOfChecksums(Strings & files) const; + static void appendFilesOfChecksums(Strings & files); /// Loads marks index granularity into memory virtual void loadIndexGranularity(); @@ -529,12 +529,12 @@ private: /// For the older format version calculates rows count from the size of a column with a fixed size. void loadRowsCount(); - void appendFilesOfRowsCount(Strings & files) const; + static void appendFilesOfRowsCount(Strings & files); /// Loads ttl infos in json format from file ttl.txt. If file doesn't exists assigns ttl infos with all zeros void loadTTLInfos(); - void appendFilesOfTTLInfos(Strings & files) const; + static void appendFilesOfTTLInfos(Strings & files); void loadPartitionAndMinMaxIndex(); @@ -549,7 +549,7 @@ private: /// any specifial compression. void loadDefaultCompressionCodec(); - void appendFilesOfDefaultCompressionCodec(Strings & files) const; + static void appendFilesOfDefaultCompressionCodec(Strings & files); /// Found column without specific compression and return codec /// for this column with default parameters. diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql index c15c2883436..70de4e0de9e 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -2,8 +2,8 @@ -- Create table under database with engine ordinary. set mutations_sync = 1; -DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; +DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql index 6bd8425a8ea..61452368b52 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -2,8 +2,8 @@ -- Create table under database with engine atomic. set mutations_sync = 1; -DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; +DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Atomic; CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 2c490c80d70..5aff175f97e 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -4,8 +4,8 @@ -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; +DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index c7cc2a09899..7a7846bab05 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -4,8 +4,8 @@ -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_metadata_cache ; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; +DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Atomic; CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); From 770ff591552bcc47461da06ef00650eda138351d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 31 Dec 2021 13:32:00 +0800 Subject: [PATCH 048/584] fix building --- src/Storages/MergeTree/MergeTreePartition.cpp | 2 +- src/Storages/MergeTree/MergeTreePartition.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index f1a669eeb2f..8a85e374062 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -465,7 +465,7 @@ KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr & } -void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files) const +void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 644ddd5ba88..b1bf64550c6 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -54,7 +54,7 @@ public: void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); - void appendFiles(const MergeTreeData & storage, Strings & files) const; + static void appendFiles(const MergeTreeData & storage, Strings & files); /// Adjust partition key and execute its expression on block. Return sample block according to used expression. static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context); From e3da4f07a9779308a6d9eef4fabde3f681f3f4fe Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 1 Jan 2022 08:36:54 +0800 Subject: [PATCH 049/584] fix stateless test --- tests/queries/0_stateless/01233_check_part_meta_cache.sql | 2 +- .../0_stateless/01233_check_part_meta_cache_in_atomic.sql | 2 +- .../0_stateless/01233_check_part_meta_cache_replicated.sql | 7 +++---- .../01233_check_part_meta_cache_replicated_in_atomic.sql | 7 +++---- 4 files changed, 8 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql index 70de4e0de9e..6c1a1232cab 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -1,4 +1,4 @@ ---Tags: no-fasttest +-- Tags: no-parallel, no-fasttest -- Create table under database with engine ordinary. set mutations_sync = 1; diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql index 61452368b52..af8d6f888a7 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-parallel, no-fasttest -- Create table under database with engine atomic. set mutations_sync = 1; diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 5aff175f97e..955c9b49957 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -1,5 +1,4 @@ --- Tags: no-fasttest --- Tags: zookeeper +-- Tags: no-fasttest, no-parallel, zookeeper -- Create table under database with engine ordinary. set mutations_sync = 1; @@ -7,7 +6,7 @@ set replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. @@ -70,7 +69,7 @@ with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadat -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache ; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index 7a7846bab05..6943f721f70 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -1,5 +1,4 @@ --- Tags: no-fasttest --- Tags: zookeeper +-- Tags: no-fasttest, zookeeper, no-parallel -- Create table under database with engine ordinary. set mutations_sync = 1; @@ -7,7 +6,7 @@ set replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Atomic; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. @@ -70,7 +69,7 @@ with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadat -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k TTL p + INTERVAL 15 YEAR settings use_metadata_cache = 1; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k TTL p + INTERVAL 15 YEAR settings use_metadata_cache = 1; -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); From eaf5a72fb79cfbdfb0f4fa227777deb9b5781ee1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 1 Jan 2022 09:06:08 +0800 Subject: [PATCH 050/584] fix stateless test --- .../0_stateless/01233_check_part_meta_cache_replicated.sql | 1 + .../01233_check_part_meta_cache_replicated_in_atomic.sql | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 955c9b49957..bdb43c4e905 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -1,4 +1,5 @@ -- Tags: no-fasttest, no-parallel, zookeeper +-- Tag no-parallel: static zk path -- Create table under database with engine ordinary. set mutations_sync = 1; diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index 6943f721f70..4e491c301f2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -1,4 +1,5 @@ -- Tags: no-fasttest, zookeeper, no-parallel +-- Tag no-parallel: static zk path -- Create table under database with engine ordinary. set mutations_sync = 1; From e13e1f5d7e164aec7b275d4bd1b883d0c2d1cbf4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 11:44:10 +0800 Subject: [PATCH 051/584] add unit test && use enum_name --- programs/server/Server.cpp | 12 ++- src/Functions/checkPartMetadataCache.cpp | 4 +- src/Interpreters/Context.cpp | 7 +- .../tests/gtest_merge_tree_metadata_cache.cpp | 80 +++++++++++++++++++ src/Processors/examples/CMakeLists.txt | 4 - .../examples/merge_tree_metadata_cache.cpp | 51 ------------ src/Storages/MergeTree/IMergeTreeDataPart.h | 17 ++-- 7 files changed, 102 insertions(+), 73 deletions(-) create mode 100644 src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp delete mode 100644 src/Processors/examples/merge_tree_metadata_cache.cpp diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f4d388d559d..ed863271e11 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -823,7 +823,17 @@ if (ThreadFuzzer::instance().isEffective()) /// Initialize merge tree metadata cache { size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); - global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + + try + { + global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + } + catch (...) + { + /// Rename rocksdb directory and reinitialize merge tree metadata cache + fs::rename(path / "rocksdb", path / "rocksdb.old"); + global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + } } #endif diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp index a57607d23f6..ddcb44f5372 100644 --- a/src/Functions/checkPartMetadataCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -39,9 +39,9 @@ public: static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } static constexpr DataPartStates part_states - = {DataPartState::Committed, + = {DataPartState::Active, DataPartState::Temporary, - DataPartState::PreCommitted, + DataPartState::PreActive, DataPartState::Outdated, DataPartState::Deleting, DataPartState::DeleteOnDestroy}; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3986b5bf822..0cb81ba4056 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2289,11 +2289,10 @@ void Context::initializeMergeTreeMetadataCache(const String & dir, size_t size) table_options.block_cache = cache; options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); + if (status != rocksdb::Status::OK()) - { - String message = "Fail to open rocksdb path at: " + dir + " status:" + status.ToString(); - throw Exception(message, ErrorCodes::SYSTEM_ERROR); - } + throw Exception(ErrorCodes::SYSTEM_ERROR, "Fail to open rocksdb path at: {} status:{}", dir, status.ToString()); + shared->merge_tree_metadata_cache = std::make_shared(db); } #endif diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp new file mode 100644 index 00000000000..0679cbf529a --- /dev/null +++ b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp @@ -0,0 +1,80 @@ +#include + +#if USE_ROCKSDB +#include +#include +#include +#include + +using namespace DB; + +class MergeTreeMetadataCacheTest : public ::testing::Test +{ +public: + void SetUp() override + { + auto shared_context = Context::createShared(); + global_context = Context::createGlobal(shared_context.get()); + global_context->makeGlobalContext(); + global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); + cache = global_context->getMergeTreeMetadataCache(); + } + + void TearDown() override + { + global_context->shutdown(); + } + + ContextMutablePtr global_context; + MergeTreeMetadataCachePtr cache; +}; + +TEST_F(MergeTreeMetadataCacheTest, testCommon) +{ + std::vector files + = {"columns.txt", "checksums.txt", "primary.idx", "count.txt", "partition.dat", "minmax_p.idx", "default_compression_codec.txt"}; + String prefix = "data/test_metadata_cache/check_part_metadata_cache/201806_1_1_0_4/"; + + for (const auto & file : files) + { + auto status = cache->put(prefix + file, prefix + file); + ASSERT_EQ(status.code(), rocksdb::Status::Code::kOk); + } + + for (const auto & file : files) + { + String value; + auto status = cache->get(prefix + file, value); + ASSERT_EQ(status.code(), rocksdb::Status::Code::kOk); + ASSERT_EQ(value, prefix + file); + } + + Strings keys; + Strings values; + cache->getByPrefix(prefix, keys, values); + ASSERT_EQ(keys.size(), files.size()); + ASSERT_EQ(values.size(), files.size()); + for (size_t i=0; i < files.size(); ++i) + { + ASSERT_EQ(values[i], prefix + keys[i]); + } + + for (const auto & file : files) + { + auto status = cache->del(prefix + file); + ASSERT_EQ(status.code(), rocksdb::Status::Code::kOk); + } + + for (const auto & file : files) + { + String value; + auto status = cache->get(prefix + file, value); + ASSERT_EQ(status.code(), rocksdb::Status::Code::kNotFound); + } + + cache->getByPrefix(prefix, keys, values); + ASSERT_EQ(keys.size(), 0); + ASSERT_EQ(values.size(), 0); +} + +#endif diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index 2b6b9128e4c..e69de29bb2d 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,4 +0,0 @@ -if (USE_ROCKSDB) - add_executable (merge_tree_metadata_cache merge_tree_metadata_cache.cpp) - target_link_libraries (merge_tree_metadata_cache PRIVATE dbms) -endif() \ No newline at end of file diff --git a/src/Processors/examples/merge_tree_metadata_cache.cpp b/src/Processors/examples/merge_tree_metadata_cache.cpp deleted file mode 100644 index c726eb7ce5a..00000000000 --- a/src/Processors/examples/merge_tree_metadata_cache.cpp +++ /dev/null @@ -1,51 +0,0 @@ -#include -#include - -int main() -{ - using namespace DB; - auto shared_context = Context::createShared(); - auto global_context = Context::createGlobal(shared_context.get()); - global_context->makeGlobalContext(); - global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); - - auto cache = global_context->getMergeTreeMetadataCache(); - - std::vector files - = {"columns.txt", "checksums.txt", "primary.idx", "count.txt", "partition.dat", "minmax_p.idx", "default_compression_codec.txt"}; - String prefix = "data/test_metadata_cache/check_part_metadata_cache/201806_1_1_0_4/"; - - for (const auto & file : files) - { - auto status = cache->put(prefix + file, prefix + file); - std::cout << "put " << file << " " << status.ToString() << std::endl; - } - - for (const auto & file : files) - { - String value; - auto status = cache->get(prefix + file, value); - std::cout << "get " << file << " " << status.ToString() << " " << value << std::endl; - } - - - for (const auto & file : files) - { - auto status = cache->del(prefix + file); - std::cout << "del " << file << " " << status.ToString() << std::endl; - } - - for (const auto & file : files) - { - String value; - auto status = cache->get(prefix + file, value); - std::cout << "get " << file << " " << status.ToString() << " " << value << std::endl; - } - - Strings keys; - Strings values; - cache->getByPrefix(prefix, keys, values); - for (size_t i=0; i #include #include #include @@ -47,21 +48,15 @@ class IMergeTreeDataPart : public std::enable_shared_from_this Date: Tue, 4 Jan 2022 11:48:07 +0800 Subject: [PATCH 052/584] remove redudant judge --- src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 5 +---- src/Storages/MergeTree/MergeTreeMetadataCache.h | 6 +++++- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index d9dacadcead..7025a79018b 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -59,10 +59,7 @@ void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, void MergeTreeMetadataCache::shutdown() { - if (rocksdb) - { - rocksdb->Close(); - } + rocksdb->Close(); } } diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index f8d7c52cb06..286c7ebb08e 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -16,7 +16,11 @@ class MergeTreeMetadataCache public: using Status = rocksdb::Status; - explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } + explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} + { + assert(rocksdb); + } + MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; From 92b3f198aa6932b1d7205c727ed7a0828fbe1e59 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 11:52:15 +0800 Subject: [PATCH 053/584] throw exception if clickhouse use merge tree metdata cache but not compiled with rocksdb --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fb04d406c98..d9d9388bdc2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -321,11 +321,7 @@ MergeTreeData::MergeTreeData( #if !USE_ROCKSDB if (use_metadata_cache) { - LOG_WARNING( - log, - "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." - "set use_metadata_cache to false forcely"); - use_metadata_cache = false; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." "set use_metadata_cache to false forcely"); } #endif From 94d1f7ccb1bc76c96d765f112286ecda1fafe0fa Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 12:36:04 +0800 Subject: [PATCH 054/584] enable continue if cache data corrupted --- programs/server/Server.cpp | 23 +++++++++++++++-------- programs/server/config.xml | 7 +++++-- src/Core/SettingsEnums.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++---- 4 files changed, 25 insertions(+), 15 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ed863271e11..8641623ac07 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -812,17 +812,16 @@ if (ThreadFuzzer::instance().isEffective()) /// Directory with metadata of tables, which was marked as dropped by Atomic database fs::create_directories(path / "metadata_dropped/"); - -#if USE_ROCKSDB - fs::create_directories(path / "rocksdb/"); -#endif } #if USE_ROCKSDB /// Initialize merge tree metadata cache + if (config().has("merge_tree_metadata_cache")) { - size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); + fs::create_directories(path / "rocksdb/"); + size_t size = config().getUInt64("merge_tree_metadata_cache.lru_cache_size", 256 << 20); + bool continue_if_corrupted = config().getBool("merge_tree_metadata_cache.continue_if_corrupted", false); try { @@ -830,9 +829,17 @@ if (ThreadFuzzer::instance().isEffective()) } catch (...) { - /// Rename rocksdb directory and reinitialize merge tree metadata cache - fs::rename(path / "rocksdb", path / "rocksdb.old"); - global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + if (continue_if_corrupted) + { + /// Rename rocksdb directory and reinitialize merge tree metadata cache + time_t now = time(nullptr); + fs::rename(path / "rocksdb", path / ("rocksdb.old." + std::to_string(now))); + global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + } + else + { + throw; + } } } #endif diff --git a/programs/server/config.xml b/programs/server/config.xml index 470c4dfa35f..c5f8e7eeb94 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1292,6 +1292,9 @@ --> - - 268435456 + + diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 106589f5d24..5d2640da319 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -171,5 +171,4 @@ DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparingMode) DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule) - } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d9d9388bdc2..4e0a0a162f6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -318,11 +318,12 @@ MergeTreeData::MergeTreeData( LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', " "'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason); -#if !USE_ROCKSDB if (use_metadata_cache) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." "set use_metadata_cache to false forcely"); - } +#if USE_ROCKSDB + if (!getContext()->getMergeTreeMetadataCache()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if not config in config.xml"); +#else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb"); #endif common_assignee_trigger = [this] (bool delay) noexcept From 37c7c282d76ef77f204528bb0779ba17685e6fca Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 13:41:11 +0800 Subject: [PATCH 055/584] fix unit test --- src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp | 3 ++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 4 ++++ 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp index 0679cbf529a..8371d7ff6b9 100644 --- a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp @@ -13,7 +13,7 @@ class MergeTreeMetadataCacheTest : public ::testing::Test public: void SetUp() override { - auto shared_context = Context::createShared(); + shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); @@ -25,6 +25,7 @@ public: global_context->shutdown(); } + SharedContextHolder shared_context; ContextMutablePtr global_context; MergeTreeMetadataCachePtr cache; }; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c0672f9e98f..2e71c22e144 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -700,7 +700,7 @@ void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, b appendFilesOfColumns(files); appendFilesOfChecksums(files); appendFilesOfIndexGranularity(files); - appendFilesofIndex(files); + appendFilesOfIndex(files); appendFilesOfRowsCount(files); appendFilesOfPartitionAndMinMaxIndex(files); appendFilesOfTTLInfos(files); @@ -817,7 +817,7 @@ void IMergeTreeDataPart::loadIndex() } } -void IMergeTreeDataPart::appendFilesofIndex(Strings & files) const +void IMergeTreeDataPart::appendFilesOfIndex(Strings & files) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (parent_part) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 150c258a4b6..0463d44abee 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -518,7 +518,7 @@ private: /// Loads index file. void loadIndex(); - void appendFilesofIndex(Strings & files) const; + void appendFilesOfIndex(Strings & files) const; /// Load rows count for this part from disk (for the newer storage format version). /// For the older format version calculates rows count from the size of a column with a fixed size. diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index 7025a79018b..fa3825fd3be 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -17,6 +17,8 @@ namespace DB MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) { auto options = rocksdb::WriteOptions(); + options.sync = true; + options.disableWAL = false; auto status = rocksdb->Put(options, key, value); ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCachePut); return status; @@ -25,6 +27,8 @@ MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, c MergeTreeMetadataCache::Status MergeTreeMetadataCache::del(const String & key) { auto options = rocksdb::WriteOptions(); + options.sync = true; + options.disableWAL = false; auto status = rocksdb->Delete(options, key); ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheDelete); LOG_TRACE(log, "Delete key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); From abefefc719712141d394b301c02b52774370655b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 18:17:23 +0800 Subject: [PATCH 056/584] support multi disk --- .../tests/gtest_merge_tree_metadata_cache.cpp | 31 ++++++++++--------- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 +-- src/Storages/MergeTree/PartMetadataCache.cpp | 31 ++++++++++++------- src/Storages/MergeTree/PartMetadataCache.h | 16 +++++++--- 4 files changed, 51 insertions(+), 31 deletions(-) diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp index 8371d7ff6b9..1005739098c 100644 --- a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp @@ -20,10 +20,7 @@ public: cache = global_context->getMergeTreeMetadataCache(); } - void TearDown() override - { - global_context->shutdown(); - } + void TearDown() override { global_context->shutdown(); } SharedContextHolder shared_context; ContextMutablePtr global_context; @@ -50,14 +47,16 @@ TEST_F(MergeTreeMetadataCacheTest, testCommon) ASSERT_EQ(value, prefix + file); } - Strings keys; - Strings values; - cache->getByPrefix(prefix, keys, values); - ASSERT_EQ(keys.size(), files.size()); - ASSERT_EQ(values.size(), files.size()); - for (size_t i=0; i < files.size(); ++i) { - ASSERT_EQ(values[i], prefix + keys[i]); + Strings keys; + Strings values; + cache->getByPrefix(prefix, keys, values); + ASSERT_EQ(keys.size(), files.size()); + ASSERT_EQ(values.size(), files.size()); + for (size_t i = 0; i < files.size(); ++i) + { + ASSERT_EQ(values[i], keys[i]); + } } for (const auto & file : files) @@ -73,9 +72,13 @@ TEST_F(MergeTreeMetadataCacheTest, testCommon) ASSERT_EQ(status.code(), rocksdb::Status::Code::kNotFound); } - cache->getByPrefix(prefix, keys, values); - ASSERT_EQ(keys.size(), 0); - ASSERT_EQ(values.size(), 0); + { + Strings keys; + Strings values; + cache->getByPrefix(prefix, keys, values); + ASSERT_EQ(keys.size(), 0); + ASSERT_EQ(values.size(), 0); + } } #endif diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 2e71c22e144..c49232397c7 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -343,7 +343,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( #if USE_ROCKSDB if (use_metadata_cache) metadata_cache = std::make_shared( - storage.getContext()->getMergeTreeMetadataCache(), storage.relative_data_path, relative_path, parent_part); + storage.getContext()->getMergeTreeMetadataCache(), volume->getDisk()->getPath(), storage.relative_data_path, relative_path, parent_part); #endif } @@ -375,7 +375,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( #if USE_ROCKSDB if (use_metadata_cache) metadata_cache = std::make_shared( - storage.getContext()->getMergeTreeMetadataCache(), storage.relative_data_path, relative_path, parent_part); + storage.getContext()->getMergeTreeMetadataCache(), volume->getDisk()->getName(), storage.relative_data_path, relative_path, parent_part); #endif } diff --git a/src/Storages/MergeTree/PartMetadataCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp index eee04d24405..eff6d9cc0a9 100644 --- a/src/Storages/MergeTree/PartMetadataCache.cpp +++ b/src/Storages/MergeTree/PartMetadataCache.cpp @@ -28,7 +28,8 @@ std::unique_ptr PartMetadataCache::readOrSet(const DiskPtr & disk, const String & file_name, String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; - auto status = cache->get(file_path, value); + String key = getKey(file_path); + auto status = cache->get(key, value); if (!status.ok()) { ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); @@ -41,7 +42,7 @@ PartMetadataCache::readOrSet(const DiskPtr & disk, const String & file_name, Str if (in) { readStringUntilEOF(value, *in); - cache->put(file_path, value); + cache->put(key, value); } } else @@ -57,7 +58,8 @@ void PartMetadataCache::batchSet(const DiskPtr & disk, const Strings & file_name String read_value; for (const auto & file_name : file_names) { - const String file_path = fs::path(getFullRelativePath()) / file_name; + String file_path = fs::path(getFullRelativePath()) / file_name; + String key = getKey(file_path); if (!disk->exists(file_path)) continue; @@ -66,10 +68,10 @@ void PartMetadataCache::batchSet(const DiskPtr & disk, const Strings & file_name continue; readStringUntilEOF(text, *in); - auto status = cache->put(file_path, text); + auto status = cache->put(key, text); if (!status.ok()) { - status = cache->get(file_path, read_value); + status = cache->get(key, read_value); if (status.IsNotFound() || read_value == text) continue; throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); @@ -82,11 +84,12 @@ void PartMetadataCache::batchDelete(const Strings & file_names) for (const auto & file_name : file_names) { String file_path = fs::path(getFullRelativePath()) / file_name; - auto status = cache->del(file_path); + String key = getKey(file_path); + auto status = cache->del(key); if (!status.ok()) { String read_value; - status = cache->get(file_path, read_value); + status = cache->get(key, read_value); if (status.IsNotFound()) continue; throw Exception(ErrorCodes::LOGICAL_ERROR, "drop meta failed status:{}, file_path:{}", status.ToString(), file_path); @@ -97,15 +100,16 @@ void PartMetadataCache::batchDelete(const Strings & file_names) void PartMetadataCache::set(const String & file_name, const String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; + String key = getKey(file_path); String read_value; - auto status = cache->get(file_path, read_value); + auto status = cache->get(key, read_value); if (status == rocksdb::Status::OK() && value == read_value) return; - status = cache->put(file_path, value); + status = cache->put(key, value); if (!status.ok()) { - status = cache->get(file_path, read_value); + status = cache->get(key, read_value); if (status.IsNotFound() || read_value == value) return; @@ -115,7 +119,7 @@ void PartMetadataCache::set(const String & file_name, const String & value) void PartMetadataCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const { - String prefix = fs::path(getFullRelativePath()) / ""; + String prefix = getKey(fs::path(getFullRelativePath()) / ""); Strings values; cache->getByPrefix(prefix, files, values); size_t size = files.size(); @@ -132,5 +136,10 @@ String PartMetadataCache::getFullRelativePath() const return fs::path(relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; } +String PartMetadataCache::getKey(const String & file_path) const +{ + return disk_name + ":" + file_path; +} + } #endif diff --git a/src/Storages/MergeTree/PartMetadataCache.h b/src/Storages/MergeTree/PartMetadataCache.h index 427b9524afd..c2904738b9c 100644 --- a/src/Storages/MergeTree/PartMetadataCache.h +++ b/src/Storages/MergeTree/PartMetadataCache.h @@ -24,8 +24,14 @@ class PartMetadataCache public: using uint128 = CityHash_v1_0_2::uint128; - PartMetadataCache(const MergeTreeMetadataCachePtr & cache_, const String & relative_data_path_, const String & relative_path_, const IMergeTreeDataPart * parent_part_) + PartMetadataCache( + const MergeTreeMetadataCachePtr & cache_, + const String & disk_name_, + const String & relative_data_path_, + const String & relative_path_, + const IMergeTreeDataPart * parent_part_) : cache(cache_) + , disk_name(disk_name_) , relative_data_path(relative_data_path_) , relative_path(relative_path_) , parent_part(parent_part_) @@ -40,11 +46,13 @@ public: void getFilesAndCheckSums(Strings & files, std::vector & checksums) const; private: - std::string getFullRelativePath() const; + String getFullRelativePath() const; + String getKey(const String & file_path) const; MergeTreeMetadataCachePtr cache; - const String & relative_data_path; // relative path of table to disk - const String & relative_path; // relative path of part to table + const String & disk_name; + const String & relative_data_path; /// Relative path of table to disk + const String & relative_path; /// Relative path of part to table const IMergeTreeDataPart * parent_part; }; From c0a9c2b9161a994a0fc26e927c8d09ac8c0d596c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 5 Jan 2022 19:51:50 +0800 Subject: [PATCH 057/584] refactor metadatacache to reduce using of USE_ROCKSDB --- src/Functions/checkPartMetadataCache.cpp | 2 +- src/IO/CMakeLists.txt | 4 +- src/IO/ReadBufferFromString.h | 7 + src/IO/examples/read_buffer.cpp | 31 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 349 +++--------------- src/Storages/MergeTree/IMergeTreeDataPart.h | 46 +-- .../MergeTree/IPartMetadataManager.cpp | 11 + src/Storages/MergeTree/IPartMetadataManager.h | 42 +++ src/Storages/MergeTree/MergeTreeData.h | 1 + src/Storages/MergeTree/MergeTreePartition.cpp | 15 +- src/Storages/MergeTree/MergeTreePartition.h | 11 +- src/Storages/MergeTree/PartMetadataCache.cpp | 145 -------- src/Storages/MergeTree/PartMetadataCache.h | 62 ---- .../MergeTree/PartMetadataManagerOrdinary.cpp | 33 ++ .../MergeTree/PartMetadataManagerOrdinary.h | 28 ++ .../PartMetadataManagerWithCache.cpp | 194 ++++++++++ .../MergeTree/PartMetadataManagerWithCache.h | 40 ++ 17 files changed, 451 insertions(+), 570 deletions(-) create mode 100644 src/Storages/MergeTree/IPartMetadataManager.cpp create mode 100644 src/Storages/MergeTree/IPartMetadataManager.h delete mode 100644 src/Storages/MergeTree/PartMetadataCache.cpp delete mode 100644 src/Storages/MergeTree/PartMetadataCache.h create mode 100644 src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp create mode 100644 src/Storages/MergeTree/PartMetadataManagerOrdinary.h create mode 100644 src/Storages/MergeTree/PartMetadataManagerWithCache.cpp create mode 100644 src/Storages/MergeTree/PartMetadataManagerWithCache.h diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp index ddcb44f5372..d65c14095f7 100644 --- a/src/Functions/checkPartMetadataCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -134,7 +134,7 @@ public: cache_checksums.reserve(file_number); disk_checksums.reserve(file_number); - part->checkMetadataCache(keys, cache_checksums, disk_checksums); + // part->checkMetadataCache(keys, cache_checksums, disk_checksums); for (size_t i = 0; i < keys.size(); ++i) { col_key.insert(keys[i]); diff --git a/src/IO/CMakeLists.txt b/src/IO/CMakeLists.txt index f676f415eea..970602896c9 100644 --- a/src/IO/CMakeLists.txt +++ b/src/IO/CMakeLists.txt @@ -1,3 +1,3 @@ -if (ENABLE_EXAMPLES) +#if (ENABLE_EXAMPLES) add_subdirectory (examples) -endif () +#endif () diff --git a/src/IO/ReadBufferFromString.h b/src/IO/ReadBufferFromString.h index 09646e9b41f..7ea6afc3543 100644 --- a/src/IO/ReadBufferFromString.h +++ b/src/IO/ReadBufferFromString.h @@ -15,4 +15,11 @@ public: explicit ReadBufferFromString(std::string_view s) : ReadBufferFromMemory(s.data(), s.size()) {} }; + +class ReadBufferFromOwnString : public String, public ReadBufferFromString +{ +public: + explicit ReadBufferFromOwnString(const String & s_): String(s_), ReadBufferFromString(*this) {} +}; + } diff --git a/src/IO/examples/read_buffer.cpp b/src/IO/examples/read_buffer.cpp index ea3da690ca5..85675c0d613 100644 --- a/src/IO/examples/read_buffer.cpp +++ b/src/IO/examples/read_buffer.cpp @@ -2,18 +2,15 @@ #include -#include -#include +#include #include +#include +#include - -int main(int, char **) +int readAndPrint(DB::ReadBuffer & in) { try { - std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; - DB::ReadBufferFromString in(s); - DB::Int64 a; DB::Float64 b; DB::String c, d; @@ -31,12 +28,32 @@ int main(int, char **) std::cout << a << ' ' << b << ' ' << c << '\t' << '\'' << d << '\'' << std::endl; std::cout << in.count() << std::endl; + return 0; } catch (const DB::Exception & e) { std::cerr << e.what() << ", " << e.displayText() << std::endl; return 1; } +} + +int main(int, char **) +{ + { + std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; + DB::ReadBufferFromString in(s); + if (readAndPrint(in)) + std::cout << "readAndPrint from ReadBufferFromString failed" << std::endl; + } + + + std::shared_ptr in; + { + std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; + in = std::make_shared(s); + } + if (readAndPrint(*in)) + std::cout << "readAndPrint from ReadBufferFromOwnString failed" << std::endl; return 0; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c49232397c7..4c88580e069 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include #include #include @@ -59,19 +61,8 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) -{ - size_t file_size = disk->getFileSize(path); - return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); -} - -#if USE_ROCKSDB void IMergeTreeDataPart::MinMaxIndex::load( - const MergeTreeData & data, const PartMetadataCachePtr & cache, const DiskPtr & disk, const String & part_path) -#else -void IMergeTreeDataPart::MinMaxIndex::load( - const MergeTreeData & data, const DiskPtr & disk, const String & part_path) -#endif + const MergeTreeData & data, const PartMetadataManagerPtr & manager) { auto metadata_snapshot = data.getInMemoryMetadataPtr(); const auto & partition_key = metadata_snapshot->getPartitionKey(); @@ -80,31 +71,11 @@ void IMergeTreeDataPart::MinMaxIndex::load( auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); size_t minmax_idx_size = minmax_column_types.size(); - auto read_min_max_index = [&](size_t i) - { - String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); - auto file = openForReading(disk, file_name); - return file; - }; - hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { - std::unique_ptr file; -#if USE_ROCKSDB - String _; - if (cache) - { - String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; - file = cache->readOrSet(disk, file_name, _); - } - else - { - file = read_min_max_index(i); - } -#else - file = read_min_max_index(i); -#endif + String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + auto file = manager->read(file_name); auto serialization = minmax_column_types[i]->getDefaultSerialization(); Field min_val; @@ -340,11 +311,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( minmax_idx = std::make_shared(); -#if USE_ROCKSDB - if (use_metadata_cache) - metadata_cache = std::make_shared( - storage.getContext()->getMergeTreeMetadataCache(), volume->getDisk()->getPath(), storage.relative_data_path, relative_path, parent_part); -#endif + initializePartMetadataManager(); } IMergeTreeDataPart::IMergeTreeDataPart( @@ -371,12 +338,8 @@ IMergeTreeDataPart::IMergeTreeDataPart( incrementTypeMetric(part_type); minmax_idx = std::make_shared(); - -#if USE_ROCKSDB - if (use_metadata_cache) - metadata_cache = std::make_shared( - storage.getContext()->getMergeTreeMetadataCache(), volume->getDisk()->getName(), storage.relative_data_path, relative_path, parent_part); -#endif + + initializePartMetadataManager(); } IMergeTreeDataPart::~IMergeTreeDataPart() @@ -774,23 +737,9 @@ void IMergeTreeDataPart::loadIndex() loaded_index[i]->reserve(index_granularity.getMarksCount()); } - String index_path = fs::path(getFullRelativePath()) / "primary.idx"; - - std::unique_ptr index_file; -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - index_file = metadata_cache->readOrSet(volume->getDisk(), "primary.idx", _); - } - else - { - index_file = openForReading(volume->getDisk(), index_path); - } -#else - index_file = openForReading(volume->getDisk(), index_path); -#endif - + String index_name = "primary.idx"; + String index_path = fs::path(getFullRelativePath()) / index_name; + auto index_file = metadata_manager->read(index_name); size_t marks_count = index_granularity.getMarksCount(); Serializations key_serializations(key_size); @@ -854,34 +803,14 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() } String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; - - bool exists = false; - std::unique_ptr file_buf; -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - file_buf = metadata_cache->readOrSet(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, _); - exists = file_buf != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - file_buf = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - file_buf = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists(DEFAULT_COMPRESSION_CODEC_FILE_NAME); if (!exists) { default_codec = detectDefaultCompressionCodec(); } else { + auto file_buf = metadata_manager->read(DEFAULT_COMPRESSION_CODEC_FILE_NAME); String codec_line; readEscapedStringUntilEOL(codec_line, *file_buf); @@ -980,11 +909,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() { String path = getFullRelativePath(); if (!parent_part) -#if USE_ROCKSDB - partition.load(storage, metadata_cache, volume->getDisk(), path); -#else - partition.load(storage, volume->getDisk(), path); -#endif + partition.load(storage, metadata_manager); if (!isEmpty()) { @@ -992,11 +917,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() // projection parts don't have minmax_idx, and it's always initialized minmax_idx->initialized = true; else -#if USE_ROCKSDB - minmax_idx->load(storage, metadata_cache, volume->getDisk(), path); -#else - minmax_idx->load(storage, volume->getDisk(), path); -#endif + minmax_idx->load(storage, metadata_manager); } if (parent_part) return; @@ -1027,30 +948,10 @@ void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) c void IMergeTreeDataPart::loadChecksums(bool require) { const String path = fs::path(getFullRelativePath()) / "checksums.txt"; - bool exists = false; - std::unique_ptr buf; - -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - buf = metadata_cache->readOrSet(volume->getDisk(), "checksums.txt", _); - exists = buf != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - buf = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - buf = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists("checksums.txt"); if (exists) { + auto buf = metadata_manager->read("checksums.txt"); if (checksums.read(*buf)) { assertEOF(*buf); @@ -1092,7 +993,8 @@ void IMergeTreeDataPart::loadRowsCount() auto read_rows_count = [&]() { - auto buf = openForReading(volume->getDisk(), path); + // auto buf = openForReading(volume->getDisk(), path); + auto buf = metadata_manager->read("count.txt"); readIntText(rows_count, *buf); assertEOF(*buf); }; @@ -1103,30 +1005,11 @@ void IMergeTreeDataPart::loadRowsCount() } else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - auto buf = metadata_cache->readOrSet(volume->getDisk(), "count.txt", _); - if (!buf) - throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - - readIntText(rows_count, *buf); - assertEOF(*buf); - } - else - { - if (!volume->getDisk()->exists(path)) - throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - - read_rows_count(); - } -#else - if (!volume->getDisk()->exists(path)) + bool exists = metadata_manager->exists("count.txt"); + if (!exists) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); read_rows_count(); -#endif #ifndef NDEBUG /// columns have to be loaded @@ -1228,31 +1111,10 @@ void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) void IMergeTreeDataPart::loadTTLInfos() { - String path = fs::path(getFullRelativePath()) / "ttl.txt"; - bool exists = false; - std::unique_ptr in; - -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - in = metadata_cache->readOrSet(volume->getDisk(), "ttl.txt", _); - exists = in != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists("ttl.txt"); if (exists) { + auto in = metadata_manager->read("ttl.txt"); assertString("ttl format version: ", *in); size_t format_version; readText(format_version, *in); @@ -1282,31 +1144,10 @@ void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) void IMergeTreeDataPart::loadUUID() { - String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; - bool exists = false; - std::unique_ptr in; - -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - in = metadata_cache->readOrSet(volume->getDisk(), UUID_FILE_NAME, _); - exists = in != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists(UUID_FILE_NAME); if (exists) { + auto in = metadata_manager->read(UUID_FILE_NAME); readText(uuid, *in); if (uuid == UUIDHelpers::Nil) throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR); @@ -1326,27 +1167,7 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; - bool exists = false; - std::unique_ptr in; -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - in = metadata_cache->readOrSet(volume->getDisk(), "columns.txt", _); - exists = in != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists("columns.txt"); if (!exists) { /// We can get list of columns only from columns.txt in compact parts. @@ -1370,6 +1191,7 @@ void IMergeTreeDataPart::loadColumns(bool require) } else { + auto in = metadata_manager->read("columns.txt"); loaded_columns.readText(*in); for (const auto & column : loaded_columns) @@ -1449,24 +1271,12 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } } -#if USE_ROCKSDB - if (use_metadata_cache) - { - modifyAllMetadataCaches(ModifyCacheType::DROP, true); - assertMetadataCacheDropped(true); - } -#endif - + metadata_manager->deleteAll(true); + metadata_manager->assertAllDeleted(true); volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); volume->getDisk()->moveDirectory(from, to); relative_path = new_relative_path; - -#if USE_ROCKSDB - if (use_metadata_cache) - { - modifyAllMetadataCaches(ModifyCacheType::PUT, true); - } -#endif + metadata_manager->updateAll(true); SyncGuardPtr sync_guard; if (storage.getSettings()->fsync_part_directory) @@ -1475,73 +1285,6 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ storage.lockSharedData(*this); } -#if USE_ROCKSDB -void IMergeTreeDataPart::modifyAllMetadataCaches(ModifyCacheType type, bool include_projection) const -{ - assert(use_metadata_cache); - - Strings files; - appendFilesOfColumnsChecksumsIndexes(files, include_projection); - LOG_TRACE( - storage.log, - "part name:{} path:{} {} keys:{}", - name, - getFullRelativePath(), - modifyCacheTypeToString(type), - boost::algorithm::join(files, ", ")); - - switch (type) - { - case ModifyCacheType::PUT: - metadata_cache->batchSet(volume->getDisk(), files); - break; - case ModifyCacheType::DROP: - metadata_cache->batchDelete(files); - break; - } -} - -void IMergeTreeDataPart::assertMetadataCacheDropped(bool include_projection) const -{ - assert(use_metadata_cache); - - Strings files; - std::vector _; - metadata_cache->getFilesAndCheckSums(files, _); - if (files.empty()) - return; - - for (const auto & file : files) - { - String file_name = fs::path(file).filename(); - /// file belongs to current part - if (fs::path(getFullRelativePath()) / file_name == file) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Data part {} with type {} with meta file {} still in cache", name, getType().toString(), file); - } - - /// file belongs to projection part of current part - if (!parent_part && include_projection) - { - for (const auto & [projection_name, projection_part] : projection_parts) - { - if (fs::path(projection_part->getFullRelativePath()) / file_name == file) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Data part {} with type {} with meta file {} with projection name still in cache", - name, - getType().toString(), - file, - projection_name); - } - } - } - } -} -#endif - std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const { /// NOTE: It's needed for zero-copy replication @@ -1560,6 +1303,18 @@ std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const return {}; } +void IMergeTreeDataPart::initializePartMetadataManager() +{ +#if USE_ROCKSDB + if (use_metadata_cache) + metadata_manager = std::make_shared(this, storage.getContext()->getMergeTreeMetadataCache()); + else + metadata_manager = std::make_shared(this); +#else + metadata_manager = std::make_shared(this); +#endif +} + void IMergeTreeDataPart::remove() const { std::optional keep_shared_data = keepSharedDataInDecoupledStorage(); @@ -1579,13 +1334,8 @@ void IMergeTreeDataPart::remove() const return; } -#if USE_ROCKSDB - if (use_metadata_cache) - { - modifyAllMetadataCaches(ModifyCacheType::DROP); - assertMetadataCacheDropped(); - } -#endif + metadata_manager->deleteAll(false); + metadata_manager->assertAllDeleted(false); /** Atomic directory removal: * - rename directory to temporary name; @@ -1690,13 +1440,8 @@ void IMergeTreeDataPart::remove() const void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_shared_data) const { -#if USE_ROCKSDB - if (use_metadata_cache) - { - modifyAllMetadataCaches(ModifyCacheType::DROP); - assertMetadataCacheDropped(); - } -#endif + metadata_manager->deleteAll(false); + metadata_manager->assertAllDeleted(false); String to = parent_to + "/" + relative_path; auto disk = volume->getDisk(); @@ -2037,6 +1782,7 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); } +/* #if USE_ROCKSDB IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const { @@ -2107,6 +1853,7 @@ void IMergeTreeDataPart::checkMetadataCache(Strings & files, std::vector #include #include -#include +#include #include @@ -46,22 +46,6 @@ class UncompressedCache; class IMergeTreeDataPart : public std::enable_shared_from_this { public: - -#if USE_ROCKSDB - enum ModifyCacheType : uint8_t - { - PUT = 1, /// Override set - DROP = 2, /// Remove keys - }; - - static constexpr std::string_view modifyCacheTypeToString(ModifyCacheType type) - { - return magic_enum::enum_name(type); - } - - using uint128 = PartMetadataCache::uint128; -#endif - static constexpr auto DATA_FILE_EXTENSION = ".bin"; using Checksums = MergeTreeDataPartChecksums; @@ -78,6 +62,8 @@ public: using Type = MergeTreeDataPartType; + using uint128 = IPartMetadataManager::uint128; + IMergeTreeDataPart( const MergeTreeData & storage_, @@ -156,10 +142,6 @@ public: /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; -#if USE_ROCKSDB - void assertMetadataCacheDropped(bool include_projection = false) const; -#endif - void remove() const; void projectionRemove(const String & parent_to, bool keep_shared_data = false) const; @@ -320,11 +302,7 @@ public: { } -#if USE_ROCKSDB - void load(const MergeTreeData & data, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path); -#else - void load(const MergeTreeData & data, const DiskPtr & disk, const String & part_path); -#endif + void load(const MergeTreeData & data, const PartMetadataManagerPtr & manager); void store(const MergeTreeData & data, const DiskPtr & disk, const String & part_path, Checksums & checksums) const; void store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const; @@ -393,9 +371,7 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; -#if USE_ROCKSDB - virtual void checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; -#endif + // virtual void checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; bool isProjectionPart() const { return parent_part != nullptr; } @@ -471,9 +447,7 @@ protected: /// Disabled when USE_ROCKSDB is OFF, or use_metadata_cache is set true in merge tree settings bool use_metadata_cache = false; -#if USE_ROCKSDB - mutable PartMetadataCachePtr metadata_cache; -#endif + mutable PartMetadataManagerPtr metadata_manager; void removeIfNeeded(); @@ -488,6 +462,8 @@ protected: std::optional keepSharedDataInDecoupledStorage() const; + void initializePartMetadataManager(); + private: /// In compact parts order of columns is necessary NameToNumber column_name_to_position; @@ -550,10 +526,8 @@ private: /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; -#if USE_ROCKSDB - void modifyAllMetadataCaches(ModifyCacheType type, bool include_projection = false) const; - IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; -#endif + // void modifyAllMetadataCaches(ModifyCacheType type, bool include_projection = false) const; + // IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; mutable State state{State::Temporary}; }; diff --git a/src/Storages/MergeTree/IPartMetadataManager.cpp b/src/Storages/MergeTree/IPartMetadataManager.cpp new file mode 100644 index 00000000000..5e24ac2c0e1 --- /dev/null +++ b/src/Storages/MergeTree/IPartMetadataManager.cpp @@ -0,0 +1,11 @@ +#include "IPartMetadataManager.h" + +#include +#include + +namespace DB +{ +IPartMetadataManager::IPartMetadataManager(const IMergeTreeDataPart * part_) : part(part_), disk(part->volume->getDisk()) +{ +} +} diff --git a/src/Storages/MergeTree/IPartMetadataManager.h b/src/Storages/MergeTree/IPartMetadataManager.h new file mode 100644 index 00000000000..6db87e20f16 --- /dev/null +++ b/src/Storages/MergeTree/IPartMetadataManager.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class IMergeTreeDataPart; + +class SeekableReadBuffer; + +class IDisk; +using DiskPtr = std::shared_ptr; + + +class IPartMetadataManager +{ +public: + using uint128 = CityHash_v1_0_2::uint128; + + explicit IPartMetadataManager(const IMergeTreeDataPart * part_); + + virtual ~IPartMetadataManager() = default; + + virtual std::unique_ptr read(const String & file_name) const = 0; + + virtual bool exists(const String & file_name) const = 0; + + virtual void deleteAll(bool include_projection) = 0; + + virtual void assertAllDeleted(bool include_projection) const = 0; + + virtual void updateAll(bool include_projection) = 0; + +protected: + const IMergeTreeDataPart * part; + const DiskPtr disk; +}; + +using PartMetadataManagerPtr = std::shared_ptr; +} diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 4a108390a85..47dcecaa860 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -895,6 +895,7 @@ protected: friend class StorageReplicatedMergeTree; friend class MergeTreeDataWriter; friend class MergeTask; + friend class IPartMetadataManager; bool require_part_metadata; diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 8a85e374062..3d933303142 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,11 +160,13 @@ namespace }; } +/* static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) { size_t file_size = disk->getFileSize(path); return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } +*/ String MergeTreePartition::getID(const MergeTreeData & storage) const { @@ -355,21 +357,18 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe } } -#if USE_ROCKSDB -void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path) -#else -void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path) -#endif +void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataManagerPtr & manager) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) return; const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; - auto partition_file_path = part_path + "partition.dat"; - std::unique_ptr file; +/* #if USE_ROCKSDB + std::unique_ptr file; + auto partition_file_path = part_path + "partition.dat"; String _; if (metadata_cache) { @@ -382,6 +381,8 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis #else file = openForReading(disk, partition_file_path); #endif +*/ + auto file = manager->read("partition.dat"); value.resize(partition_key_sample.columns()); for (size_t i = 0; i < partition_key_sample.columns(); ++i) diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index b1bf64550c6..d408b7df6a1 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -4,12 +4,9 @@ #include #include #include +#include #include -#if USE_ROCKSDB -#include -#endif - namespace DB { @@ -41,11 +38,7 @@ public: void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; -#if USE_ROCKSDB - void load(const MergeTreeData & storage, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path); -#else - void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path); -#endif + void load(const MergeTreeData & storage, const PartMetadataManagerPtr & manager); void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; diff --git a/src/Storages/MergeTree/PartMetadataCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp deleted file mode 100644 index eff6d9cc0a9..00000000000 --- a/src/Storages/MergeTree/PartMetadataCache.cpp +++ /dev/null @@ -1,145 +0,0 @@ -#include "PartMetadataCache.h" - -#if USE_ROCKSDB -#include -#include -#include -#include -#include -#include -#include -#include - -namespace ProfileEvents -{ - extern const Event MergeTreeMetadataCacheHit; - extern const Event MergeTreeMetadataCacheMiss; -} - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -namespace DB -{ - -std::unique_ptr -PartMetadataCache::readOrSet(const DiskPtr & disk, const String & file_name, String & value) -{ - String file_path = fs::path(getFullRelativePath()) / file_name; - String key = getKey(file_path); - auto status = cache->get(key, value); - if (!status.ok()) - { - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); - if (!disk->exists(file_path)) - { - return nullptr; - } - - auto in = disk->readFile(file_path); - if (in) - { - readStringUntilEOF(value, *in); - cache->put(key, value); - } - } - else - { - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit); - } - return std::make_unique(value); -} - -void PartMetadataCache::batchSet(const DiskPtr & disk, const Strings & file_names) -{ - String text; - String read_value; - for (const auto & file_name : file_names) - { - String file_path = fs::path(getFullRelativePath()) / file_name; - String key = getKey(file_path); - if (!disk->exists(file_path)) - continue; - - auto in = disk->readFile(file_path); - if (!in) - continue; - - readStringUntilEOF(text, *in); - auto status = cache->put(key, text); - if (!status.ok()) - { - status = cache->get(key, read_value); - if (status.IsNotFound() || read_value == text) - continue; - throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); - } - } -} - -void PartMetadataCache::batchDelete(const Strings & file_names) -{ - for (const auto & file_name : file_names) - { - String file_path = fs::path(getFullRelativePath()) / file_name; - String key = getKey(file_path); - auto status = cache->del(key); - if (!status.ok()) - { - String read_value; - status = cache->get(key, read_value); - if (status.IsNotFound()) - continue; - throw Exception(ErrorCodes::LOGICAL_ERROR, "drop meta failed status:{}, file_path:{}", status.ToString(), file_path); - } - } -} - -void PartMetadataCache::set(const String & file_name, const String & value) -{ - String file_path = fs::path(getFullRelativePath()) / file_name; - String key = getKey(file_path); - String read_value; - auto status = cache->get(key, read_value); - if (status == rocksdb::Status::OK() && value == read_value) - return; - - status = cache->put(key, value); - if (!status.ok()) - { - status = cache->get(key, read_value); - if (status.IsNotFound() || read_value == value) - return; - - throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); - } -} - -void PartMetadataCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const -{ - String prefix = getKey(fs::path(getFullRelativePath()) / ""); - Strings values; - cache->getByPrefix(prefix, files, values); - size_t size = files.size(); - for (size_t i = 0; i < size; ++i) - { - ReadBufferFromString rbuf(values[i]); - HashingReadBuffer hbuf(rbuf); - checksums.push_back(hbuf.getHash()); - } -} - -String PartMetadataCache::getFullRelativePath() const -{ - return fs::path(relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -} - -String PartMetadataCache::getKey(const String & file_path) const -{ - return disk_name + ":" + file_path; -} - -} -#endif diff --git a/src/Storages/MergeTree/PartMetadataCache.h b/src/Storages/MergeTree/PartMetadataCache.h deleted file mode 100644 index c2904738b9c..00000000000 --- a/src/Storages/MergeTree/PartMetadataCache.h +++ /dev/null @@ -1,62 +0,0 @@ -#pragma once - -#include "config_core.h" - -#if USE_ROCKSDB -#include -#include - - -namespace DB -{ - -class SeekableReadBuffer; -class IMergeTreeDataPart; - -class MergeTreeMetadataCache; -using MergeTreeMetadataCachePtr = std::shared_ptr; - -class IDisk; -using DiskPtr = std::shared_ptr; - -class PartMetadataCache -{ -public: - using uint128 = CityHash_v1_0_2::uint128; - - PartMetadataCache( - const MergeTreeMetadataCachePtr & cache_, - const String & disk_name_, - const String & relative_data_path_, - const String & relative_path_, - const IMergeTreeDataPart * parent_part_) - : cache(cache_) - , disk_name(disk_name_) - , relative_data_path(relative_data_path_) - , relative_path(relative_path_) - , parent_part(parent_part_) - { - } - - std::unique_ptr - readOrSet(const DiskPtr & disk, const String & file_name, String & value); - void batchSet(const DiskPtr & disk, const Strings & file_names); - void batchDelete(const Strings & file_names); - void set(const String & file_name, const String & value); - void getFilesAndCheckSums(Strings & files, std::vector & checksums) const; - -private: - String getFullRelativePath() const; - String getKey(const String & file_path) const; - - MergeTreeMetadataCachePtr cache; - const String & disk_name; - const String & relative_data_path; /// Relative path of table to disk - const String & relative_path; /// Relative path of part to table - const IMergeTreeDataPart * parent_part; -}; - -using PartMetadataCachePtr = std::shared_ptr; - -} -#endif diff --git a/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp b/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp new file mode 100644 index 00000000000..f12af590e7d --- /dev/null +++ b/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp @@ -0,0 +1,33 @@ +#include "PartMetadataManagerOrdinary.h" + +#include +#include +#include + +namespace DB +{ + +static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) +{ + size_t file_size = disk->getFileSize(path); + return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); +} + +PartMetadataManagerOrdinary::PartMetadataManagerOrdinary(const IMergeTreeDataPart * part_) : IPartMetadataManager(part_) +{ +} + + +std::unique_ptr PartMetadataManagerOrdinary::read(const String & file_name) const +{ + String file_path = fs::path(part->getFullRelativePath() + "/" + file_name); + return openForReading(disk, file_path); +} + +bool PartMetadataManagerOrdinary::exists(const String & file_name) const +{ + return disk->exists(fs::path(part->getFullRelativePath()) / file_name); +} + + +} diff --git a/src/Storages/MergeTree/PartMetadataManagerOrdinary.h b/src/Storages/MergeTree/PartMetadataManagerOrdinary.h new file mode 100644 index 00000000000..5f236f6271d --- /dev/null +++ b/src/Storages/MergeTree/PartMetadataManagerOrdinary.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + +class PartMetadataManagerOrdinary : public IPartMetadataManager +{ +public: + explicit PartMetadataManagerOrdinary(const IMergeTreeDataPart * part_); + + ~PartMetadataManagerOrdinary() override = default; + + std::unique_ptr read(const String & file_name) const override; + + bool exists(const String & file_name) const override; + + void deleteAll(bool /*include_projection*/) override {} + + void assertAllDeleted(bool /*include_projection*/) const override {} + + void updateAll(bool /*include_projection*/) override {} + +}; + + +} diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp new file mode 100644 index 00000000000..ce10f7b625c --- /dev/null +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -0,0 +1,194 @@ +#include "PartMetadataManagerWithCache.h" + +#if USE_ROCKSDB +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event MergeTreeMetadataCacheHit; + extern const Event MergeTreeMetadataCacheMiss; +} + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB +{ +PartMetadataManagerWithCache::PartMetadataManagerWithCache(const IMergeTreeDataPart * part_, const MergeTreeMetadataCachePtr & cache_) + : IPartMetadataManager(part_), cache(cache_) +{ +} + +String PartMetadataManagerWithCache::getKeyFromFilePath(const String & file_path) const +{ + return disk->getName() + ":" + file_path; +} + +String PartMetadataManagerWithCache::getFilePathFromKey(const String & key) const +{ + return key.substr(disk->getName().size() + 1); +} + +std::unique_ptr PartMetadataManagerWithCache::read(const String & file_name) const +{ + String file_path = fs::path(part->getFullRelativePath()) / file_name; + String key = getKeyFromFilePath(file_path); + String value; + auto status = cache->get(key, value); + if (!status.ok()) + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); + auto in = disk->readFile(file_path); + readStringUntilEOF(value, *in); + cache->put(key, value); + } + else + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit); + } + return std::make_unique(value); +} + +bool PartMetadataManagerWithCache::exists(const String & file_name) const +{ + String file_path = fs::path(part->getFullRelativePath()) / file_name; + String key = getKeyFromFilePath(file_path); + String value; + auto status = cache->get(key, value); + if (status.ok()) + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit); + return true; + } + else + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); + return disk->exists(fs::path(part->getFullRelativePath()) / file_name); + } +} + +void PartMetadataManagerWithCache::deleteAll(bool include_projection) +{ + Strings file_names; + part->appendFilesOfColumnsChecksumsIndexes(file_names, include_projection); + + String value; + for (const auto & file_name : file_names) + { + String file_path = fs::path(part->getFullRelativePath()) / file_name; + String key = getKeyFromFilePath(file_path); + auto status = cache->del(key); + if (!status.ok()) + { + status = cache->get(key, value); + if (status.IsNotFound()) + continue; + + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "deleteAll failed include_projection:{} status:{}, file_path:{}", + include_projection, + status.ToString(), + file_path); + } + } +} + +void PartMetadataManagerWithCache::updateAll(bool include_projection) +{ + Strings file_names; + part->appendFilesOfColumnsChecksumsIndexes(file_names, include_projection); + + String value; + String read_value; + for (const auto & file_name : file_names) + { + String file_path = fs::path(part->getFullRelativePath()) / file_name; + if (!disk->exists(file_path)) + continue; + auto in = disk->readFile(file_path); + readStringUntilEOF(value, *in); + + String key = getKeyFromFilePath(file_path); + auto status = cache->put(key, value); + if (!status.ok()) + { + status = cache->get(key, read_value); + if (status.IsNotFound() || read_value == value) + continue; + + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "updateAll failed include_projection:{} status:{}, file_path:{}", + include_projection, + status.ToString(), + file_path); + } + } +} + +void PartMetadataManagerWithCache::assertAllDeleted(bool include_projection) const +{ + Strings keys; + std::vector _; + getKeysAndCheckSums(keys, _); + if (keys.empty()) + return; + + String file_path; + String file_name; + for (const auto & key : keys) + { + file_path = getFilePathFromKey(key); + file_name = fs::path(file_path).filename(); + + /// Metadata file belongs to current part + if (fs::path(part->getFullRelativePath()) / file_name == file_path) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Data part {} with type {} with meta file {} still in cache", + part->name, + part->getType().toString(), + file_path); + + /// File belongs to projection part of current part + if (!part->isProjectionPart() && include_projection) + { + const auto & projection_parts = part->getProjectionParts(); + for (const auto & [projection_name, projection_part] : projection_parts) + { + if (fs::path(projection_part->getFullRelativePath()) / file_name == file_path) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Data part {} with type {} with meta file {} with projection name still in cache", + part->name, + part->getType().toString(), + file_path, + projection_name); + } + } + } + } +} + +void PartMetadataManagerWithCache::getKeysAndCheckSums(Strings & keys, std::vector & checksums) const +{ + String prefix = getKeyFromFilePath(fs::path(part->getFullRelativePath()) / ""); + Strings values; + cache->getByPrefix(prefix, keys, values); + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + ReadBufferFromString rbuf(values[i]); + HashingReadBuffer hbuf(rbuf); + checksums.push_back(hbuf.getHash()); + } +} + +} +#endif diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.h b/src/Storages/MergeTree/PartMetadataManagerWithCache.h new file mode 100644 index 00000000000..76570b0684a --- /dev/null +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.h @@ -0,0 +1,40 @@ +#pragma once + +#include "config_core.h" + +#if USE_ROCKSDB +#include +#include + +namespace DB +{ + +class PartMetadataManagerWithCache : public IPartMetadataManager +{ +public: + PartMetadataManagerWithCache(const IMergeTreeDataPart * part_, const MergeTreeMetadataCachePtr & cache_); + + ~PartMetadataManagerWithCache() override = default; + + std::unique_ptr read(const String & file_name) const override; + + bool exists(const String & file_name) const override; + + void deleteAll(bool include_projection) override; + + void assertAllDeleted(bool include_projection) const override; + + void updateAll(bool include_projection) override; + +private: + String getKeyFromFilePath(const String & file_path) const; + String getFilePathFromKey(const String & key) const; + + void getKeysAndCheckSums(Strings & keys, std::vector & checksums) const; + + + MergeTreeMetadataCachePtr cache; +}; + +} +#endif From b91e0a533c0328167cf812c4f853d3a18cdbac84 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 5 Jan 2022 20:05:22 +0800 Subject: [PATCH 058/584] fix style --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 4c88580e069..99072ebcd41 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -338,7 +338,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( incrementTypeMetric(part_type); minmax_idx = std::make_shared(); - + initializePartMetadataManager(); } From 83d064c24adb5e3b509050e610120e7c5a872707 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 6 Jan 2022 11:41:24 +0800 Subject: [PATCH 059/584] fix unit test and build error --- src/IO/CMakeLists.txt | 4 ++-- src/IO/examples/write_buffer.cpp | 1 + .../tests/gtest_merge_tree_metadata_cache.cpp | 16 ++++++---------- 3 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/IO/CMakeLists.txt b/src/IO/CMakeLists.txt index 970602896c9..f676f415eea 100644 --- a/src/IO/CMakeLists.txt +++ b/src/IO/CMakeLists.txt @@ -1,3 +1,3 @@ -#if (ENABLE_EXAMPLES) +if (ENABLE_EXAMPLES) add_subdirectory (examples) -#endif () +endif () diff --git a/src/IO/examples/write_buffer.cpp b/src/IO/examples/write_buffer.cpp index 5587b8aa1a2..bca0be24b1a 100644 --- a/src/IO/examples/write_buffer.cpp +++ b/src/IO/examples/write_buffer.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp index 1005739098c..839c54c63b2 100644 --- a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp @@ -1,9 +1,11 @@ -#include +#include "config_core.h" #if USE_ROCKSDB #include #include #include +#include +#include #include using namespace DB; @@ -13,17 +15,11 @@ class MergeTreeMetadataCacheTest : public ::testing::Test public: void SetUp() override { - shared_context = Context::createShared(); - global_context = Context::createGlobal(shared_context.get()); - global_context->makeGlobalContext(); - global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); - cache = global_context->getMergeTreeMetadataCache(); + const auto & context_holder = getContext(); + context_holder.context->initializeMergeTreeMetadataCache("./db/", 256 << 20); + cache = context_holder.context->getMergeTreeMetadataCache(); } - void TearDown() override { global_context->shutdown(); } - - SharedContextHolder shared_context; - ContextMutablePtr global_context; MergeTreeMetadataCachePtr cache; }; From 3803cc3d5eddf69fafc72f5181f25be65761c4e5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 6 Jan 2022 15:42:12 +0800 Subject: [PATCH 060/584] fix bug --- programs/server/Server.cpp | 4 ++-- programs/server/config.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8641623ac07..846a1960713 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -814,7 +814,6 @@ if (ThreadFuzzer::instance().isEffective()) fs::create_directories(path / "metadata_dropped/"); } - #if USE_ROCKSDB /// Initialize merge tree metadata cache if (config().has("merge_tree_metadata_cache")) @@ -822,9 +821,10 @@ if (ThreadFuzzer::instance().isEffective()) fs::create_directories(path / "rocksdb/"); size_t size = config().getUInt64("merge_tree_metadata_cache.lru_cache_size", 256 << 20); bool continue_if_corrupted = config().getBool("merge_tree_metadata_cache.continue_if_corrupted", false); - try { + LOG_DEBUG( + log, "Initiailizing merge tree metadata cache lru_cache_size:{} continue_if_corrupted:{}", size, continue_if_corrupted); global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); } catch (...) diff --git a/programs/server/config.xml b/programs/server/config.xml index c5f8e7eeb94..4f75e233b64 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1293,8 +1293,8 @@ --> - + From cf413f16a8166af61c236e054be650ce7ffa908a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 7 Jan 2022 18:37:08 +0800 Subject: [PATCH 061/584] remove function checkPartMetadataCache --- src/Functions/checkPartMetadataCache.cpp | 156 ------------------ .../registerFunctionsMiscellaneous.cpp | 8 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 51 +----- src/Storages/MergeTree/IMergeTreeDataPart.h | 12 +- src/Storages/MergeTree/IPartMetadataManager.h | 3 + .../MergeTree/PartMetadataManagerOrdinary.h | 1 + .../PartMetadataManagerWithCache.cpp | 83 +++++++++- .../MergeTree/PartMetadataManagerWithCache.h | 2 + .../ReplicatedMergeTreePartCheckThread.cpp | 1 + src/Storages/MergeTree/checkDataPart.cpp | 1 - src/Storages/StorageMergeTree.cpp | 3 + 11 files changed, 101 insertions(+), 220 deletions(-) delete mode 100644 src/Functions/checkPartMetadataCache.cpp diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp deleted file mode 100644 index d65c14095f7..00000000000 --- a/src/Functions/checkPartMetadataCache.cpp +++ /dev/null @@ -1,156 +0,0 @@ -#include "config_core.h" - -#if USE_ROCKSDB -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; -} - -class FunctionCheckPartMetadataCache : public IFunction, WithContext -{ -public: - using uint128 = IMergeTreeDataPart::uint128; - using DataPartPtr = MergeTreeData::DataPartPtr; - using DataPartState = MergeTreeData::DataPartState; - using DataPartStates = MergeTreeData::DataPartStates; - - - static constexpr auto name = "checkPartMetadataCache"; - static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - - static constexpr DataPartStates part_states - = {DataPartState::Active, - DataPartState::Temporary, - DataPartState::PreActive, - DataPartState::Outdated, - DataPartState::Deleting, - DataPartState::DeleteOnDestroy}; - - explicit FunctionCheckPartMetadataCache(ContextPtr context_) : WithContext(context_) { } - - String getName() const override { return name; } - - bool isDeterministic() const override { return false; } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - bool isDeterministicInScopeOfQuery() const override { return false; } - - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - for (const auto & argument : arguments) - { - if (!isString(argument)) - throw Exception("The argument of function " + getName() + " must have String type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - DataTypePtr key_type = std::make_unique(); - DataTypePtr state_type = std::make_unique(); - DataTypePtr cache_checksum_type = std::make_unique(32); - DataTypePtr disk_checksum_type = std::make_unique(32); - DataTypePtr match_type = std::make_unique(); - DataTypePtr tuple_type - = std::make_unique(DataTypes{key_type, state_type, cache_checksum_type, disk_checksum_type, match_type}); - return std::make_shared(tuple_type); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - /// Get database name - const auto * arg_database = arguments[0].column.get(); - const ColumnString * column_database = checkAndGetColumnConstData(arg_database); - if (!column_database) - throw Exception("The first argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); - String database_name = column_database->getDataAt(0).toString(); - - /// Get table name - const auto * arg_table = arguments[1].column.get(); - const ColumnString * column_table = checkAndGetColumnConstData(arg_table); - if (!column_table) - throw Exception("The second argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); - String table_name = column_table->getDataAt(0).toString(); - - /// Get storage - StorageID storage_id(database_name, table_name); - auto storage = DatabaseCatalog::instance().getTable(storage_id, getContext()); - auto data = std::dynamic_pointer_cast(storage); - if (!data || !data->getSettings()->use_metadata_cache) - throw Exception("The table in function " + getName() + " must be in MergeTree Family", ErrorCodes::ILLEGAL_COLUMN); - - /// Fill in checking results. - auto col_result = result_type->createColumn(); - auto & col_arr = assert_cast(*col_result); - auto & col_tuple = assert_cast(col_arr.getData()); - col_tuple.reserve(data->fileNumberOfDataParts(part_states)); - auto & col_key = assert_cast(col_tuple.getColumn(0)); - auto & col_state = assert_cast(col_tuple.getColumn(1)); - auto & col_cache_checksum = assert_cast(col_tuple.getColumn(2)); - auto & col_disk_checksum = assert_cast(col_tuple.getColumn(3)); - auto & col_match = assert_cast(col_tuple.getColumn(4)); - auto parts = data->getDataParts(part_states); - for (const auto & part : parts) - executePart(part, col_key, col_state, col_cache_checksum, col_disk_checksum, col_match); - col_arr.getOffsets().push_back(col_tuple.size()); - return result_type->createColumnConst(input_rows_count, col_arr[0]); - } - - static void executePart( - const DataPartPtr & part, - ColumnString & col_key, - ColumnString & col_state, - ColumnFixedString & col_cache_checksum, - ColumnFixedString & col_disk_checksum, - ColumnUInt8 & col_match) - { - Strings keys; - auto state_view = part->stateString(); - String state(state_view.data(), state_view.size()); - std::vector cache_checksums; - std::vector disk_checksums; - uint8_t match = 0; - size_t file_number = part->fileNumberOfColumnsChecksumsIndexes(); - keys.reserve(file_number); - cache_checksums.reserve(file_number); - disk_checksums.reserve(file_number); - - // part->checkMetadataCache(keys, cache_checksums, disk_checksums); - for (size_t i = 0; i < keys.size(); ++i) - { - col_key.insert(keys[i]); - col_state.insert(state); - col_cache_checksum.insert(getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); - col_disk_checksum.insert(getHexUIntUppercase(disk_checksums[i].first) + getHexUIntUppercase(disk_checksums[i].second)); - - match = cache_checksums[i] == disk_checksums[i] ? 1 : 0; - col_match.insertValue(match); - } - } -}; - -void registerFunctionCheckPartMetadataCache(FunctionFactory & factory) -{ - factory.registerFunction(); -} -} -#endif diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 297e6dfb452..76d61ce509a 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -81,10 +81,6 @@ void registerFunctionServerUUID(FunctionFactory &); void registerFunctionZooKeeperSessionUptime(FunctionFactory &); void registerFunctionGetOSKernelVersion(FunctionFactory &); -#if USE_ROCKSDB -void registerFunctionCheckPartMetadataCache(FunctionFactory &); -#endif - #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); #endif @@ -171,10 +167,6 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionZooKeeperSessionUptime(factory); registerFunctionGetOSKernelVersion(factory); -#if USE_ROCKSDB - registerFunctionCheckPartMetadataCache(factory); -#endif - #if USE_ICU registerFunctionConvertCharset(factory); #endif diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 99072ebcd41..909c6e42c9e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -61,8 +61,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -void IMergeTreeDataPart::MinMaxIndex::load( - const MergeTreeData & data, const PartMetadataManagerPtr & manager) +void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const PartMetadataManagerPtr & manager) { auto metadata_snapshot = data.getInMemoryMetadataPtr(); const auto & partition_key = metadata_snapshot->getPartitionKey(); @@ -1782,8 +1781,6 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); } -/* -#if USE_ROCKSDB IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const { assert(use_metadata_cache); @@ -1808,52 +1805,10 @@ IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const St return in_hash.getHash(); } -void IMergeTreeDataPart::checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const +std::unordered_map IMergeTreeDataPart::checkMetadata() const { - assert(use_metadata_cache); - - /// Only applies for normal part - if (isProjectionPart()) - return; - - /// the directory of projection part is under the directory of its parent part - const auto filenames_without_checksums = getFileNamesWithoutChecksums(); - metadata_cache->getFilesAndCheckSums(files, cache_checksums); - for (const auto & file : files) - { - // std::cout << "check key:" << file << std::endl; - String file_name = fs::path(file).filename(); - - /// file belongs to normal part - if (fs::path(getFullRelativePath()) / file_name == file) - { - auto disk_checksum = getActualChecksumByFile(file); - disk_checksums.push_back(disk_checksum); - continue; - } - - /// file belongs to projection part - String proj_dir_name = fs::path(file).parent_path().filename(); - auto pos = proj_dir_name.find_last_of('.'); - if (pos == String::npos) - { - disk_checksums.push_back({}); - continue; - } - String proj_name = proj_dir_name.substr(0, pos); - auto it = projection_parts.find(proj_name); - if (it == projection_parts.end()) - { - disk_checksums.push_back({}); - continue; - } - - auto disk_checksum = it->second->getActualChecksumByFile(file); - disk_checksums.push_back(disk_checksum); - } + return metadata_manager->check(); } -#endif -*/ bool isCompactPart(const MergeTreeDataPartPtr & data_part) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2bb6f570b6a..ecbc2a32aa4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -371,8 +371,6 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; - // virtual void checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; - bool isProjectionPart() const { return parent_part != nullptr; } const IMergeTreeDataPart * getParentPart() const { return parent_part; } @@ -418,6 +416,12 @@ public: /// Required for distinguish different copies of the same part on S3 String getUniqueId() const; + /// Get checksums of metadata file in part directory + IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; + + /// Check metadata in cache is consistent with actual metadata on disk(if use_metadata_cache is true) + std::unordered_map checkMetadata() const; + protected: /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk @@ -464,6 +468,7 @@ protected: void initializePartMetadataManager(); + private: /// In compact parts order of columns is necessary NameToNumber column_name_to_position; @@ -526,9 +531,6 @@ private: /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; - // void modifyAllMetadataCaches(ModifyCacheType type, bool include_projection = false) const; - // IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; - mutable State state{State::Temporary}; }; diff --git a/src/Storages/MergeTree/IPartMetadataManager.h b/src/Storages/MergeTree/IPartMetadataManager.h index 6db87e20f16..17786c90761 100644 --- a/src/Storages/MergeTree/IPartMetadataManager.h +++ b/src/Storages/MergeTree/IPartMetadataManager.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -33,6 +34,8 @@ public: virtual void updateAll(bool include_projection) = 0; + virtual std::unordered_map check() const = 0; + protected: const IMergeTreeDataPart * part; const DiskPtr disk; diff --git a/src/Storages/MergeTree/PartMetadataManagerOrdinary.h b/src/Storages/MergeTree/PartMetadataManagerOrdinary.h index 5f236f6271d..a655431296a 100644 --- a/src/Storages/MergeTree/PartMetadataManagerOrdinary.h +++ b/src/Storages/MergeTree/PartMetadataManagerOrdinary.h @@ -22,6 +22,7 @@ public: void updateAll(bool /*include_projection*/) override {} + std::unordered_map check() const override { return {}; } }; diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index ce10f7b625c..b088f63c0d0 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -1,6 +1,8 @@ #include "PartMetadataManagerWithCache.h" #if USE_ROCKSDB +#include +#include #include #include #include @@ -11,13 +13,16 @@ namespace ProfileEvents extern const Event MergeTreeMetadataCacheMiss; } +namespace DB +{ + namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int CORRUPTED_DATA; + extern const int NO_SUCH_PROJECTION_IN_TABLE; } -namespace DB -{ PartMetadataManagerWithCache::PartMetadataManagerWithCache(const IMergeTreeDataPart * part_, const MergeTreeMetadataCachePtr & cache_) : IPartMetadataManager(part_), cache(cache_) { @@ -190,5 +195,79 @@ void PartMetadataManagerWithCache::getKeysAndCheckSums(Strings & keys, std::vect } } +std::unordered_map PartMetadataManagerWithCache::check() const +{ + /// Only applies for normal part stored on disk + if (part->isProjectionPart() || !part->isStoredOnDisk()) + return {}; + + /// the directory of projection part is under the directory of its parent part + const auto filenames_without_checksums = part->getFileNamesWithoutChecksums(); + + std::unordered_map results; + Strings keys; + std::vector cache_checksums; + std::vector disk_checksums; + getKeysAndCheckSums(keys, cache_checksums); + for (size_t i = 0; i < keys.size(); ++i) + { + const auto & key = keys[i]; + String file_path = getFilePathFromKey(key); + String file_name = fs::path(file_path).filename(); + results.emplace(file_name, cache_checksums[i]); + + /// File belongs to normal part + if (fs::path(part->getFullRelativePath()) / file_name == file_path) + { + auto disk_checksum = part->getActualChecksumByFile(file_path); + if (disk_checksum != cache_checksums[i]) + throw Exception( + ErrorCodes::CORRUPTED_DATA, + "Checksums doesn't match in part {}. Expected: {}. Found {}.", + part->name, + getHexUIntUppercase(disk_checksum.first) + getHexUIntUppercase(disk_checksum.second), + getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); + + disk_checksums.push_back(disk_checksum); + continue; + } + + /// File belongs to projection part + String proj_dir_name = fs::path(file_path).parent_path().filename(); + auto pos = proj_dir_name.find_last_of('.'); + if (pos == String::npos) + { + throw Exception( + ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE, + "There is no projection in part: {} contains file: {} with directory name: {}", + part->name, + file_path, + proj_dir_name); + } + + String proj_name = proj_dir_name.substr(0, pos); + const auto & projection_parts = part->getProjectionParts(); + auto it = projection_parts.find(proj_name); + if (it == projection_parts.end()) + { + throw Exception( + ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE, + "There is no projection {} in part: {} contains file: {}", + proj_name, part->name, file_path); + } + + auto disk_checksum = it->second->getActualChecksumByFile(file_path); + if (disk_checksum != cache_checksums[i]) + throw Exception( + ErrorCodes::CORRUPTED_DATA, + "Checksums doesn't match in projection part {} {}. Expected: {}. Found {}.", + part->name, proj_name, + getHexUIntUppercase(disk_checksum.first) + getHexUIntUppercase(disk_checksum.second), + getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); + disk_checksums.push_back(disk_checksum); + } + return results; +} + } #endif diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.h b/src/Storages/MergeTree/PartMetadataManagerWithCache.h index 76570b0684a..8b1472f5457 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.h +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.h @@ -26,6 +26,8 @@ public: void updateAll(bool include_projection) override; + std::unordered_map check() const override; + private: String getKeyFromFilePath(const String & file_path) const; String getFilePathFromKey(const String & key) const; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 8fcaee66007..85d929b5ea4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -399,6 +399,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na LOG_WARNING(log, "We have part {} covering part {}", part->name, part_name); } + part->checkMetadata(); return {part_name, true, ""}; } diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index eabd901eb24..0f35e30c5d0 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -270,7 +270,6 @@ IMergeTreeDataPart::Checksums checkDataPart( if (require_checksums || !checksums_txt.files.empty()) checksums_txt.checkEqual(checksums_data, check_uncompressed); - return checksums_data; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 11815d9ceef..03a789a7725 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1612,6 +1612,8 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ auto out = disk->writeFile(tmp_checksums_path, 4096); part->checksums.write(*out); disk->moveFile(tmp_checksums_path, checksums_path); + + part->checkMetadata(); results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } catch (const Exception & ex) @@ -1628,6 +1630,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ try { checkDataPart(part, true); + part->checkMetadata(); results.emplace_back(part->name, true, ""); } catch (const Exception & ex) From 3ed13e789612010ad55dcb10586e6d226c33ba27 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 7 Jan 2022 21:06:10 +0800 Subject: [PATCH 062/584] refactor stateless test and add move part/partition integration test --- programs/server/config.xml | 4 +- .../configs/logs_config.xml | 4 + tests/integration/test_multiple_disks/test.py | 17 +- ..._check_table_with_metadata_cache.reference | 672 ++++++++++++++++++ .../01233_check_table_with_metadata_cache.sh | 90 +++ 5 files changed, 780 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference create mode 100644 tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh diff --git a/programs/server/config.xml b/programs/server/config.xml index 4f75e233b64..ec49d516849 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1293,8 +1293,8 @@ --> - + diff --git a/tests/integration/test_multiple_disks/configs/logs_config.xml b/tests/integration/test_multiple_disks/configs/logs_config.xml index 2ee8bb55f38..b0643c8bdad 100644 --- a/tests/integration/test_multiple_disks/configs/logs_config.xml +++ b/tests/integration/test_multiple_disks/configs/logs_config.xml @@ -14,4 +14,8 @@ part_log
500 + + 268435456 + true + diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index db541edde9c..cfce95fab69 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -632,12 +632,13 @@ def get_paths_for_partition_from_part_log(node, table, partition_id): return paths.strip().split('\n') -@pytest.mark.parametrize("name,engine", [ - pytest.param("altering_mt", "MergeTree()", id="mt"), +@pytest.mark.parametrize("name,engine,use_metadata_cache", [ + pytest.param("altering_mt", "MergeTree()", "false", id="mt"), + pytest.param("altering_mt", "MergeTree()", "true", id="mt_use_metadata_cache"), # ("altering_replicated_mt","ReplicatedMergeTree('/clickhouse/altering_replicated_mt', '1')",), # SYSTEM STOP MERGES doesn't disable merges assignments ]) -def test_alter_move(start_cluster, name, engine): +def test_alter_move(start_cluster, name, engine, use_metadata_cache): try: node1.query(""" CREATE TABLE IF NOT EXISTS {name} ( @@ -646,8 +647,8 @@ def test_alter_move(start_cluster, name, engine): ) ENGINE = {engine} ORDER BY tuple() PARTITION BY toYYYYMM(EventDate) - SETTINGS storage_policy='jbods_with_external' - """.format(name=name, engine=engine)) + SETTINGS storage_policy='jbods_with_external', use_metadata_cache={use_metadata_cache} + """.format(name=name, engine=engine, use_metadata_cache=use_metadata_cache)) node1.query("SYSTEM STOP MERGES {}".format(name)) # to avoid conflicts @@ -655,6 +656,8 @@ def test_alter_move(start_cluster, name, engine): node1.query("INSERT INTO {} VALUES(toDate('2019-03-16'), 66)".format(name)) node1.query("INSERT INTO {} VALUES(toDate('2019-04-10'), 42)".format(name)) node1.query("INSERT INTO {} VALUES(toDate('2019-04-11'), 43)".format(name)) + assert node1.query("CHECK TABLE " + name) == "1\n" + used_disks = get_used_disks_for_table(node1, name) assert all(d.startswith("jbod") for d in used_disks), "All writes should go to jbods" @@ -664,6 +667,7 @@ def test_alter_move(start_cluster, name, engine): time.sleep(1) node1.query("ALTER TABLE {} MOVE PART '{}' TO VOLUME 'external'".format(name, first_part)) + assert node1.query("CHECK TABLE " + name) == "1\n" disk = node1.query( "SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(name, first_part)).strip() @@ -672,6 +676,7 @@ def test_alter_move(start_cluster, name, engine): time.sleep(1) node1.query("ALTER TABLE {} MOVE PART '{}' TO DISK 'jbod1'".format(name, first_part)) + assert node1.query("CHECK TABLE " + name) == "1\n" disk = node1.query( "SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(name, first_part)).strip() @@ -680,6 +685,7 @@ def test_alter_move(start_cluster, name, engine): time.sleep(1) node1.query("ALTER TABLE {} MOVE PARTITION 201904 TO VOLUME 'external'".format(name)) + assert node1.query("CHECK TABLE " + name) == "1\n" disks = node1.query( "SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201904' and active = 1".format( name)).strip().split('\n') @@ -690,6 +696,7 @@ def test_alter_move(start_cluster, name, engine): time.sleep(1) node1.query("ALTER TABLE {} MOVE PARTITION 201904 TO DISK 'jbod2'".format(name)) + assert node1.query("CHECK TABLE " + name) == "1\n" disks = node1.query( "SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201904' and active = 1".format( name)).strip().split('\n') diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference new file mode 100644 index 00000000000..5957d23fe82 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference @@ -0,0 +1,672 @@ +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh new file mode 100644 index 00000000000..5a3fd98c3be --- /dev/null +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -0,0 +1,90 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +table_engines="ReplicatedMergeTree" +database_engines="Ordinary Atomic" +use_metadata_caches="false true" +use_projections="false true" + +for table_engine in $table_engines; do + for database_engine in $database_engines; do + for use_metadata_cache in $use_metadata_caches; do + for use_projection in $use_projections; do + echo "database engine:${database_engine}; table engine:${table_engine}; use metadata cache:${use_metadata_cache}; use projection:${use_projection}" + + ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC;" + ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS test_metadata_cache;" + ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE test_metadata_cache ENGINE = ${database_engine};" + + table_engine_clause="" + if [[ "$table_engine" == "ReplicatedMergeTree" ]]; then + table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/test_metadata_cache/check_part_metadata_cache', 'r1')" + elif [[ "$table_engine" == "MergeTree" ]]; then + table_engine_clause="ENGINE MergeTree()" + fi + + projection_clause="" + if [[ "$use_projection" == "true" ]]; then + projection_clause=", projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)" + fi + ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_metadata_cache.check_part_metadata_cache (p Date, k UInt64, v1 UInt64, v2 Int64${projection_clause}) $table_engine_clause PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = ${use_metadata_cache};" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Insert first batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Insert second batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # First update. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Second update. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # First delete. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Second delete. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Insert third batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Drop one partition. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Add column. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Delete column. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Add TTL. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Modify TTL. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Truncate table. + ${CLICKHOUSE_CLIENT} --echo --query "TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + done + done + done +done From 3d0e960bc0d6250c634811172d6cc019d36ae9ab Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 7 Jan 2022 21:07:06 +0800 Subject: [PATCH 063/584] fix stateless test --- .../0_stateless/01233_check_table_with_metadata_cache.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index 5a3fd98c3be..017c7977745 100644 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -table_engines="ReplicatedMergeTree" +table_engines="MergeTree ReplicatedMergeTree" database_engines="Ordinary Atomic" use_metadata_caches="false true" use_projections="false true" From fdea2bc9caa2f2b1757d93768e0afe9ac2f30a15 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 8 Jan 2022 09:55:02 +0800 Subject: [PATCH 064/584] remove old stateless tests --- .../01233_check_part_meta_cache.reference | 28 ---- .../01233_check_part_meta_cache.sql | 125 ----------------- ..._check_part_meta_cache_in_atomic.reference | 28 ---- .../01233_check_part_meta_cache_in_atomic.sql | 126 ----------------- ...check_part_meta_cache_replicated.reference | 28 ---- ...01233_check_part_meta_cache_replicated.sql | 128 ------------------ ..._meta_cache_replicated_in_atomic.reference | 28 ---- ...k_part_meta_cache_replicated_in_atomic.sql | 128 ------------------ 8 files changed, 619 deletions(-) delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache.reference delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache.sql delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.reference b/tests/queries/0_stateless/01233_check_part_meta_cache.reference deleted file mode 100644 index 2bd6025fea2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.reference +++ /dev/null @@ -1,28 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql deleted file mode 100644 index 6c1a1232cab..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ /dev/null @@ -1,125 +0,0 @@ --- Tags: no-parallel, no-fasttest - --- Create table under database with engine ordinary. -set mutations_sync = 1; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; -DROP DATABASE IF EXISTS test_metadata_cache; -CREATE DATABASE test_metadata_cache ENGINE = Ordinary; -CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert third batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete column. -alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache; -CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert third batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference deleted file mode 100644 index 2bd6025fea2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference +++ /dev/null @@ -1,28 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql deleted file mode 100644 index af8d6f888a7..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ /dev/null @@ -1,126 +0,0 @@ --- Tags: no-parallel, no-fasttest - --- Create table under database with engine atomic. -set mutations_sync = 1; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; -DROP DATABASE IF EXISTS test_metadata_cache; -CREATE DATABASE test_metadata_cache ENGINE = Atomic; -CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete column. -alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 30; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 60; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; -CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- nsert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference deleted file mode 100644 index 2bd6025fea2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference +++ /dev/null @@ -1,28 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql deleted file mode 100644 index bdb43c4e905..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ /dev/null @@ -1,128 +0,0 @@ --- Tags: no-fasttest, no-parallel, zookeeper --- Tag no-parallel: static zk path - --- Create table under database with engine ordinary. -set mutations_sync = 1; -set replication_alter_partitions_sync = 2; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; -DROP DATABASE IF EXISTS test_metadata_cache; -CREATE DATABASE test_metadata_cache ENGINE = Ordinary; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - ---Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete column. -alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache ; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - ---Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference deleted file mode 100644 index 2bd6025fea2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference +++ /dev/null @@ -1,28 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql deleted file mode 100644 index 4e491c301f2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ /dev/null @@ -1,128 +0,0 @@ --- Tags: no-fasttest, zookeeper, no-parallel --- Tag no-parallel: static zk path - --- Create table under database with engine ordinary. -set mutations_sync = 1; -set replication_alter_partitions_sync = 2; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; -DROP DATABASE IF EXISTS test_metadata_cache; -CREATE DATABASE test_metadata_cache ENGINE = Atomic; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - ---Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete column. -alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k TTL p + INTERVAL 15 YEAR settings use_metadata_cache = 1; - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - ---Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - From 51e544b447cf23c5dc0a4a01136dda3364c6a364 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 8 Jan 2022 10:11:21 +0800 Subject: [PATCH 065/584] fix stateless test --- .../0_stateless/01233_check_table_with_metadata_cache.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) mode change 100644 => 100755 tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh old mode 100644 new mode 100755 index 017c7977745..e4b8e2f6792 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: setting use_metadata_cache=true is not supported in fasttest, because clickhouse binary in fasttest is build without RocksDB. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -21,7 +23,7 @@ for table_engine in $table_engines; do table_engine_clause="" if [[ "$table_engine" == "ReplicatedMergeTree" ]]; then - table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/test_metadata_cache/check_part_metadata_cache', 'r1')" + table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_metadata_cache/check_part_metadata_cache', 'r1')" elif [[ "$table_engine" == "MergeTree" ]]; then table_engine_clause="ENGINE MergeTree()" fi From 5bff268f84d029261a9e75bccf2111450299723e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 11:45:06 +0800 Subject: [PATCH 066/584] optimize code as advices --- src/Interpreters/Context.cpp | 6 +++++- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - src/Storages/MergeTree/IPartMetadataManager.h | 16 +++++++++++++++- src/Storages/MergeTree/MergeTreePartition.cpp | 8 -------- .../MergeTree/PartMetadataManagerWithCache.h | 14 ++++++++++++++ 5 files changed, 34 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0cb81ba4056..d51da66872f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2291,7 +2291,11 @@ void Context::initializeMergeTreeMetadataCache(const String & dir, size_t size) rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); if (status != rocksdb::Status::OK()) - throw Exception(ErrorCodes::SYSTEM_ERROR, "Fail to open rocksdb path at: {} status:{}", dir, status.ToString()); + throw Exception( + ErrorCodes::SYSTEM_ERROR, + "Fail to open rocksdb path at: {} status:{}. You can try to remove the cache (this will not affect any table data).", + dir, + status.ToString()); shared->merge_tree_metadata_cache = std::make_shared(db); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 7e222607cb6..c8b693727da 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Storages/MergeTree/IPartMetadataManager.h b/src/Storages/MergeTree/IPartMetadataManager.h index 17786c90761..876000de412 100644 --- a/src/Storages/MergeTree/IPartMetadataManager.h +++ b/src/Storages/MergeTree/IPartMetadataManager.h @@ -14,7 +14,12 @@ class SeekableReadBuffer; class IDisk; using DiskPtr = std::shared_ptr; - +/// Interface for managing metadata of merge tree part. +/// IPartMetadataManager has two implementations: +/// - PartMetadataManagerOrdinary: manage metadata from disk directly. deleteAll/assertAllDeleted/updateAll/check +/// are all empty implementations because they are not needed for PartMetadataManagerOrdinary(those operations +/// are done implicitly when removing or renaming part directory). +/// - PartMetadataManagerWithCache: manage metadata from RocksDB cache and disk. class IPartMetadataManager { public: @@ -24,16 +29,25 @@ public: virtual ~IPartMetadataManager() = default; + /// Read metadata content and return SeekableReadBuffer object. virtual std::unique_ptr read(const String & file_name) const = 0; + /// Return true if metadata exists in part. virtual bool exists(const String & file_name) const = 0; + /// Delete all metadatas in part. + /// If include_projection is true, also delete metadatas in projection parts. virtual void deleteAll(bool include_projection) = 0; + /// Assert that all metadatas in part are deleted. + /// If include_projection is true, also assert that all metadatas in projection parts are deleted. virtual void assertAllDeleted(bool include_projection) const = 0; + /// Update all metadatas in part. + /// If include_projection is true, also update metadatas in projection parts. virtual void updateAll(bool include_projection) = 0; + /// Check all metadatas in part. virtual std::unordered_map check() const = 0; protected: diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 3d933303142..c16840ed1bf 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,14 +160,6 @@ namespace }; } -/* -static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) -{ - size_t file_size = disk->getFileSize(path); - return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); -} -*/ - String MergeTreePartition::getID(const MergeTreeData & storage) const { return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block); diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.h b/src/Storages/MergeTree/PartMetadataManagerWithCache.h index 8b1472f5457..06e7a85ba2b 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.h +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.h @@ -9,6 +9,8 @@ namespace DB { +/// PartMetadataManagerWithCache stores metadatas of part in RocksDB as cache layer to speed up +/// loading process of merge tree table. class PartMetadataManagerWithCache : public IPartMetadataManager { public: @@ -16,22 +18,34 @@ public: ~PartMetadataManagerWithCache() override = default; + /// First read the metadata from RocksDB cache, then from disk. std::unique_ptr read(const String & file_name) const override; + /// First judge existence of the metadata in RocksDB cache, then in disk. bool exists(const String & file_name) const override; + /// Delete all metadatas in part from RocksDB cache. void deleteAll(bool include_projection) override; + /// Assert all metadatas in part from RocksDB cache are deleted. void assertAllDeleted(bool include_projection) const override; + /// Update all metadatas in part from RocksDB cache. + /// Need to be called after part directory is renamed. void updateAll(bool include_projection) override; + /// Check if all metadatas in part from RocksDB cache are up to date. std::unordered_map check() const override; private: + /// Get cache key from path of metadata file. + /// Format: :relative/full/path/of/metadata/file String getKeyFromFilePath(const String & file_path) const; + + /// Get metadata file path from cache key. String getFilePathFromKey(const String & key) const; + /// Get cache keys and checksums of corresponding metadata in a part(including projection parts) void getKeysAndCheckSums(Strings & keys, std::vector & checksums) const; From 3d409a4526b3ba7331e61460d320783c8371edb1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 12:20:39 +0800 Subject: [PATCH 067/584] add metrics MergeTreeMetadataCacheSize --- src/Interpreters/AsynchronousMetrics.cpp | 11 +++++++++++ src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 7 +++++++ src/Storages/MergeTree/MergeTreeMetadataCache.h | 1 + 3 files changed, 19 insertions(+) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index d1c5fbebbc7..451014755b8 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -573,6 +574,15 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } } +#if USE_ROCKSDB + { + if (auto metadata_cache = getContext()->getMergeTreeMetadataCache()) + { + new_values["MergeTreeMetadataCacheSize"] = metadata_cache->getEstimateNumKeys(); + } + } +#endif + #if USE_EMBEDDED_COMPILER { if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) @@ -583,6 +593,7 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } #endif + new_values["Uptime"] = getContext()->getUptimeSeconds(); /// Process process memory usage according to OS diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index fa3825fd3be..5df9850b318 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -61,6 +61,13 @@ void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); } +uint64_t MergeTreeMetadataCache::getEstimateNumKeys() const +{ + uint64_t keys = 0; + rocksdb->GetAggregatedIntProperty("rocksdb.estimate-num-keys", &keys); + return keys; +} + void MergeTreeMetadataCache::shutdown() { rocksdb->Close(); diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index 286c7ebb08e..a389d9a14ad 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -28,6 +28,7 @@ public: Status del(const String & key); Status get(const String & key, String & value); void getByPrefix(const String & prefix, Strings & keys, Strings & values); + uint64_t getEstimateNumKeys() const; void shutdown(); private: From 7c7f45bce23a73052fddd1b83efa64e0cf253c3a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 12:32:38 +0800 Subject: [PATCH 068/584] fix comment --- src/Storages/MergeTree/PartMetadataManagerWithCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index b088f63c0d0..3d68497f5b0 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -201,7 +201,7 @@ std::unordered_map PartMetadataManagerWit if (part->isProjectionPart() || !part->isStoredOnDisk()) return {}; - /// the directory of projection part is under the directory of its parent part + /// The directory of projection part is under the directory of its parent part const auto filenames_without_checksums = part->getFileNamesWithoutChecksums(); std::unordered_map results; From 2cae8d552cf2ddc05a8db66359f4e014ddf95f4e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 19:17:12 +0800 Subject: [PATCH 069/584] remove useless code --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 ------- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 9 --------- src/Storages/MergeTree/MergeTreeData.h | 2 -- 4 files changed, 19 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c8995309c4b..a2cceb540af 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -681,13 +681,6 @@ void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, b } } -size_t IMergeTreeDataPart::fileNumberOfColumnsChecksumsIndexes() const -{ - Strings files; - appendFilesOfColumnsChecksumsIndexes(files, true); - return files.size(); -} - void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index c8b693727da..c0c61c6ca0a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -149,7 +149,6 @@ public: /// Load checksums from checksums.txt if exists. Load index if required. void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency); void appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection = false) const; - size_t fileNumberOfColumnsChecksumsIndexes() const; String getMarksFileExtension() const { return index_granularity_info.marks_file_extension; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7ee8e8f196b..a18b7125e54 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1341,15 +1341,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size()); } -size_t MergeTreeData::fileNumberOfDataParts(const DataPartStates & states) const -{ - size_t result = 0; - auto parts = getDataParts(states); - for (const auto & part : parts) - result += part->fileNumberOfColumnsChecksumsIndexes(); - return result; -} - /// Is the part directory old. /// True if its modification time and the modification time of all files inside it is less then threshold. /// (Only files on the first level of nesting are considered). diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 80d0f0a9bff..cc6b1c78b74 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -427,8 +427,6 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); - size_t fileNumberOfDataParts(const DataPartStates & states) const; - String getLogName() const { return log_name; } Int64 getMaxBlockNumber() const; From 7e227040cb0e25f6ac8eaba688e43708080123f0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 19:52:54 +0800 Subject: [PATCH 070/584] optimize getMergeTreeMetadataCache --- src/Interpreters/Context.cpp | 4 ++++ src/Storages/MergeTree/MergeTreeData.cpp | 5 +---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d51da66872f..d8bdb024462 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2029,6 +2029,10 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const #if USE_ROCKSDB MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const { + if (!shared->merge_tree_metadata_cache) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Merge tree metadata cache is not initialized, please add config merge_tree_metadata_cache in config.xml and restart"); return shared->merge_tree_metadata_cache; } #endif diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a18b7125e54..1168335647f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -319,11 +319,8 @@ MergeTreeData::MergeTreeData( LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', " "'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason); +#if !USE_ROCKSDB if (use_metadata_cache) -#if USE_ROCKSDB - if (!getContext()->getMergeTreeMetadataCache()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if not config in config.xml"); -#else throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb"); #endif From 90a92dd14a7bae2b780bfad82dd327bc3f8a5bf7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 11 Jan 2022 11:09:52 +0800 Subject: [PATCH 071/584] fix stateless test --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- src/Interpreters/Context.cpp | 8 +++++++- src/Interpreters/Context.h | 1 + tests/queries/0_stateless/01161_all_system_tables.sh | 2 +- 4 files changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 451014755b8..4b9bcd42e51 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -576,7 +576,7 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti #if USE_ROCKSDB { - if (auto metadata_cache = getContext()->getMergeTreeMetadataCache()) + if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache()) { new_values["MergeTreeMetadataCacheSize"] = metadata_cache->getEstimateNumKeys(); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d8bdb024462..925f3048262 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2029,10 +2029,16 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const #if USE_ROCKSDB MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const { - if (!shared->merge_tree_metadata_cache) + auto cache = tryGetMergeTreeMetadataCache(); + if (!cache) throw Exception( ErrorCodes::LOGICAL_ERROR, "Merge tree metadata cache is not initialized, please add config merge_tree_metadata_cache in config.xml and restart"); + return cache; +} + +MergeTreeMetadataCachePtr Context::tryGetMergeTreeMetadataCache() const +{ return shared->merge_tree_metadata_cache; } #endif diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b8616e8b634..1d618ba9ac6 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -686,6 +686,7 @@ public: #if USE_ROCKSDB MergeTreeMetadataCachePtr getMergeTreeMetadataCache() const; + MergeTreeMetadataCachePtr tryGetMergeTreeMetadataCache() const; #endif #if USE_NURAFT diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index 9b19cc97d16..1a653763ad3 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -12,7 +12,7 @@ function run_selects() { thread_num=$1 readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables - WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' + WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' and name!='merge_tree_metadata_cache' AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num") for t in "${tables_arr[@]}" From d31871d253f70ee79b3cae3e382b1f7ec02aee10 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 11 Jan 2022 15:17:00 +0800 Subject: [PATCH 072/584] commit again --- src/Common/tests/gtest_global_context.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 9bd7c2490d6..1b7eacf6f11 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -18,8 +18,11 @@ struct ContextHolder ContextHolder(ContextHolder &&) = default; }; -inline const ContextHolder & getContext() +const ContextHolder & getContext() { - static ContextHolder holder; - return holder; + static ContextHolder * holder; + static std::once_flag once; + std::call_once(once, [&]() { holder = new ContextHolder(); }); + return *holder; } + From 2b8853db279bc1b55a10ff21b2b07a3a08fc1276 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Tue, 11 Jan 2022 20:30:40 -0600 Subject: [PATCH 073/584] Update tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh Co-authored-by: Azat Khuzhin --- .../0_stateless/01233_check_table_with_metadata_cache.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index aedecb0328f..efa94e9775a 100755 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -1,7 +1,8 @@ #!/usr/bin/env bash # Tags: no-fasttest # Tag no-fasttest: setting use_metadata_cache=true is not supported in fasttest, because clickhouse binary in fasttest is build without RocksDB. - +# To suppress Warning messages from CHECK TABLE +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 6567cd7abc185d0496fd850c5d193efeb90d2109 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 13 Jan 2022 15:27:41 +0800 Subject: [PATCH 074/584] fix all the stateless test --- src/Common/tests/gtest_global_context.cpp | 17 ++++++++--- src/Common/tests/gtest_global_context.h | 11 +++++++ src/Interpreters/Context.cpp | 21 +------------- .../MergeTree/MergeTreeMetadataCache.cpp | 29 +++++++++++++++++++ .../MergeTree/MergeTreeMetadataCache.h | 2 ++ .../tests/gtest_merge_tree_metadata_cache.cpp | 10 +++++-- 6 files changed, 63 insertions(+), 27 deletions(-) rename src/{Interpreters => Storages/MergeTree}/tests/gtest_merge_tree_metadata_cache.cpp (90%) diff --git a/src/Common/tests/gtest_global_context.cpp b/src/Common/tests/gtest_global_context.cpp index 9b51e1bb99c..ec86c953c5b 100644 --- a/src/Common/tests/gtest_global_context.cpp +++ b/src/Common/tests/gtest_global_context.cpp @@ -2,8 +2,17 @@ const ContextHolder & getContext() { - static ContextHolder * holder; - static std::once_flag once; - std::call_once(once, [&]() { holder = new ContextHolder(); }); - return *holder; + return getMutableContext(); +} + +ContextHolder & getMutableContext() +{ + static ContextHolder holder; + return holder; +} + +void destroyContext() +{ + auto & holder = getMutableContext(); + return holder.destroy(); } diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 7756be7ce9b..f846a0dbe4f 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -16,6 +16,17 @@ struct ContextHolder } ContextHolder(ContextHolder &&) = default; + + void destroy() + { + context->shutdown(); + context.reset(); + shared_context.reset(); + } }; const ContextHolder & getContext(); + +ContextHolder & getMutableContext(); + +void destroyContext(); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 925f3048262..111ffc9b3f8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -131,7 +131,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int INVALID_SETTING_VALUE; extern const int UNKNOWN_READ_METHOD; - extern const int SYSTEM_ERROR; } @@ -2289,25 +2288,7 @@ void Context::initializeTraceCollector() #if USE_ROCKSDB void Context::initializeMergeTreeMetadataCache(const String & dir, size_t size) { - rocksdb::Options options; - rocksdb::BlockBasedTableOptions table_options; - rocksdb::DB * db; - - options.create_if_missing = true; - options.statistics = rocksdb::CreateDBStatistics(); - auto cache = rocksdb::NewLRUCache(size); - table_options.block_cache = cache; - options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); - rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); - - if (status != rocksdb::Status::OK()) - throw Exception( - ErrorCodes::SYSTEM_ERROR, - "Fail to open rocksdb path at: {} status:{}. You can try to remove the cache (this will not affect any table data).", - dir, - status.ToString()); - - shared->merge_tree_metadata_cache = std::make_shared(db); + shared->merge_tree_metadata_cache = MergeTreeMetadataCache::create(dir, size); } #endif diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index 5df9850b318..555838204ff 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -14,6 +14,33 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int SYSTEM_ERROR; +} + +std::unique_ptr MergeTreeMetadataCache::create(const String & dir, size_t size) +{ + assert(size != 0); + rocksdb::Options options; + rocksdb::BlockBasedTableOptions table_options; + rocksdb::DB * db; + + options.create_if_missing = true; + options.statistics = rocksdb::CreateDBStatistics(); + auto cache = rocksdb::NewLRUCache(size); + table_options.block_cache = cache; + options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); + rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); + if (status != rocksdb::Status::OK()) + throw Exception( + ErrorCodes::SYSTEM_ERROR, + "Fail to open rocksdb path at: {} status:{}. You can try to remove the cache (this will not affect any table data).", + dir, + status.ToString()); + return std::make_unique(db); +} + MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) { auto options = rocksdb::WriteOptions(); @@ -59,6 +86,7 @@ void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, } LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetadataCache items:{}", prefix, keys.size()); ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); + delete it; } uint64_t MergeTreeMetadataCache::getEstimateNumKeys() const @@ -71,6 +99,7 @@ uint64_t MergeTreeMetadataCache::getEstimateNumKeys() const void MergeTreeMetadataCache::shutdown() { rocksdb->Close(); + rocksdb.reset(); } } diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index a389d9a14ad..8e40eaf7310 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -21,6 +21,8 @@ public: assert(rocksdb); } + static std::unique_ptr create(const String & dir, size_t size); + MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp similarity index 90% rename from src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp rename to src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp index 839c54c63b2..f513d1b2553 100644 --- a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp @@ -15,9 +15,13 @@ class MergeTreeMetadataCacheTest : public ::testing::Test public: void SetUp() override { - const auto & context_holder = getContext(); - context_holder.context->initializeMergeTreeMetadataCache("./db/", 256 << 20); - cache = context_holder.context->getMergeTreeMetadataCache(); + cache = MergeTreeMetadataCache::create("./db/", 268435456); + } + + void TearDown() override + { + cache->shutdown(); + cache.reset(); } MergeTreeMetadataCachePtr cache; From cfb18e7ea0972101c52ee5861de40a065d53aad5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 17 Jan 2022 10:05:36 +0800 Subject: [PATCH 075/584] remove useless header --- src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp b/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp index f513d1b2553..33a82845545 100644 --- a/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include using namespace DB; From e64ba110ef8e0ac6cb719e2ca4a9a253b54d4549 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 17 Jan 2022 19:55:44 +0800 Subject: [PATCH 076/584] fix path error --- src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp b/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp index f12af590e7d..184521cb6cf 100644 --- a/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp @@ -20,7 +20,7 @@ PartMetadataManagerOrdinary::PartMetadataManagerOrdinary(const IMergeTreeDataPar std::unique_ptr PartMetadataManagerOrdinary::read(const String & file_name) const { - String file_path = fs::path(part->getFullRelativePath() + "/" + file_name); + String file_path = fs::path(part->getFullRelativePath()) / file_name; return openForReading(disk, file_path); } From 55382ccb1edb7682e78f3794a8e3ae019ee65d4b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 18 Jan 2022 10:37:37 +0800 Subject: [PATCH 077/584] fix stateless ests --- .../01233_check_table_with_metadata_cache.sh | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index efa94e9775a..07833f4cb39 100755 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -12,6 +12,13 @@ database_engines=(Ordinary Atomic) use_metadata_caches=(false true) use_projections=(false true) +function materialize_projection_if_needed() +{ + if [[ "$use_projection" == "true" ]]; then + ${CLICKHOUSE_CLIENT} --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache MATERIALIZE PROJECTION p1" + fi +} + for table_engine in "${table_engines[@]}"; do for database_engine in "${database_engines[@]}"; do for use_metadata_cache in "${use_metadata_caches[@]}"; do @@ -34,58 +41,72 @@ for table_engine in "${table_engines[@]}"; do projection_clause=", projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)" fi ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_metadata_cache.check_part_metadata_cache (p Date, k UInt64, v1 UInt64, v2 Int64${projection_clause}) $table_engine_clause PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = ${use_metadata_cache};" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Insert first batch of data. ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Insert second batch of data. ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # First update. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Second update. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # First delete. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Second delete. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Insert third batch of data. ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Drop one partition. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Add column. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Delete column. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Add TTL. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Modify TTL. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Truncate table. ${CLICKHOUSE_CLIENT} --echo --query "TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" done done From d73d4e831e20d44b31a24e10fd0df4f5cfcf5240 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 19 Jan 2022 17:08:01 +0800 Subject: [PATCH 078/584] fix bug of check table when create data part with wide format and projection --- src/Storages/MergeTree/checkDataPart.cpp | 2 +- .../0_stateless/01710_projection_part_check.reference | 1 + tests/queries/0_stateless/01710_projection_part_check.sql | 8 ++++++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 0f35e30c5d0..075e9e9fbc8 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -153,7 +153,7 @@ IMergeTreeDataPart::Checksums checkDataPart( [&](const ISerialization::SubstreamPath & substream_path) { String projection_file_name = ISerialization::getFileNameForStream(projection_column, substream_path) + ".bin"; - checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_path + projection_file_name); + projection_checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_path + projection_file_name); }); } } diff --git a/tests/queries/0_stateless/01710_projection_part_check.reference b/tests/queries/0_stateless/01710_projection_part_check.reference index 2f7ad3359c0..813e663bdfc 100644 --- a/tests/queries/0_stateless/01710_projection_part_check.reference +++ b/tests/queries/0_stateless/01710_projection_part_check.reference @@ -1,2 +1,3 @@ all_1_1_0 1 all_2_2_0 1 +201805_1_1_0 1 diff --git a/tests/queries/0_stateless/01710_projection_part_check.sql b/tests/queries/0_stateless/01710_projection_part_check.sql index 39fb6a89fc8..142b1363d3c 100644 --- a/tests/queries/0_stateless/01710_projection_part_check.sql +++ b/tests/queries/0_stateless/01710_projection_part_check.sql @@ -8,3 +8,11 @@ insert into tp select number, number from numbers(5); check table tp settings check_query_single_value_result=0; drop table if exists tp; + +CREATE TABLE tp (`p` Date, `k` UInt64, `v1` UInt64, `v2` Int64, PROJECTION p1 ( SELECT p, sum(k), sum(v1), sum(v2) GROUP BY p) ) ENGINE = MergeTree PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tp (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); + +CHECK TABLE tp settings check_query_single_value_result=0; + +DROP TABLE if exists tp; \ No newline at end of file From 9253ed8836bb274cc82685f0e6ad6160b80e1d96 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 19 Jan 2022 18:28:09 +0800 Subject: [PATCH 079/584] fix bug of checkPartData --- ..._check_table_with_metadata_cache.reference | 704 +++++++++++++++++- .../01233_check_table_with_metadata_cache.sh | 139 ++-- 2 files changed, 751 insertions(+), 92 deletions(-) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference index 5957d23fe82..c3206dcb9cb 100644 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference @@ -1,4 +1,4 @@ -database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -40,7 +40,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -82,7 +82,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -124,7 +124,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -166,7 +166,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -208,7 +208,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -250,7 +250,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -292,7 +292,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -334,7 +334,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -376,7 +376,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -418,7 +418,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -460,7 +460,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -502,7 +502,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -544,7 +544,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -586,7 +586,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -628,7 +628,679 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index 07833f4cb39..468a259ac66 100755 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -11,103 +11,90 @@ table_engines=(MergeTree ReplicatedMergeTree) database_engines=(Ordinary Atomic) use_metadata_caches=(false true) use_projections=(false true) - -function materialize_projection_if_needed() -{ - if [[ "$use_projection" == "true" ]]; then - ${CLICKHOUSE_CLIENT} --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache MATERIALIZE PROJECTION p1" - fi -} +use_compact_data_parts=(false true) for table_engine in "${table_engines[@]}"; do for database_engine in "${database_engines[@]}"; do for use_metadata_cache in "${use_metadata_caches[@]}"; do for use_projection in "${use_projections[@]}"; do - echo "database engine:${database_engine}; table engine:${table_engine}; use metadata cache:${use_metadata_cache}; use projection:${use_projection}" + for use_compact_data_part in "${use_compact_data_parts[@]}"; do + echo "database engine:${database_engine}; table engine:${table_engine}; use metadata cache:${use_metadata_cache}; use projection:${use_projection}; use_compact_data_part:${use_compact_data_part}" - ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC;" - ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS test_metadata_cache;" - ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE test_metadata_cache ENGINE = ${database_engine};" + ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC;" + ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS test_metadata_cache;" + ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE test_metadata_cache ENGINE = ${database_engine};" - table_engine_clause="" - if [[ "$table_engine" == "ReplicatedMergeTree" ]]; then - table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_metadata_cache/check_part_metadata_cache', 'r1')" - elif [[ "$table_engine" == "MergeTree" ]]; then - table_engine_clause="ENGINE MergeTree()" - fi + table_engine_clause="" + if [[ "$table_engine" == "ReplicatedMergeTree" ]]; then + table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_metadata_cache/check_part_metadata_cache', 'r1')" + elif [[ "$table_engine" == "MergeTree" ]]; then + table_engine_clause="ENGINE MergeTree()" + fi - projection_clause="" - if [[ "$use_projection" == "true" ]]; then - projection_clause=", projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)" - fi - ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_metadata_cache.check_part_metadata_cache (p Date, k UInt64, v1 UInt64, v2 Int64${projection_clause}) $table_engine_clause PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = ${use_metadata_cache};" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + projection_clause="" + if [[ "$use_projection" == "true" ]]; then + projection_clause=", projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)" + fi - # Insert first batch of data. - ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + compact_data_part_clause=", min_bytes_for_wide_part = 10485760" + if [[ $use_compact_data_part == "true" ]]; then + compact_data_part_clause=", min_bytes_for_wide_part = 0" + fi + ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_metadata_cache.check_part_metadata_cache (p Date, k UInt64, v1 UInt64, v2 Int64${projection_clause}) $table_engine_clause PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = ${use_metadata_cache} ${compact_data_part_clause}" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Insert second batch of data. - ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Insert first batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # First update. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Insert second batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Second update. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # First update. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # First delete. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Second update. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Second delete. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # First delete. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Insert third batch of data. - ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Second delete. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Drop one partition. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Insert third batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Add column. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Drop one partition. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Delete column. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Add column. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Add TTL. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Delete column. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Modify TTL. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Add TTL. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Truncate table. - ${CLICKHOUSE_CLIENT} --echo --query "TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Modify TTL. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Truncate table. + ${CLICKHOUSE_CLIENT} --echo --query "TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + done done done done 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 080/584] 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 dfd199b047ec1e100b1a8ba6654535a1d68c3b2f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 24 Jan 2022 11:51:11 +0800 Subject: [PATCH 081/584] fix integration test failed --- src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 1 - src/Storages/MergeTree/MergeTreeMetadataCache.h | 5 +++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index 555838204ff..40e84deb259 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -27,7 +27,6 @@ std::unique_ptr MergeTreeMetadataCache::create(const Str rocksdb::DB * db; options.create_if_missing = true; - options.statistics = rocksdb::CreateDBStatistics(); auto cache = rocksdb::NewLRUCache(size); table_options.block_cache = cache; options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index 8e40eaf7310..65c5eada200 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -16,14 +16,15 @@ class MergeTreeMetadataCache public: using Status = rocksdb::Status; + static std::unique_ptr create(const String & dir, size_t size); + explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { assert(rocksdb); } - static std::unique_ptr create(const String & dir, size_t size); - MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; + MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; Status put(const String & key, const String & value); From 5398af7c054ef1cfcf170e7c8857a3816040dbf8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jan 2022 18:59:29 +0800 Subject: [PATCH 082/584] update again --- src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index 40e84deb259..0f67346810f 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -19,6 +19,7 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; } + std::unique_ptr MergeTreeMetadataCache::create(const String & dir, size_t size) { assert(size != 0); 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 083/584] 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 e813f6413f12887d0a7a4fec1b5455379073bbe4 Mon Sep 17 00:00:00 2001 From: Pablo Alegre Date: Wed, 29 Dec 2021 16:56:58 +0100 Subject: [PATCH 084/584] Add groupSortedArray() function --- .../reference/groupsortedarray.md | 48 ++++ .../aggregate-functions/reference/index.md | 1 + .../AggregateFunctionGroupSortedArray.cpp | 163 +++++++++++++ .../AggregateFunctionGroupSortedArray.h | 229 ++++++++++++++++++ .../AggregateFunctionGroupSortedArrayData.h | 162 +++++++++++++ .../registerAggregateFunctions.cpp | 2 + tests/performance/group_sorted_array.xml | 25 ++ .../02158_groupsortedarray.reference | 14 ++ .../0_stateless/02158_groupsortedarray.sql | 38 +++ 9 files changed, 682 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/groupsortedarray.md create mode 100644 src/AggregateFunctions/AggregateFunctionGroupSortedArray.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionGroupSortedArray.h create mode 100644 src/AggregateFunctions/AggregateFunctionGroupSortedArrayData.h create mode 100644 tests/performance/group_sorted_array.xml create mode 100644 tests/queries/0_stateless/02158_groupsortedarray.reference create mode 100644 tests/queries/0_stateless/02158_groupsortedarray.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupsortedarray.md b/docs/en/sql-reference/aggregate-functions/reference/groupsortedarray.md new file mode 100644 index 00000000000..06c004173b8 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/groupsortedarray.md @@ -0,0 +1,48 @@ +--- +toc_priority: 108 +--- + +# groupSortedArray {#groupSortedArray} + +Returns an array with the first N items in ascending order. + +``` sql +groupSortedArray(N)(column) +``` + +**Arguments** + +- `N` – The number of elements to return. + +If the parameter is omitted, default value 10 is used. + +**Arguments** + +- `x` – The value. +- `expr` — Optional. The field or expresion to sort by. If not set values are sorted by themselves. [Integer](../../../sql-reference/data-types/int-uint.md). + +**Example** + +Gets the first 10 numbers: + +``` sql +SELECT groupSortedArray(10)(number) FROM numbers(100) +``` + +``` text +┌─groupSortedArray(10)(number)─┐ +│ [0,1,2,3,4,5,6,7,8,9] │ +└──────────────────────────────┘ +``` + +Or the last 10: + +``` sql +SELECT groupSortedArray(10)(number, -number) FROM numbers(100) +``` + +``` text +┌─groupSortedArray(10)(number, negate(number))─┐ +│ [99,98,97,96,95,94,93,92,91,90] │ +└──────────────────────────────────────────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index 59befed8785..14a8ecc9dcf 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -42,6 +42,7 @@ ClickHouse-specific aggregate functions: - [groupBitmapAnd](../../../sql-reference/aggregate-functions/reference/groupbitmapand.md) - [groupBitmapOr](../../../sql-reference/aggregate-functions/reference/groupbitmapor.md) - [groupBitmapXor](../../../sql-reference/aggregate-functions/reference/groupbitmapxor.md) +- [groupSortedArray](../../../sql-reference/aggregate-functions/reference/groupsortedarray.md) - [sumWithOverflow](../../../sql-reference/aggregate-functions/reference/sumwithoverflow.md) - [sumMap](../../../sql-reference/aggregate-functions/reference/summap.md) - [minMap](../../../sql-reference/aggregate-functions/reference/minmap.md) diff --git a/src/AggregateFunctions/AggregateFunctionGroupSortedArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupSortedArray.cpp new file mode 100644 index 00000000000..e52091fc597 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionGroupSortedArray.cpp @@ -0,0 +1,163 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +static inline constexpr UInt64 GROUP_SORTED_ARRAY_MAX_SIZE = 0xFFFFFF; +static inline constexpr UInt64 GROUP_SORTED_ARRAY_DEFAULT_THRESHOLD = 10; + + +namespace DB +{ +struct Settings; + +namespace ErrorCodes +{ + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +namespace +{ + template + class AggregateFunctionGroupSortedArrayNumeric : public AggregateFunctionGroupSortedArray + { + using AggregateFunctionGroupSortedArray::AggregateFunctionGroupSortedArray; + }; + + template + class AggregateFunctionGroupSortedArrayFieldType + : public AggregateFunctionGroupSortedArray + { + using AggregateFunctionGroupSortedArray::AggregateFunctionGroupSortedArray; + DataTypePtr getReturnType() const override { return std::make_shared(std::make_shared()); } + }; + + template + static IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt64 threshold, const Array & params) + { + if (argument_types.empty()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Got empty arguments list"); + + WhichDataType which(argument_types[0]); + if (which.idx == TypeIndex::Date) + return new AggregateFunctionGroupSortedArrayFieldType(threshold, argument_types, params); + if (which.idx == TypeIndex::DateTime) + return new AggregateFunctionGroupSortedArrayFieldType(threshold, argument_types, params); + + if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) + { + return new AggregateFunctionGroupSortedArray(threshold, argument_types, params); + } + else + { + return new AggregateFunctionGroupSortedArray(threshold, argument_types, params); + } + } + + template