Inhibit randomization in more tests

This commit is contained in:
Alexey Milovidov 2023-08-12 19:36:57 +02:00
parent 75de1a1a62
commit 28ce14ed72
10 changed files with 17 additions and 16 deletions

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS multidimensional;
CREATE TABLE multidimensional ENGINE = MergeTree ORDER BY number AS SELECT number, arrayMap(x -> (x, [x], [[x]], (x, toString(x))), arrayMap(x -> range(x), range(number % 10))) AS value FROM system.numbers LIMIT 100000;
CREATE TABLE multidimensional ENGINE = MergeTree ORDER BY number SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi' AS SELECT number, arrayMap(x -> (x, [x], [[x]], (x, toString(x))), arrayMap(x -> range(x), range(number % 10))) AS value FROM system.numbers LIMIT 100000;
SELECT sum(cityHash64(toString(value))) FROM multidimensional;

View File

@ -1,6 +1,6 @@
DROP TABLE IF EXISTS mt_pk;
CREATE TABLE mt_pk ENGINE = MergeTree PARTITION BY d ORDER BY x
CREATE TABLE mt_pk ENGINE = MergeTree PARTITION BY d ORDER BY x SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'
AS SELECT toDate(number % 32) AS d, number AS x FROM system.numbers LIMIT 10000010;
SELECT x FROM mt_pk ORDER BY x ASC LIMIT 10000000, 1;

View File

@ -4,7 +4,7 @@ DROP TABLE IF EXISTS tt_01373;
CREATE TABLE tt_01373
(a Int64, d Int64, val Int64)
ENGINE = SummingMergeTree PARTITION BY (a) ORDER BY (d);
ENGINE = SummingMergeTree PARTITION BY (a) ORDER BY (d) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
SYSTEM STOP MERGES tt_01373;

View File

@ -4,7 +4,7 @@ SET allow_asynchronous_read_from_io_pool_for_merge_tree = 0;
SET do_not_merge_across_partitions_select_final = 1;
SET max_threads = 16;
CREATE TABLE select_final (t DateTime, x Int32, string String) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY (x, t);
CREATE TABLE select_final (t DateTime, x Int32, string String) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY (x, t) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO select_final SELECT toDate('2000-01-01'), number, '' FROM numbers(2);
INSERT INTO select_final SELECT toDate('2000-01-01'), number + 1, '' FROM numbers(2);

View File

@ -5,7 +5,8 @@ CREATE TABLE wide_to_comp (a Int, b Int, c Int)
settings vertical_merge_algorithm_min_rows_to_activate = 1,
vertical_merge_algorithm_min_columns_to_activate = 1,
min_bytes_for_wide_part = 0,
min_rows_for_wide_part = 0;
min_rows_for_wide_part = 0,
index_granularity = 8192, index_granularity_bytes = '10Mi';
SYSTEM STOP merges wide_to_comp;

View File

@ -1,7 +1,7 @@
drop table if exists t_distinct_limit;
create table t_distinct_limit (d Date, id Int64)
engine = MergeTree partition by toYYYYMM(d) order by d;
engine = MergeTree partition by toYYYYMM(d) order by d SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
set max_threads = 10;

View File

@ -20,7 +20,7 @@ create table morton_numbers_mask_02457(
n4 UInt8
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into morton_numbers_mask_02457
select n1.number, n2.number, n3.number, n4.number
@ -37,7 +37,7 @@ create table morton_numbers_mask_1_02457(
n4 UInt64
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into morton_numbers_mask_1_02457
select untuple(mortonDecode((1,2,1,2), mortonEncode((1,2,1,2), n1, n2, n3, n4)))
@ -64,7 +64,7 @@ create table morton_numbers_mask_02457(
n2 UInt8
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into morton_numbers_mask_02457
select n1.number, n2.number
@ -77,7 +77,7 @@ create table morton_numbers_mask_2_02457(
n2 UInt64
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into morton_numbers_mask_2_02457
select untuple(mortonDecode((1,4), mortonEncode((1,4), n1, n2)))
@ -105,7 +105,7 @@ create table morton_numbers_mask_02457(
n3 UInt8,
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into morton_numbers_mask_02457
select n1.number, n2.number, n3.number
@ -120,7 +120,7 @@ create table morton_numbers_mask_3_02457(
n3 UInt64
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into morton_numbers_mask_3_02457
select untuple(mortonDecode((1,1,2), mortonEncode((1,1,2), n1, n2, n3)))

View File

@ -1,5 +1,5 @@
drop table if exists test_table;
CREATE TABLE test_table (string_value String) ENGINE = MergeTree ORDER BY string_value;
CREATE TABLE test_table (string_value String) ENGINE = MergeTree ORDER BY string_value SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
system stop merges test_table;
insert into test_table select * from (
select 'test_value_1'

View File

@ -7,7 +7,7 @@ CREATE TABLE t_update_empty_nested
)
ENGINE = MergeTree
ORDER BY id
SETTINGS min_bytes_for_wide_part = 0;
SETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192, index_granularity_bytes = '10Mi';
SET mutations_sync = 2;

View File

@ -2,7 +2,7 @@ set mutations_sync = 2;
drop table if exists t_delete_skip_index;
create table t_delete_skip_index (x UInt32, y String, index i y type minmax granularity 3) engine = MergeTree order by tuple();
create table t_delete_skip_index (x UInt32, y String, index i y type minmax granularity 3) engine = MergeTree order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t_delete_skip_index select number, toString(number) from numbers(8192 * 10);
select count() from t_delete_skip_index where y in (4, 5);
@ -12,7 +12,7 @@ select count() from t_delete_skip_index where y in (4, 5);
drop table if exists t_delete_skip_index;
drop table if exists t_delete_projection;
create table t_delete_projection (x UInt32, y UInt64, projection p (select sum(y))) engine = MergeTree order by tuple();
create table t_delete_projection (x UInt32, y UInt64, projection p (select sum(y))) engine = MergeTree order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t_delete_projection select number, toString(number) from numbers(8192 * 10);
select sum(y) from t_delete_projection settings optimize_use_projections = 0;