mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
fix tests
This commit is contained in:
parent
f00e6b5a7a
commit
c068593a0d
@ -2,6 +2,7 @@ drop table if exists summing_merge_tree_aggregate_function;
|
||||
drop table if exists summing_merge_tree_null;
|
||||
|
||||
---- partition merge
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
create table summing_merge_tree_aggregate_function (
|
||||
d Date,
|
||||
k UInt64,
|
||||
|
@ -1,4 +1,6 @@
|
||||
DROP TABLE IF EXISTS primary_key;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = MergeTree(d, -x, 1);
|
||||
|
||||
INSERT INTO primary_key (x) VALUES (1), (2), (3);
|
||||
|
@ -1,5 +1,6 @@
|
||||
drop table if exists prewhere_column_missing;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
create table prewhere_column_missing (d Date default '2015-01-01', x UInt64) engine=MergeTree(d, x, 1);
|
||||
|
||||
insert into prewhere_column_missing (x) values (0);
|
||||
|
@ -1,5 +1,6 @@
|
||||
drop table if exists aliases_test;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
create table aliases_test (
|
||||
date Date, id UInt64,
|
||||
array default ['zero','one','two'],
|
||||
|
@ -1,5 +1,6 @@
|
||||
drop table if exists aliases_test;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
create table aliases_test (date default today(), id default rand(), array default [0, 1, 2]) engine=MergeTree(date, id, 1);
|
||||
|
||||
insert into aliases_test (id) values (0);
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS sample_00276;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
|
||||
SET max_block_size = 10;
|
||||
|
||||
|
@ -1,4 +1,6 @@
|
||||
DROP TABLE IF EXISTS merge;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
|
||||
|
||||
INSERT INTO merge (x) VALUES (1), (2), (3);
|
||||
|
@ -3,6 +3,7 @@
|
||||
set max_threads = 1;
|
||||
drop table if exists enums;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
create table enums (
|
||||
d Date default '2015-12-29', k default 0,
|
||||
e Enum8('world' = 2, 'hello' = 1), sign Enum8('minus' = -1, 'plus' = 1),
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
DROP TABLE IF EXISTS array_pk;
|
||||
CREATE TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
|
||||
|
||||
|
@ -2,6 +2,7 @@ DROP TABLE IF EXISTS sample_00314_1;
|
||||
DROP TABLE IF EXISTS sample_00314_2;
|
||||
DROP TABLE IF EXISTS sample_merge_00314;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE sample_00314_1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
|
||||
CREATE TABLE sample_00314_2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS pk;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE pk (d Date DEFAULT '2000-01-01', x UInt64, y UInt64, z UInt64) ENGINE = MergeTree(d, (x, y, z), 1);
|
||||
|
||||
INSERT INTO pk (x, y, z) VALUES (1, 11, 1235), (1, 11, 4395), (1, 22, 3545), (1, 22, 6984), (1, 33, 4596), (2, 11, 4563), (2, 11, 4578), (2, 11, 3572), (2, 22, 5786), (2, 22, 5786), (2, 22, 2791), (2, 22, 2791), (3, 33, 2791), (3, 33, 2791), (3, 33, 1235), (3, 44, 4935), (3, 44, 4578), (3, 55, 5786), (3, 55, 2791), (3, 55, 1235);
|
||||
|
@ -1,4 +1,5 @@
|
||||
DROP TABLE IF EXISTS index_for_like;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE index_for_like (s String, d Date DEFAULT today()) ENGINE = MergeTree(d, (s, d), 1);
|
||||
|
||||
INSERT INTO index_for_like (s) VALUES ('Hello'), ('Hello, World'), ('Hello, World 1'), ('Hello 1'), ('Goodbye'), ('Goodbye, World'), ('Goodbye 1'), ('Goodbye, World 1');
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS pk_set;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE pk_set (d Date, n UInt64, host String, code UInt64) ENGINE = MergeTree(d, (n, host, code), 1);
|
||||
INSERT INTO pk_set (n, host, code) VALUES (1, 'market', 100), (11, 'news', 100);
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
DROP TABLE IF EXISTS summing_composite_key;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE summing_composite_key (d Date, k UInt64, FirstMap Nested(k1 UInt32, k2ID Int8, s Float64), SecondMap Nested(k1ID UInt64, k2Key String, k3Type Int32, s Int64)) ENGINE = SummingMergeTree(d, k, 1);
|
||||
|
||||
INSERT INTO summing_composite_key VALUES ('2000-01-01', 1, [1,2], ['3','4'], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 1, [2,1], ['4','3'], [20,22], [2,2,1], [5,5,0], [-3,-3,-33], [10,100,1000]), ('2000-01-01', 2, [1,2], ['3','4'], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 2, [2,1,1], ['4','3','3'], [20,22,33], [2,2], [5,5], [-3,-3], [10,100]), ('2000-01-01', 2, [1,2], ['3','4'], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]);
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS replace;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE replace ( EventDate Date, Id UInt64, Data String, Version UInt32) ENGINE = ReplacingMergeTree(EventDate, Id, 8192, Version);
|
||||
INSERT INTO replace VALUES ('2016-06-02', 1, 'version 1', 1);
|
||||
INSERT INTO replace VALUES ('2016-06-02', 2, 'version 1', 1);
|
||||
|
@ -1,6 +1,7 @@
|
||||
DROP TABLE IF EXISTS nested1;
|
||||
DROP TABLE IF EXISTS nested2;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE nested1 (d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, b String)) ENGINE = MergeTree(d, x, 1);
|
||||
INSERT INTO nested1 (x, n.a, n.b) VALUES (1, ['Hello', 'World'], ['abc', 'def']), (2, [], []);
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS prewhere_defaults;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE prewhere_defaults (d Date DEFAULT '2000-01-01', k UInt64 DEFAULT 0, x UInt16) ENGINE = MergeTree(d, k, 1);
|
||||
|
||||
INSERT INTO prewhere_defaults (x) VALUES (1);
|
||||
|
@ -1,4 +1,5 @@
|
||||
DROP TABLE IF EXISTS enum_pk;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE enum_pk (date Date DEFAULT '0000-00-00', x Enum8('0' = 0, '1' = 1, '2' = 2), d Enum8('0' = 0, '1' = 1, '2' = 2)) ENGINE = MergeTree(date, x, 1);
|
||||
INSERT INTO enum_pk (x, d) VALUES ('0', '0')('1', '1')('0', '0')('1', '1')('1', '1')('0', '0')('0', '0')('2', '2')('0', '0')('1', '1')('1', '1')('1', '1')('1', '1')('0', '0');
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
DROP TABLE IF EXISTS enum_nested_alter;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE enum_nested_alter
|
||||
(d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, e Enum8('Hello' = 1), b UInt8))
|
||||
ENGINE = MergeTree(d, x, 1);
|
||||
|
@ -1,4 +1,5 @@
|
||||
DROP TABLE IF EXISTS const_in_const;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE const_in_const (id UInt64, date Date, uid UInt32, name String, Sign Int8) ENGINE = CollapsingMergeTree(date, intHash32(uid), (id, date, intHash32(uid)), 8192, Sign);
|
||||
INSERT INTO const_in_const VALUES(1, now(), 1, 'test1', 1);
|
||||
INSERT INTO const_in_const VALUES(2, now(), 1, 'test2', 1);
|
||||
|
@ -1,4 +1,5 @@
|
||||
DROP TABLE IF EXISTS merge_tree;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE merge_tree (x UInt64, date Date) ENGINE = MergeTree(date, x, 1);
|
||||
|
||||
INSERT INTO merge_tree VALUES (1, '2000-01-01');
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS agg_func_col;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE agg_func_col (p Date, k UInt8, d AggregateFunction(sum, UInt64) DEFAULT arrayReduce('sumState', [toUInt64(200)])) ENGINE = AggregatingMergeTree(p, k, 1);
|
||||
INSERT INTO agg_func_col (k) VALUES (0);
|
||||
INSERT INTO agg_func_col (k, d) SELECT 1 AS k, arrayReduce('sumState', [toUInt64(100)]) AS d;
|
||||
|
@ -16,7 +16,7 @@ function get_num_parts {
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $name"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE $name (
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE $name (
|
||||
date Date,
|
||||
Sign Int8,
|
||||
ki UInt64,
|
||||
|
@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0"
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO preferred_block_size_bytes (s) SELECT '16_bytes_-_-_-_' AS s FROM system.numbers LIMIT 10, 90"
|
||||
$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE preferred_block_size_bytes"
|
||||
$CLICKHOUSE_CLIENT --preferred_block_size_bytes=26 -q "SELECT DISTINCT blockSize(), ignore(p, s) FROM preferred_block_size_bytes"
|
||||
@ -18,7 +18,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes"
|
||||
# PREWHERE using empty column
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS pbs"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0"
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000"
|
||||
$CLICKHOUSE_CLIENT -q "ALTER TABLE pbs ADD COLUMN s UInt8 DEFAULT 0"
|
||||
$CLICKHOUSE_CLIENT --preferred_block_size_bytes=100000 -q "SELECT count() FROM pbs PREWHERE s = 0"
|
||||
@ -29,7 +29,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE pbs"
|
||||
# Nullable PREWHERE
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0"
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1-q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT sum(d), sum(f), max(d) FROM nullable_prewhere PREWHERE NOT isNull(f)"
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere"
|
||||
|
@ -1,4 +1,6 @@
|
||||
DROP TABLE IF EXISTS nullable_alter;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE nullable_alter (d Date DEFAULT '2000-01-01', x String) ENGINE = MergeTree(d, d, 1);
|
||||
|
||||
INSERT INTO nullable_alter (x) VALUES ('Hello'), ('World');
|
||||
|
@ -1,4 +1,5 @@
|
||||
DROP TABLE IF EXISTS tab_00481;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE tab_00481 (date Date, value UInt64, s String, m FixedString(16)) ENGINE = MergeTree(date, (date, value), 8);
|
||||
INSERT INTO tab_00481 SELECT today() as date, number as value, '' as s, toFixedString('', 16) as m from system.numbers limit 42;
|
||||
SET preferred_max_column_in_block_size_bytes = 32;
|
||||
|
@ -1,4 +1,6 @@
|
||||
drop table if exists `table_00483`;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
create table `table_00483` (date Date, `Struct.Key1` Array(UInt64), `Struct.Key2` Array(UInt64), padding FixedString(16)) engine = MergeTree(date, (date), 16);
|
||||
insert into `table_00483` select today() as date, [number], [number + 1], toFixedString('', 16) from system.numbers limit 100;
|
||||
set preferred_max_column_in_block_size_bytes = 96;
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS pk;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE pk (d Date DEFAULT '2000-01-01', x DateTime, y UInt64, z UInt64) ENGINE = MergeTree(d, (toStartOfMinute(x), y, z), 1);
|
||||
|
||||
INSERT INTO pk (x, y, z) VALUES (1, 11, 1235), (2, 11, 4395), (3, 22, 3545), (4, 22, 6984), (5, 33, 4596), (61, 11, 4563), (62, 11, 4578), (63, 11, 3572), (64, 22, 5786), (65, 22, 5786), (66, 22, 2791), (67, 22, 2791), (121, 33, 2791), (122, 33, 2791), (123, 33, 1235), (124, 44, 4935), (125, 44, 4578), (126, 55, 5786), (127, 55, 2791), (128, 55, 1235);
|
||||
|
@ -8,8 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS advertiser";
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS advertiser_test";
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE advertiser ( action_date Date, adblock UInt8, imps Int64 ) Engine = SummingMergeTree( action_date, ( adblock ), 8192, ( imps ) )";
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE advertiser_test ( action_date Date, adblock UInt8, imps Int64, Hash UInt64 ) Engine = SummingMergeTree( action_date, ( adblock, Hash ), 8192, ( imps ) )";
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE advertiser ( action_date Date, adblock UInt8, imps Int64 ) Engine = SummingMergeTree( action_date, ( adblock ), 8192, ( imps ) )";
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 -q "CREATE TABLE advertiser_test ( action_date Date, adblock UInt8, imps Int64, Hash UInt64 ) Engine = SummingMergeTree( action_date, ( adblock, Hash ), 8192, ( imps ) )";
|
||||
|
||||
# This test will fail. It's ok.
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO advertiser_test SELECT *, sipHash64( CAST(adblock AS String) ), CAST(1 AS Int8) FROM advertiser;" 2>/dev/null
|
||||
|
@ -2,6 +2,7 @@ DROP TABLE IF EXISTS m3;
|
||||
DROP TABLE IF EXISTS m1;
|
||||
DROP TABLE IF EXISTS x;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE x (d Date, t DateTime) ENGINE = MergeTree(d, (d, t), 1);
|
||||
|
||||
CREATE MATERIALIZED VIEW m1 (d Date, t DateTime, c UInt64) ENGINE = SummingMergeTree(d, (d, t), 1) AS SELECT d, toStartOfMinute(x.t) as t, count() as c FROM x GROUP BY d, t;
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
set optimize_on_insert = 0;
|
||||
|
||||
drop table if exists mult_tab;
|
||||
|
@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT <<"EOF"
|
||||
DROP TABLE IF EXISTS `test_log`
|
||||
EOF
|
||||
|
||||
$CLICKHOUSE_CLIENT <<"EOF"
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 <<"EOF"
|
||||
CREATE TABLE `test_log` (
|
||||
date Date,
|
||||
datetime DateTime,
|
||||
|
@ -2,6 +2,7 @@ DROP TABLE IF EXISTS sample_00579_1;
|
||||
DROP TABLE IF EXISTS sample_00579_2;
|
||||
DROP TABLE IF EXISTS sample_merge_00579;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE sample_00579_1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
|
||||
CREATE TABLE sample_00579_2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
DROP TABLE IF EXISTS test;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE test(date Date, keys Array(Nullable(UInt8))) ENGINE = MergeTree(date, date, 1);
|
||||
INSERT INTO test VALUES ('2017-09-10', [1, 2, 3, 4, 5, 6, 7, NULL]);
|
||||
SELECT * FROM test LIMIT 1;
|
||||
|
@ -12,6 +12,7 @@ CREATE TABLE test_00616
|
||||
ENGINE = MergeTree(date, x, 4096);
|
||||
|
||||
INSERT INTO test_00616 VALUES ('2018-03-21', 1, 1), ('2018-03-21', 1, 2);
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE replacing_00616 ENGINE = ReplacingMergeTree(date, x, 4096, ver) AS SELECT * FROM test_00616;
|
||||
|
||||
SELECT * FROM test_00616 ORDER BY ver;
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
|
||||
DROP DATABASE IF EXISTS truncate_test;
|
||||
DROP TABLE IF EXISTS truncate_test_log;
|
||||
DROP TABLE IF EXISTS truncate_test_memory;
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS tab_00625;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE tab_00625
|
||||
(
|
||||
date Date,
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS partition_id;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE IF NOT EXISTS partition_id (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
|
||||
|
||||
INSERT INTO partition_id VALUES (100, 1), (200, 2), (300, 3);
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS test_00974;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE test_00974
|
||||
(
|
||||
date Date,
|
||||
|
@ -30,6 +30,7 @@ SELECT * FROM t2;
|
||||
DROP DATABASE IF EXISTS test_01109_other_atomic;
|
||||
DROP DATABASE IF EXISTS test_01109_ordinary;
|
||||
CREATE DATABASE test_01109_other_atomic;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
CREATE DATABASE test_01109_ordinary ENGINE=Ordinary;
|
||||
|
||||
CREATE TABLE test_01109_other_atomic.t3 ENGINE=MergeTree() ORDER BY tuple()
|
||||
|
@ -1,4 +1,6 @@
|
||||
DROP TABLE IF EXISTS mass_table_312;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE mass_table_312 (d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, b String)) ENGINE = MergeTree(d, x, 1);
|
||||
INSERT INTO mass_table_312 SELECT * FROM generateRandom('`d` Date,`x` UInt64,`n.a` Array(String),`n.b` Array(String)', 1, 10, 2) LIMIT 100;
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
DROP DATABASE IF EXISTS test_01190;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE DATABASE test_01190 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic
|
||||
USE test_01190;
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
drop table if exists merge_distributed;
|
||||
drop table if exists merge_distributed1;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
create table merge_distributed1 ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign);
|
||||
insert into merge_distributed1 values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3);
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
drop table if exists merge_tree;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
create table merge_tree ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign);
|
||||
|
||||
insert into merge_tree values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3)
|
||||
|
@ -1,5 +1,6 @@
|
||||
DROP TABLE IF EXISTS ttl_old_syntax;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE ttl_old_syntax (d Date, i Int) ENGINE = MergeTree(d, i, 8291);
|
||||
ALTER TABLE ttl_old_syntax MODIFY TTL toDate('2020-01-01'); -- { serverError 36 }
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
DROP TABLE IF EXISTS t;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE t (d Date, z UInt32) ENGINE = MergeTree(d, (z), 1);
|
||||
|
||||
INSERT INTO t VALUES ('2017-01-01', 1);
|
||||
|
@ -18,7 +18,7 @@ function test_db_comments()
|
||||
local ENGINE_NAME="$1"
|
||||
echo "engine : ${ENGINE_NAME}"
|
||||
|
||||
$CLICKHOUSE_CLIENT -nm <<EOF
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -nm <<EOF
|
||||
DROP DATABASE IF EXISTS ${DB_NAME};
|
||||
CREATE DATABASE ${DB_NAME} ENGINE = ${ENGINE_NAME} COMMENT 'Test DB with comment';
|
||||
EOF
|
||||
|
@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
for Engine in Atomic Ordinary; do
|
||||
$CLICKHOUSE_LOCAL --query """
|
||||
$CLICKHOUSE_LOCAL --allow_deprecated_database_ordinary=1 --query """
|
||||
CREATE DATABASE foo_$Engine Engine=$Engine;
|
||||
DROP DATABASE foo_$Engine;
|
||||
"""
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
DROP DATABASE IF EXISTS 02265_atomic_db;
|
||||
DROP DATABASE IF EXISTS 02265_ordinary_db;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user