Merge pull request #53378 from ClickHouse/inhibit-randomization-in-more-tests

Inhibit randomization in more tests
This commit is contained in:
Alexey Milovidov 2023-08-14 08:01:35 +03:00 committed by GitHub
commit 06a3caf7d6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
72 changed files with 145 additions and 140 deletions

View File

@ -13,13 +13,13 @@ SET group_by_two_level_threshold = 100000;
SET max_bytes_before_external_group_by = '1Mi';
-- method: key_string & key_string_two_level
CREATE TABLE t_00284_str(s String) ENGINE = MergeTree() ORDER BY tuple();
CREATE TABLE t_00284_str(s String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6);
INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6);
SELECT s, count() FROM t_00284_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42;
-- method: low_cardinality_key_string & low_cardinality_key_string_two_level
CREATE TABLE t_00284_lc_str(s LowCardinality(String)) ENGINE = MergeTree() ORDER BY tuple();
CREATE TABLE t_00284_lc_str(s LowCardinality(String)) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6);
INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6);
SELECT s, count() FROM t_00284_lc_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42;

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 nested;
CREATE TABLE nested (x UInt64, filter UInt8, n Nested(a UInt64)) ENGINE = MergeTree ORDER BY x;
CREATE TABLE nested (x UInt64, filter UInt8, n Nested(a UInt64)) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO nested SELECT number, number % 2, range(number % 10) FROM system.numbers LIMIT 100000;
ALTER TABLE nested ADD COLUMN n.b Array(UInt64);

View File

@ -1,6 +1,6 @@
DROP TABLE IF EXISTS count;
CREATE TABLE count (x UInt64) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE count (x UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO count SELECT * FROM numbers(1234567);
SELECT count() FROM count;

View File

@ -1,6 +1,5 @@
drop table if exists lc_dict_reading;
create table lc_dict_reading (val UInt64, str StringWithDictionary, pat String) engine = MergeTree order by val;
create table lc_dict_reading (val UInt64, str StringWithDictionary, pat String) engine = MergeTree order by val SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into lc_dict_reading select number, if(number < 8192 * 4, number % 100, number) as s, s from system.numbers limit 1000000;
select sum(toUInt64(str)), sum(toUInt64(pat)) from lc_dict_reading where val < 8129 or val > 8192 * 4;
drop table if exists lc_dict_reading;

View File

@ -8,8 +8,8 @@ select 'MergeTree';
drop table if exists lc_small_dict;
drop table if exists lc_big_dict;
create table lc_small_dict (str StringWithDictionary) engine = MergeTree order by str;
create table lc_big_dict (str StringWithDictionary) engine = MergeTree order by str;
create table lc_small_dict (str StringWithDictionary) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
create table lc_big_dict (str StringWithDictionary) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into lc_small_dict select toString(number % 1000) from system.numbers limit 1000000;
insert into lc_big_dict select toString(number) from system.numbers limit 1000000;
@ -25,4 +25,3 @@ select sum(toUInt64OrZero(str)) from lc_big_dict;
drop table if exists lc_small_dict;
drop table if exists lc_big_dict;

View File

@ -13,7 +13,7 @@ uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase())
echo "DROP TABLE IF EXISTS tab_00738 SYNC;
DROP TABLE IF EXISTS mv SYNC;
CREATE TABLE tab_00738(a Int) ENGINE = MergeTree() ORDER BY a;
CREATE TABLE tab_00738(a Int) ENGINE = MergeTree() ORDER BY a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
-- The matview will take at least 2 seconds to be finished (10000000 * 0.0000002)
CREATE MATERIALIZED VIEW mv UUID '$uuid' ENGINE = Log AS SELECT sleepEachRow(0.0000002) FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n
@ -63,4 +63,3 @@ drop_inner_id
wait
drop_at_exit

View File

@ -6,14 +6,14 @@ CREATE TABLE IF NOT EXISTS test_move_partition_src (
val UInt32
) Engine = MergeTree()
PARTITION BY pk
ORDER BY (pk, val);
ORDER BY (pk, val) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE IF NOT EXISTS test_move_partition_dest (
pk UInt8,
val UInt32
) Engine = MergeTree()
PARTITION BY pk
ORDER BY (pk, val);
ORDER BY (pk, val) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO test_move_partition_src SELECT number % 2, number FROM system.numbers LIMIT 10000000;

View File

@ -2,7 +2,7 @@
DROP TABLE IF EXISTS topk;
CREATE TABLE topk (val1 String, val2 UInt32) ENGINE = MergeTree ORDER BY val1;
CREATE TABLE topk (val1 String, val2 UInt32) ENGINE = MergeTree ORDER BY val1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO topk WITH number % 7 = 0 AS frequent SELECT toString(frequent ? number % 10 : number), frequent ? 999999999 : number FROM numbers(4000000);

View File

@ -59,7 +59,8 @@ function thread6()
CREATE TABLE alter_table_$REPLICA (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16))
ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$REPLICA') ORDER BY a PARTITION BY b % 10
SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0,
cleanup_thread_preferred_points_per_iteration=0, replicated_max_mutations_in_one_entry = $(($RANDOM / 50));";
cleanup_thread_preferred_points_per_iteration=0, replicated_max_mutations_in_one_entry = $(($RANDOM / 50)),
index_granularity = 8192, index_granularity_bytes = '10Mi';";
sleep 0.$RANDOM;
done
}

View File

@ -3,7 +3,7 @@
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);
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) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
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);
select 'count';

View File

@ -22,7 +22,7 @@ CREATE TABLE IF NOT EXISTS test_01035_avg (
d64 Decimal64(18) DEFAULT toDecimal64(u64 / 1000000, 8),
d128 Decimal128(20) DEFAULT toDecimal128(i128 / 100000, 20),
d256 Decimal256(40) DEFAULT toDecimal256(i256 / 100000, 40)
) ENGINE = MergeTree() ORDER BY i64;
) ENGINE = MergeTree() ORDER BY i64 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
SELECT avg(i8), avg(i16), avg(i32), avg(i64), avg(i128), avg(i256),
avg(u8), avg(u16), avg(u32), avg(u64), avg(u128), avg(u256),

