Enable all tests using ATTACH/FETCH on Replicated database

This commit is contained in:
Kevin Michel 2021-09-21 17:47:48 +02:00
parent 008c3c812b
commit 14b1c438cc
No known key found for this signature in database
GPG Key ID: 9F95C41F2EB138FC
19 changed files with 16 additions and 36 deletions

View File

@ -1,5 +1,4 @@
-- Tags: no-replicated-database, no-parallel -- Tags: no-parallel
-- Tag no-replicated-database: Unsupported type of ALTER query
DROP TABLE IF EXISTS src; DROP TABLE IF EXISTS src;
DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS dst;

View File

@ -1,6 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: zookeeper, no-replicated-database, no-parallel # Tags: zookeeper, no-parallel
# Tag no-replicated-database: Unsupported type of ALTER query
# Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem # Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem
# (it tries to do it as quick as possible, but it still performed in separate thread asynchronously) # (it tries to do it as quick as possible, but it still performed in separate thread asynchronously)

View File

@ -1,5 +1,4 @@
-- Tags: no-replicated-database, no-parallel -- Tags: no-parallel
-- Tag no-replicated-database: Unsupported type of ALTER query
DROP TABLE IF EXISTS alter_attach; DROP TABLE IF EXISTS alter_attach;
CREATE TABLE alter_attach (x UInt64, p UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p; CREATE TABLE alter_attach (x UInt64, p UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p;

View File

@ -1,5 +1,4 @@
-- Tags: no-replicated-database, no-parallel -- Tags: no-parallel
-- Tag no-replicated-database: Unsupported type of ALTER query
SET send_logs_level = 'fatal'; SET send_logs_level = 'fatal';

View File

@ -1,6 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: no-replicated-database, no-parallel # Tags: no-parallel
# Tag no-replicated-database: Unsupported type of ALTER query
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh

View File

@ -1,5 +1,4 @@
-- Tags: no-replicated-database, no-parallel -- Tags: no-parallel
-- Tag no-replicated-database: Unsupported type of ALTER query
DROP TABLE IF EXISTS table_01; DROP TABLE IF EXISTS table_01;

View File

@ -1,5 +1,4 @@
-- Tags: no-replicated-database, no-parallel -- Tags: no-parallel
-- Tag no-replicated-database: Unsupported type of ALTER query
CREATE TABLE test CREATE TABLE test
( (

View File

@ -1,5 +1,4 @@
-- Tags: no-replicated-database, no-parallel -- Tags: no-parallel
-- Tag no-replicated-database: Unsupported type of ALTER query
DROP TABLE IF EXISTS test; 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 AS SELECT number, toString(rand()) x from numbers(10000000);

View File

@ -1,5 +1,4 @@
-- Tags: no-replicated-database, no-parallel -- Tags: no-parallel
-- Tag no-replicated-database: Unsupported type of ALTER query
DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2;

View File

@ -1,6 +1,6 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: zookeeper, no-replicated-database, no-parallel # Tags: zookeeper, no-replicated-database, no-parallel
# Tag no-replicated-database: Unsupported type of ALTER query # Tag no-replicated-database: Fails due to additional replicas or shards
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh

View File

@ -1,6 +1,3 @@
-- Tags: no-replicated-database
-- Tag no-replicated-database: Unsupported type of ALTER query
DROP TABLE IF EXISTS mt_01451; DROP TABLE IF EXISTS mt_01451;
CREATE TABLE mt_01451 (v UInt8) ENGINE = MergeTree() order by tuple(); CREATE TABLE mt_01451 (v UInt8) ENGINE = MergeTree() order by tuple();

View File

@ -1,5 +1,5 @@
-- Tags: long, replica, no-replicated-database -- Tags: long, replica, no-replicated-database
-- Tag no-replicated-database: Unsupported type of ALTER query -- Tag no-replicated-database: Fails due to additional replicas or shards
SET replication_alter_partitions_sync = 2; SET replication_alter_partitions_sync = 2;

View File

@ -1,5 +1,4 @@
-- Tags: long, no-replicated-database -- Tags: long
-- Tag no-replicated-database: Unsupported type of ALTER query
DROP TABLE IF EXISTS test_01640; DROP TABLE IF EXISTS test_01640;
DROP TABLE IF EXISTS restore_01640; DROP TABLE IF EXISTS restore_01640;

View File

@ -1,6 +1,3 @@
-- Tags: no-replicated-database
-- Tag no-replicated-database: Unsupported type of ALTER query
CREATE TABLE table_2009_part (`i` Int64, `d` Date, `s` String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY i; CREATE TABLE table_2009_part (`i` Int64, `d` Date, `s` String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY i;
ALTER TABLE table_2009_part ATTACH PARTITION tuple(arrayJoin([0, 1])); -- {serverError 248} ALTER TABLE table_2009_part ATTACH PARTITION tuple(arrayJoin([0, 1])); -- {serverError 248}

View File

@ -1,5 +1,4 @@
-- Tags: replica, no-replicated-database -- Tags: replica
-- Tag no-replicated-database: Unsupported type of ALTER query
create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2)) engine = MergeTree() ORDER BY a; create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2)) engine = MergeTree() ORDER BY a;
insert into enum_alter_issue values ('one'), ('two'); insert into enum_alter_issue values ('one'), ('two');

View File

@ -1,5 +1,5 @@
-- Tags: zookeeper, no-replicated-database -- Tags: zookeeper, no-replicated-database
-- Tag no-replicated-database: Unsupported type of ALTER query -- Tag no-replicated-database: Fails due to additional replicas or shards
create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2), b Int) create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2), b Int)
engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02012/enum_alter_issue', 'r1') engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02012/enum_alter_issue', 'r1')

View File

@ -1,5 +1,5 @@
-- Tags: zookeeper, no-replicated-database -- Tags: zookeeper, no-replicated-database
-- Tag no-replicated-database: Unsupported type of ALTER query -- Tag no-replicated-database: Fails due to additional replicas or shards
drop table if exists enum_alter_issue; drop table if exists enum_alter_issue;
create table enum_alter_issue (a Enum16('one' = 1, 'two' = 2), b Int) create table enum_alter_issue (a Enum16('one' = 1, 'two' = 2), b Int)

View File

@ -1,6 +1,3 @@
-- Tags: no-replicated-database
-- Tag no-replicated-database: Unsupported type of ALTER query
DROP TABLE IF EXISTS test.partitions; DROP TABLE IF EXISTS test.partitions;
CREATE TABLE test.partitions (EventDate Date, CounterID UInt32) ENGINE = MergeTree(EventDate, CounterID, 8192); CREATE TABLE test.partitions (EventDate Date, CounterID UInt32) ENGINE = MergeTree(EventDate, CounterID, 8192);
INSERT INTO test.partitions SELECT EventDate + UserID % 365 AS EventDate, CounterID FROM test.hits WHERE CounterID = 1704509; INSERT INTO test.partitions SELECT EventDate + UserID % 365 AS EventDate, CounterID FROM test.hits WHERE CounterID = 1704509;

View File

@ -1,5 +1,5 @@
-- Tags: no-tsan, no-replicated-database -- Tags: no-tsan, no-replicated-database
-- Tag no-replicated-database: Unsupported type of ALTER query -- Tag no-replicated-database: Fails due to additional replicas or shards
DROP TABLE IF EXISTS fixed_granularity_table; DROP TABLE IF EXISTS fixed_granularity_table;