Merge pull request #34092 from Avogar/random-settings

Randomize some settings in functional tests
This commit is contained in:
Kruglov Pavel 2022-03-04 13:57:49 +03:00 committed by GitHub
commit cbb913a67e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
62 changed files with 201 additions and 27 deletions

View File

@ -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"))

View File

@ -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();

View File

@ -260,6 +260,8 @@ protected:
std::vector<HostAndPort> hosts_and_ports{};
bool allow_repeated_settings = false;
bool cancelled = false;
};

View File

@ -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))

View File

@ -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);
};
/*

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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'

View File

@ -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);

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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';

View File

@ -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.

View File

@ -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);

View File

@ -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

View File

@ -1,3 +1,5 @@
SET convert_query_to_cnf = 0;
DROP TABLE IF EXISTS n;
DROP TABLE IF EXISTS r;

View File

@ -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;

View File

@ -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;

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash
# Tags: no-random-settings
unset CLICKHOUSE_LOG_COMMENT

View File

@ -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;

View File

@ -1,3 +1,5 @@
set max_threads = 0;
drop table if exists testX;
drop table if exists testXA;
drop table if exists testXB;

View File

@ -1,3 +1,5 @@
-- Tags: no-random-settings
show settings like 'send_timeout';
SHOW SETTINGS ILIKE '%CONNECT_timeout%';
SHOW CHANGED SETTINGS ILIKE '%MEMORY%';

View File

@ -1,4 +1,5 @@
-- Tags: no-parallel
set prefer_localhost_replica = 1;
drop table if exists null_01293;
drop table if exists dist_01293;

View File

@ -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;

View File

@ -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;

View File

@ -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

View File

@ -1,3 +1,5 @@
SET convert_query_to_cnf = 0;
DROP TABLE IF EXISTS t0;
CREATE TABLE t0

View File

@ -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;

View File

@ -1,5 +1,7 @@
-- Tags: no-parallel
SET prefer_localhost_replica = 1;
DROP DATABASE IF EXISTS test_01457;
CREATE DATABASE test_01457;

View File

@ -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;

View File

@ -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)

View File

@ -3,6 +3,7 @@
DROP TABLE IF EXISTS nested;
SET flatten_nested = 0;
SET use_uncompressed_cache = 0;
CREATE TABLE nested
(

View File

@ -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');

View File

@ -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);

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -1,3 +1,5 @@
SET group_by_two_level_threshold = 10000;
CREATE TABLE group_bitmap_data_test
(
`pickup_date` Date,

View File

@ -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);

View File

@ -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

View File

@ -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;

View File

@ -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"

View File

@ -1,5 +1,7 @@
-- Tags: no-fasttest
SET max_block_size = 65505;
SELECT 'uniqTheta many agrs';
SELECT

View File

@ -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);

View File

@ -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)

View File

@ -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';

View File

@ -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();

View File

@ -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;

View File

@ -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

View File

@ -1,3 +1,4 @@
set group_by_two_level_threshold = 100000;
set enable_positional_arguments = 1;
drop table if exists test;

View File

@ -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);

View File

@ -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)"

View File

@ -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);

View File

@ -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"

View File

@ -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"

View File

@ -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;

View File

@ -1,3 +1,5 @@
-- Tags: no-random-settings
SET max_bytes_before_external_group_by = 200000000;
SET max_memory_usage = 1500000000;

View File

@ -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;

View File

@ -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