mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Inhibit index_granularity_bytes randomization in some tests
This commit is contained in:
parent
2b4f5f82da
commit
b1e0e587e6
@ -1,5 +1,5 @@
|
||||
DROP TABLE IF EXISTS size_hint;
|
||||
CREATE TABLE size_hint (s Array(String)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 1000;
|
||||
CREATE TABLE size_hint (s Array(String)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 1000, index_granularity_bytes = '10Mi';
|
||||
|
||||
SET max_block_size = 1000;
|
||||
SET max_memory_usage = 1000000000;
|
||||
|
@ -65,7 +65,7 @@ CREATE TABLE large_alter_table_00804 (
|
||||
somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)),
|
||||
id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC),
|
||||
data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4)
|
||||
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, min_bytes_for_wide_part = 0;
|
||||
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_bytes_for_wide_part = 0;
|
||||
|
||||
INSERT INTO large_alter_table_00804 SELECT toDate('2019-01-01'), number, toString(number + rand()) FROM system.numbers LIMIT 300000;
|
||||
|
||||
|
@ -114,7 +114,7 @@ CREATE TABLE compression_codec_multiple_with_key (
|
||||
somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta),
|
||||
id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta),
|
||||
data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4)
|
||||
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2;
|
||||
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
|
||||
INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!');
|
||||
|
@ -23,7 +23,7 @@ CREATE TABLE minmax_idx
|
||||
INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
@ -48,4 +48,4 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE i32 = 5 AND i32 + f64
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"
|
||||
|
@ -19,7 +19,7 @@ CREATE TABLE minmax_idx1
|
||||
idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r1')
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
CREATE TABLE minmax_idx2
|
||||
(
|
||||
@ -36,7 +36,7 @@ CREATE TABLE minmax_idx2
|
||||
idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r2')
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
|
||||
/* many small inserts => table will make merges */
|
||||
|
@ -22,7 +22,7 @@ CREATE TABLE set_idx
|
||||
INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE set(6) GRANULARITY 3
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO set_idx VALUES
|
||||
(0, 5, 4.7, 6.5, 'cba', 'b', '2014-01-04'),
|
||||
|
@ -14,7 +14,7 @@ CREATE TABLE set_idx
|
||||
INDEX idx (i32) TYPE set(2) GRANULARITY 1
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 6;"
|
||||
SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="
|
||||
INSERT INTO set_idx
|
||||
@ -24,4 +24,4 @@ SELECT number, number FROM system.numbers LIMIT 100"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE i32 > 0 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE set_idx;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE set_idx;"
|
||||
|
@ -20,7 +20,7 @@ CREATE TABLE bloom_filter_idx
|
||||
INDEX bf (s, lower(s)) TYPE ngrambf_v1(3, 512, 2, 0) GRANULARITY 1
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY k
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
CREATE TABLE bloom_filter_idx2
|
||||
@ -30,7 +30,7 @@ CREATE TABLE bloom_filter_idx2
|
||||
INDEX bf (s, lower(s)) TYPE ngrambf_v1(3, 512, 2, 0) GRANULARITY 1
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY k
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO bloom_filter_idx VALUES
|
||||
@ -113,7 +113,7 @@ CREATE TABLE bloom_filter_idx3
|
||||
INDEX bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY k
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO bloom_filter_idx3 VALUES
|
||||
(0, 'ClickHouse is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).'),
|
||||
|
@ -126,7 +126,7 @@ CREATE TABLE compression_codec_multiple_with_key_replicated (
|
||||
somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta),
|
||||
id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta),
|
||||
data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4)
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2;
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
|
||||
INSERT INTO compression_codec_multiple_with_key_replicated VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!');
|
||||
|
@ -15,7 +15,7 @@ CREATE TABLE minmax_idx
|
||||
INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
(0, 1, 1),
|
||||
|
@ -16,7 +16,7 @@ CREATE TABLE minmax_idx
|
||||
) ENGINE = MergeTree()
|
||||
PARTITION BY i32
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
|
@ -17,7 +17,7 @@ CREATE TABLE minmax_idx
|
||||
) ENGINE = MergeTree()
|
||||
PARTITION BY i32
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
|
@ -1,7 +1,7 @@
|
||||
|
||||
DROP TABLE IF EXISTS single_column_bloom_filter;
|
||||
|
||||
CREATE TABLE single_column_bloom_filter (u64 UInt64, i32 Int32, i64 UInt64, INDEX idx (i32) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 6;
|
||||
CREATE TABLE single_column_bloom_filter (u64 UInt64, i32 Int32, i64 UInt64, INDEX idx (i32) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO single_column_bloom_filter SELECT number AS u64, number AS i32, number AS i64 FROM system.numbers LIMIT 100;
|
||||
|
||||
@ -28,7 +28,7 @@ DROP TABLE IF EXISTS single_column_bloom_filter;
|
||||
|
||||
DROP TABLE IF EXISTS bloom_filter_types_test;
|
||||
|
||||
CREATE TABLE bloom_filter_types_test (order_key UInt64, i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, f32 Float32, f64 Float64, date Date, date_time DateTime('Asia/Istanbul'), str String, fixed_string FixedString(5), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
CREATE TABLE bloom_filter_types_test (order_key UInt64, i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, f32 Float32, f64 Float64, date Date, date_time DateTime('Asia/Istanbul'), str String, fixed_string FixedString(5), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO bloom_filter_types_test SELECT number AS order_key, toInt8(number) AS i8, toInt16(number) AS i16, toInt32(number) AS i32, toInt64(number) AS i64, toUInt8(number) AS u8, toUInt16(number) AS u16, toUInt32(number) AS u32, toUInt64(number) AS u64, toFloat32(number) AS f32, toFloat64(number) AS f64, toDate(number, 'Asia/Istanbul') AS date, toDateTime(number, 'Asia/Istanbul') AS date_time, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100;
|
||||
|
||||
SELECT COUNT() FROM bloom_filter_types_test WHERE i8 = 1 SETTINGS max_rows_to_read = 6;
|
||||
@ -52,7 +52,7 @@ DROP TABLE IF EXISTS bloom_filter_types_test;
|
||||
|
||||
DROP TABLE IF EXISTS bloom_filter_array_types_test;
|
||||
|
||||
CREATE TABLE bloom_filter_array_types_test (order_key Array(UInt64), i8 Array(Int8), i16 Array(Int16), i32 Array(Int32), i64 Array(Int64), u8 Array(UInt8), u16 Array(UInt16), u32 Array(UInt32), u64 Array(UInt64), f32 Array(Float32), f64 Array(Float64), date Array(Date), date_time Array(DateTime('Asia/Istanbul')), str Array(String), fixed_string Array(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
CREATE TABLE bloom_filter_array_types_test (order_key Array(UInt64), i8 Array(Int8), i16 Array(Int16), i32 Array(Int32), i64 Array(Int64), u8 Array(UInt8), u16 Array(UInt16), u32 Array(UInt32), u64 Array(UInt64), f32 Array(Float32), f64 Array(Float64), date Array(Date), date_time Array(DateTime('Asia/Istanbul')), str Array(String), fixed_string Array(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Asia/Istanbul')) AS date, groupArray(toDateTime(number, 'Asia/Istanbul')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers LIMIT 15);
|
||||
INSERT INTO bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Asia/Istanbul')) AS date, groupArray(toDateTime(number, 'Asia/Istanbul')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 5 LIMIT 15);
|
||||
INSERT INTO bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Asia/Istanbul')) AS date, groupArray(toDateTime(number, 'Asia/Istanbul')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 10 LIMIT 15);
|
||||
@ -106,7 +106,7 @@ DROP TABLE IF EXISTS bloom_filter_array_types_test;
|
||||
|
||||
DROP TABLE IF EXISTS bloom_filter_null_types_test;
|
||||
|
||||
CREATE TABLE bloom_filter_null_types_test (order_key UInt64, i8 Nullable(Int8), i16 Nullable(Int16), i32 Nullable(Int32), i64 Nullable(Int64), u8 Nullable(UInt8), u16 Nullable(UInt16), u32 Nullable(UInt32), u64 Nullable(UInt64), f32 Nullable(Float32), f64 Nullable(Float64), date Nullable(Date), date_time Nullable(DateTime('Asia/Istanbul')), str Nullable(String), fixed_string Nullable(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
CREATE TABLE bloom_filter_null_types_test (order_key UInt64, i8 Nullable(Int8), i16 Nullable(Int16), i32 Nullable(Int32), i64 Nullable(Int64), u8 Nullable(UInt8), u16 Nullable(UInt16), u32 Nullable(UInt32), u64 Nullable(UInt64), f32 Nullable(Float32), f64 Nullable(Float64), date Nullable(Date), date_time Nullable(DateTime('Asia/Istanbul')), str Nullable(String), fixed_string Nullable(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO bloom_filter_null_types_test SELECT number AS order_key, toInt8(number) AS i8, toInt16(number) AS i16, toInt32(number) AS i32, toInt64(number) AS i64, toUInt8(number) AS u8, toUInt16(number) AS u16, toUInt32(number) AS u32, toUInt64(number) AS u64, toFloat32(number) AS f32, toFloat64(number) AS f64, toDate(number, 'Asia/Istanbul') AS date, toDateTime(number, 'Asia/Istanbul') AS date_time, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100;
|
||||
INSERT INTO bloom_filter_null_types_test SELECT 0 AS order_key, NULL AS i8, NULL AS i16, NULL AS i32, NULL AS i64, NULL AS u8, NULL AS u16, NULL AS u32, NULL AS u64, NULL AS f32, NULL AS f64, NULL AS date, NULL AS date_time, NULL AS str, NULL AS fixed_string;
|
||||
|
||||
@ -146,7 +146,7 @@ DROP TABLE IF EXISTS bloom_filter_null_types_test;
|
||||
|
||||
DROP TABLE IF EXISTS bloom_filter_lc_null_types_test;
|
||||
|
||||
CREATE TABLE bloom_filter_lc_null_types_test (order_key UInt64, str LowCardinality(Nullable(String)), fixed_string LowCardinality(Nullable(FixedString(5))), INDEX idx (str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
CREATE TABLE bloom_filter_lc_null_types_test (order_key UInt64, str LowCardinality(Nullable(String)), fixed_string LowCardinality(Nullable(FixedString(5))), INDEX idx (str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO bloom_filter_lc_null_types_test SELECT number AS order_key, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100;
|
||||
INSERT INTO bloom_filter_lc_null_types_test SELECT 0 AS order_key, NULL AS str, NULL AS fixed_string;
|
||||
|
||||
@ -183,7 +183,7 @@ CREATE TABLE bloom_filter_array_lc_null_types_test (
|
||||
fixed_string Array(LowCardinality(Nullable(FixedString(5)))),
|
||||
INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string)
|
||||
TYPE bloom_filter GRANULARITY 1)
|
||||
ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, allow_nullable_key = 1;
|
||||
ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi', allow_nullable_key = 1;
|
||||
|
||||
INSERT INTO bloom_filter_array_lc_null_types_test
|
||||
SELECT groupArray(number) AS order_key,
|
||||
@ -286,19 +286,19 @@ SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(fixed_string
|
||||
DROP TABLE IF EXISTS bloom_filter_array_lc_null_types_test;
|
||||
|
||||
DROP TABLE IF EXISTS bloom_filter_array_offsets_lc_str;
|
||||
CREATE TABLE bloom_filter_array_offsets_lc_str (order_key int, str Array(LowCardinality(String)), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024;
|
||||
CREATE TABLE bloom_filter_array_offsets_lc_str (order_key int, str Array(LowCardinality(String)), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO bloom_filter_array_offsets_lc_str SELECT number AS i, if(i%2, ['value'], []) FROM system.numbers LIMIT 10000;
|
||||
SELECT count() FROM bloom_filter_array_offsets_lc_str WHERE has(str, 'value');
|
||||
DROP TABLE IF EXISTS bloom_filter_array_offsets_lc_str;
|
||||
|
||||
DROP TABLE IF EXISTS bloom_filter_array_offsets_str;
|
||||
CREATE TABLE bloom_filter_array_offsets_str (order_key int, str Array(String), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024;
|
||||
CREATE TABLE bloom_filter_array_offsets_str (order_key int, str Array(String), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO bloom_filter_array_offsets_str SELECT number AS i, if(i%2, ['value'], []) FROM system.numbers LIMIT 10000;
|
||||
SELECT count() FROM bloom_filter_array_offsets_str WHERE has(str, 'value');
|
||||
DROP TABLE IF EXISTS bloom_filter_array_offsets_str;
|
||||
|
||||
DROP TABLE IF EXISTS bloom_filter_array_offsets_i;
|
||||
CREATE TABLE bloom_filter_array_offsets_i (order_key int, i Array(int), INDEX idx i TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024;
|
||||
CREATE TABLE bloom_filter_array_offsets_i (order_key int, i Array(int), INDEX idx i TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO bloom_filter_array_offsets_i SELECT number AS i, if(i%2, [99999], []) FROM system.numbers LIMIT 10000;
|
||||
SELECT count() FROM bloom_filter_array_offsets_i WHERE has(i, 99999);
|
||||
DROP TABLE IF EXISTS bloom_filter_array_offsets_i;
|
||||
|
@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test_00961;
|
||||
|
||||
CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32)
|
||||
ENGINE = MergeTree PARTITION BY d ORDER BY (a, b)
|
||||
SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0;
|
||||
SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
|
||||
|
||||
@ -18,4 +18,3 @@ FROM system.parts
|
||||
WHERE table = 'test_00961' and database = currentDatabase();
|
||||
|
||||
DROP TABLE test_00961;
|
||||
|
||||
|
@ -15,7 +15,7 @@ CREATE TABLE bloom_filter_idx
|
||||
INDEX bf (s, lower(s)) TYPE ngrambf_v1(3, 512, 2, 0) GRANULARITY 1
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY k
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO bloom_filter_idx VALUES
|
||||
(0, 'ClickHouse - столбцовая система управления базами данных (СУБД)'),
|
||||
|
@ -14,7 +14,7 @@ CREATE TABLE set_idx
|
||||
INDEX idx (s) TYPE set(2) GRANULARITY 1
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY k
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO set_idx VALUES
|
||||
(0, 'ClickHouse - столбцовая система управления базами данных (СУБД)'),
|
||||
|
@ -21,7 +21,7 @@ CREATE TABLE indices_mutaions1
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/indices_mutaions', 'r1')
|
||||
PARTITION BY i32
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
CREATE TABLE indices_mutaions2
|
||||
(
|
||||
@ -32,7 +32,7 @@ CREATE TABLE indices_mutaions2
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/indices_mutaions', 'r2')
|
||||
PARTITION BY i32
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO indices_mutaions1 VALUES
|
||||
|
@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_for_concurrent_alter"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE table_for_concurrent_alter (id UInt64, Data String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096;";
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE table_for_concurrent_alter (id UInt64, Data String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096, index_granularity_bytes = '10Mi';";
|
||||
|
||||
n=0
|
||||
while [ "$n" -lt 50 ];
|
||||
|
@ -1,12 +1,12 @@
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 1, parts_to_delay_insert = 1
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi'
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 1, parts_to_delay_insert = 1
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 100, parts_to_delay_insert = 100
|
||||
2
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 30
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String,\n `Data2` UInt64\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 15
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 30
|
||||
CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String,\n `Data2` UInt64\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 15
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi'
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 1, parts_to_delay_insert = 1
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, merge_with_ttl_timeout = 300, max_concurrent_queries = 1
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi'
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi'
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', merge_with_ttl_timeout = 300, max_concurrent_queries = 1
|
||||
CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi'
|
||||
|
@ -17,7 +17,7 @@ DROP TABLE IF EXISTS table_for_alter;
|
||||
CREATE TABLE table_for_alter (
|
||||
id UInt64,
|
||||
Data String
|
||||
) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096;
|
||||
) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096, index_granularity_bytes = '10Mi';
|
||||
|
||||
ALTER TABLE table_for_alter MODIFY SETTING index_granularity=555; -- { serverError 472 }
|
||||
|
||||
@ -62,7 +62,7 @@ DROP TABLE IF EXISTS table_for_reset_setting;
|
||||
CREATE TABLE table_for_reset_setting (
|
||||
id UInt64,
|
||||
Data String
|
||||
) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096;
|
||||
) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096, index_granularity_bytes = '10Mi';
|
||||
|
||||
ALTER TABLE table_for_reset_setting MODIFY SETTING index_granularity=555; -- { serverError 472 }
|
||||
|
||||
|
@ -5,7 +5,7 @@ CREATE TABLE bloom_filter
|
||||
id UInt64,
|
||||
s String,
|
||||
INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1
|
||||
) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 8;
|
||||
) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 8, index_granularity_bytes = '10Mi';
|
||||
|
||||
insert into bloom_filter select number, 'yyy,uuu' from numbers(1024);
|
||||
insert into bloom_filter select number+2000, 'abc,def,zzz' from numbers(8);
|
||||
@ -38,4 +38,4 @@ SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'yyy'); -- { serverError 158
|
||||
-- this syntax is not supported by tokenbf
|
||||
SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'zzz') == 1; -- { serverError 158 }
|
||||
|
||||
DROP TABLE bloom_filter;
|
||||
DROP TABLE bloom_filter;
|
||||
|
@ -1,5 +1,5 @@
|
||||
DROP TABLE IF EXISTS empty_pk;
|
||||
CREATE TABLE empty_pk (x UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 256;
|
||||
CREATE TABLE empty_pk (x UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 256, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO empty_pk SELECT number FROM numbers(100000);
|
||||
|
||||
|
@ -25,7 +25,7 @@ CREATE TABLE minmax_idx
|
||||
INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2, min_rows_for_wide_part = 1000000"
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_rows_for_wide_part = 1000000"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
@ -50,4 +50,4 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE i32 = 5 AND i32 + f64
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"
|
||||
|
@ -13,7 +13,7 @@ DROP TABLE IF EXISTS mv_checkouts2target;
|
||||
-- that is the final table, which is filled incrementally from 2 different sources
|
||||
|
||||
CREATE TABLE target_table Engine=SummingMergeTree() ORDER BY id
|
||||
SETTINGS index_granularity=128
|
||||
SETTINGS index_granularity=128, index_granularity_bytes = '10Mi'
|
||||
AS
|
||||
SELECT
|
||||
number as id,
|
||||
|
@ -8,7 +8,7 @@ CREATE TABLE minmax_compact
|
||||
) ENGINE = MergeTree()
|
||||
PARTITION BY i32
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2, min_rows_for_wide_part = 1000000;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_rows_for_wide_part = 1000000;
|
||||
|
||||
INSERT INTO minmax_compact VALUES (0, 2, 1), (1, 1, 1), (2, 1, 1), (3, 1, 1), (4, 1, 1), (5, 2, 1), (6, 1, 2), (7, 1, 2), (8, 1, 2), (9, 1, 2);
|
||||
|
||||
|
@ -6,7 +6,7 @@ CREATE TABLE t
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY number
|
||||
SETTINGS index_granularity = 128;
|
||||
SETTINGS index_granularity = 128, index_granularity_bytes = '10Mi';
|
||||
|
||||
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
|
||||
INSERT INTO t SELECT number FROM numbers(10000000);
|
||||
|
@ -1,6 +1,6 @@
|
||||
DROP TABLE IF EXISTS test_01307;
|
||||
|
||||
CREATE TABLE test_01307 (id UInt64, val String, INDEX ind val TYPE bloom_filter() GRANULARITY 1) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 2;
|
||||
CREATE TABLE test_01307 (id UInt64, val String, INDEX ind val TYPE bloom_filter() GRANULARITY 1) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO test_01307 (id, val) select number as id, toString(number) as val from numbers(4);
|
||||
SELECT count() FROM test_01307 WHERE identity(val) = '2';
|
||||
SELECT count() FROM test_01307 WHERE val = '2';
|
||||
|
@ -1,5 +1,5 @@
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1000;
|
||||
CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1000, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO test SELECT * FROM numbers(1000000);
|
||||
OPTIMIZE TABLE test;
|
||||
|
||||
|
@ -43,7 +43,7 @@ CREATE TABLE goal
|
||||
`GoalID` UInt32,
|
||||
`Visits` AggregateFunction(sumIf, Int8, UInt8),
|
||||
`GoalReaches` AggregateFunction(sum, Int8)
|
||||
) ENGINE = AggregatingMergeTree PARTITION BY toStartOfMonth(StartDate) ORDER BY (CounterID, StartDate, GoalID) SETTINGS index_granularity = 256;
|
||||
) ENGINE = AggregatingMergeTree PARTITION BY toStartOfMonth(StartDate) ORDER BY (CounterID, StartDate, GoalID) SETTINGS index_granularity = 256, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO visits (`CounterID`,`StartDate`,`StartTime`,`Sign`,`GoalsID`) VALUES (1, toDate('2000-01-01'), toDateTime(toDate('2000-01-01')), 1, [1]);
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
DROP TABLE IF EXISTS skip_idx_comp_parts;
|
||||
CREATE TABLE skip_idx_comp_parts (a Int, b Int, index b_idx b TYPE minmax GRANULARITY 4)
|
||||
ENGINE = MergeTree ORDER BY a
|
||||
SETTINGS index_granularity=256, merge_max_block_size=100;
|
||||
SETTINGS index_granularity=256, index_granularity_bytes = '10Mi', merge_max_block_size=100;
|
||||
|
||||
SYSTEM STOP MERGES skip_idx_comp_parts;
|
||||
|
||||
|
@ -7,7 +7,7 @@ CREATE TABLE bloom_filter_nullable_index
|
||||
INDEX idx (str) TYPE bloom_filter GRANULARITY 1
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO bloom_filter_nullable_index VALUES (1, 'test');
|
||||
INSERT INTO bloom_filter_nullable_index VALUES (2, 'test2');
|
||||
|
@ -5,7 +5,7 @@ DROP TABLE IF EXISTS t_sparse_full;
|
||||
|
||||
CREATE TABLE t_sparse_full (id UInt64, u UInt64, s String)
|
||||
ENGINE = MergeTree ORDER BY id
|
||||
SETTINGS index_granularity = 32,
|
||||
SETTINGS index_granularity = 32, index_granularity_bytes = '10Mi',
|
||||
ratio_of_defaults_for_sparse_serialization = 0.1;
|
||||
|
||||
SYSTEM STOP MERGES t_sparse_full;
|
||||
|
@ -1,7 +1,7 @@
|
||||
SET max_block_size = 10, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_threads = 20;
|
||||
|
||||
DROP TABLE IF EXISTS bloom_filter;
|
||||
CREATE TABLE bloom_filter (`id` UInt64, `s` String, INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8;
|
||||
CREATE TABLE bloom_filter (`id` UInt64, `s` String, INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO bloom_filter SELECT number, 'yyy,uuu' FROM numbers(1024);
|
||||
|
||||
SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc');
|
||||
|
@ -6,7 +6,7 @@ SET enable_filesystem_cache=0;
|
||||
|
||||
CREATE TABLE order_by_desc (u UInt32, s String)
|
||||
ENGINE MergeTree ORDER BY u PARTITION BY u % 100
|
||||
SETTINGS index_granularity = 1024;
|
||||
SETTINGS index_granularity = 1024, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO order_by_desc SELECT number, repeat('a', 1024) FROM numbers(1024 * 300);
|
||||
OPTIMIZE TABLE order_by_desc FINAL;
|
||||
|
@ -2,7 +2,7 @@ SET allow_experimental_analyzer = 1;
|
||||
|
||||
DROP TABLE IF EXISTS test_tuple_filter;
|
||||
|
||||
CREATE TABLE test_tuple_filter (id UInt32, value String, log_date Date) Engine=MergeTree() ORDER BY id PARTITION BY log_date SETTINGS index_granularity = 3;
|
||||
CREATE TABLE test_tuple_filter (id UInt32, value String, log_date Date) Engine=MergeTree() ORDER BY id PARTITION BY log_date SETTINGS index_granularity = 3, index_granularity_bytes = '10Mi';
|
||||
|
||||
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');
|
||||
|
||||
|
@ -38,7 +38,7 @@ DROP TABLE IF EXISTS t_read_in_order;
|
||||
|
||||
CREATE TABLE t_read_in_order(a UInt32, b UInt32)
|
||||
ENGINE = MergeTree ORDER BY (a, b)
|
||||
SETTINGS index_granularity = 3;
|
||||
SETTINGS index_granularity = 3, index_granularity_bytes = '10Mi';
|
||||
|
||||
SYSTEM STOP MERGES t_read_in_order;
|
||||
|
||||
|
@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t_max_rows_to_read;
|
||||
|
||||
CREATE TABLE t_max_rows_to_read (a UInt64)
|
||||
ENGINE = MergeTree ORDER BY a
|
||||
SETTINGS index_granularity = 4;
|
||||
SETTINGS index_granularity = 4, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO t_max_rows_to_read SELECT number FROM numbers(100);
|
||||
|
||||
|
@ -8,7 +8,7 @@ CREATE TABLE t_02267
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY b
|
||||
SETTINGS index_granularity = 500;
|
||||
SETTINGS index_granularity = 500, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO t_02267 (b, a, c) SELECT 0, ['x'], ['1','2','3','4','5','6'] FROM numbers(1) ;
|
||||
INSERT INTO t_02267 (b, a, c) SELECT 1, [], ['1','2','3','4','5','6'] FROM numbers(300000);
|
||||
|
@ -8,7 +8,7 @@ DROP TABLE IF EXISTS tab;
|
||||
|
||||
CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(2))
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO tab VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'), (104, 'Dlick a04'), (105, 'Elick a05'), (106, 'Alick a06'), (107, 'Blick a07'), (108, 'Click a08'), (109, 'Dlick a09'), (110, 'Elick a10'), (111, 'Alick b01'), (112, 'Blick b02'), (113, 'Click b03'), (114, 'Dlick b04'), (115, 'Elick b05'), (116, 'Alick b06'), (117, 'Blick b07'), (118, 'Click b08'), (119, 'Dlick b09'), (120, 'Elick b10');
|
||||
|
||||
@ -61,7 +61,7 @@ DROP TABLE IF EXISTS tab_x;
|
||||
|
||||
CREATE TABLE tab_x(k UInt64, s String, INDEX af(s) TYPE inverted())
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO tab_x VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'), (104, 'Dlick a04'), (105, 'Elick a05'), (106, 'Alick a06'), (107, 'Blick a07'), (108, 'Click a08'), (109, 'Dlick a09'), (110, 'Elick a10'), (111, 'Alick b01'), (112, 'Blick b02'), (113, 'Click b03'), (114, 'Dlick b04'), (115, 'Elick b05'), (116, 'Alick b06'), (117, 'Blick b07'), (118, 'Click b08'), (119, 'Dlick b09'), (120, 'Elick b10');
|
||||
|
||||
@ -114,7 +114,7 @@ DROP TABLE IF EXISTS tab;
|
||||
|
||||
create table tab (k UInt64, s Array(String), INDEX af(s) TYPE inverted(2))
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO tab SELECT rowNumberInBlock(), groupArray(s) FROM tab_x GROUP BY k%10;
|
||||
|
||||
@ -141,7 +141,7 @@ DROP TABLE IF EXISTS tab;
|
||||
|
||||
CREATE TABLE tab (k UInt64, s Map(String,String), INDEX af(mapKeys(s)) TYPE inverted(2))
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO tab VALUES (101, {'Alick':'Alick a01'}), (102, {'Blick':'Blick a02'}), (103, {'Click':'Click a03'}), (104, {'Dlick':'Dlick a04'}), (105, {'Elick':'Elick a05'}), (106, {'Alick':'Alick a06'}), (107, {'Blick':'Blick a07'}), (108, {'Click':'Click a08'}), (109, {'Dlick':'Dlick a09'}), (110, {'Elick':'Elick a10'}), (111, {'Alick':'Alick b01'}), (112, {'Blick':'Blick b02'}), (113, {'Click':'Click b03'}), (114, {'Dlick':'Dlick b04'}), (115, {'Elick':'Elick b05'}), (116, {'Alick':'Alick b06'}), (117, {'Blick':'Blick b07'}), (118, {'Click':'Click b08'}), (119, {'Dlick':'Dlick b09'}), (120, {'Elick':'Elick b10'});
|
||||
|
||||
@ -181,7 +181,7 @@ DROP TABLE IF EXISTS tab;
|
||||
|
||||
CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(2))
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO tab VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'), (104, 'Dlick a04'), (105, 'Elick a05'), (106, 'Alick a06'), (107, 'Blick a07'), (108, 'Click a08'), (109, 'Dlick a09'), (110, 'Elick b10'), (111, 'Alick b01'), (112, 'Blick b02'), (113, 'Click b03'), (114, 'Dlick b04'), (115, 'Elick b05'), (116, 'Alick b06'), (117, 'Blick b07'), (118, 'Click b08'), (119, 'Dlick b09'), (120, 'Elick b10');
|
||||
INSERT INTO tab VALUES (201, 'rick c01'), (202, 'mick c02'), (203, 'nick c03');
|
||||
@ -210,7 +210,7 @@ DROP TABLE IF EXISTS tab;
|
||||
CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(2))
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO tab VALUES (101, 'Alick 好'), (102, 'clickhouse你好'), (103, 'Click 你'), (104, 'Dlick 你a好'), (105, 'Elick 好好你你'), (106, 'Alick 好a好a你a你');
|
||||
|
||||
@ -332,4 +332,3 @@ SELECT read_rows==512 from system.query_log
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==1
|
||||
LIMIT 1;
|
||||
|
||||
|
@ -17,7 +17,7 @@ CREATE TABLE 02354_annoy_l2
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5;
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO 02354_annoy_l2 VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]);
|
||||
|
||||
@ -64,7 +64,7 @@ CREATE TABLE 02354_annoy_l2
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5;
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO 02354_annoy_l2 VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]);
|
||||
|
||||
@ -95,7 +95,7 @@ CREATE TABLE 02354_annoy_cosine
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5;
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO 02354_annoy_cosine VALUES (1, [0.0, 0.0, 10.0]), (2, [0.2, 0.0, 10.0]), (3, [-0.3, 0.0, 10.0]), (4, [0.5, 0.0, 10.1]), (5, [0.8, 0.0, 10.0]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]);
|
||||
|
||||
@ -124,7 +124,7 @@ CREATE TABLE 02354_annoy_cosine
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5;
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO 02354_annoy_cosine VALUES (1, [0.0, 0.0, 10.0]), (2, [0.2, 0.0, 10.0]), (3, [-0.3, 0.0, 10.0]), (4, [0.5, 0.0, 10.1]), (5, [0.8, 0.0, 10.0]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]);
|
||||
|
||||
@ -156,7 +156,7 @@ CREATE TABLE 02354_annoy
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5; -- {serverError 7 }
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 7 }
|
||||
|
||||
-- Index must be created on Array(Float32) or Tuple(Float32)
|
||||
|
||||
@ -168,7 +168,7 @@ CREATE TABLE 02354_annoy
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5; -- {serverError 44 }
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 }
|
||||
|
||||
|
||||
CREATE TABLE 02354_annoy
|
||||
@ -179,7 +179,7 @@ CREATE TABLE 02354_annoy
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5; -- {serverError 44 }
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 }
|
||||
|
||||
CREATE TABLE 02354_annoy
|
||||
(
|
||||
@ -189,7 +189,7 @@ CREATE TABLE 02354_annoy
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5; -- {serverError 44 }
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 }
|
||||
|
||||
CREATE TABLE 02354_annoy
|
||||
(
|
||||
@ -199,7 +199,7 @@ CREATE TABLE 02354_annoy
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5; -- {serverError 44 }
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 }
|
||||
|
||||
CREATE TABLE 02354_annoy
|
||||
(
|
||||
@ -209,4 +209,4 @@ CREATE TABLE 02354_annoy
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS index_granularity=5; -- {serverError 44 }"
|
||||
SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 }"
|
||||
|
@ -8,7 +8,7 @@ CREATE TABLE t_in_tuple_index
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY (PLATFORM, USER_ID, ID)
|
||||
SETTINGS index_granularity = 2048;
|
||||
SETTINGS index_granularity = 2048, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO t_in_tuple_index VALUES ('1', 33, 'insta'), ('2', 33, 'insta');
|
||||
|
||||
|
@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t_max_rows_to_read;
|
||||
|
||||
CREATE TABLE t_max_rows_to_read (a UInt64)
|
||||
ENGINE = MergeTree ORDER BY a
|
||||
SETTINGS index_granularity = 4;
|
||||
SETTINGS index_granularity = 4, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO t_max_rows_to_read SELECT number FROM numbers(100);
|
||||
|
||||
|
@ -6,7 +6,7 @@ CREATE TABLE bloom_filter_nullable_index__fuzz_0
|
||||
`str` Nullable(String),
|
||||
INDEX idx str TYPE bloom_filter GRANULARITY 1
|
||||
)
|
||||
ENGINE = MergeTree ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
ENGINE = MergeTree ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO bloom_filter_nullable_index__fuzz_0 VALUES (1, 'test');
|
||||
INSERT INTO bloom_filter_nullable_index__fuzz_0 VALUES (2, 'test2');
|
||||
@ -18,7 +18,7 @@ CREATE TABLE bloom_filter_nullable_index__fuzz_1
|
||||
`str` String,
|
||||
INDEX idx str TYPE bloom_filter GRANULARITY 1
|
||||
)
|
||||
ENGINE = MergeTree ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
ENGINE = MergeTree ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO bloom_filter_nullable_index__fuzz_0 VALUES (1, 'test');
|
||||
INSERT INTO bloom_filter_nullable_index__fuzz_0 VALUES (2, 'test2');
|
||||
|
@ -1,6 +1,6 @@
|
||||
DROP TABLE IF EXISTS test_rlp;
|
||||
|
||||
CREATE TABLE test_rlp (a Int32, b Int32) ENGINE=MergeTree() ORDER BY a SETTINGS index_granularity=5;
|
||||
CREATE TABLE test_rlp (a Int32, b Int32) ENGINE=MergeTree() ORDER BY a SETTINGS index_granularity=5, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO test_rlp SELECT number, number FROM numbers(15);
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
-- { echoOn }
|
||||
CREATE TABLE test_filter(a Int32, b Int32, c Int32) ENGINE = MergeTree() ORDER BY a SETTINGS index_granularity = 3;
|
||||
CREATE TABLE test_filter(a Int32, b Int32, c Int32) ENGINE = MergeTree() ORDER BY a SETTINGS index_granularity = 3, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO test_filter SELECT number, number+1, (number/2 + 1) % 2 FROM numbers(15);
|
||||
SELECT _part_offset, intDiv(_part_offset, 3) as granule, * FROM test_filter ORDER BY _part_offset;
|
||||
0 0 0 1 1
|
||||
|
@ -1,7 +1,7 @@
|
||||
DROP TABLE IF EXISTS test_filter;
|
||||
|
||||
-- { echoOn }
|
||||
CREATE TABLE test_filter(a Int32, b Int32, c Int32) ENGINE = MergeTree() ORDER BY a SETTINGS index_granularity = 3;
|
||||
CREATE TABLE test_filter(a Int32, b Int32, c Int32) ENGINE = MergeTree() ORDER BY a SETTINGS index_granularity = 3, index_granularity_bytes = '10Mi';
|
||||
|
||||
INSERT INTO test_filter SELECT number, number+1, (number/2 + 1) % 2 FROM numbers(15);
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user