diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index ac0463b447a..d3fac8670e8 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -19,9 +19,9 @@ Detailed description / Documentation draft: ... -By adding documentation, you'll allow users to try your new feature immediately, not when someone else will have time to document it later. Documentation is necessary for all features that affect user experience in any way. You can add brief documentation draft above, or add documentation right into your patch as Markdown files in [docs](https://github.com/ClickHouse/ClickHouse/tree/master/docs) folder. +> By adding documentation, you'll allow users to try your new feature immediately, not when someone else will have time to document it later. Documentation is necessary for all features that affect user experience in any way. You can add brief documentation draft above, or add documentation right into your patch as Markdown files in [docs](https://github.com/ClickHouse/ClickHouse/tree/master/docs) folder. -If you are doing this for the first time, it's recommended to read the lightweight [Contributing to ClickHouse Documentation](https://github.com/ClickHouse/ClickHouse/tree/master/docs/README.md) guide first. +> If you are doing this for the first time, it's recommended to read the lightweight [Contributing to ClickHouse Documentation](https://github.com/ClickHouse/ClickHouse/tree/master/docs/README.md) guide first. -Information about CI checks: https://clickhouse.tech/docs/en/development/continuous-integration/ +> Information about CI checks: https://clickhouse.tech/docs/en/development/continuous-integration/ diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index ed5c81e89fa..745e020c8bb 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -259,10 +259,25 @@ private: Poco::Logger * log; BaseDaemon & daemon; - void onTerminate(const std::string & message, UInt32 thread_num) const + void onTerminate(std::string_view message, UInt32 thread_num) const { + size_t pos = message.find('\n'); + LOG_FATAL(log, "(version {}{}, {}) (from thread {}) {}", - VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num, message); + VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num, message.substr(0, pos)); + + /// Print trace from std::terminate exception line-by-line to make it easy for grep. + while (pos != std::string_view::npos) + { + ++pos; + size_t next_pos = message.find('\n', pos); + size_t size = next_pos; + if (next_pos != std::string_view::npos) + size = next_pos - pos; + + LOG_FATAL(log, "{}", message.substr(pos, size)); + pos = next_pos; + } } void onFault( diff --git a/contrib/NuRaft b/contrib/NuRaft index 976874b7aa7..0ce94900930 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 976874b7aa7f422bf4ea595bb7d1166c617b1c26 +Subproject commit 0ce9490093021c63564cca159571a8b27772ad48 diff --git a/contrib/nuraft-cmake/CMakeLists.txt b/contrib/nuraft-cmake/CMakeLists.txt index 725e86195e1..d9e0aa6efc7 100644 --- a/contrib/nuraft-cmake/CMakeLists.txt +++ b/contrib/nuraft-cmake/CMakeLists.txt @@ -22,6 +22,7 @@ set(SRCS "${LIBRARY_DIR}/src/launcher.cxx" "${LIBRARY_DIR}/src/srv_config.cxx" "${LIBRARY_DIR}/src/snapshot_sync_req.cxx" + "${LIBRARY_DIR}/src/snapshot_sync_ctx.cxx" "${LIBRARY_DIR}/src/handle_timeout.cxx" "${LIBRARY_DIR}/src/handle_append_entries.cxx" "${LIBRARY_DIR}/src/cluster_config.cxx" diff --git a/docker/test/stress/stress b/docker/test/stress/stress index c98a527c1fe..c71722809d7 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -20,6 +20,7 @@ def get_skip_list_cmd(path): def get_options(i): options = [] + client_options = [] if 0 < i: options.append("--order=random") @@ -27,25 +28,29 @@ def get_options(i): options.append("--db-engine=Ordinary") if i % 3 == 2: - options.append('''--client-option='allow_experimental_database_replicated=1' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)) + options.append('''--db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)) + client_options.append('allow_experimental_database_replicated=1') # If database name is not specified, new database is created for each functional test. # Run some threads with one database for all tests. if i % 2 == 1: options.append(" --database=test_{}".format(i)) - if i % 7 == 0: - options.append(" --client-option='join_use_nulls=1'") + if i % 5 == 1: + client_options.append("join_use_nulls=1") - if i % 14 == 0: - options.append(' --client-option="join_algorithm=\'partial_merge\'"') + if i % 15 == 6: + client_options.append("join_algorithm='partial_merge'") - if i % 21 == 0: - options.append(' --client-option="join_algorithm=\'auto\'"') - options.append(' --client-option="max_rows_in_join=1000"') + if i % 15 == 11: + client_options.append("join_algorithm='auto'") + client_options.append('max_rows_in_join=1000') if i == 13: - options.append(" --client-option='memory_tracker_fault_probability=0.00001'") + client_options.append('memory_tracker_fault_probability=0.001') + + if client_options: + options.append(" --client-option " + ' '.join(client_options)) return ' '.join(options) diff --git a/programs/copier/CMakeLists.txt b/programs/copier/CMakeLists.txt index dfb067b00f9..57e0996ed78 100644 --- a/programs/copier/CMakeLists.txt +++ b/programs/copier/CMakeLists.txt @@ -11,7 +11,6 @@ set (CLICKHOUSE_COPIER_LINK clickhouse_functions clickhouse_table_functions clickhouse_aggregate_functions - clickhouse_dictionaries string_utils PUBLIC diff --git a/programs/local/CMakeLists.txt b/programs/local/CMakeLists.txt index b61f0ea33b7..530128c2041 100644 --- a/programs/local/CMakeLists.txt +++ b/programs/local/CMakeLists.txt @@ -6,7 +6,6 @@ set (CLICKHOUSE_LOCAL_LINK clickhouse_aggregate_functions clickhouse_common_config clickhouse_common_io - clickhouse_dictionaries clickhouse_functions clickhouse_parsers clickhouse_storages_system diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 739d1004025..281c25d50eb 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -13,7 +13,6 @@ set (CLICKHOUSE_SERVER_LINK clickhouse_common_config clickhouse_common_io clickhouse_common_zookeeper - clickhouse_dictionaries clickhouse_functions clickhouse_parsers clickhouse_storages_system diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 023dc4ed15b..b13be0aba29 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 718bb58a237..b833241b2bc 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -21,13 +22,14 @@ namespace ErrorCodes HedgedConnections::HedgedConnections( const ConnectionPoolWithFailoverPtr & pool_, - const Settings & settings_, + ContextPtr context_, const ConnectionTimeouts & timeouts_, const ThrottlerPtr & throttler_, PoolMode pool_mode, std::shared_ptr table_to_check_) - : hedged_connections_factory(pool_, &settings_, timeouts_, table_to_check_) - , settings(settings_) + : hedged_connections_factory(pool_, &context_->getSettingsRef(), timeouts_, table_to_check_) + , context(std::move(context_)) + , settings(context->getSettingsRef()) , drain_timeout(settings.drain_timeout) , allow_changing_replica_until_first_data_packet(settings.allow_changing_replica_until_first_data_packet) , throttler(throttler_) diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index f41142346ed..b0bff8e7c5d 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -72,7 +72,7 @@ public: }; HedgedConnections(const ConnectionPoolWithFailoverPtr & pool_, - const Settings & settings_, + ContextPtr context_, const ConnectionTimeouts & timeouts_, const ThrottlerPtr & throttler, PoolMode pool_mode, @@ -188,6 +188,7 @@ private: Packet last_received_packet; Epoll epoll; + ContextPtr context; const Settings & settings; /// The following two fields are from settings but can be referenced outside the lifetime of diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index bc26edd7feb..21e874691c1 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -102,7 +102,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( if (main_table) table_to_check = std::make_shared(main_table.getQualifiedName()); - return std::make_shared(pool, current_settings, timeouts, throttler, pool_mode, table_to_check); + return std::make_shared(pool, context, timeouts, throttler, pool_mode, table_to_check); } #endif diff --git a/src/DataStreams/formatBlock.cpp b/src/DataStreams/formatBlock.cpp new file mode 100644 index 00000000000..e38540256ac --- /dev/null +++ b/src/DataStreams/formatBlock.cpp @@ -0,0 +1,15 @@ +#include +#include +#include + +namespace DB +{ +void formatBlock(BlockOutputStreamPtr & out, const Block & block) +{ + out->writePrefix(); + out->write(block); + out->writeSuffix(); + out->flush(); +} + +} diff --git a/src/DataStreams/formatBlock.h b/src/DataStreams/formatBlock.h new file mode 100644 index 00000000000..939b72682c3 --- /dev/null +++ b/src/DataStreams/formatBlock.h @@ -0,0 +1,9 @@ +#pragma once + +#include + +namespace DB +{ +void formatBlock(BlockOutputStreamPtr & out, const Block & block); + +} diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 2d53ac4321e..54ed07092d3 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -1,7 +1,7 @@ #include "DictionarySourceHelpers.h" #include #include -#include +#include #include #include #include "DictionaryStructure.h" @@ -18,14 +18,6 @@ namespace ErrorCodes extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; } -void formatBlock(BlockOutputStreamPtr & out, const Block & block) -{ - out->writePrefix(); - out->write(block); - out->writeSuffix(); - out->flush(); -} - /// For simple key Block blockForIds( diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 6c9a321aa36..6fed4c7181c 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -13,15 +13,8 @@ namespace DB { -class IBlockOutputStream; -using BlockOutputStreamPtr = std::shared_ptr; - struct DictionaryStructure; -/// Write keys to block output stream. - -void formatBlock(BlockOutputStreamPtr & out, const Block & block); - /// For simple key Block blockForIds( diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 7f4a3a5b45f..daf79965428 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 5ece5af0f0b..9eacda343cf 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index b1b1968454c..ea26e9b7a2a 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -1,6 +1,7 @@ #include "HTTPDictionarySource.h" #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a268468ea59..651f74622bb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3214,7 +3214,11 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc const auto & partition_ast = ast->as(); if (!partition_ast.value) + { + if (!MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version)) + throw Exception("Invalid partition format: " + partition_ast.id, ErrorCodes::INVALID_PARTITION_VALUE); return partition_ast.id; + } if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index 94430de422e..24c18843935 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -21,6 +21,40 @@ MergeTreePartInfo MergeTreePartInfo::fromPartName(const String & part_name, Merg } +bool MergeTreePartInfo::validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version) +{ + if (partition_id.empty()) + return false; + + ReadBufferFromString in(partition_id); + + if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + { + UInt32 min_yyyymmdd = 0; + UInt32 max_yyyymmdd = 0; + if (!tryReadIntText(min_yyyymmdd, in) + || !checkChar('_', in) + || !tryReadIntText(max_yyyymmdd, in) + || !checkChar('_', in)) + { + return false; + } + } + else + { + while (!in.eof()) + { + char c; + readChar(c, in); + + if (c == '_') + break; + } + } + + return in.eof(); +} + bool MergeTreePartInfo::tryParsePartName(const String & part_name, MergeTreePartInfo * part_info, MergeTreeDataFormatVersion format_version) { ReadBufferFromString in(part_name); diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index 8b77442bf8b..063f234e50f 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -86,6 +86,9 @@ struct MergeTreePartInfo return static_cast(max_block - min_block + 1); } + /// Simple sanity check for partition ID. Checking that it's not too long or too short, doesn't contain a lot of '_'. + static bool validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version); + static MergeTreePartInfo fromPartName(const String & part_name, MergeTreeDataFormatVersion format_version); // -V1071 static bool tryParsePartName(const String & part_name, MergeTreePartInfo * part_info, MergeTreeDataFormatVersion format_version); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 93b6243d6f5..362d1d225d9 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -175,7 +175,8 @@ def configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file): database = 'test_{suffix}'.format(suffix=random_str()) with open(stderr_file, 'w') as stderr: - clickhouse_proc_create = Popen(shlex.split(testcase_args.testcase_client), stdin=PIPE, stdout=PIPE, stderr=stderr, universal_newlines=True) + client_cmd = testcase_args.testcase_client + " " + get_additional_client_options(args) + clickhouse_proc_create = Popen(shlex.split(client_cmd), stdin=PIPE, stdout=PIPE, stderr=stderr, universal_newlines=True) try: clickhouse_proc_create.communicate(("CREATE DATABASE " + database + get_db_engine(testcase_args, database)), timeout=testcase_args.timeout) except TimeoutExpired: @@ -937,7 +938,8 @@ def main(args): def create_common_database(args, db_name): create_database_retries = 0 while create_database_retries < MAX_RETRIES: - clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) + client_cmd = args.client + " " + get_additional_client_options(args) + clickhouse_proc_create = Popen(shlex.split(client_cmd), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) (stdout, stderr) = clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name))) if not need_retry(stdout, stderr): break diff --git a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh index 323dd88efab..97e4da5f9e3 100755 --- a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh +++ b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh @@ -6,20 +6,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -n --query " -DROP USER IF EXISTS quoted_by_ip; -DROP USER IF EXISTS quoted_by_forwarded_ip; +CREATE USER quoted_by_ip_${CLICKHOUSE_DATABASE}; +CREATE USER quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}; -DROP QUOTA IF EXISTS quota_by_ip; -DROP QUOTA IF EXISTS quota_by_forwarded_ip; +GRANT SELECT, CREATE ON *.* TO quoted_by_ip_${CLICKHOUSE_DATABASE}; +GRANT SELECT, CREATE ON *.* TO quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}; -CREATE USER quoted_by_ip; -CREATE USER quoted_by_forwarded_ip; - -GRANT SELECT, CREATE ON *.* TO quoted_by_ip; -GRANT SELECT, CREATE ON *.* TO quoted_by_forwarded_ip; - -CREATE QUOTA quota_by_ip KEYED BY ip_address FOR RANDOMIZED INTERVAL 1 YEAR MAX QUERIES = 1 TO quoted_by_ip; -CREATE QUOTA quota_by_forwarded_ip KEYED BY forwarded_ip_address FOR RANDOMIZED INTERVAL 1 YEAR MAX QUERIES = 1 TO quoted_by_forwarded_ip; +CREATE QUOTA quota_by_ip_${CLICKHOUSE_DATABASE} KEYED BY ip_address FOR RANDOMIZED INTERVAL 1 YEAR MAX QUERIES = 1 TO quoted_by_ip_${CLICKHOUSE_DATABASE}; +CREATE QUOTA quota_by_forwarded_ip_${CLICKHOUSE_DATABASE} KEYED BY forwarded_ip_address FOR RANDOMIZED INTERVAL 1 YEAR MAX QUERIES = 1 TO quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}; " # Note: the test can be flaky if the randomized interval will end while the loop is run. But with year long interval it's unlikely. @@ -28,39 +22,39 @@ CREATE QUOTA quota_by_forwarded_ip KEYED BY forwarded_ip_address FOR RANDOMIZED echo '--- Test with quota by immediate IP ---' while true; do - $CLICKHOUSE_CLIENT --user quoted_by_ip --query "SELECT count() FROM numbers(10)" 2>/dev/null || break + ${CLICKHOUSE_CURL} --fail -sS "${CLICKHOUSE_URL}&user=quoted_by_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" 2>/dev/null || break done | uniq -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=quoted_by_ip" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=quoted_by_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' # X-Forwarded-For is ignored for quota by immediate IP address -${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_ip" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' +${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' echo '--- Test with quota by forwarded IP ---' while true; do - $CLICKHOUSE_CLIENT --user quoted_by_forwarded_ip --query "SELECT count() FROM numbers(10)" 2>/dev/null || break + ${CLICKHOUSE_CURL} --fail -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" 2>/dev/null || break done | uniq -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' # X-Forwarded-For is respected for quota by forwarded IP address while true; do - ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip" -d "SELECT count() FROM numbers(10)" | grep -oP '^10$' || break + ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oP '^10$' || break done | uniq -${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' +${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' # Only the last IP address is trusted -${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 5.6.7.8, 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' +${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 5.6.7.8, 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' -${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4, 5.6.7.8' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip" -d "SELECT count() FROM numbers(10)" +${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4, 5.6.7.8' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" $CLICKHOUSE_CLIENT -n --query " -DROP QUOTA IF EXISTS quota_by_ip; +DROP QUOTA IF EXISTS quota_by_ip_${CLICKHOUSE_DATABASE}; DROP QUOTA IF EXISTS quota_by_forwarded_ip; -DROP USER IF EXISTS quoted_by_ip; -DROP USER IF EXISTS quoted_by_forwarded_ip; +DROP USER IF EXISTS quoted_by_ip_${CLICKHOUSE_DATABASE}; +DROP USER IF EXISTS quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}; " diff --git a/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.reference b/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql b/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql new file mode 100644 index 00000000000..baf6c1fbf8f --- /dev/null +++ b/tests/queries/0_stateless/01925_broken_partition_id_zookeeper.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS broken_partition; + +CREATE TABLE broken_partition +( + date Date, + key UInt64 +) +ENGINE = ReplicatedMergeTree('/clickhouse/test_01925_{database}/rmt', 'r1') +ORDER BY tuple() +PARTITION BY date; + +ALTER TABLE broken_partition DROP PARTITION ID '20210325_0_13241_6_12747'; --{serverError 248} + +ALTER TABLE broken_partition DROP PARTITION ID '20210325_0_13241_6_12747'; --{serverError 248} + +DROP TABLE IF EXISTS broken_partition;