mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge pull request #34092 from Avogar/random-settings
Randomize some settings in functional tests
This commit is contained in:
commit
cbb913a67e
@ -1126,7 +1126,12 @@ void Client::processOptions(const OptionsDescription & options_description,
|
||||
{
|
||||
const auto & name = setting.getName();
|
||||
if (options.count(name))
|
||||
config().setString(name, options[name].as<String>());
|
||||
{
|
||||
if (allow_repeated_settings)
|
||||
config().setString(name, options[name].as<Strings>().back());
|
||||
else
|
||||
config().setString(name, options[name].as<String>());
|
||||
}
|
||||
}
|
||||
|
||||
if (options.count("config-file") && options.count("config"))
|
||||
|
@ -1872,6 +1872,8 @@ void ClientBase::readArguments(
|
||||
prev_port_arg = port_arg;
|
||||
}
|
||||
}
|
||||
else if (arg == "--allow_repeated_settings"sv)
|
||||
allow_repeated_settings = true;
|
||||
else
|
||||
common_arguments.emplace_back(arg);
|
||||
}
|
||||
@ -1884,7 +1886,10 @@ void ClientBase::readArguments(
|
||||
|
||||
void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments)
|
||||
{
|
||||
cmd_settings.addProgramOptions(options_description.main_description.value());
|
||||
if (allow_repeated_settings)
|
||||
cmd_settings.addProgramOptionsAsMultitokens(options_description.main_description.value());
|
||||
else
|
||||
cmd_settings.addProgramOptions(options_description.main_description.value());
|
||||
/// Parse main commandline options.
|
||||
auto parser = po::command_line_parser(arguments).options(options_description.main_description.value()).allow_unregistered();
|
||||
po::parsed_options parsed = parser.run();
|
||||
|
@ -260,6 +260,8 @@ protected:
|
||||
|
||||
std::vector<HostAndPort> hosts_and_ports{};
|
||||
|
||||
bool allow_repeated_settings = false;
|
||||
|
||||
bool cancelled = false;
|
||||
};
|
||||
|
||||
|
@ -89,6 +89,14 @@ void Settings::addProgramOptions(boost::program_options::options_description & o
|
||||
}
|
||||
}
|
||||
|
||||
void Settings::addProgramOptionsAsMultitokens(boost::program_options::options_description & options)
|
||||
{
|
||||
for (const auto & field : all())
|
||||
{
|
||||
addProgramOptionAsMultitoken(options, field);
|
||||
}
|
||||
}
|
||||
|
||||
void Settings::addProgramOption(boost::program_options::options_description & options, const SettingFieldRef & field)
|
||||
{
|
||||
const std::string_view name = field.getName();
|
||||
@ -97,6 +105,14 @@ void Settings::addProgramOption(boost::program_options::options_description & op
|
||||
name.data(), boost::program_options::value<std::string>()->composing()->notifier(on_program_option), field.getDescription())));
|
||||
}
|
||||
|
||||
void Settings::addProgramOptionAsMultitoken(boost::program_options::options_description & options, const SettingFieldRef & field)
|
||||
{
|
||||
const std::string_view name = field.getName();
|
||||
auto on_program_option = boost::function1<void, const Strings &>([this, name](const Strings & values) { set(name, values.back()); });
|
||||
options.add(boost::shared_ptr<boost::program_options::option_description>(new boost::program_options::option_description(
|
||||
name.data(), boost::program_options::value<Strings>()->multitoken()->composing()->notifier(on_program_option), field.getDescription())));
|
||||
}
|
||||
|
||||
void Settings::checkNoSettingNamesAtTopLevel(const Poco::Util::AbstractConfiguration & config, const String & config_path)
|
||||
{
|
||||
if (config.getBool("skip_check_for_incorrect_settings", false))
|
||||
|
@ -722,6 +722,11 @@ struct Settings : public BaseSettings<SettingsTraits>, public IHints<2, Settings
|
||||
/// (Don't forget to call notify() on the `variables_map` after parsing it!)
|
||||
void addProgramOptions(boost::program_options::options_description & options);
|
||||
|
||||
/// Adds program options as to set the settings from a command line.
|
||||
/// Allows to set one setting multiple times, the last value will be used.
|
||||
/// (Don't forget to call notify() on the `variables_map` after parsing it!)
|
||||
void addProgramOptionsAsMultitokens(boost::program_options::options_description & options);
|
||||
|
||||
/// Check that there is no user-level settings at the top level in config.
|
||||
/// This is a common source of mistake (user don't know where to write user-level setting).
|
||||
static void checkNoSettingNamesAtTopLevel(const Poco::Util::AbstractConfiguration & config, const String & config_path);
|
||||
@ -729,6 +734,8 @@ struct Settings : public BaseSettings<SettingsTraits>, public IHints<2, Settings
|
||||
std::vector<String> getAllRegisteredNames() const override;
|
||||
|
||||
void addProgramOption(boost::program_options::options_description & options, const SettingFieldRef & field);
|
||||
|
||||
void addProgramOptionAsMultitoken(boost::program_options::options_description & options, const SettingFieldRef & field);
|
||||
};
|
||||
|
||||
/*
|
||||
|
@ -337,6 +337,26 @@ class FailureReason(enum.Enum):
|
||||
INTERNAL_ERROR = "Test internal error: "
|
||||
|
||||
|
||||
class SettingsRandomizer:
|
||||
settings = {
|
||||
"max_insert_threads": lambda: 0 if random.random() < 0.5 else random.randint(1, 16),
|
||||
"group_by_two_level_threshold": lambda: 1 if random.random() < 0.1 else 2 ** 60 if random.random() < 0.11 else 100000,
|
||||
"group_by_two_level_threshold_bytes": lambda: 1 if random.random() < 0.1 else 2 ** 60 if random.random() < 0.11 else 50000000,
|
||||
"distributed_aggregation_memory_efficient": lambda: random.randint(0, 1),
|
||||
"fsync_metadata": lambda: random.randint(0, 1),
|
||||
"priority": lambda: int(abs(random.gauss(0, 2))),
|
||||
"output_format_parallel_formatting": lambda: random.randint(0, 1),
|
||||
"input_format_parallel_parsing": lambda: random.randint(0, 1),
|
||||
}
|
||||
|
||||
@staticmethod
|
||||
def get_random_settings():
|
||||
random_settings = []
|
||||
for setting, generator in SettingsRandomizer.settings.items():
|
||||
random_settings.append(setting + "=" + str(generator()) + "")
|
||||
return random_settings
|
||||
|
||||
|
||||
class TestResult:
|
||||
def __init__(self, case_name: str, status: TestStatus, reason: Optional[FailureReason], total_time: float, description: str):
|
||||
self.case_name: str = case_name
|
||||
@ -417,6 +437,29 @@ class TestCase:
|
||||
|
||||
return testcase_args
|
||||
|
||||
def add_random_settings(self, client_options):
|
||||
if self.tags and 'no-random-settings' in self.tags:
|
||||
return client_options
|
||||
|
||||
if len(self.base_url_params) == 0:
|
||||
os.environ['CLICKHOUSE_URL_PARAMS'] = '&'.join(self.random_settings)
|
||||
else:
|
||||
os.environ['CLICKHOUSE_URL_PARAMS'] = self.base_url_params + '&' + '&'.join(self.random_settings)
|
||||
|
||||
new_options = "--allow_repeated_settings --" + " --".join(self.random_settings)
|
||||
os.environ['CLICKHOUSE_CLIENT_OPT'] = self.base_client_options + new_options + ' '
|
||||
return client_options + new_options
|
||||
|
||||
def remove_random_settings_from_env(self):
|
||||
os.environ['CLICKHOUSE_URL_PARAMS'] = self.base_url_params
|
||||
os.environ['CLICKHOUSE_CLIENT_OPT'] = self.base_client_options
|
||||
|
||||
def add_info_about_settings(self, description):
|
||||
if self.tags and 'no-random-settings' in self.tags:
|
||||
return description
|
||||
|
||||
return description + "\n" + "Settings used in the test: " + "--" + " --".join(self.random_settings) + "\n"
|
||||
|
||||
def __init__(self, suite, case: str, args, is_concurrent: bool):
|
||||
self.case: str = case # case file name
|
||||
self.tags: Set[str] = suite.all_tags[case] if case in suite.all_tags else set()
|
||||
@ -432,6 +475,10 @@ class TestCase:
|
||||
self.testcase_args = None
|
||||
self.runs_count = 0
|
||||
|
||||
self.random_settings = SettingsRandomizer.get_random_settings()
|
||||
self.base_url_params = os.environ['CLICKHOUSE_URL_PARAMS'] if 'CLICKHOUSE_URL_PARAMS' in os.environ else ''
|
||||
self.base_client_options = os.environ['CLICKHOUSE_CLIENT_OPT'] if 'CLICKHOUSE_CLIENT_OPT' in os.environ else ''
|
||||
|
||||
# should skip test, should increment skipped_total, skip reason
|
||||
def should_skip_test(self, suite) -> Optional[FailureReason]:
|
||||
tags = self.tags
|
||||
@ -673,10 +720,13 @@ class TestCase:
|
||||
|
||||
self.runs_count += 1
|
||||
self.testcase_args = self.configure_testcase_args(args, self.case_file, suite.suite_tmp_path)
|
||||
client_options = self.add_random_settings(client_options)
|
||||
proc, stdout, stderr, total_time = self.run_single_test(server_logs_level, client_options)
|
||||
|
||||
result = self.process_result_impl(proc, stdout, stderr, total_time)
|
||||
result.check_if_need_retry(args, stdout, stderr, self.runs_count)
|
||||
if result.status == TestStatus.FAIL:
|
||||
result.description = self.add_info_about_settings(result.description)
|
||||
return result
|
||||
except KeyboardInterrupt as e:
|
||||
raise e
|
||||
@ -684,17 +734,20 @@ class TestCase:
|
||||
return TestResult(self.name, TestStatus.FAIL,
|
||||
FailureReason.INTERNAL_QUERY_FAIL,
|
||||
0.,
|
||||
self.get_description_from_exception_info(sys.exc_info()))
|
||||
self.add_info_about_settings(self.get_description_from_exception_info(sys.exc_info())))
|
||||
except (ConnectionRefusedError, ConnectionResetError):
|
||||
return TestResult(self.name, TestStatus.FAIL,
|
||||
FailureReason.SERVER_DIED,
|
||||
0.,
|
||||
self.get_description_from_exception_info(sys.exc_info()))
|
||||
self.add_info_about_settings(self.get_description_from_exception_info(sys.exc_info())))
|
||||
except:
|
||||
return TestResult(self.name, TestStatus.UNKNOWN,
|
||||
FailureReason.INTERNAL_ERROR,
|
||||
0.,
|
||||
self.get_description_from_exception_info(sys.exc_info()))
|
||||
finally:
|
||||
self.remove_random_settings_from_env()
|
||||
|
||||
|
||||
class TestSuite:
|
||||
@staticmethod
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: replica, distributed
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas = 0;
|
||||
SET max_parallel_replicas = 2;
|
||||
DROP TABLE IF EXISTS report;
|
||||
|
||||
|
@ -2,13 +2,11 @@
|
||||
"value": 4611686018427387904
|
||||
"name": "value",
|
||||
"value": "4611686018427387904"
|
||||
value
|
||||
value
|
||||
Cannot modify 'output_format_json_quote_64bit_integers' setting in readonly mode
|
||||
OK
|
||||
OK
|
||||
"name": "value",
|
||||
"value": "9223372036854775808"
|
||||
"name": "value",
|
||||
"value": 9223372036854775808
|
||||
value
|
||||
value
|
||||
Cannot modify 'output_format_json_quote_64bit_integers' setting in readonly mode
|
||||
OK
|
||||
OK
|
||||
|
@ -9,13 +9,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
$CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=0 | grep value
|
||||
$CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=1 | grep value
|
||||
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode'
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode'
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
|
||||
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" | grep value
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" | grep value
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&session_timeout=3600" -d 'SET readonly = 1'
|
||||
#${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&session_timeout=3600" -d 'SET readonly = 1'
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode.' && echo "OK" || echo "FAIL"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode.'
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode'
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET send_logs_level = 'fatal';
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
DROP TABLE IF EXISTS test_00808;
|
||||
CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date);
|
||||
|
@ -1,4 +1,6 @@
|
||||
SET enable_optimize_predicate_expression = 0;
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
select * from system.one l cross join system.one r;
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET enable_optimize_predicate_expression = 0;
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE IF EXISTS t2;
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: shard
|
||||
|
||||
SET prefer_localhost_replica = 1;
|
||||
|
||||
SELECT count() FROM remote('127.0.0.1,localhos', system.one); -- { serverError 198 }
|
||||
SELECT count() FROM remote('127.0.0.1|localhos', system.one);
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel, no-fasttest
|
||||
-- Tags: no-parallel, no-fasttest, no-random-settings
|
||||
|
||||
SET max_memory_usage = 32000000;
|
||||
SET join_on_disk_max_files_to_merge = 4;
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-tsan, no-asan, no-msan, no-replicated-database
|
||||
-- Tags: no-tsan, no-asan, no-msan, no-replicated-database, no-random-settings
|
||||
-- Tag no-tsan: Fine thresholds on memory usage
|
||||
-- Tag no-asan: Fine thresholds on memory usage
|
||||
-- Tag no-msan: Fine thresholds on memory usage
|
||||
@ -7,6 +7,8 @@
|
||||
-- sizeof(HLL) is (2^K * 6 / 8)
|
||||
-- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400
|
||||
|
||||
SET use_uncompressed_cache = 0;
|
||||
|
||||
-- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements
|
||||
SELECT 'UInt32';
|
||||
SET max_memory_usage = 4000000;
|
||||
@ -19,6 +21,8 @@ SELECT 'UInt64';
|
||||
SET max_memory_usage = 4000000;
|
||||
SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(reinterpretAsString(number % 4096)) u FROM numbers(4096 * 100) GROUP BY k); -- { serverError 241 }
|
||||
SET max_memory_usage = 9830400;
|
||||
|
||||
|
||||
SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(reinterpretAsString(number % 4096)) u FROM numbers(4096 * 100) GROUP BY k);
|
||||
|
||||
SELECT 'K=16';
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: replica, distributed
|
||||
|
||||
set allow_experimental_parallel_reading_from_replicas=0;
|
||||
|
||||
drop table if exists test_max_parallel_replicas_lr;
|
||||
|
||||
-- If you wonder why the table is named with "_lr" suffix in this test.
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: distributed
|
||||
|
||||
set allow_experimental_parallel_reading_from_replicas = 0;
|
||||
|
||||
drop table if exists sample_final;
|
||||
create table sample_final (CounterID UInt32, EventDate Date, EventTime DateTime, UserID UInt64, Sign Int8) engine = CollapsingMergeTree(Sign) order by (CounterID, EventDate, intHash32(UserID), EventTime) sample by intHash32(UserID);
|
||||
insert into sample_final select number / (8192 * 4), toDate('2019-01-01'), toDateTime('2019-01-01 00:00:01') + number, number / (8192 * 2), number % 3 = 1 ? -1 : 1 from numbers(1000000);
|
||||
|
@ -1,5 +1,6 @@
|
||||
SET enable_optimize_predicate_expression = 1;
|
||||
SET joined_subquery_requires_alias = 0;
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/3885
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/5485
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
DROP TABLE IF EXISTS n;
|
||||
DROP TABLE IF EXISTS r;
|
||||
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
-- set insert_distributed_sync = 1; -- see https://github.com/ClickHouse/ClickHouse/issues/18971
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas = 0; -- see https://github.com/ClickHouse/ClickHouse/issues/34525
|
||||
|
||||
DROP TABLE IF EXISTS local_01099_a;
|
||||
DROP TABLE IF EXISTS local_01099_b;
|
||||
DROP TABLE IF EXISTS distributed_01099_a;
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET prefer_localhost_replica = 1;
|
||||
|
||||
DROP DATABASE IF EXISTS test_01155_ordinary;
|
||||
DROP DATABASE IF EXISTS test_01155_atomic;
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-random-settings
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
set optimize_arithmetic_operations_in_aggregate_functions = 1;
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
explain syntax select min((n as a) + (1 as b)) c from (select number n from numbers(10)) where a > 0 and b > 0 having c > 0;
|
||||
select min((n as a) + (1 as b)) c from (select number n from numbers(10)) where a > 0 and b > 0 having c > 0;
|
||||
|
@ -1,3 +1,5 @@
|
||||
set max_threads = 0;
|
||||
|
||||
drop table if exists testX;
|
||||
drop table if exists testXA;
|
||||
drop table if exists testXB;
|
||||
|
@ -1,3 +1,5 @@
|
||||
-- Tags: no-random-settings
|
||||
|
||||
show settings like 'send_timeout';
|
||||
SHOW SETTINGS ILIKE '%CONNECT_timeout%';
|
||||
SHOW CHANGED SETTINGS ILIKE '%MEMORY%';
|
||||
|
@ -1,4 +1,5 @@
|
||||
-- Tags: no-parallel
|
||||
set prefer_localhost_replica = 1;
|
||||
|
||||
drop table if exists null_01293;
|
||||
drop table if exists dist_01293;
|
||||
|
@ -1,3 +1,5 @@
|
||||
set max_block_size = 65505;
|
||||
|
||||
set optimize_group_by_function_keys = 1;
|
||||
|
||||
SELECT round(max(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 2, number % 3, (number % 2 + number % 3) % 2 ORDER BY k;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t;
|
||||
|
||||
CREATE TABLE t (x UInt8) ENGINE = MergeTree ORDER BY x;
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
# Tags: no-fasttest, no-random-settings
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
DROP TABLE IF EXISTS t0;
|
||||
|
||||
CREATE TABLE t0
|
||||
|
@ -1,5 +1,6 @@
|
||||
set log_queries = 1;
|
||||
set max_threads = 16;
|
||||
set prefer_localhost_replica = 1;
|
||||
|
||||
select sum(number) from remote('127.0.0.{1|2}', numbers_mt(1000000)) group by number % 2 order by number % 2;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET prefer_localhost_replica = 1;
|
||||
|
||||
DROP DATABASE IF EXISTS test_01457;
|
||||
|
||||
CREATE DATABASE test_01457;
|
||||
|
@ -1,4 +1,7 @@
|
||||
-- Tags: no-s3-storage
|
||||
|
||||
SET use_uncompressed_cache = 0;
|
||||
|
||||
SELECT '====array====';
|
||||
DROP TABLE IF EXISTS t_arr;
|
||||
CREATE TABLE t_arr (a Array(UInt32)) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: distributed
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas = 0;
|
||||
|
||||
DROP TABLE IF EXISTS test5346;
|
||||
|
||||
CREATE TABLE test5346 (`Id` String, `Timestamp` DateTime, `updated` DateTime)
|
||||
|
@ -3,6 +3,7 @@
|
||||
DROP TABLE IF EXISTS nested;
|
||||
|
||||
SET flatten_nested = 0;
|
||||
SET use_uncompressed_cache = 0;
|
||||
|
||||
CREATE TABLE nested
|
||||
(
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: replica
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas=0;
|
||||
|
||||
DROP TABLE IF EXISTS t;
|
||||
CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x;
|
||||
INSERT INTO t VALUES ('Hello');
|
||||
|
@ -1,3 +1,6 @@
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
DROP TABLE IF EXISTS prewhere_move;
|
||||
CREATE TABLE prewhere_move (x Int, y String) ENGINE = MergeTree ORDER BY tuple();
|
||||
INSERT INTO prewhere_move SELECT number, toString(number) FROM numbers(1000);
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET use_uncompressed_cache = 0;
|
||||
|
||||
DROP TABLE IF EXISTS adaptive_table;
|
||||
|
||||
--- If granularity of consequent blocks differs a lot, then adaptive
|
||||
|
@ -64,7 +64,7 @@ $CLICKHOUSE_CLIENT -q "
|
||||
settings enable_optimize_predicate_expression=0"
|
||||
|
||||
echo "> one condition of filter should be pushed down after aggregating, other two conditions are ANDed"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
$CLICKHOUSE_CLIENT --convert_query_to_cnf=0 -q "
|
||||
explain actions = 1 select s, y from (
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
) where y != 0 and s - 8 and s - 4
|
||||
@ -77,7 +77,7 @@ $CLICKHOUSE_CLIENT -q "
|
||||
settings enable_optimize_predicate_expression=0"
|
||||
|
||||
echo "> two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
$CLICKHOUSE_CLIENT --convert_query_to_cnf=0 -q "
|
||||
explain actions = 1 select s, y from (
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
) where y != 0 and s != 8 and y - 4
|
||||
@ -127,7 +127,7 @@ $CLICKHOUSE_CLIENT -q "
|
||||
settings enable_optimize_predicate_expression=0"
|
||||
|
||||
echo "> filter is pushed down before sorting steps"
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
$CLICKHOUSE_CLIENT --convert_query_to_cnf=0 -q "
|
||||
explain actions = 1 select x, y from (
|
||||
select number % 2 as x, number % 3 as y from numbers(6) order by y desc
|
||||
) where x != 0 and y != 0
|
||||
|
@ -27,6 +27,8 @@ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
|
||||
SOFTWARE.
|
||||
*/
|
||||
|
||||
SET max_insert_threads = 0;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
|
||||
CREATE TABLE test
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET group_by_two_level_threshold = 10000;
|
||||
|
||||
CREATE TABLE group_bitmap_data_test
|
||||
(
|
||||
`pickup_date` Date,
|
||||
|
@ -1,3 +1,6 @@
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
DROP TABLE IF EXISTS prewhere_move_select_final;
|
||||
|
||||
CREATE TABLE prewhere_move_select_final (x Int, y Int, z Int) ENGINE = ReplacingMergeTree() ORDER BY (x, y);
|
||||
|
@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: zstd' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | zstd -d | tail -n30 | head -n23
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: zstd' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) SETTINGS max_block_size=65505 FORMAT JSON" | zstd -d | tail -n30 | head -n23
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: distributed
|
||||
|
||||
SET prefer_localhost_replica = 1;
|
||||
|
||||
DROP TABLE IF EXISTS tt6;
|
||||
|
||||
CREATE TABLE tt6
|
||||
@ -13,6 +15,8 @@ CREATE TABLE tt6
|
||||
)
|
||||
ENGINE = Distributed('test_shard_localhost', '', 'tt7', rand());
|
||||
|
||||
DROP TABLE IF EXISTS tt7;
|
||||
|
||||
CREATE TABLE tt7 as tt6 ENGINE = Distributed('test_shard_localhost', '', 'tt6', rand());
|
||||
|
||||
INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError 581 }
|
||||
@ -28,3 +32,4 @@ INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError 306}
|
||||
SELECT * FROM tt6; -- { serverError 306 }
|
||||
|
||||
DROP TABLE tt6;
|
||||
DROP TABLE tt7;
|
||||
|
@ -4,6 +4,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --optimize_move_to_prewhere=1 --convert_query_to_cnf=0"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "drop table if exists test_index"
|
||||
$CLICKHOUSE_CLIENT -q "drop table if exists idx"
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
SET max_block_size = 65505;
|
||||
|
||||
SELECT 'uniqTheta many agrs';
|
||||
|
||||
SELECT
|
||||
|
@ -1,4 +1,5 @@
|
||||
set short_circuit_function_evaluation = 'enable';
|
||||
set convert_query_to_cnf = 0;
|
||||
|
||||
select if(number > 0, intDiv(number + 100, number), throwIf(number)) from numbers(10);
|
||||
select multiIf(number == 0, 0, number == 1, intDiv(1, number), number == 2, intDiv(1, number - 1), number == 3, intDiv(1, number - 2), intDiv(1, number - 3)) from numbers(10);
|
||||
|
@ -1,3 +1,6 @@
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
SET convert_query_to_cnf = 0;
|
||||
|
||||
DROP TABLE IF EXISTS t_move_to_prewhere;
|
||||
|
||||
CREATE TABLE t_move_to_prewhere (id UInt32, a UInt8, b UInt8, c UInt8, fat_string String)
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
|
||||
CREATE TABLE t1 ( s String, f Float32, e UInt16 ) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = '100G';
|
||||
|
@ -1,6 +1,8 @@
|
||||
-- Tags: no-replicated-database
|
||||
-- Tag no-replicated-database: Different query_id
|
||||
|
||||
SET prefer_localhost_replica = 1;
|
||||
|
||||
DROP TABLE IF EXISTS tmp;
|
||||
|
||||
CREATE TABLE tmp ENGINE = TinyLog AS SELECT queryID();
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: distributed
|
||||
|
||||
set prefer_localhost_replica = 1;
|
||||
|
||||
-- { echo }
|
||||
explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0;
|
||||
explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1;
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
set optimize_skip_unused_shards=1;
|
||||
set optimize_distributed_group_by_sharding_key=1;
|
||||
set prefer_localhost_replica=1;
|
||||
|
||||
-- { echo }
|
||||
explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized
|
||||
|
@ -1,3 +1,4 @@
|
||||
set group_by_two_level_threshold = 100000;
|
||||
set enable_positional_arguments = 1;
|
||||
|
||||
drop table if exists test;
|
||||
|
@ -5,6 +5,7 @@ CREATE TABLE test_tuple_filter (id UInt32, value String, log_date Date) Engine=M
|
||||
INSERT INTO test_tuple_filter VALUES (1,'A','2021-01-01'),(2,'B','2021-01-01'),(3,'C','2021-01-01'),(4,'D','2021-01-02'),(5,'E','2021-01-02');
|
||||
|
||||
SET force_primary_key = 1;
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
|
||||
SELECT * FROM test_tuple_filter WHERE (id, value) = (1, 'A');
|
||||
SELECT * FROM test_tuple_filter WHERE (1, 'A') = (id, value);
|
||||
|
@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# do not print any ProfileEvents packets
|
||||
$CLICKHOUSE_CLIENT -q 'select * from numbers(1e5) format Null' |& grep -c 'SelectedRows'
|
||||
# print only last (and also number of rows to provide more info in case of failures)
|
||||
$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' 2> >(grep -o -e '\[ 0 \] SelectedRows: .*$' -e Exception) 1> >(wc -l)
|
||||
$CLICKHOUSE_CLIENT --max_block_size=65505 --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' 2> >(grep -o -e '\[ 0 \] SelectedRows: .*$' -e Exception) 1> >(wc -l)
|
||||
# print everything
|
||||
profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')"
|
||||
test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)"
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
|
||||
DROP TABLE IF EXISTS 02131_multiply_row_policies_on_same_column;
|
||||
CREATE TABLE 02131_multiply_row_policies_on_same_column (x UInt8) ENGINE = MergeTree ORDER BY x;
|
||||
INSERT INTO 02131_multiply_row_policies_on_same_column VALUES (1), (2), (3), (4);
|
||||
|
@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000));" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress"
|
||||
$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000) settings max_block_size=65505);" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress"
|
||||
|
@ -7,4 +7,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
echo "#1"
|
||||
${CLICKHOUSE_CLIENT} --query='SELECT count() FROM numbers(100) FORMAT JSON;' | grep -a -v "elapsed"
|
||||
echo "#2"
|
||||
${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery"
|
||||
${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) SETTINGS max_block_size = 65505 FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery"
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET optimize_move_to_prewhere = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t_02156_mt1;
|
||||
DROP TABLE IF EXISTS t_02156_mt2;
|
||||
DROP TABLE IF EXISTS t_02156_log;
|
||||
|
@ -1,3 +1,5 @@
|
||||
-- Tags: no-random-settings
|
||||
|
||||
SET max_bytes_before_external_group_by = 200000000;
|
||||
|
||||
SET max_memory_usage = 1500000000;
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
DROP TABLE IF EXISTS test.avro;
|
||||
|
||||
SET max_threads = 1, max_block_size = 8192, min_insert_block_size_rows = 8192, min_insert_block_size_bytes = 1048576; -- lower memory usage
|
||||
SET max_threads = 1, max_insert_threads = 0, max_block_size = 8192, min_insert_block_size_rows = 8192, min_insert_block_size_bytes = 1048576; -- lower memory usage
|
||||
|
||||
CREATE TABLE test.avro AS test.hits ENGINE = File(Avro);
|
||||
INSERT INTO test.avro SELECT * FROM test.hits LIMIT 10000;
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-tsan
|
||||
# Tags: no-tsan, no-random-settings
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
Loading…
Reference in New Issue
Block a user