mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Enable all tests using ATTACH/FETCH on Replicated database
This commit is contained in:
parent
008c3c812b
commit
14b1c438cc
@ -1,5 +1,4 @@
|
||||
-- Tags: no-replicated-database, no-parallel
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS src;
|
||||
DROP TABLE IF EXISTS dst;
|
||||
|
@ -1,6 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper, no-replicated-database, no-parallel
|
||||
# Tag no-replicated-database: Unsupported type of ALTER query
|
||||
# Tags: zookeeper, no-parallel
|
||||
|
||||
# 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)
|
||||
|
@ -1,5 +1,4 @@
|
||||
-- Tags: no-replicated-database, no-parallel
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS alter_attach;
|
||||
CREATE TABLE alter_attach (x UInt64, p UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p;
|
||||
|
@ -1,5 +1,4 @@
|
||||
-- Tags: no-replicated-database, no-parallel
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-replicated-database, no-parallel
|
||||
# Tag no-replicated-database: Unsupported type of ALTER query
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,4 @@
|
||||
-- Tags: no-replicated-database, no-parallel
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS table_01;
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
-- Tags: no-replicated-database, no-parallel
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- Tags: no-parallel
|
||||
|
||||
CREATE TABLE test
|
||||
(
|
||||
|
@ -1,5 +1,4 @@
|
||||
-- Tags: no-replicated-database, no-parallel
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- 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);
|
||||
|
@ -1,5 +1,4 @@
|
||||
-- Tags: no-replicated-database, no-parallel
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS t2;
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# 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)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,6 +1,3 @@
|
||||
-- Tags: no-replicated-database
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
|
||||
DROP TABLE IF EXISTS mt_01451;
|
||||
|
||||
CREATE TABLE mt_01451 (v UInt8) ENGINE = MergeTree() order by tuple();
|
||||
|
@ -1,5 +1,5 @@
|
||||
-- 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;
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
-- Tags: long, no-replicated-database
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- Tags: long
|
||||
|
||||
DROP TABLE IF EXISTS test_01640;
|
||||
DROP TABLE IF EXISTS restore_01640;
|
||||
|
@ -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;
|
||||
|
||||
ALTER TABLE table_2009_part ATTACH PARTITION tuple(arrayJoin([0, 1])); -- {serverError 248}
|
||||
|
@ -1,5 +1,4 @@
|
||||
-- Tags: replica, no-replicated-database
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
-- Tags: replica
|
||||
|
||||
create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2)) engine = MergeTree() ORDER BY a;
|
||||
insert into enum_alter_issue values ('one'), ('two');
|
||||
|
@ -1,5 +1,5 @@
|
||||
-- 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)
|
||||
engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02012/enum_alter_issue', 'r1')
|
||||
|
@ -1,5 +1,5 @@
|
||||
-- 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;
|
||||
create table enum_alter_issue (a Enum16('one' = 1, 'two' = 2), b Int)
|
||||
|
@ -1,6 +1,3 @@
|
||||
-- Tags: no-replicated-database
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
|
||||
DROP TABLE IF EXISTS test.partitions;
|
||||
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;
|
||||
|
@ -1,5 +1,5 @@
|
||||
-- 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;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user