fix tests

This commit is contained in:
Alexander Tokmakov 2021-06-01 13:24:06 +03:00
parent fe482109da
commit 94d2aed336
5 changed files with 32 additions and 14 deletions

View File

@ -22,6 +22,7 @@
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/formatAST.h>
#include <Common/Macros.h>
@ -317,10 +318,25 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_
if (!replicated_table || !create->storage->engine->arguments)
return;
ASTs & args = create->storage->engine->arguments->children;
ASTs & args_ref = create->storage->engine->arguments->children;
ASTs args = args_ref;
if (args.size() < 2)
return;
/// It can be a constant expression. Try to evaluate it, ignore exception if we cannot.
bool has_expression_argument = args_ref[0]->as<ASTFunction>() || args_ref[0]->as<ASTFunction>();
if (has_expression_argument)
{
try
{
args[0] = evaluateConstantExpressionAsLiteral(args_ref[0]->clone(), query_context);
args[1] = evaluateConstantExpressionAsLiteral(args_ref[1]->clone(), query_context);
}
catch (...)
{
}
}
ASTLiteral * arg1 = args[0]->as<ASTLiteral>();
ASTLiteral * arg2 = args[1]->as<ASTLiteral>();
if (!arg1 || !arg2 || arg1->value.getType() != Field::Types::String || arg2->value.getType() != Field::Types::String)
@ -348,12 +364,12 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_
if (maybe_shard_macros && maybe_replica_macros)
return;
if (enable_functional_tests_helper)
if (enable_functional_tests_helper && !has_expression_argument)
{
if (maybe_path.empty() || maybe_path.back() != '/')
maybe_path += '/';
arg1->value = maybe_path + "auto_{shard}";
arg2->value = maybe_replica + "auto_{replica}";
args_ref[0]->as<ASTLiteral>()->value = maybe_path + "auto_{shard}";
args_ref[1]->as<ASTLiteral>()->value = maybe_replica + "auto_{replica}";
return;
}

View File

@ -7,12 +7,12 @@ DROP TABLE IF EXISTS alter_compression_codec2;
CREATE TABLE alter_compression_codec1 (
somedate Date CODEC(LZ4),
id UInt64 CODEC(NONE)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs', '1') PARTITION BY somedate ORDER BY id;
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '1_{replica}') PARTITION BY somedate ORDER BY id;
CREATE TABLE alter_compression_codec2 (
somedate Date CODEC(LZ4),
id UInt64 CODEC(NONE)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs', '2') PARTITION BY somedate ORDER BY id;
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/'||currentDatabase()||'alter_compression_codecs/{shard}', '2_{replica}') PARTITION BY somedate ORDER BY id;
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 1);
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 2);

View File

@ -10,8 +10,8 @@
3 3
4 4
0
CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01710_projection_fetch_default\', \'2\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192
CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192
2
CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01710_projection_fetch_default\', \'2\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192
CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01710_projection_fetch_default\', \'2\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192
CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01710_projection_fetch_default\', \'2\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192
CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192
CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n ),\n PROJECTION pp\n (\n SELECT \n x,\n count()\n GROUP BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192
CREATE TABLE default.tp_2\n(\n `x` Int32,\n `y` Int32,\n PROJECTION p\n (\n SELECT \n x,\n y\n ORDER BY x\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{shard}/01710_projection_fetch_default\', \'2_{replica}\')\nORDER BY y\nSETTINGS min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32, index_granularity = 8192

View File

@ -1,9 +1,9 @@
drop table if exists tp_1;
drop table if exists tp_2;
create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/01710_projection_fetch_' || currentDatabase(), '1') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32;
create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '1_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32;
create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/01710_projection_fetch_' || currentDatabase(), '2') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32;
create table tp_2 (x Int32, y Int32, projection p (select x, y order by x)) engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/01710_projection_fetch_' || currentDatabase(), '2_{replica}') order by y settings min_rows_for_compact_part = 2, min_rows_for_wide_part = 4, min_bytes_for_compact_part = 16, min_bytes_for_wide_part = 32;
insert into tp_1 select number, number from numbers(3);

View File

@ -109,6 +109,7 @@
"01153_attach_mv_uuid"
],
"database-replicated": [
/// Unclassified
"memory_tracking",
"memory_usage",
"live_view",
@ -167,8 +168,9 @@
/// Does not support renaming of multiple tables in single query
"00634_rename_view",
"00140_rename",
"01783_http_chunk_size",
"01710_projection_fetch"
/// Requires investigation
"00953_zookeeper_suetin_deduplication_bug",
"01783_http_chunk_size"
],
"polymorphic-parts": [
"01508_partition_pruning_long", /// bug, shoud be fixed