deprecate old MergeTree syntax

This commit is contained in:
Alexander Tokmakov 2022-06-23 10:37:52 +02:00
parent 2c5a88faba
commit f00e6b5a7a
138 changed files with 173 additions and 50 deletions

View File

@ -591,6 +591,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
\
M(Bool, allow_unrestricted_reads_from_keeper, false, "Allow unrestricted (without condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper", 0) \
M(Bool, allow_deprecated_database_ordinary, false, "Allow to create databases with deprecated Ordinary engine", 0) \
M(Bool, allow_deprecated_syntax_for_merge_tree, false, "Allow to create *MergeTree tables with deprecated engine definition syntax", 0) \
\
/** Experimental functions */ \
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \

View File

@ -303,6 +303,11 @@ static StoragePtr create(const StorageFactory::Arguments & args)
arg_idx, e.message(), getMergeTreeVerboseHelp(is_extended_storage_def));
}
}
else if (!args.attach && !args.local_context.lock()->getSettingsRef().allow_deprecated_syntax_for_merge_tree)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "This syntax for *MergeTree engine is deprecated. "
"Use extended storage definition syntax with ORDER BY/PRIMARY KEY clause.");
}
/// For Replicated.
String zookeeper_path;

View File

@ -5,6 +5,7 @@
<log_queries>1</log_queries>
<!-- Just to test settings_pull -->
<max_rows_in_distinct>5</max_rows_in_distinct>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</default>
</profiles>

View File

@ -3,6 +3,7 @@
<profiles>
<default>
<log_queries>1</log_queries>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</default>
</profiles>

View File

@ -3,6 +3,7 @@
<profiles>
<default>
<log_queries>1</log_queries>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</default>
</profiles>

View File

@ -1,4 +1,9 @@
<clickhouse>
<profiles>
<default>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</default>
</profiles>
<users>
<default>
<password></password>

View File

@ -37,7 +37,7 @@ def started_cluster():
CREATE DATABASE shard_{shard};
CREATE TABLE shard_{shard}.replicated(date Date, id UInt32, shard_id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{replica}', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
""".format(
shard=shard, replica=node.name
)

View File

@ -32,7 +32,7 @@ def started_cluster():
node.query(
"""
CREATE TABLE replicated (d Date, x UInt32) ENGINE =
ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{instance}', d, d, 8192)""".format(
ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{instance}') PARTITION BY toYYYYMM(d) ORDER BY d""".format(
shard=shard, instance=node.name
)
)

View File

@ -3,6 +3,7 @@
<!-- Default profile settings. -->
<default>
<log_queries>1</log_queries>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</default>
</profiles>
</clickhouse>

View File

@ -3,6 +3,7 @@
<!-- Default profile settings. -->
<default>
<log_queries>1</log_queries>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</default>
</profiles>
</clickhouse>

View File

@ -40,7 +40,7 @@ def test_deduplication_window_in_seconds(started_cluster):
node1.query(
"""
CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}', date, id, 8192)"""
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id"""
)
node.query("INSERT INTO simple VALUES (0, 0)")
@ -77,7 +77,7 @@ def test_deduplication_works_in_case_of_intensive_inserts(started_cluster):
node1.query(
"""
CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}', date, id, 8192)"""
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id"""
)
node1.query("INSERT INTO simple VALUES (0, 0)")

View File

@ -19,7 +19,7 @@ def _fill_nodes(nodes, shard):
CREATE DATABASE test;
CREATE TABLE test_table(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
""".format(
shard=shard, replica=node.name
)

View File

