mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-15 20:24:07 +00:00
Bring some test changes too
This commit is contained in:
parent
f3c36defab
commit
af2fa85352
@ -1,5 +1,6 @@
|
||||
-- Tags: long, replica, no-replicated-database
|
||||
-- Tags: long, replica, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Old syntax is not allowed
|
||||
-- no-shared-merge-tree: implemented another test
|
||||
|
||||
DROP TABLE IF EXISTS replicated_alter1;
|
||||
DROP TABLE IF EXISTS replicated_alter2;
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: long, zookeeper, no-replicated-database
|
||||
-- Tags: long, zookeeper, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Old syntax is not allowed
|
||||
-- Tag no-parallel: leftovers
|
||||
-- no-shared-merge-tree implemented another test
|
||||
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long, no-debug, no-asan, no-tsan, no-msan
|
||||
# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-random-settings, no-random-merge-tree-settings
|
||||
# Test can be very long in case when we have busy background pool + s3 storage
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,4 +1,5 @@
|
||||
-- Tags: long, replica
|
||||
-- Tags: long, replica, no-shared-merge-tree
|
||||
-- no-shared-merge-tree: require sync replica, added new test
|
||||
|
||||
DROP TABLE IF EXISTS replicated_truncate1;
|
||||
DROP TABLE IF EXISTS replicated_truncate2;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: replica, no-replicated-database, no-fasttest
|
||||
# Tags: replica, no-replicated-database, no-fasttest, no-shared-merge-tree
|
||||
# Tag no-replicated-database: Old syntax is not allowed
|
||||
# no-shared-merge-tree: implemented separate test
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,11 +1,15 @@
|
||||
-- Tags: replica
|
||||
|
||||
-- May affect part names
|
||||
SET prefer_warmed_unmerged_parts_seconds=0;
|
||||
SET ignore_cold_parts_seconds=0;
|
||||
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661 SYNC;
|
||||
DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661 SYNC;
|
||||
CREATE TABLE partitioned_by_tuple_replica1_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '1') PARTITION BY (d, x) ORDER BY (d, x, w);
|
||||
CREATE TABLE partitioned_by_tuple_replica2_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '2') PARTITION BY (d, x) ORDER BY (d, x, w);
|
||||
CREATE TABLE partitioned_by_tuple_replica1_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '1') PARTITION BY (d, x) ORDER BY (d, x, w) SETTINGS cache_populated_by_fetch=0;
|
||||
CREATE TABLE partitioned_by_tuple_replica2_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '2') PARTITION BY (d, x) ORDER BY (d, x, w) SETTINGS cache_populated_by_fetch=0;
|
||||
|
||||
INSERT INTO partitioned_by_tuple_replica1_00661 VALUES ('2000-01-02', 1, 'first', 3);
|
||||
INSERT INTO partitioned_by_tuple_replica1_00661 VALUES ('2000-01-01', 2, 'first', 2);
|
||||
|
@ -1,8 +1,8 @@
|
||||
*** Check data after fetch of merged part ***
|
||||
all_0_2_1 1
|
||||
all_0_2_1 2
|
||||
all_0_2_1 3
|
||||
0_2 1
|
||||
0_2 2
|
||||
0_2 3
|
||||
*** Check data after fetch/clone of mutated part ***
|
||||
all_0_2_1_3 1
|
||||
all_0_2_1_3 2
|
||||
all_0_2_1_3 3
|
||||
0_2 3 1
|
||||
0_2 3 2
|
||||
0_2 3 3
|
||||
|
@ -1,5 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper
|
||||
# Tags: zookeeper, no-shared-merge-tree
|
||||
# no-shared-merge-tree: no fetches in shared merge tree
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
@ -34,7 +35,7 @@ ${CLICKHOUSE_CLIENT} --query="ATTACH TABLE fetches_r2"
|
||||
${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA fetches_r2"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT '*** Check data after fetch of merged part ***'"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT _part, * FROM fetches_r2 ORDER BY x"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT substring(_part, 5, 3) as block_range, x FROM fetches_r2 ORDER BY x"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DETACH TABLE fetches_r2"
|
||||
|
||||
@ -49,7 +50,7 @@ ${CLICKHOUSE_CLIENT} --query="ATTACH TABLE fetches_r2"
|
||||
${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA fetches_r2"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT '*** Check data after fetch/clone of mutated part ***'"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT _part, * FROM fetches_r2 ORDER BY x"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT substring(_part, 5, 3) as block_range, substring(_part, 11, 2) as mutation_version, x FROM fetches_r2 ORDER BY x"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="
|
||||
DROP TABLE fetches_r1 SYNC;
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: long, zookeeper
|
||||
|
||||
SET prefer_warmed_unmerged_parts_seconds = 0;
|
||||
|
||||
DROP TABLE IF EXISTS byte_identical_r1;
|
||||
DROP TABLE IF EXISTS byte_identical_r2;
|
||||
|
||||
@ -16,6 +18,7 @@ SYSTEM SYNC REPLICA byte_identical_r1;
|
||||
SYSTEM SYNC REPLICA byte_identical_r2;
|
||||
SET replication_alter_partitions_sync=2;
|
||||
OPTIMIZE TABLE byte_identical_r1 PARTITION tuple() FINAL;
|
||||
SYSTEM SYNC REPLICA byte_identical_r2;
|
||||
|
||||
SELECT x, t1.y - t2.y FROM byte_identical_r1 t1 SEMI LEFT JOIN byte_identical_r2 t2 USING x ORDER BY x;
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: long, zookeeper, no-replicated-database
|
||||
-- Tags: long, zookeeper, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Fails due to additional replicas or shards
|
||||
-- Tag no-shared-merge-tree: no-shared-merge-tree: No quorum
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: long, zookeeper, no-replicated-database
|
||||
-- Tags: long, zookeeper, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Fails due to additional replicas or shards
|
||||
-- Tag no-shared-merge-tree: no-shared-merge-tree: No quorum
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: long, zookeeper, no-replicated-database
|
||||
-- Tags: long, zookeeper, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Fails due to additional replicas or shards
|
||||
-- Tag no-shared-merge-tree: no-shared-merge-tree: No quorum
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: long, zookeeper, no-replicated-database
|
||||
-- Tags: long, zookeeper, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Fails due to additional replicas or shards
|
||||
-- Tag no-shared-merge-tree: no-shared-merge-tree: No quorum
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: long, zookeeper, no-replicated-database
|
||||
-- Tags: long, zookeeper, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Fails due to additional replicas or shards
|
||||
-- Tag no-shared-merge-tree: no-shared-merge-tree: No quorum
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: long, zookeeper, no-replicated-database
|
||||
-- Tags: long, zookeeper, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Fails due to additional replicas or shards
|
||||
-- Tag no-shared-merge-tree: no-shared-merge-tree: No quorum
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: long, replica, no-replicated-database
|
||||
-- Tags: long, replica, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Old syntax is not allowed
|
||||
-- no-shared-merge-tree -- old syntax not supported, for new syntax additional test
|
||||
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: replica
|
||||
# Tags: replica, no-shared-merge-tree
|
||||
# no-shared-merge-tree -- non minimalistic header is not supported
|
||||
|
||||
set -e
|
||||
|
||||
@ -12,11 +13,15 @@ REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')")
|
||||
|
||||
$CLICKHOUSE_CLIENT -m -q "
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS part_header_r1;
|
||||
DROP TABLE IF EXISTS part_header_r2;
|
||||
|
||||
SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries
|
||||
SET replication_alter_partitions_sync = 2;
|
||||
-- May affect part names
|
||||
set prefer_warmed_unmerged_parts_seconds=0;
|
||||
set ignore_cold_parts_seconds=0;
|
||||
|
||||
CREATE TABLE part_header_r1(x UInt32, y UInt32)
|
||||
ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/{shard}', '1{replica}') ORDER BY x
|
||||
@ -24,14 +29,16 @@ CREATE TABLE part_header_r1(x UInt32, y UInt32)
|
||||
old_parts_lifetime = 1,
|
||||
cleanup_delay_period = 0,
|
||||
cleanup_delay_period_random_add = 0,
|
||||
cleanup_thread_preferred_points_per_iteration=0;
|
||||
cleanup_thread_preferred_points_per_iteration=0,
|
||||
cache_populated_by_fetch=0;
|
||||
CREATE TABLE part_header_r2(x UInt32, y UInt32)
|
||||
ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/{shard}', '2{replica}') ORDER BY x
|
||||
SETTINGS use_minimalistic_part_header_in_zookeeper = 1,
|
||||
old_parts_lifetime = 1,
|
||||
cleanup_delay_period = 0,
|
||||
cleanup_delay_period_random_add = 0,
|
||||
cleanup_thread_preferred_points_per_iteration=0;
|
||||
cleanup_thread_preferred_points_per_iteration=0,
|
||||
cache_populated_by_fetch=0;
|
||||
|
||||
SELECT '*** Test fetches ***';
|
||||
INSERT INTO part_header_r1 VALUES (1, 1);
|
||||
@ -64,6 +71,10 @@ done
|
||||
|
||||
$CLICKHOUSE_CLIENT -m -q "
|
||||
|
||||
-- May affect part names
|
||||
set prefer_warmed_unmerged_parts_seconds=0;
|
||||
set ignore_cold_parts_seconds=0;
|
||||
|
||||
SELECT '*** Test part removal ***';
|
||||
SELECT '*** replica 1 ***';
|
||||
SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1';
|
||||
|
@ -1,6 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: replica, no-debug, no-fasttest
|
||||
# Tags: replica, no-debug, no-fasttest, no-shared-merge-tree
|
||||
# no-fasttest: Waiting for failed mutations is slow: https://github.com/ClickHouse/ClickHouse/issues/67936
|
||||
# no-shared-merge-tree: kill mutation looks different for shared merge tree, implemented another test
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,6 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long, replica, no-replicated-database
|
||||
# Tags: long, replica, no-replicated-database, no-shared-merge-tree
|
||||
# Tag no-replicated-database: Old syntax is not allowed
|
||||
# no-shared-merge-tree: implemented another test with new syntax
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -15,7 +15,8 @@ CREATE TABLE zero_rows_per_granule (
|
||||
min_rows_for_wide_part = 0,
|
||||
enable_vertical_merge_algorithm=1,
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0;
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1);
|
||||
|
||||
@ -48,7 +49,8 @@ CREATE TABLE four_rows_per_granule (
|
||||
min_rows_for_wide_part = 0,
|
||||
enable_vertical_merge_algorithm=1,
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0;
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1);
|
||||
|
||||
|
@ -9,7 +9,7 @@ CREATE TABLE zero_rows_per_granule (
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0;
|
||||
) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, min_index_granularity_bytes = 10, write_final_mark = 0, min_bytes_for_wide_part = 0, min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -36,7 +36,7 @@ CREATE TABLE two_rows_per_granule (
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 40, min_index_granularity_bytes = 10, write_final_mark = 0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0;
|
||||
) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 40, min_index_granularity_bytes = 10, write_final_mark = 0, min_bytes_for_wide_part = 0, min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -97,7 +97,7 @@ CREATE TABLE huge_granularity_small_blocks (
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 1000000, write_final_mark = 0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0;
|
||||
) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 1000000, write_final_mark = 0, min_bytes_for_wide_part = 0, min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -128,7 +128,7 @@ CREATE TABLE adaptive_granularity_alter (
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0;
|
||||
) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, min_index_granularity_bytes = 100, write_final_mark = 0, min_bytes_for_wide_part = 0, min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -189,7 +189,7 @@ CREATE TABLE zero_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
|
||||
INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
@ -225,7 +225,7 @@ CREATE TABLE two_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -260,7 +260,7 @@ CREATE TABLE four_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -300,7 +300,7 @@ CREATE TABLE huge_granularity_small_blocks (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -339,7 +339,7 @@ CREATE TABLE adaptive_granularity_alter (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
|
||||
INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
@ -16,7 +16,7 @@ CREATE TABLE zero_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO zero_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -51,7 +51,7 @@ CREATE TABLE two_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO two_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -86,7 +86,7 @@ CREATE TABLE four_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO four_rows_per_granule (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -134,7 +134,7 @@ CREATE TABLE huge_granularity_small_blocks (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO huge_granularity_small_blocks (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -174,7 +174,7 @@ CREATE TABLE adaptive_granularity_alter (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
@ -182,7 +182,6 @@ SELECT COUNT(*) FROM adaptive_granularity_alter;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter' and database=currentDatabase() and active=1;
|
||||
|
||||
|
||||
ALTER TABLE adaptive_granularity_alter MODIFY COLUMN v1 Int16;
|
||||
|
||||
DETACH TABLE adaptive_granularity_alter;
|
||||
|
@ -18,7 +18,7 @@ CREATE TABLE zero_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1);
|
||||
|
||||
@ -54,7 +54,7 @@ CREATE TABLE four_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1);
|
||||
|
||||
@ -107,7 +107,7 @@ CREATE TABLE six_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0,
|
||||
min_bytes_for_wide_part = 0,
|
||||
min_rows_for_wide_part = 0;
|
||||
min_bytes_for_full_part_storage = 0;
|
||||
|
||||
|
||||
INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 1, 1000, 2000, -1, 2);
|
||||
|
@ -1,6 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper, no-replicated-database, no-fasttest
|
||||
# Tags: zookeeper, no-replicated-database, no-fasttest, no-shared-merge-tree
|
||||
# Tag no-replicated-database: Requires investigation
|
||||
# no-shared-merge-tree: relies on zookeeper structure of rmt
|
||||
|
||||
set -e
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: zookeeper, no-replicated-database
|
||||
-- Tags: zookeeper, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- Tag no-shared-merge-tree: for smt works
|
||||
|
||||
DROP TABLE IF EXISTS replicated_table_for_alter1;
|
||||
DROP TABLE IF EXISTS replicated_table_for_alter2;
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-random-settings, no-random-merge-tree-settings
|
||||
-- Tags: no-random-settings, no-random-merge-tree-settings, no-msan, no-tsan, no-asan, no-debug
|
||||
-- small number of insert threads can make insert terribly slow, especially with some build like msan
|
||||
DROP TABLE IF EXISTS mt;
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: replica, no-parallel
|
||||
# Tags: replica, no-parallel, no-shared-merge-tree
|
||||
# no-shared-merge-tree: impossible to stop fetches
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper
|
||||
# Tags: zookeeper, no-shared-merge-tree
|
||||
# no-shared-merge-tree: non determenistic is just allowed with shared merge tree
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -48,7 +48,7 @@ fi
|
||||
$CLICKHOUSE_CLIENT -q "SELECT '---MaterializedView---'"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW mv (a UInt32, s String) engine = MergeTree ORDER BY s SETTINGS min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0 POPULATE AS SELECT a, s FROM s1 WHERE a % 7 = 0"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT a, s FROM mv ORDER BY s LIMIT 10"
|
||||
rows_read=$($CLICKHOUSE_CLIENT -q "SELECT a, s FROM mv ORDER BY s LIMIT 10 FORMAT JSON" --max_threads=1 --max_block_size=20 --optimize_read_in_order=1 | grep "rows_read" | sed 's/[^0-9]*//g')
|
||||
rows_read=$($CLICKHOUSE_CLIENT -q "SELECT a, s FROM mv ORDER BY s LIMIT 10 FORMAT JSON" --max_threads=1 --max_block_size=20 --optimize_read_in_order=1 --allow_prefetched_read_pool_for_remote_filesystem=0 --allow_prefetched_read_pool_for_local_filesystem=0 | grep "rows_read" | sed 's/[^0-9]*//g')
|
||||
|
||||
if [[ $rows_read -lt 500 ]]
|
||||
then echo "OK"
|
||||
|
@ -1,5 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper, no-parallel
|
||||
# Tags: zookeeper, no-parallel, no-shared-merge-tree
|
||||
# no-shared-merge-tree -- we have similar tests for SMT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: long, zookeeper, no-replicated-database
|
||||
-- Tags: long, zookeeper, no-replicated-database, no-shared-merge-tree
|
||||
-- Tag no-replicated-database: Old syntax is not allowed
|
||||
-- no-shared-merge-tree: old syntax not allowed
|
||||
|
||||
DROP TABLE IF EXISTS test_alter_on_mutation;
|
||||
|
||||
@ -50,7 +51,6 @@ ALTER TABLE test_alter_on_mutation ADD COLUMN value String DEFAULT '10';
|
||||
|
||||
SELECT sum(cast(value as UInt64)) from test_alter_on_mutation;
|
||||
|
||||
-- TODO(alesap)
|
||||
OPTIMIZE table test_alter_on_mutation FINAL;
|
||||
|
||||
ALTER TABLE test_alter_on_mutation MODIFY COLUMN value UInt64 DEFAULT 10;
|
||||
|
Loading…
Reference in New Issue
Block a user