View File

@ -21,7 +21,7 @@ function wait_mutation_to_start()
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS table_for_mutations"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE table_for_mutations(k UInt32, v1 UInt64) ENGINE MergeTree ORDER BY k PARTITION BY modulo(k, 2)"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE table_for_mutations(k UInt32, v1 UInt64) ENGINE MergeTree ORDER BY k PARTITION BY modulo(k, 2) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'"
${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES table_for_mutations"
@ -48,7 +48,7 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS table_for_mutations"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS replicated_table_for_mutations"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE replicated_table_for_mutations(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/replicated_table_for_mutations', '1') ORDER BY k PARTITION BY modulo(k, 2)"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE replicated_table_for_mutations(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/replicated_table_for_mutations', '1') ORDER BY k PARTITION BY modulo(k, 2) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'"
${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES replicated_table_for_mutations"

View File

@ -5,9 +5,9 @@ DROP TABLE IF EXISTS table_for_synchronous_mutations2;
SELECT 'Replicated';
CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '1') ORDER BY k;
CREATE TABLE table_for_synchronous_mutations1(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '1') ORDER BY k SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '2') ORDER BY k;
CREATE TABLE table_for_synchronous_mutations2(k UInt32, v1 UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_01049/table_for_synchronous_mutations', '2') ORDER BY k SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO table_for_synchronous_mutations1 select number, number from numbers(100000);
@ -29,7 +29,7 @@ SELECT 'Normal';
DROP TABLE IF EXISTS table_for_synchronous_mutations_no_replication;
CREATE TABLE table_for_synchronous_mutations_no_replication(k UInt32, v1 UInt64) ENGINE MergeTree ORDER BY k;
CREATE TABLE table_for_synchronous_mutations_no_replication(k UInt32, v1 UInt64) ENGINE MergeTree ORDER BY k SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO table_for_synchronous_mutations_no_replication select number, number from numbers(100000);

View File

@ -1,7 +1,7 @@
-- Tags: no-parallel
DROP TABLE IF EXISTS test;
CREATE TABLE test Engine = MergeTree ORDER BY number AS SELECT number, toString(rand()) x from numbers(10000000);
CREATE TABLE test Engine = MergeTree ORDER BY number SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi' AS SELECT number, toString(rand()) x from numbers(10000000);
SELECT count() FROM test;

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS movement"
$CLICKHOUSE_CLIENT -n --query "CREATE TABLE movement (date DateTime('Asia/Istanbul')) Engine = MergeTree ORDER BY (toStartOfHour(date));"
$CLICKHOUSE_CLIENT -n --query "CREATE TABLE movement (date DateTime('Asia/Istanbul')) Engine = MergeTree ORDER BY (toStartOfHour(date)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';"
$CLICKHOUSE_CLIENT --query "insert into movement select toDateTime('2020-01-22 00:00:00', 'Asia/Istanbul') + number%(23*3600) from numbers(1000000);"

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

@ -15,7 +15,7 @@ done
for i in $(seq $REPLICAS); do
$CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_add_drop_$i (key UInt64, value0 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_add_drop_column', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000"
$CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_add_drop_$i (key UInt64, value0 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_add_drop_column', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192, index_granularity_bytes = '10Mi'"
done
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_add_drop_1 SELECT number, number + 10 from numbers(100000)"

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS pk_func;
CREATE TABLE pk_func(d DateTime, ui UInt32) ENGINE = MergeTree ORDER BY toDate(d);
CREATE TABLE pk_func(d DateTime, ui UInt32) ENGINE = MergeTree ORDER BY toDate(d) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO pk_func SELECT '2020-05-05 01:00:00', number FROM numbers(1000000);
INSERT INTO pk_func SELECT '2020-05-06 01:00:00', number FROM numbers(1000000);
@ -10,7 +10,7 @@ SELECT * FROM pk_func ORDER BY toDate(d), ui LIMIT 5;
DROP TABLE pk_func;
DROP TABLE IF EXISTS nORX;
CREATE TABLE nORX (`A` Int64, `B` Int64, `V` Int64) ENGINE = MergeTree ORDER BY (A, negate(B));
CREATE TABLE nORX (`A` Int64, `B` Int64, `V` Int64) ENGINE = MergeTree ORDER BY (A, negate(B)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO nORX SELECT 111, number, number FROM numbers(10000000);
SELECT *

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "drop table if exists huge_strings"
$CLICKHOUSE_CLIENT -q "create table huge_strings (n UInt64, l UInt64, s String, h UInt64) engine=MergeTree order by n"
$CLICKHOUSE_CLIENT -q "create table huge_strings (n UInt64, l UInt64, s String, h UInt64) engine=MergeTree order by n SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'"
# Timeouts are increased, because test can be slow with sanitizers and parallel runs.

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS null_in;
CREATE TABLE null_in (dt DateTime, idx int, i Nullable(int), s Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx;
CREATE TABLE null_in (dt DateTime, idx int, i Nullable(int), s Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO null_in VALUES (1, 1, 1, '1') (2, 2, NULL, NULL) (3, 3, 3, '3') (4, 4, NULL, NULL) (5, 5, 5, '5');
@ -81,7 +81,7 @@ DROP TABLE IF EXISTS null_in;
DROP TABLE IF EXISTS null_in_subquery;
CREATE TABLE null_in_subquery (dt DateTime, idx int, i Nullable(UInt64)) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx;
CREATE TABLE null_in_subquery (dt DateTime, idx int, i Nullable(UInt64)) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO null_in_subquery SELECT number % 3, number, number FROM system.numbers LIMIT 99999;
SELECT count() == 33333 FROM null_in_subquery WHERE i in (SELECT i FROM null_in_subquery WHERE dt = 0);
@ -111,7 +111,7 @@ DROP TABLE IF EXISTS null_in_subquery;
DROP TABLE IF EXISTS null_in_tuple;
CREATE TABLE null_in_tuple (dt DateTime, idx int, t Tuple(Nullable(UInt64), Nullable(String))) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx;
CREATE TABLE null_in_tuple (dt DateTime, idx int, t Tuple(Nullable(UInt64), Nullable(String))) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO null_in_tuple VALUES (1, 1, (1, '1')) (2, 2, (2, NULL)) (3, 3, (NULL, '3')) (4, 4, (NULL, NULL))
SET transform_null_in = 0;

View File

@ -1,6 +1,6 @@
DROP TABLE IF EXISTS ES;
create table ES(A String) Engine=MergeTree order by tuple();
create table ES(A String) Engine=MergeTree order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into ES select toString(number) from numbers(10000000);
SET max_execution_time = 100,

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 @@ CREATE TABLE t_01411(
str LowCardinality(String),
arr Array(LowCardinality(String)) default [str]
) ENGINE = MergeTree()
ORDER BY tuple();
ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t_01411 (str) SELECT concat('asdf', toString(number % 10000)) FROM numbers(1000000);
@ -24,7 +24,7 @@ CREATE TABLE t_01411_num(
num UInt8,
arr Array(LowCardinality(Int64)) default [num]
) ENGINE = MergeTree()
ORDER BY tuple();
ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t_01411_num (num) SELECT number % 1000 FROM numbers(1000000);

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

@ -9,14 +9,14 @@ drop table if exists table_map;
drop table if exists table_map;
create table table_map (a Map(String, UInt64)) engine = MergeTree() order by a;
create table table_map (a Map(String, UInt64)) engine = MergeTree() order by a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into table_map select map('key1', number, 'key2', number * 2) from numbers(1111, 3);
select a['key1'], a['key2'] from table_map;
drop table if exists table_map;
-- MergeTree Engine
drop table if exists table_map;
create table table_map (a Map(String, String), b String) engine = MergeTree() order by a;
create table table_map (a Map(String, String), b String) engine = MergeTree() order by a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into table_map values ({'name':'zhangsan', 'gender':'male'}, 'name'), ({'name':'lisi', 'gender':'female'}, 'gender');
select a[b] from table_map;
select b from table_map where a = map('name','lisi', 'gender', 'female');
@ -24,21 +24,21 @@ drop table if exists table_map;
-- Big Integer type
create table table_map (d DATE, m Map(Int8, UInt256)) ENGINE = MergeTree() order by d;
create table table_map (d DATE, m Map(Int8, UInt256)) ENGINE = MergeTree() order by d SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into table_map values ('2020-01-01', map(1, 0, 2, 1));
select * from table_map;
drop table table_map;
-- Integer type
create table table_map (d DATE, m Map(Int8, Int8)) ENGINE = MergeTree() order by d;
create table table_map (d DATE, m Map(Int8, Int8)) ENGINE = MergeTree() order by d SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into table_map values ('2020-01-01', map(1, 0, 2, -1));
select * from table_map;
drop table table_map;
-- Unsigned Int type
drop table if exists table_map;
create table table_map(a Map(UInt8, UInt64), b UInt8) Engine = MergeTree() order by b;
create table table_map(a Map(UInt8, UInt64), b UInt8) Engine = MergeTree() order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into table_map select map(number, number+5), number from numbers(1111,4);
select a[b] from table_map;
drop table if exists table_map;
@ -46,7 +46,7 @@ drop table if exists table_map;
-- Array Type
drop table if exists table_map;
create table table_map(a Map(String, Array(UInt8))) Engine = MergeTree() order by a;
create table table_map(a Map(String, Array(UInt8))) Engine = MergeTree() order by a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into table_map values(map('k1', [1,2,3], 'k2', [4,5,6])), (map('k0', [], 'k1', [100,20,90]));
insert into table_map select map('k1', [number, number + 2, number * 2]) from numbers(6);
insert into table_map select map('k2', [number, number + 2, number * 2]) from numbers(6);
@ -56,7 +56,7 @@ drop table if exists table_map;
SELECT CAST(([1, 2, 3], ['1', '2', 'foo']), 'Map(UInt8, String)') AS map, map[1];
CREATE TABLE table_map (n UInt32, m Map(String, Int))
ENGINE = MergeTree ORDER BY n SETTINGS min_bytes_for_wide_part = 0;
ENGINE = MergeTree ORDER BY n SETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192, index_granularity_bytes = '10Mi';
-- coversion from Tuple(Array(K), Array(V))
INSERT INTO table_map SELECT number, (arrayMap(x -> toString(x), range(number % 10 + 2)), range(number % 10 + 2)) FROM numbers(100000);
@ -67,7 +67,7 @@ SELECT sum(m['1']), sum(m['7']), sum(m['100']) FROM table_map;
DROP TABLE IF EXISTS table_map;
CREATE TABLE table_map (n UInt32, m Map(String, Int))
ENGINE = MergeTree ORDER BY n;
ENGINE = MergeTree ORDER BY n SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
-- coversion from Tuple(Array(K), Array(V))
INSERT INTO table_map SELECT number, (arrayMap(x -> toString(x), range(number % 10 + 2)), range(number % 10 + 2)) FROM numbers(100000);

View File

@ -7,7 +7,7 @@ set max_insert_threads = 4;
create table stack(item_id Int64, brand_id Int64, rack_id Int64, dt DateTime, expiration_dt DateTime, quantity UInt64)
Engine = MergeTree
partition by toYYYYMM(dt)
order by (brand_id, toStartOfHour(dt));
order by (brand_id, toStartOfHour(dt)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into stack
select number%99991, number%11, number%1111, toDateTime('2020-01-01 00:00:00')+number/100,

View File

@ -35,7 +35,7 @@ $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --
# InMemory - [0..5]
# Compact - (5..10]
# Wide - >10
$CLICKHOUSE_CLIENT --query="CREATE TABLE data_01600 (part_type String, key Int) ENGINE = MergeTree PARTITION BY part_type ORDER BY key SETTINGS min_bytes_for_wide_part=0, min_rows_for_wide_part=10"
$CLICKHOUSE_CLIENT --query="CREATE TABLE data_01600 (part_type String, key Int) ENGINE = MergeTree PARTITION BY part_type ORDER BY key SETTINGS min_bytes_for_wide_part=0, min_rows_for_wide_part=10, index_granularity = 8192, index_granularity_bytes = '10Mi'"
# InMemory
$CLICKHOUSE_CLIENT --query="INSERT INTO data_01600 SELECT 'InMemory', number FROM system.numbers LIMIT 1"

View File

@ -5,7 +5,7 @@ set enable_filesystem_cache=0;
set enable_filesystem_cache_on_write_operations=0;
drop table if exists t;
create table t (x UInt64, s String) engine = MergeTree order by x;
create table t (x UInt64, s String) engine = MergeTree order by x SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t SELECT
number,
if(number < (8129 * 1024), arrayStringConcat(arrayMap(x -> toString(x), range(number % 128)), ' '), '')

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

@ -23,12 +23,12 @@ select * from tbl WHERE indexHint(p in (select toInt64(number) - 2 from numbers(
0 3 0
drop table tbl;
drop table if exists XXXX;
create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=128;
create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=128, index_granularity_bytes = '10Mi';
insert into XXXX select number*60, 0 from numbers(100000);
SELECT sum(t) FROM XXXX WHERE indexHint(t = 42);
487680
drop table if exists XXXX;
create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192;
create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192, index_granularity_bytes = '10Mi';
insert into XXXX select number*60, 0 from numbers(100000);
SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1;
100000

View File

@ -18,7 +18,7 @@ drop table tbl;
drop table if exists XXXX;
create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=128;
create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=128, index_granularity_bytes = '10Mi';
insert into XXXX select number*60, 0 from numbers(100000);
@ -26,7 +26,7 @@ SELECT sum(t) FROM XXXX WHERE indexHint(t = 42);
drop table if exists XXXX;
create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192;
create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192, index_granularity_bytes = '10Mi';
insert into XXXX select number*60, 0 from numbers(100000);

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS ttt01746;
CREATE TABLE ttt01746 (d Date, n UInt64) ENGINE = MergeTree() PARTITION BY toMonday(d) ORDER BY n;
CREATE TABLE ttt01746 (d Date, n UInt64) ENGINE = MergeTree() PARTITION BY toMonday(d) ORDER BY n SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO ttt01746 SELECT toDate('2021-02-14') + (number % 30) AS d, number AS n FROM numbers(1500000);
set optimize_move_to_prewhere=0;
SELECT arraySort(x -> x.2, [tuple('a', 10)]) AS X FROM ttt01746 WHERE d >= toDate('2021-03-03') - 2 ORDER BY n LIMIT 1;

View File

@ -9,6 +9,7 @@ CREATE TABLE Test
ENGINE = MergeTree()
PRIMARY KEY (String1,String2)
ORDER BY (String1,String2)
SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'
AS
SELECT
'String1_' || toString(number) as String1,
@ -39,15 +40,15 @@ DROP TABLE IF EXISTS Test;
select x, y from (select [0, 1, 2] as y, 1 as a, 2 as b) array join y as x where a = 1 and b = 2 and (x = 1 or x != 1) and x = 1;
DROP TABLE IF EXISTS t;
create table t(a UInt8) engine=MergeTree order by a;
create table t(a UInt8) engine=MergeTree order by a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t select * from numbers(2);
select a from t t1 join t t2 on t1.a = t2.a where t1.a;
DROP TABLE IF EXISTS t;
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id;
CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time;
CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t1 values (101, '2023-05-28 00:00:00'), (102, '2023-05-28 00:00:00');
insert into t2 values ('2023-05-31 00:00:00');

View File

@ -1,6 +1,6 @@
-- Tags: no-parallel, long
DROP TABLE IF EXISTS bloom_filter_null_array;
CREATE TABLE bloom_filter_null_array (v Array(Int32), INDEX idx v TYPE bloom_filter GRANULARITY 3) ENGINE = MergeTree() ORDER BY v;
CREATE TABLE bloom_filter_null_array (v Array(Int32), INDEX idx v TYPE bloom_filter GRANULARITY 3) ENGINE = MergeTree() ORDER BY v SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO bloom_filter_null_array SELECT [number] FROM numbers(10000000);
SELECT COUNT() FROM bloom_filter_null_array;
SELECT COUNT() FROM bloom_filter_null_array WHERE has(v, 0);

View File

@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t_sparse;
CREATE TABLE t_sparse (id UInt64, u UInt64, s String)
ENGINE = MergeTree ORDER BY id
SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9;
SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9, index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t_sparse SELECT
number,

View File

@ -7,7 +7,7 @@ SET allow_experimental_object_type = 1;
CREATE TABLE t_json_sparse (data JSON)
ENGINE = MergeTree ORDER BY tuple()
SETTINGS ratio_of_defaults_for_sparse_serialization = 0.1,
min_bytes_for_wide_part = 0;
min_bytes_for_wide_part = 0, index_granularity = 8192, index_granularity_bytes = '10Mi';
SYSTEM STOP MERGES t_json_sparse;

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test(a Int, b Int) Engine=ReplacingMergeTree order by a;
CREATE TABLE test(a Int, b Int) Engine=ReplacingMergeTree order by a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO test select number, number from numbers(5);
INSERT INTO test select number, number from numbers(5,2);
set max_threads =1;

View File

@ -8,6 +8,6 @@ select count() as c, x in ('a', 'bb') as g from tab group by g order by c;
drop table if exists tab;
-- https://github.com/ClickHouse/ClickHouse/issues/44503
CREATE TABLE test(key Int32) ENGINE = MergeTree ORDER BY (key);
CREATE TABLE test(key Int32) ENGINE = MergeTree ORDER BY (key) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into test select intDiv(number,100) from numbers(10000000);
SELECT COUNT() FROM test WHERE key <= 100000 AND (NOT (toLowCardinality('') IN (SELECT '')));

View File

@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS d"
${CLICKHOUSE_CLIENT} --query "CREATE TABLE d (oid UInt64) ENGINE = MergeTree ORDER BY oid"
${CLICKHOUSE_CLIENT} --query "CREATE TABLE d (oid UInt64) ENGINE = MergeTree ORDER BY oid SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'"
${CLICKHOUSE_CLIENT} --min_insert_block_size_rows 0 --min_insert_block_size_bytes 0 --max_block_size 8192 --query "insert into d select * from numbers(1000000)"
# In previous ClickHouse versions there was a mistake that makes quantileDeterministic functions not really deterministic (in edge cases).

View File

@ -6,15 +6,18 @@ DROP TABLE IF EXISTS partslost_2;
CREATE TABLE partslost_0 (x String) ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}_02067_lost/partslost', '0') ORDER BY tuple()
SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, old_parts_lifetime = 1,
cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0;
cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0,
index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE partslost_1 (x String) ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}_02067_lost/partslost', '1') ORDER BY tuple()
SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, old_parts_lifetime = 1,
cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0;
cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0,
index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE partslost_2 (x String) ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}_02067_lost/partslost', '2') ORDER BY tuple()
SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, old_parts_lifetime = 1,
cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0;
cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0,
index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO partslost_0 SELECT toString(number) AS x from system.numbers LIMIT 10000;

View File

@ -5,7 +5,7 @@ SET read_in_order_two_level_merge_threshold=100;
DROP TABLE IF EXISTS t_read_in_order;
CREATE TABLE t_read_in_order(date Date, i UInt64, v UInt64)
ENGINE = MergeTree ORDER BY (date, i);
ENGINE = MergeTree ORDER BY (date, i) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t_read_in_order SELECT '2020-10-10', number % 10, number FROM numbers(100000);
INSERT INTO t_read_in_order SELECT '2020-10-11', number % 10, number FROM numbers(100000);
@ -55,7 +55,7 @@ SELECT a, b FROM t_read_in_order WHERE a = 1 ORDER BY b DESC SETTINGS read_in_or
DROP TABLE t_read_in_order;
CREATE TABLE t_read_in_order(dt DateTime, d Decimal64(5), v UInt64)
ENGINE = MergeTree ORDER BY (toStartOfDay(dt), d);
ENGINE = MergeTree ORDER BY (toStartOfDay(dt), d) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t_read_in_order SELECT toDateTime('2020-10-10 00:00:00') + number, 1 / (number % 100 + 1), number FROM numbers(1000);

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_index_hypothesis"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_index_hypothesis (a UInt32, b UInt32, INDEX t a != b TYPE hypothesis GRANULARITY 1) ENGINE = MergeTree ORDER BY a"
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_index_hypothesis (a UInt32, b UInt32, INDEX t a != b TYPE hypothesis GRANULARITY 1) ENGINE = MergeTree ORDER BY a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'"
$CLICKHOUSE_CLIENT -q "INSERT INTO t_index_hypothesis SELECT number, number + 1 FROM numbers(10000000)"

View File

@ -17,9 +17,9 @@ prepare_table() {
table_name="t_hash_table_sizes_stats_$RANDOM$RANDOM"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $table_name;"
if [ -z "$1" ]; then
$CLICKHOUSE_CLIENT -q "CREATE TABLE $table_name(number UInt64) Engine=MergeTree() ORDER BY tuple();"
$CLICKHOUSE_CLIENT -q "CREATE TABLE $table_name(number UInt64) Engine=MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';"
else
$CLICKHOUSE_CLIENT -q "CREATE TABLE $table_name(number UInt64) Engine=MergeTree() ORDER BY $1;"
$CLICKHOUSE_CLIENT -q "CREATE TABLE $table_name(number UInt64) Engine=MergeTree() ORDER BY $1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';"
fi
$CLICKHOUSE_CLIENT -q "SYSTEM STOP MERGES $table_name;"
for ((i = 1; i <= max_threads; i++)); do

View File

@ -19,9 +19,9 @@ prepare_table() {
table_name="t_hash_table_sizes_stats_$RANDOM$RANDOM"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $table_name;"
if [ -z "$1" ]; then
$CLICKHOUSE_CLIENT -q "CREATE TABLE $table_name(number UInt64) Engine=MergeTree() ORDER BY tuple();"
$CLICKHOUSE_CLIENT -q "CREATE TABLE $table_name(number UInt64) Engine=MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';"
else
$CLICKHOUSE_CLIENT -q "CREATE TABLE $table_name(number UInt64) Engine=MergeTree() ORDER BY $1;"
$CLICKHOUSE_CLIENT -q "CREATE TABLE $table_name(number UInt64) Engine=MergeTree() ORDER BY $1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';"
fi
$CLICKHOUSE_CLIENT -q "SYSTEM STOP MERGES $table_name;"
for ((i = 1; i <= max_threads; i++)); do

View File

@ -1,6 +1,6 @@
-- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug
drop table if exists tab_lc;
CREATE TABLE tab_lc (x UInt64, y LowCardinality(String)) engine = MergeTree order by x;
CREATE TABLE tab_lc (x UInt64, y LowCardinality(String)) engine = MergeTree order by x SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into tab_lc select number, toString(number % 10) from numbers(20000000);
optimize table tab_lc;
select count() from tab_lc where y == '0' settings local_filesystem_read_prefetch=1;

View File

@ -5,8 +5,8 @@ SET max_threads=0;
DROP TABLE IF EXISTS left;
DROP TABLE IF EXISTS right;
CREATE TABLE left ( key UInt32, value String ) ENGINE = MergeTree ORDER BY key;
CREATE TABLE right ( key UInt32, value String ) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE left ( key UInt32, value String ) ENGINE = MergeTree ORDER BY key SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE right ( key UInt32, value String ) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO left SELECT number, toString(number) FROM numbers(25367182);
INSERT INTO right SELECT number, toString(number) FROM numbers(23124707);

View File

@ -1,8 +1,8 @@
DROP TABLE IF EXISTS ev;
DROP TABLE IF EXISTS idx;
CREATE TABLE ev (a Int32, b Int32) Engine=MergeTree() ORDER BY a;
CREATE TABLE idx (a Int32) Engine=MergeTree() ORDER BY a;
CREATE TABLE ev (a Int32, b Int32) Engine=MergeTree() ORDER BY a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE idx (a Int32) Engine=MergeTree() ORDER BY a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO ev SELECT number, number FROM numbers(10000000);
INSERT INTO idx SELECT number * 5 FROM numbers(1000);

View File

@ -12,7 +12,8 @@ CREATE TABLE t_1
)
ENGINE = MergeTree
PARTITION BY toYYYYMM(p_time)
ORDER BY order_0;
ORDER BY order_0
SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE t_random_1
(

View File

@ -2,8 +2,8 @@
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
CREATE TABLE t1 (key UInt32, s String) ENGINE = MergeTree ORDER BY key;
CREATE TABLE t2 (key UInt32, s String) ENGINE = MergeTree ORDER BY key;
CREATE TABLE t1 (key UInt32, s String) ENGINE = MergeTree ORDER BY key SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE t2 (key UInt32, s String) ENGINE = MergeTree ORDER BY key SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
{% set ltable_size = 10000000 -%}
{% set rtable_size = 1000000 -%}

View File

@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test_table;
CREATE TABLE test_table (`n` UInt64, `s` String)
ENGINE = MergeTree
PRIMARY KEY n ORDER BY n;
PRIMARY KEY n ORDER BY n SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO test_table SELECT number, concat('some string ', CAST(number, 'String')) FROM numbers(1000000);

View File

@ -5,7 +5,8 @@ DROP TABLE IF EXISTS t_sparse_s3;
CREATE TABLE t_sparse_s3 (id UInt32, cond UInt8, s String)
engine = MergeTree ORDER BY id
settings ratio_of_defaults_for_sparse_serialization = 0.01, storage_policy = 's3_cache',
min_bytes_for_wide_part = 0, min_compress_block_size = 1;
min_bytes_for_wide_part = 0, min_compress_block_size = 1,
index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t_sparse_s3 SELECT 1, number % 2, '' FROM numbers(8192);
INSERT INTO t_sparse_s3 SELECT 2, number % 2, '' FROM numbers(24576);

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

@ -2,7 +2,7 @@
DROP TABLE IF EXISTS t_2354_dist_with_external_aggr;
create table t_2354_dist_with_external_aggr(a UInt64, b String, c FixedString(100)) engine = MergeTree order by tuple();
create table t_2354_dist_with_external_aggr(a UInt64, b String, c FixedString(100)) engine = MergeTree order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t_2354_dist_with_external_aggr select number, toString(number) as s, toFixedString(s, 100) from numbers_mt(5e7);

View File

@ -9,7 +9,7 @@ CREATE TABLE t_modify_from_lc_1
a LowCardinality(UInt32) CODEC(NONE)
)
ENGINE = MergeTree ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0;
SETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE t_modify_from_lc_2
(
@ -17,7 +17,7 @@ CREATE TABLE t_modify_from_lc_2
a LowCardinality(UInt32) CODEC(NONE)
)
ENGINE = MergeTree ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0;
SETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO t_modify_from_lc_1 SELECT number, number FROM numbers(100000);
INSERT INTO t_modify_from_lc_2 SELECT number, number FROM numbers(100000);

View File

@ -21,7 +21,7 @@ n8 UInt64,
n9 UInt64
)
ENGINE = MergeTree
ORDER BY n0 SETTINGS min_bytes_for_wide_part = 1;"
ORDER BY n0 SETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192, index_granularity_bytes = '10Mi';"
${CLICKHOUSE_CLIENT} -q "INSERT INTO test select number, number % 3, number % 5, number % 10, number % 13, number % 15, number % 17, number % 18, number % 22, number % 25 from numbers(1000000)"
${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES test"

View File

@ -1,7 +1,7 @@
-- From https://github.com/ClickHouse/ClickHouse/issues/41814
drop table if exists test;
create table test(a UInt64, m UInt64, d DateTime) engine MergeTree partition by toYYYYMM(d) order by (a, m, d);
create table test(a UInt64, m UInt64, d DateTime) engine MergeTree partition by toYYYYMM(d) order by (a, m, d) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into test select number, number, '2022-01-01 00:00:00' from numbers(1000000);
@ -12,7 +12,7 @@ drop table test;
-- From https://github.com/ClickHouse/ClickHouse/issues/34063
drop table if exists test_null_filter;
create table test_null_filter(key UInt64, value UInt32) engine MergeTree order by key;
create table test_null_filter(key UInt64, value UInt32) engine MergeTree order by key SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into test_null_filter select number, number from numbers(10000000);

View File

@ -11,7 +11,7 @@ create table morton_numbers_02457(
n8 UInt8
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
SELECT '----- CONST -----';
select mortonEncode(1,2,3,4);
@ -45,7 +45,7 @@ create table morton_numbers_1_02457(
n8 UInt64
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into morton_numbers_1_02457
select untuple(mortonDecode(8, mortonEncode(n1, n2, n3, n4, n5, n6, n7, n8)))
@ -80,7 +80,7 @@ create table morton_numbers_2_02457(
n4 UInt64
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into morton_numbers_2_02457
select untuple(mortonDecode(4, mortonEncode(n1, n2, n3, n4)))
@ -114,7 +114,7 @@ create table morton_numbers_3_02457(
n2 UInt64
)
Engine=MergeTree()
ORDER BY n1;
ORDER BY n1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into morton_numbers_3_02457
select untuple(mortonDecode(2, mortonEncode(n1, n2)))

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,7 +1,7 @@
DROP TABLE IF EXISTS 02481_mergetree;
DROP TABLE IF EXISTS 02481_merge;
CREATE TABLE 02481_mergetree(x UInt64, y UInt64, arr Array(String)) ENGINE = MergeTree ORDER BY x SAMPLE BY x;
CREATE TABLE 02481_mergetree(x UInt64, y UInt64, arr Array(String)) ENGINE = MergeTree ORDER BY x SAMPLE BY x SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
CREATE TABLE 02481_merge(x UInt64, y UInt64, arr Array(String)) ENGINE = Merge(currentDatabase(), '^(02481_mergetree)$');

View File

@ -10,7 +10,7 @@ CREATE TABLE gen
)
ENGINE = GenerateRandom;
CREATE TABLE github_events AS gen ENGINE=MergeTree ORDER BY (event_type, repo_name, created_at);
CREATE TABLE github_events AS gen ENGINE=MergeTree ORDER BY (event_type, repo_name, created_at) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO github_events SELECT * FROM gen LIMIT 100000;

View File

@ -18,7 +18,7 @@ select count() from (select throwIf(count() != 2) from t1 group by a);
drop table t1;
create table t2(a UInt32) engine=MergeTree order by tuple() partition by a % 8;
create table t2(a UInt32) engine=MergeTree order by tuple() partition by a % 8 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
system stop merges t2;
@ -31,7 +31,7 @@ select count() from (select throwIf(count() != 2) from t2 group by a);
drop table t2;
create table t3(a UInt32) engine=MergeTree order by tuple() partition by a % 16;
create table t3(a UInt32) engine=MergeTree order by tuple() partition by a % 16 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
system stop merges t3;
@ -53,7 +53,7 @@ drop table t3;
set optimize_aggregation_in_order = 1;
create table t4(a UInt32) engine=MergeTree order by a partition by a % 4;
create table t4(a UInt32) engine=MergeTree order by a partition by a % 4 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
system stop merges t4;
@ -66,7 +66,7 @@ select count() from (select throwIf(count() != 2) from t4 group by a);
drop table t4;
create table t5(a UInt32) engine=MergeTree order by a partition by a % 8;
create table t5(a UInt32) engine=MergeTree order by a partition by a % 8 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
system stop merges t5;
@ -79,7 +79,7 @@ select count() from (select throwIf(count() != 2) from t5 group by a);
drop table t5;
create table t6(a UInt32) engine=MergeTree order by a partition by a % 16;
create table t6(a UInt32) engine=MergeTree order by a partition by a % 16 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
system stop merges t6;
@ -94,7 +94,7 @@ drop table t6;
set optimize_aggregation_in_order = 0;
create table t7(a UInt32) engine=MergeTree order by a partition by intDiv(a, 2);
create table t7(a UInt32) engine=MergeTree order by a partition by intDiv(a, 2) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t7 select number from numbers_mt(100);
@ -104,7 +104,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t7;
create table t8(a UInt32) engine=MergeTree order by a partition by intDiv(a, 2) * 2 + 1;
create table t8(a UInt32) engine=MergeTree order by a partition by intDiv(a, 2) * 2 + 1 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t8 select number from numbers_mt(100);
@ -114,7 +114,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t8;
create table t9(a UInt32) engine=MergeTree order by a partition by intDiv(a, 2);
create table t9(a UInt32) engine=MergeTree order by a partition by intDiv(a, 2) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t9 select number from numbers_mt(100);
@ -124,7 +124,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t9;
create table t10(a UInt32, b UInt32) engine=MergeTree order by a partition by (intDiv(a, 2), intDiv(b, 3));
create table t10(a UInt32, b UInt32) engine=MergeTree order by a partition by (intDiv(a, 2), intDiv(b, 3)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t10 select number, number from numbers_mt(100);
@ -135,7 +135,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t10;
-- multiplication by 2 is not injective, so optimization is not applicable
create table t11(a UInt32, b UInt32) engine=MergeTree order by a partition by (intDiv(a, 2), intDiv(b, 3));
create table t11(a UInt32, b UInt32) engine=MergeTree order by a partition by (intDiv(a, 2), intDiv(b, 3)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t11 select number, number from numbers_mt(100);
@ -155,7 +155,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t12;
create table t13(a UInt32, b UInt32) engine=MergeTree order by a partition by (intDiv(a, 2), intDiv(b, 3));
create table t13(a UInt32, b UInt32) engine=MergeTree order by a partition by (intDiv(a, 2), intDiv(b, 3)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t13 select number, number from numbers_mt(100);
@ -165,7 +165,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t13;
create table t14(a UInt32, b UInt32) engine=MergeTree order by a partition by intDiv(a, 2) + intDiv(b, 3);
create table t14(a UInt32, b UInt32) engine=MergeTree order by a partition by intDiv(a, 2) + intDiv(b, 3) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t14 select number, number from numbers_mt(100);
@ -176,7 +176,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t14;
-- to few partitions --
create table t15(a UInt32, b UInt32) engine=MergeTree order by a partition by a < 90;
create table t15(a UInt32, b UInt32) engine=MergeTree order by a partition by a < 90 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t15 select number, number from numbers_mt(100);
@ -188,7 +188,7 @@ settings force_aggregate_partitions_independently = 0;
drop table t15;
-- to many partitions --
create table t16(a UInt32, b UInt32) engine=MergeTree order by a partition by a % 16;
create table t16(a UInt32, b UInt32) engine=MergeTree order by a partition by a % 16 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t16 select number, number from numbers_mt(100);
@ -200,7 +200,7 @@ settings force_aggregate_partitions_independently = 0, max_number_of_partitions_
drop table t16;
-- to big skew --
create table t17(a UInt32, b UInt32) engine=MergeTree order by a partition by a < 90;
create table t17(a UInt32, b UInt32) engine=MergeTree order by a partition by a < 90 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t17 select number, number from numbers_mt(100);
@ -211,7 +211,7 @@ settings force_aggregate_partitions_independently = 0, max_threads = 4;
drop table t17;
create table t18(a UInt32, b UInt32) engine=MergeTree order by a partition by a;
create table t18(a UInt32, b UInt32) engine=MergeTree order by a partition by a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t18 select number, number from numbers_mt(50);
@ -221,7 +221,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t18;
create table t19(a UInt32, b UInt32) engine=MergeTree order by a partition by a;
create table t19(a UInt32, b UInt32) engine=MergeTree order by a partition by a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t19 select number, number from numbers_mt(50);
@ -231,7 +231,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t19;
create table t20(a UInt32, b UInt32) engine=MergeTree order by a partition by a;
create table t20(a UInt32, b UInt32) engine=MergeTree order by a partition by a SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t20 select number, number from numbers_mt(50);
@ -241,7 +241,7 @@ select replaceRegexpOne(explain, '^[ ]*(.*)', '\\1') from (
drop table t20;
create table t21(a UInt64, b UInt64) engine=MergeTree order by a partition by a % 16;
create table t21(a UInt64, b UInt64) engine=MergeTree order by a partition by a % 16 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t21 select number, number from numbers_mt(1e6);
@ -249,7 +249,7 @@ select a from t21 group by a limit 10 format Null;
drop table t21;
create table t22(a UInt32, b UInt32) engine=SummingMergeTree order by a partition by a % 16;
create table t22(a UInt32, b UInt32) engine=SummingMergeTree order by a partition by a % 16 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into t22 select number, number from numbers_mt(1e6);

View File

@ -3,7 +3,7 @@ DROP TABLE IF EXISTS lwd_test_02521;
CREATE TABLE lwd_test_02521 (id UInt64, value String, event_time DateTime)
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';
INSERT INTO lwd_test_02521 SELECT number, randomString(10), now() - INTERVAL 2 MONTH FROM numbers(50000);
INSERT INTO lwd_test_02521 SELECT number, randomString(10), now() FROM numbers(50000);
@ -42,4 +42,4 @@ SELECT 'Count', count() FROM lwd_test_02521;
-- { echoOff }
DROP TABLE lwd_test_02521;
DROP TABLE lwd_test_02521;

View File

@ -11,7 +11,7 @@ drop table if exists bug_delta_gorilla;
create table bug_delta_gorilla
(value_bug UInt64 codec (Delta, Gorilla))
engine = MergeTree
order by tuple()
order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'
as (select 0 from numbers(30000000));
select count(*)
@ -32,7 +32,7 @@ select 'The same issue in a much smaller repro happens also in Debug builds';
create table bug_delta_gorilla (val UInt64 codec (Delta, Gorilla))
engine = MergeTree
order by val;
order by val SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into bug_delta_gorilla values (0)(1)(3);
select * from bug_delta_gorilla;

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

@ -1,6 +1,6 @@
DROP TABLE IF EXISTS 02581_trips;
CREATE TABLE 02581_trips(id UInt32, id2 UInt32, description String) ENGINE=MergeTree ORDER BY id;
CREATE TABLE 02581_trips(id UInt32, id2 UInt32, description String) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
-- Make multiple parts
INSERT INTO 02581_trips SELECT number, number, '' FROM numbers(10000);

View File

@ -13,7 +13,7 @@ CREATE TABLE fx_1m (
)
ENGINE = MergeTree()
PARTITION BY toYear(dt_close)
ORDER BY (symbol, dt_close);
ORDER BY (symbol, dt_close) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
-- add projection
ALTER TABLE fx_1m

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;

View File

@ -15,8 +15,8 @@ SELECT count() FROM test.hits WHERE EventDate IN (toDate('2014-03-18'), toDate('
SELECT count() FROM test.hits WHERE EventDate = concat('2014-0', '3-18');
DROP TABLE IF EXISTS test.hits_indexed_by_time;
CREATE TABLE test.hits_indexed_by_time (EventDate Date, EventTime DateTime('Asia/Dubai')) ENGINE = MergeTree ORDER BY (EventDate, EventTime);
INSERT INTO test.hits_indexed_by_time SELECT EventDate, EventTime FROM test.hits;
CREATE TABLE test.hits_indexed_by_time (EventDate Date, EventTime DateTime('Asia/Dubai')) ENGINE = MergeTree ORDER BY (EventDate, EventTime) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO test.hits_indexed_by_time SELECT EventDate, EventTime FROM test.hits SETTINGS max_block_size = 65000;
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime = '2014-03-18 01:02:03';
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime < '2014-03-18 01:02:03';

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS hits_none;
CREATE TABLE hits_none (Title String CODEC(NONE)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE hits_none (Title String CODEC(NONE)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO hits_none SELECT Title FROM test.hits;
SET min_bytes_to_use_mmap_io = 1;

View File

@ -4,9 +4,9 @@ drop table if exists distinct_in_order sync;
drop table if exists ordinary_distinct sync;
select '-- DISTINCT columns are the same as in ORDER BY';
create table distinct_in_order (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate);
create table distinct_in_order (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into distinct_in_order select distinct CounterID, EventDate from test.hits order by CounterID, EventDate settings optimize_distinct_in_order=1;
create table ordinary_distinct (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate);
create table ordinary_distinct (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into ordinary_distinct select distinct CounterID, EventDate from test.hits order by CounterID, EventDate settings optimize_distinct_in_order=0;
select distinct * from distinct_in_order except select * from ordinary_distinct;
@ -14,9 +14,9 @@ drop table if exists distinct_in_order sync;
drop table if exists ordinary_distinct sync;
select '-- DISTINCT columns has prefix in ORDER BY columns';
create table distinct_in_order (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate);
create table distinct_in_order (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into distinct_in_order select distinct CounterID, EventDate from test.hits order by CounterID settings optimize_distinct_in_order=1;
create table ordinary_distinct (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate);
create table ordinary_distinct (CounterID UInt32, EventDate Date) engine=MergeTree() order by (CounterID, EventDate) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into ordinary_distinct select distinct CounterID, EventDate from test.hits order by CounterID settings optimize_distinct_in_order=0;
select distinct * from distinct_in_order except select * from ordinary_distinct;