@ -52,7 +52,7 @@ CREATE TABLE distributed (x UInt32) ENGINE = Distributed('test_cluster', 'defaul
)
remote.query(
"CREATE TABLE local2 (d Date, x UInt32, s String) ENGINE = MergeTree(d, x, 8192)"
"CREATE TABLE local2 (d Date, x UInt32, s String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x"
)
instance_test_inserts_batching.query(
"""
@ -61,7 +61,7 @@ CREATE TABLE distributed (d Date, x UInt32) ENGINE = Distributed('test_cluster',
)
instance_test_inserts_local_cluster.query(
"CREATE TABLE local (d Date, x UInt32) ENGINE = MergeTree(d, x, 8192)"
"CREATE TABLE local (d Date, x UInt32) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x"
)
instance_test_inserts_local_cluster.query(
"""
@ -71,12 +71,12 @@ CREATE TABLE distributed_on_local (d Date, x UInt32) ENGINE = Distributed('test_
node1.query(
"""
CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node1', date, id, 8192)
CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node1') PARTITION BY toYYYYMM(date) ORDER BY id
"""
)
node2.query(
"""
CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node2', date, id, 8192)
CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node2') PARTITION BY toYYYYMM(date) ORDER BY id
"""
)
@ -94,12 +94,12 @@ CREATE TABLE distributed (date Date, id UInt32) ENGINE = Distributed('shard_with
shard1.query(
"""
CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree(d, x, 8192)"""
CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x"""
)
shard2.query(
"""
CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree(d, x, 8192)"""
CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x"""
)
shard1.query(
@ -143,7 +143,7 @@ CREATE TABLE distributed_one_replica_no_internal_replication (date Date, id UInt
node2.query(
"""
CREATE TABLE single_replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/single_replicated', 'node2', date, id, 8192)
CREATE TABLE single_replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/single_replicated', 'node2') PARTITION BY toYYYYMM(date) ORDER BY id
"""
)
@ -228,11 +228,11 @@ def test_inserts_batching(started_cluster):
# 4. Full batch of inserts after ALTER (that have different block structure).
# 5. What was left to insert with the column structure before ALTER.
expected = """\
20000101_20000101_1_1_0\t[1]
20000101_20000101_2_2_0\t[2,3,4]
20000101_20000101_3_3_0\t[5,6,7]
20000101_20000101_4_4_0\t[10,11,12]
20000101_20000101_5_5_0\t[8,9]
20000101__1_1_0\t[1]
20000101__2_2_0\t[2,3,4]
20000101__3_3_0\t[5,6,7]
20000101__4_4_0\t[10,11,12]
20000101__5_5_0\t[8,9]
"""
assert TSV(result) == TSV(expected)

View File

@ -23,7 +23,7 @@ def started_cluster():
for node in (node1, node2):
node.query(
"""
CREATE TABLE local_table(date Date, val UInt64) ENGINE = MergeTree(date, (date, val), 8192);
CREATE TABLE local_table(date Date, val UInt64) ENGINE = MergeTree() PARTITION BY toYYYYMM(date) ORDER BY (date, val);
"""
)

View File

@ -3,6 +3,7 @@
<default>
<distributed_directory_monitor_batch_inserts>1</distributed_directory_monitor_batch_inserts>
<min_insert_block_size_rows>3</min_insert_block_size_rows>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</default>
</profiles>
</clickhouse>

View File

@ -25,7 +25,7 @@ def started_cluster():
def test_empty_parts_alter_delete(started_cluster):
node1.query(
"CREATE TABLE empty_parts_delete (d Date, key UInt64, value String) \
ENGINE = ReplicatedMergeTree('/clickhouse/tables/empty_parts_delete', 'r1', d, key, 8192)"
ENGINE = ReplicatedMergeTree('/clickhouse/tables/empty_parts_delete', 'r1') PARTITION BY toYYYYMM(d) ORDER BY key"
)
node1.query("INSERT INTO empty_parts_delete VALUES (toDate('2020-10-10'), 1, 'a')")
@ -44,7 +44,7 @@ def test_empty_parts_alter_delete(started_cluster):
def test_empty_parts_summing(started_cluster):
node1.query(
"CREATE TABLE empty_parts_summing (d Date, key UInt64, value Int64) \
ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/empty_parts_summing', 'r1', d, key, 8192)"
ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/empty_parts_summing', 'r1') PARTITION BY toYYYYMM(d) ORDER BY key"
)
node1.query("INSERT INTO empty_parts_summing VALUES (toDate('2020-10-10'), 1, 1)")

View File

@ -17,7 +17,7 @@ def started_cluster():
try:
cluster.start()
instance_test_mutations.query(
"""CREATE TABLE test_mutations_with_ast_elements(date Date, a UInt64, b String) ENGINE = MergeTree(date, (a, date), 8192)"""
"""CREATE TABLE test_mutations_with_ast_elements(date Date, a UInt64, b String) ENGINE = MergeTree PARTITION BY toYYYYMM(date) ORDER BY (a, date)"""
)
instance_test_mutations.query(
"""INSERT INTO test_mutations_with_ast_elements SELECT '2019-07-29' AS date, 1, toString(number) FROM numbers(1) SETTINGS force_index_by_date = 0, force_primary_key = 0"""

View File

@ -44,7 +44,7 @@ def test_random_inserts(started_cluster):
node1.query(
"""
CREATE TABLE simple ON CLUSTER test_cluster (date Date, i UInt32, s String)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}', date, i, 8192)"""
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY i"""
)
with PartitionManager() as pm_random_drops:

View File

@ -20,13 +20,13 @@ def _fill_nodes(nodes, shard):
CREATE DATABASE test;
CREATE TABLE real_table(date Date, id UInt32, dummy UInt32)
ENGINE = MergeTree(date, id, 8192);
ENGINE = MergeTree PARTITION BY toYYYYMM(date) ORDER BY id;
CREATE TABLE other_table(date Date, id UInt32, dummy UInt32)
ENGINE = MergeTree(date, id, 8192);
ENGINE = MergeTree PARTITION BY toYYYYMM(date) ORDER BY id;
CREATE TABLE test_table(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
""".format(
shard=shard, replica=node.name
)

View File

@ -393,7 +393,7 @@ def test_alters_from_different_replicas(started_cluster):
main_node.query(
"CREATE TABLE testdb.concurrent_test "
"(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) "
"ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192);"
"ENGINE = MergeTree PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID);"
)
main_node.query(
@ -443,7 +443,7 @@ def test_alters_from_different_replicas(started_cluster):
" `Added0` UInt32,\\n `Added1` UInt32,\\n `Added2` UInt32,\\n `AddedNested1.A` Array(UInt32),\\n"
" `AddedNested1.B` Array(UInt64),\\n `AddedNested1.C` Array(String),\\n `AddedNested2.A` Array(UInt32),\\n"
" `AddedNested2.B` Array(UInt64)\\n)\\n"
"ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192)"
"ENGINE = MergeTree\\nPARTITION BY toYYYYMM(StartDate)\\nORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)\\nSETTINGS index_granularity = 8192"
)
assert_create_query([main_node, competing_node], "testdb.concurrent_test", expected)

View File

@ -10,7 +10,7 @@ def _fill_nodes(nodes, shard):
"""
CREATE DATABASE test;
CREATE TABLE test_table(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
""".format(
shard=shard, replica=node.name
)

View File

@ -40,7 +40,7 @@ def started_cluster():
node.query(
"""
CREATE TABLE sometable(date Date, id UInt32, value Int32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '{replica}', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
""".format(
replica=node.name
),
@ -51,7 +51,7 @@ def started_cluster():
node.query(
"""
CREATE TABLE someothertable(date Date, id UInt32, value Int32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/someothertable', '{replica}', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/someothertable', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
""".format(
replica=node.name
),

View File

@ -27,7 +27,7 @@ def started_cluster():
log_type UInt32,
account_id String
)
ENGINE = MergeTree(event_date, (event_time, account_id), 8192);
ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_time, account_id);
"""
)

View File

@ -48,7 +48,7 @@ def test_chroot_with_same_root(started_cluster):
node.query(
"""
CREATE TABLE simple (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
""".format(
replica=node.name
)
@ -68,7 +68,7 @@ def test_chroot_with_different_root(started_cluster):
node.query(
"""
CREATE TABLE simple_different (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple_different', '{replica}', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple_different', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
""".format(
replica=node.name
)

View File

@ -35,7 +35,7 @@ def test_identity(started_cluster):
node1.query(
"""
CREATE TABLE simple (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
""".format(
replica=node1.name
)
@ -45,6 +45,6 @@ def test_identity(started_cluster):
node2.query(
"""
CREATE TABLE simple (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '1', date, id, 8192);
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '1') PARTITION BY toYYYYMM(date) ORDER BY id;
"""
)

View File

@ -4,6 +4,7 @@
<max_partitions_per_insert_block>1024</max_partitions_per_insert_block>
<max_insert_threads>1</max_insert_threads>
<max_memory_usage>20G</max_memory_usage>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</settings>
<substitutions>

View File

@ -1,4 +1,8 @@
<test>
<settings>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</settings>
<create_query>CREATE TABLE tbl0 (`ds` Date, `x1` String, `x2` UInt32, `x3` UInt32, `x4` UInt32, `bm` AggregateFunction(groupBitmap, UInt32)) ENGINE = MergeTree PARTITION BY (ds, x1) ORDER BY (x2, x3, x4) SETTINGS index_granularity = 1</create_query>
<create_query>CREATE TABLE tbl (`ds` Date, `y1` UInt32, `x4` UInt32, `y2` UInt32, `y3` UInt32, `bm` AggregateFunction(groupBitmap, UInt32), `y4` UInt32 DEFAULT 0) ENGINE = MergeTree PARTITION BY (ds) ORDER BY (x4, y2, y3) SETTINGS index_granularity = 8192, max_parts_in_total = 10000000</create_query>

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS alter_test;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE alter_test (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192);
INSERT INTO alter_test VALUES (1, '2014-01-01', 2, 3, [1,2,3], ['a','b','c'], 4);

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS empty_summing;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE empty_summing (d Date, k UInt64, v Int8) ENGINE=SummingMergeTree(d, k, 8192);
INSERT INTO empty_summing VALUES ('2015-01-01', 1, 10);

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS stored_aggregates;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE stored_aggregates
(
d Date,

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS stored_aggregates;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE stored_aggregates
(
d Date,

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS stored_aggregates;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE stored_aggregates
(
d Date,

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS stored_aggregates;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE stored_aggregates
(
d Date,

View File

@ -1,6 +1,7 @@
-- Tags: no-backward-compatibility-check
DROP TABLE IF EXISTS alter_00061;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE alter_00061 (d Date, k UInt64, i32 Int32) ENGINE=MergeTree(d, k, 8192);
INSERT INTO alter_00061 VALUES ('2015-01-01', 10, 42);

View File

@ -6,6 +6,7 @@ DROP TABLE IF EXISTS replicated_alter2;
SET replication_alter_partitions_sync = 2;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00062/alter', 'r1', d, k, 8192);
CREATE TABLE replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00062/alter', 'r2', d, k, 8192);

View File

@ -13,6 +13,7 @@ drop table defaulted;
create table defaulted (col1 Int8, col2 UInt64 default (SELECT dummy+99 from system.one)) engine=Memory; --{serverError 116}
set allow_deprecated_syntax_for_merge_tree=1;
create table defaulted (payload String, date materialized today(), key materialized 0 * rand()) engine=MergeTree(date, key, 8192);
desc table defaulted;
insert into defaulted (payload) values ('hello clickhouse');

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS summing_merge_tree;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE summing_merge_tree (d Date, a String, x UInt32, y UInt64, z Float64) ENGINE = SummingMergeTree(d, a, 8192);
INSERT INTO summing_merge_tree VALUES ('2000-01-01', 'Hello', 1, 2, 3);

View File

@ -3,6 +3,7 @@
DROP TABLE IF EXISTS report1;
DROP TABLE IF EXISTS report2;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE report1(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
CREATE TABLE report2(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);

View File

@ -8,6 +8,7 @@ DROP TABLE IF EXISTS test_table;
DROP TABLE IF EXISTS test_view;
DROP TABLE IF EXISTS test_view_filtered;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE test_table (EventDate Date, CounterID UInt32, UserID UInt64, EventTime DateTime('America/Los_Angeles'), UTCEventTime DateTime('UTC')) ENGINE = MergeTree(EventDate, CounterID, 8192);
CREATE MATERIALIZED VIEW test_view (Rows UInt64, MaxHitTime DateTime('America/Los_Angeles')) ENGINE = Memory AS SELECT count() AS Rows, max(UTCEventTime) AS MaxHitTime FROM test_table;
CREATE MATERIALIZED VIEW test_view_filtered (EventDate Date, CounterID UInt32) ENGINE = Memory POPULATE AS SELECT CounterID, EventDate FROM test_table WHERE EventDate < '2013-01-01';

View File

@ -2,6 +2,7 @@
-- Tag no-replicated-database: Old syntax is not allowed
DROP TABLE IF EXISTS alter_00121;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t1', 'r1', d, (d), 8192);
INSERT INTO alter_00121 VALUES ('2014-01-01', 1);

View File

@ -4,6 +4,7 @@ SET allow_experimental_parallel_reading_from_replicas = 0;
SET max_parallel_replicas = 2;
DROP TABLE IF EXISTS report;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE report(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
INSERT INTO report(id,event_date,priority,description) VALUES (1, '2015-01-01', 1, 'foo')(2, '2015-02-01', 2, 'bar')(3, '2015-03-01', 3, 'foo')(4, '2015-04-01', 4, 'bar')(5, '2015-05-01', 5, 'foo');

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS prewhere;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE prewhere (d Date, a String, b String) ENGINE = MergeTree(d, d, 8192);
INSERT INTO prewhere VALUES ('2015-01-01', 'hello', 'world');

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS default;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE default (d Date DEFAULT toDate(t), t DateTime) ENGINE = MergeTree(d, t, 8192);
INSERT INTO default (t) VALUES ('1234567890');
SELECT toStartOfMonth(d), toUInt32(t) FROM default;

View File

@ -1,5 +1,6 @@
drop table if exists nested_map;
set allow_deprecated_syntax_for_merge_tree=1;
create table nested_map (d default today(), k UInt64, payload default rand(), SomeMap Nested(ID UInt32, Num Int64)) engine=SummingMergeTree(d, k, 8192);
insert into nested_map (k, `SomeMap.ID`, `SomeMap.Num`) values (0,[1],[100]),(1,[1],[100]),(2,[1],[100]),(3,[1,2],[100,150]);

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS alter_00147;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE alter_00147 (d Date DEFAULT toDate('2015-01-01'), n Nested(x String)) ENGINE = MergeTree(d, d, 8192);
INSERT INTO alter_00147 (`n.x`) VALUES (['Hello', 'World']);

View File

@ -29,6 +29,7 @@ select count() from summing_merge_tree_aggregate_function;
drop table summing_merge_tree_aggregate_function;
---- sum + uniq + uniqExact
set allow_deprecated_syntax_for_merge_tree=1;
create table summing_merge_tree_aggregate_function (
d materialized today(),
k UInt64,

View File

@ -1,5 +1,6 @@
drop table if exists nested_map_multiple_values;
set allow_deprecated_syntax_for_merge_tree=1;
create table nested_map_multiple_values (d materialized today(), k UInt64, payload materialized rand(), SomeMap Nested(ID UInt32, Num1 Int64, Num2 Float64)) engine=SummingMergeTree(d, k, 8192);
insert into nested_map_multiple_values values (0,[1],[100],[1.0]),(1,[1],[100],[1.0]),(2,[1],[100],[1.0]),(3,[1,2],[100,150],[1.0,1.5]);

View File

@ -11,9 +11,9 @@ function create {
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS aggregating_00155"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS replacing_00155"
$CLICKHOUSE_CLIENT --query="CREATE TABLE summing_00155 (d Date DEFAULT today(), x UInt64, s UInt64 DEFAULT 1) ENGINE = SummingMergeTree(d, x, 8192)"
$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE summing_00155 (d Date DEFAULT today(), x UInt64, s UInt64 DEFAULT 1) ENGINE = SummingMergeTree(d, x, 8192)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE collapsing_00155 (d Date DEFAULT today(), x UInt64, s Int8 DEFAULT 1) ENGINE = CollapsingMergeTree(d, x, 8192, s)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE aggregating_00155 (d Date DEFAULT today(), x UInt64, s AggregateFunction(sum, UInt64)) ENGINE = AggregatingMergeTree(d, x, 8192)"
$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE aggregating_00155 (d Date DEFAULT today(), x UInt64, s AggregateFunction(sum, UInt64)) ENGINE = AggregatingMergeTree(d, x, 8192)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE replacing_00155 (d Date DEFAULT today(), x UInt64, s Int8 DEFAULT 1, v UInt64) ENGINE = ReplacingMergeTree(d, (x), 8192, v)"
}

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS mt_00168;
DROP TABLE IF EXISTS mt_00168_buffer;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE mt_00168 (EventDate Date, UTCEventTime DateTime, MoscowEventDate Date DEFAULT toDate(UTCEventTime)) ENGINE = MergeTree(EventDate, UTCEventTime, 8192);
CREATE TABLE mt_00168_buffer AS mt_00168 ENGINE = Buffer(currentDatabase(), mt_00168, 16, 10, 100, 10000, 1000000, 10000000, 100000000);
DESC TABLE mt_00168;

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS aggregating_00191;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE aggregating_00191 (d Date DEFAULT '2000-01-01', k UInt64, u AggregateFunction(uniq, UInt64)) ENGINE = AggregatingMergeTree(d, k, 8192);
INSERT INTO aggregating_00191 (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000) GROUP BY k;

View File

@ -3,6 +3,7 @@
DROP TABLE IF EXISTS parallel_replicas;
DROP TABLE IF EXISTS parallel_replicas_backup;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE parallel_replicas (d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = MergeTree(d, cityHash64(u, s), (x, d, cityHash64(u, s)), 8192);
INSERT INTO parallel_replicas (x, u, s) VALUES (1, 2, 'A'),(3, 4, 'B'),(5, 6, 'C'),(7, 8, 'D'),(9,10,'E');
INSERT INTO parallel_replicas (x, u, s) VALUES (11, 12, 'F'),(13, 14, 'G'),(15, 16, 'H'),(17, 18, 'I'),(19,20,'J');

View File

@ -6,6 +6,7 @@ SET replication_alter_partitions_sync = 2;
DROP TABLE IF EXISTS attach_r1;
DROP TABLE IF EXISTS attach_r2;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00236/01/attach', 'r1', d, d, 8192);
CREATE TABLE attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00236/01/attach', 'r2', d, d, 8192);

View File

@ -5,6 +5,7 @@ SELECT * FROM defaults;
DROP TABLE defaults;
DROP TABLE IF EXISTS elog_cut;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE elog_cut
(
date Date DEFAULT toDate(uts),

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS sorted;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE sorted (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
INSERT INTO sorted (x) SELECT intDiv(number, 100000) AS x FROM system.numbers LIMIT 1000000;

View File

@ -73,6 +73,7 @@ SELECT * FROM merge ORDER BY _part_index, x;
DROP TABLE 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, 8192);
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS index;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE index
(
key Int32,

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS aggregates;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE aggregates (d Date, s AggregateFunction(uniq, UInt64)) ENGINE = MergeTree(d, d, 8192);
INSERT INTO aggregates

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS alter_00394;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE alter_00394 (d Date, k UInt64, i32 Int32, n Nested(ui8 UInt8, s String)) ENGINE=MergeTree(d, k, 8192);
INSERT INTO alter_00394 VALUES ('2015-01-01', 3, 30, [1,2,3], ['1','12','123']);

View File

@ -14,6 +14,7 @@ ORDER BY str ASC;
DROP TABLE replaceall;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE replaceall (date Date DEFAULT today(), fs FixedString(16)) ENGINE = MergeTree(date, (date, fs), 8192);
INSERT INTO replaceall (fs) VALUES ('54db0d43009d\0\0\0\0'), ('fe2b58224766cf10'), ('54db0d43009d\0\0\0\0'), ('fe2b58224766cf10');

View File

@ -8,6 +8,7 @@ SELECT NULL + NULL;
SELECT '----- MergeTree engine -----';
DROP TABLE IF EXISTS test1_00395;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE test1_00395(
col1 UInt64, col2 Nullable(UInt64),
col3 String, col4 Nullable(String),

View File

@ -8,6 +8,7 @@ function perform()
{
local query=$1
TZ=UTC $CLICKHOUSE_CLIENT \
--allow_deprecated_syntax_for_merge_tree=1 \
--use_client_time_zone=1 \
--input_format_values_interpret_expressions=0 \
--query "$query" 2>/dev/null

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS nulls;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE nulls (d Date, x Nullable(UInt64)) ENGINE = MergeTree(d, d, 8192);
INSERT INTO nulls SELECT toDate('2000-01-01'), number % 10 != 0 ? number : NULL FROM system.numbers LIMIT 10000;
SELECT count() FROM nulls WHERE x IS NULL;

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
ch="$CLICKHOUSE_CLIENT --stacktrace -q"
ch="$CLICKHOUSE_CLIENT --allow_deprecated_syntax_for_merge_tree=1 --stacktrace -q"
$ch "DROP TABLE IF EXISTS clear_column1"
$ch "DROP TABLE IF EXISTS clear_column2"

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS cast_enums;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE cast_enums
(
type Enum8('session' = 1, 'pageview' = 2, 'click' = 3),

View File

@ -1,4 +1,5 @@
drop table if exists one_table;
set allow_deprecated_syntax_for_merge_tree=1;
create table one_table (date Date, one UInt64) engine = MergeTree(date, (date, one), 8192);
insert into one_table select today(), toUInt64(1) from system.numbers limit 100000;
SET preferred_block_size_bytes = 8192;

View File

@ -1,4 +1,5 @@
set allow_deprecated_syntax_for_merge_tree=1;
drop table if exists test_ins_arr;
create table test_ins_arr (date Date, val Array(UInt64)) engine = MergeTree(date, (date), 8192);
insert into test_ins_arr select toDate('2017-10-02'), [number, 42] from system.numbers limit 10000;

View File

@ -8,6 +8,7 @@ DROP TABLE IF EXISTS union1;
DROP TABLE IF EXISTS union2;
DROP TABLE IF EXISTS union3;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE union1 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = MergeTree(date, (a, date), 8192);
CREATE TABLE union2 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost, currentDatabase(), 'union1');
CREATE TABLE union3 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost, currentDatabase(), 'union2');

View File

@ -1,6 +1,7 @@
-- Tags: shard
drop table if exists tab;
set allow_deprecated_syntax_for_merge_tree=1;
create table tab (date Date, val UInt64, val2 UInt8 default 42, val3 UInt8 default val2 + 1, val4 UInt64 alias val) engine = MergeTree(date, (date, val), 8192);
desc tab;
select '-';

View File

@ -2,6 +2,7 @@
-- Tag no-replicated-database: Old syntax is not allowed
DROP TABLE IF EXISTS deduplication_by_partition;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE deduplication_by_partition(d Date, x UInt32) ENGINE =
ReplicatedMergeTree('/clickhouse/tables/{database}/test_00516/deduplication_by_partition', 'r1', d, x, 8192);

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS agg_over_nullable;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE agg_over_nullable (
partition Date,
timestamp DateTime,

View File

@ -1,3 +1,5 @@
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE test
(
dt Date,

View File

@ -1,5 +1,6 @@
drop table IF EXISTS joinbug;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE joinbug (
event_date Date MATERIALIZED toDate(created, 'Asia/Istanbul'),
id UInt64,

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS test_00563;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE test_00563 ( dt Date, site_id Int32, site_key String ) ENGINE = MergeTree(dt, (site_id, site_key, dt), 8192);
INSERT INTO test_00563 (dt,site_id, site_key) VALUES ('2018-1-29', 100, 'key');
SELECT * FROM test_00563 WHERE toInt32(site_id) IN (100);

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS test;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE IF NOT EXISTS test( id UInt32, track UInt8, codec String, content String, rdate Date DEFAULT '2018-02-03', track_id String DEFAULT concat(concat(concat(toString(track), '-'), codec), content) ) ENGINE=MergeTree(rdate, (id, track_id), 8192);
INSERT INTO test(id, track, codec) VALUES(1, 0, 'h264');

View File

@ -8,6 +8,7 @@ DROP DATABASE IF EXISTS none;
DROP TABLE IF EXISTS test_00571;
DROP TABLE IF EXISTS test_materialized_00571;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE DATABASE none;
CREATE TABLE test_00571 ( date Date, platform Enum8('a' = 0, 'b' = 1, 'c' = 2), app Enum8('a' = 0, 'b' = 1) ) ENGINE = MergeTree(date, (platform, app), 8192);
CREATE MATERIALIZED VIEW test_materialized_00571 ENGINE = MergeTree(date, (platform, app), 8192) POPULATE AS SELECT date, platform, app FROM (SELECT * FROM test_00571);

View File

@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
exception_pattern="Code: 44.*Cannot drop column \`id\`, because column \`id2\` depends on it"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_00575;"
${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_00575 (dt Date DEFAULT now(), id UInt32, id2 UInt32 DEFAULT id + 1) ENGINE = MergeTree(dt, dt, 8192);"
${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query "CREATE TABLE test_00575 (dt Date DEFAULT now(), id UInt32, id2 UInt32 DEFAULT id + 1) ENGINE = MergeTree(dt, dt, 8192);"
${CLICKHOUSE_CLIENT} --query "INSERT INTO test_00575(dt,id) VALUES ('2018-02-22',3), ('2018-02-22',4), ('2018-02-22',5);"
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_00575 ORDER BY id;"
echo "$(${CLICKHOUSE_CLIENT} --query "ALTER TABLE test_00575 DROP COLUMN id;" --server_logs_file=/dev/null 2>&1 | grep -c "$exception_pattern")"

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS t_00575;
set allow_deprecated_syntax_for_merge_tree=1;
create table t_00575(d Date) engine MergeTree(d, d, 8192);
insert into t_00575 values ('2018-02-20');

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS Test_00584;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE Test_00584 (
createdDate Date,
str String,

View File

@ -3,6 +3,7 @@
DROP TABLE IF EXISTS alias_local10;
DROP TABLE IF EXISTS alias10;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE alias_local10 (
Id Int8,
EventDate Date DEFAULT '2000-01-01',

View File

@ -7,6 +7,7 @@ SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS test_00597;
DROP TABLE IF EXISTS test_view_00597;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE test_00597(date Date, id Int8, name String, value Int64) ENGINE = MergeTree(date, (id, date), 8192);
CREATE VIEW test_view_00597 AS SELECT * FROM test_00597;

View File

@ -6,6 +6,7 @@ DROP TABLE IF EXISTS test_mv_00609;
create table test_00609 (a Int8) engine=Memory;
insert into test_00609 values (1);
set allow_deprecated_syntax_for_merge_tree=1;
create materialized view test_mv_00609 uuid '00000609-1000-4000-8000-000000000001' Engine=MergeTree(date, (a), 8192) populate as select a, toDate('2000-01-01') date from test_00609;
select * from test_mv_00609; -- OK

View File

@ -1,6 +1,7 @@
DROP TABLE IF EXISTS tab_00610;
DROP TABLE IF EXISTS mv_00610;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE tab_00610(d Date, x UInt32) ENGINE MergeTree(d, x, 8192);
CREATE MATERIALIZED VIEW mv_00610(d Date, y UInt64) ENGINE MergeTree(d, y, 8192) AS SELECT d, x + 1 AS y FROM tab_00610;

View File

@ -1,6 +1,7 @@
-- Tags: distributed
drop table if exists tab;
set allow_deprecated_syntax_for_merge_tree=1;
create table tab (date Date, time DateTime, data String) ENGINE = MergeTree(date, (time, data), 8192);
insert into tab values ('2018-01-21','2018-01-21 15:12:13','test');
select time FROM remote('127.0.0.{1,2}', currentDatabase(), tab) WHERE date = '2018-01-21' limit 2;

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS test_00615;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE test_00615
(
dt Date,

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS regression_for_in_operator_view;
DROP TABLE IF EXISTS regression_for_in_operator;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE regression_for_in_operator (d Date, v UInt32, g String) ENGINE=MergeTree(d, d, 8192);
CREATE MATERIALIZED VIEW regression_for_in_operator_view ENGINE=AggregatingMergeTree(d, (d,g), 8192) AS SELECT d, g, maxState(v) FROM regression_for_in_operator GROUP BY d, g;

View File

@ -1,4 +1,5 @@
drop table if exists test54378;
set allow_deprecated_syntax_for_merge_tree=1;
create table test54378 (part_date Date, pk_date Date, date Date) Engine=MergeTree(part_date, pk_date, 8192);
insert into test54378 values ('2018-04-19', '2018-04-19', '2018-04-19');

View File

@ -4,6 +4,7 @@
DROP TABLE IF EXISTS replicated_truncate1;
DROP TABLE IF EXISTS replicated_truncate2;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE replicated_truncate1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00623/truncate', 'r1', d, k, 8192);
CREATE TABLE replicated_truncate2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_00623/truncate', 'r2', d, k, 8192);

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS final_test;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE final_test (id String, version Date) ENGINE = ReplacingMergeTree(version, id, 8192);
INSERT INTO final_test (id, version) VALUES ('2018-01-01', '2018-01-01');
SELECT * FROM final_test FINAL PREWHERE id == '2018-01-02';

View File

@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations(d Date, x UInt32, s String, a UInt32 ALIAS x + 1, m MATERIALIZED x + 2) ENGINE MergeTree(d, intDiv(x, 10), 8192)"
${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations(d Date, x UInt32, s String, a UInt32 ALIAS x + 1, m MATERIALIZED x + 2) ENGINE MergeTree(d, intDiv(x, 10), 8192)"
# Test a mutation on empty table
${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations DELETE WHERE x = 1"

View File

@ -12,8 +12,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r1(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r1', d, intDiv(x, 10), 8192)"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r2', d, intDiv(x, 10), 8192)"
${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r1(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r1', d, intDiv(x, 10), 8192)"
${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r2', d, intDiv(x, 10), 8192)"
# Test a mutation on empty table
${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2"

View File

@ -1,6 +1,7 @@
-- Tags: shard
DROP TABLE IF EXISTS remote_test;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE remote_test(uid String, its UInt32, action_code String, day Date) ENGINE = MergeTree(day, (uid, its), 8192);
INSERT INTO remote_test SELECT toString(number) AS uid, number % 3 AS its, toString(number % 3) AS action_code, '2000-01-01' FROM system.numbers LIMIT 10000;
SELECT level, COUNT() FROM (SELECT uid, windowFunnel(3600)(toUInt32(its), action_code != '', action_code = '2') AS level FROM remote('127.0.0.{2,3}', currentDatabase(), remote_test) GROUP BY uid) GROUP BY level;

View File

@ -1,5 +1,6 @@
drop table if exists table;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE table (a UInt32, date Date, b UInt64, c UInt64, str String, d Int8, arr Array(UInt64), arr_alias Array(UInt64) ALIAS arr) ENGINE = MergeTree(date, intHash32(c), (a, date, intHash32(c), b), 8192);
SELECT alias2 AS alias3

View File

@ -1,4 +1,5 @@
drop table if exists trepl;
set allow_deprecated_syntax_for_merge_tree=1;
create table trepl(d Date,a Int32, b Int32) engine = ReplacingMergeTree(d, (a,b), 8192);
insert into trepl values ('2018-09-19', 1, 1);
select b from trepl FINAL prewhere a < 1000;

View File

@ -1,4 +1,5 @@
drop table if exists t_00712_2;
set allow_deprecated_syntax_for_merge_tree=1;
create table t_00712_2 (date Date, counter UInt64, sampler UInt64, alias_col alias date + 1) engine = MergeTree(date, intHash32(sampler), (counter, date, intHash32(sampler)), 8192);
insert into t_00712_2 values ('2018-01-01', 1, 1);
select alias_col from t_00712_2 sample 1 / 2 where date = '2018-01-01' and counter = 1 and sampler = 1;

View File

@ -7,6 +7,7 @@ DROP TABLE IF EXISTS test_local_2;
DROP TABLE IF EXISTS test_distributed_1;
DROP TABLE IF EXISTS test_distributed_2;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE test_local_1 (date Date, value UInt32) ENGINE = MergeTree(date, date, 8192);
CREATE TABLE test_local_2 (date Date, value UInt32) ENGINE = MergeTree(date, date, 8192);
CREATE TABLE test_distributed_1 AS test_local_1 ENGINE = Distributed('test_shard_localhost', currentDatabase(), test_local_1, rand());

View File

@ -1,6 +1,7 @@
SET send_logs_level = 'fatal';
drop table if exists t1_00729;
set allow_deprecated_syntax_for_merge_tree=1;
create table t1_00729 (id UInt64, val Array(String),nid UInt64, eDate Date)ENGINE = MergeTree(eDate, (id, eDate), 8192);
insert into t1_00729 (id,val,nid,eDate) values (1,['background','foreground','heading','image'],1,'2018-09-27');

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
settings="--log_queries=1 --log_query_threads=1 --log_profile_events=1 --log_query_settings=1"
settings="--log_queries=1 --log_query_threads=1 --log_profile_events=1 --log_query_settings=1 --allow_deprecated_syntax_for_merge_tree=1"
# Test insert logging on each block and checkPacket() method

View File

@ -1,4 +1,5 @@
DROP TABLE IF EXISTS decimal_sum;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE decimal_sum
(
date Date,

View File

@ -1,5 +1,6 @@
DROP TABLE IF EXISTS arraytest;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE arraytest ( created_date Date DEFAULT toDate(created_at), created_at DateTime DEFAULT now(), strings Array(String) DEFAULT emptyArrayString()) ENGINE = MergeTree(created_date, cityHash64(created_at), (created_date, cityHash64(created_at)), 8192);
INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', 'ccccc']);

View File

@ -1,6 +1,7 @@
drop table if exists radacct;
drop table if exists mv_traffic_by_tadig15min;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE radacct ( radacctid UInt64, f3gppchargingid Nullable(String), f3gppggsnaddress Nullable(String), f3gppggsnmccmnc Nullable(String), f3gppgprsqos Nullable(String), f3gppimeisv Nullable(String), f3gppimsi Nullable(UInt64), f3gppimsimccmnc Nullable(String), f3gpploci Nullable(String), f3gppnsapi Nullable(String), f3gpprattype Nullable(String), f3gppsgsnaddress Nullable(String), f3gppsgsnmccmnc Nullable(String), acctdelaytime Nullable(UInt32), acctinputoctets Nullable(UInt64), acctinputpackets Nullable(UInt64), acctoutputoctets Nullable(UInt64), acctoutputpackets Nullable(UInt64), acctsessionid String, acctstatustype Nullable(String), acctuniqueid String, calledstationid Nullable(String), callingstationid Nullable(String), framedipaddress Nullable(String), nasidentifier Nullable(String), nasipaddress Nullable(String), acctstarttime Nullable(DateTime), acctstoptime Nullable(DateTime), acctsessiontime Nullable(UInt32), acctterminatecause Nullable(String), acctstartdelay Nullable(UInt32), acctstopdelay Nullable(UInt32), connectinfo_start Nullable(String), connectinfo_stop Nullable(String), timestamp DateTime, username Nullable(String), realm Nullable(String), f3gppimsi_int UInt64, f3gppsgsnaddress_int Nullable(UInt32), timestamp_date Date, tac Nullable(String), mnc Nullable(String), tadig LowCardinality(String), country LowCardinality(String), tadig_op_ip Nullable(String) DEFAULT CAST('TADIG NOT FOUND', 'Nullable(String)'), mcc Nullable(UInt16) MATERIALIZED toUInt16OrNull(substring(f3gppsgsnmccmnc, 1, 6))) ENGINE = MergeTree(timestamp_date, (timestamp, radacctid, acctuniqueid), 8192);
insert into radacct values (1, 'a', 'b', 'c', 'd', 'e', 2, 'a', 'b', 'c', 'd', 'e', 'f', 3, 4, 5, 6, 7, 'a', 'Stop', 'c', 'd', 'e', 'f', 'g', 'h', '2018-10-10 15:54:21', '2018-10-10 15:54:21', 8, 'a', 9, 10, 'a', 'b', '2018-10-10 15:54:21', 'a', 'b', 11, 12, '2018-10-10', 'a', 'b', 'c', 'd', 'e');

Some files were not shown because too many files have changed in this diff Show More