未验证 提交 15a0f39e 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #7974 from ClickHouse/enable-experimental-indices-and-make-settings-obsolete

Make data skipping indices non-experimental. Make settings about data skipping indices, multiple JOINs and CROSS to INNER JOIN conversion obsolete.
......@@ -358,11 +358,8 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.", 0) \
M(SettingBool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \
M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \
M(SettingBool, allow_experimental_multiple_joins_emulation, true, "Emulate multiple joins using subselects", 0) \
M(SettingBool, allow_experimental_cross_to_join_conversion, true, "Convert CROSS JOIN to INNER JOIN if possible", 0) \
M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \
M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.", 0) \
M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.", 0) \
\
M(SettingBool, experimental_use_processors, false, "Use processors pipeline.", 0) \
\
......@@ -390,11 +387,13 @@ struct Settings : public SettingsCollection<Settings>
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\
M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \
M(SettingBool, compile, false, "Obsolete setting, does nothing. Will be removed after 2020-03-13", 0) \
M(SettingBool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13", 0) \
M(SettingUInt64, min_count_to_compile, 0, "Obsolete setting, does nothing. Will be removed after 2020-03-13", 0) \
M(SettingBool, allow_experimental_multiple_joins_emulation, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(SettingBool, allow_experimental_cross_to_join_conversion, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(SettingBool, allow_experimental_data_skipping_indices, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(SettingBool, merge_tree_uniform_read_distribution, true, "Obsolete setting, does nothing. Will be removed after 2020-05-20", 0) \
DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS)
/** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings).
......
......@@ -242,17 +242,11 @@ InterpreterSelectQuery::InterpreterSelectQuery(
throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(),
ErrorCodes::TOO_DEEP_SUBQUERIES);
if (settings.allow_experimental_cross_to_join_conversion)
{
CrossToInnerJoinVisitor::Data cross_to_inner;
CrossToInnerJoinVisitor(cross_to_inner).visit(query_ptr);
}
CrossToInnerJoinVisitor::Data cross_to_inner;
CrossToInnerJoinVisitor(cross_to_inner).visit(query_ptr);
if (settings.allow_experimental_multiple_joins_emulation)
{
JoinToSubqueryTransformVisitor::Data join_to_subs_data{*context};
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr);
}
JoinToSubqueryTransformVisitor::Data join_to_subs_data{*context};
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr);
max_streams = settings.max_threads;
auto & query = getSelectQuery();
......
......@@ -239,7 +239,7 @@ static const ASTTablesInSelectQueryElement * getFirstTableJoin(const ASTSelectQu
if (!joined_table)
joined_table = &tables_element;
else
throw Exception("Multiple JOIN disabled or does not support the query.", ErrorCodes::NOT_IMPLEMENTED);
throw Exception("Multiple JOIN does not support the query.", ErrorCodes::NOT_IMPLEMENTED);
}
}
......
......@@ -1357,7 +1357,7 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to)
}
void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & context)
void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & /*context*/)
{
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
auto new_columns = getColumns();
......@@ -1368,10 +1368,6 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
ASTPtr new_ttl_table_ast = ttl_table_ast;
SettingsChanges new_changes;
commands.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes);
if (getIndices().empty() && !new_indices.empty() &&
!context.getSettingsRef().allow_experimental_data_skipping_indices)
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
/// Set of columns that shouldn't be altered.
NameSet columns_alter_type_forbidden;
......
......@@ -633,10 +633,6 @@ static StoragePtr create(const StorageFactory::Arguments & args)
ErrorCodes::BAD_ARGUMENTS);
}
if (!args.attach && !indices_description.empty() && !args.local_context.getSettingsRef().allow_experimental_data_skipping_indices)
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
if (replicated)
return StorageReplicatedMergeTree::create(
zookeeper_path, replica_name, args.attach, args.database_name, args.table_name, args.relative_data_path,
......
......@@ -13,8 +13,6 @@ INSERT INTO table2 SELECT number * 2, number * 20 FROM numbers(11);
INSERT INTO table3 SELECT number * 30, number * 300 FROM numbers(10);
INSERT INTO table5 SELECT number * 5, number * 50, number * 500 FROM numbers(10);
SET allow_experimental_multiple_joins_emulation = 1;
select t1.a, t2.b, t3.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b;
select t1.a, t2.b, t5.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on t1.a = t5.a AND t2.b = t5.b;
......
......@@ -13,7 +13,6 @@ INSERT INTO table2 SELECT number * 2, number * 20 FROM numbers(11);
INSERT INTO table3 SELECT number * 30, number * 300 FROM numbers(10);
INSERT INTO table5 SELECT number * 5, number * 50, number * 500 FROM numbers(10);
SET allow_experimental_multiple_joins_emulation = 1;
SET joined_subquery_requires_alias = 1;
select t1.a, t2.b, t3.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b;
......
0 0
0 0
cross
1 1 1 1
1 1 1 2
2 2 2 \N
1 1 1 1
1 1 1 2
2 2 2 \N
cross nullable
1 1 1 1
2 2 1 2
1 1 1 1
2 2 1 2
cross nullable vs not nullable
1 1 1 1
2 2 1 2
1 1 1 1
2 2 1 2
cross self
1 1 1 1
2 2 2 2
1 1 1 1
2 2 2 2
cross one table expr
1 1 1 1
1 1 1 2
......@@ -31,23 +21,12 @@ cross one table expr
2 2 1 2
2 2 2 \N
2 2 3 \N
1 1 1 1
1 1 1 2
1 1 2 \N
1 1 3 \N
2 2 1 1
2 2 1 2
2 2 2 \N
2 2 3 \N
cross multiple ands
1 1 1 1
1 1 1 1
cross and inside and
1 1 1 1
1 1 1 1
cross split conjunction
1 1 1 1
1 1 1 1
comma
1 1 1 1
1 1 1 2
......@@ -56,26 +35,18 @@ comma nullable
1 1 1 1
2 2 1 2
cross
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
cross nullable
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\n, t2_00826\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
cross nullable vs not nullable
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = t2_00826.b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b
cross self
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nCROSS JOIN t1_00826 AS y\nWHERE (a = y.a) AND (b = y.b)
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b)\nWHERE (a = y.a) AND (b = y.b)
cross one table expr
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = b
cross multiple ands
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
cross and inside and
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
cross split conjunction
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
SET enable_debug_queries = 1;
SET enable_optimize_predicate_expression = 0;
set allow_experimental_cross_to_join_conversion = 0;
select * from system.one l cross join system.one r;
set allow_experimental_cross_to_join_conversion = 1;
select * from system.one l cross join system.one r;
DROP TABLE IF EXISTS t1_00826;
......@@ -17,50 +14,21 @@ INSERT INTO t2_00826 values (1,1), (1,2);
INSERT INTO t2_00826 (a) values (2), (3);
SELECT 'cross';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a;
SELECT 'cross nullable';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b;
SELECT 'cross nullable vs not nullable';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b;
SELECT 'cross self';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b;
SELECT 'cross one table expr';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t1_00826.b order by (t1_00826.a, t2_00826.a, t2_00826.b);
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t1_00826.b order by (t1_00826.a, t2_00826.a, t2_00826.b);
SELECT 'cross multiple ands';
SET allow_experimental_cross_to_join_conversion = 0;
--SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a = t2_00826.a;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b;
SELECT 'cross and inside and';
SET allow_experimental_cross_to_join_conversion = 0;
--SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b));
--SELECT * FROM t1_00826 x cross join t2_00826 y where t1_00826.a = t2_00826.a and (t1_00826.b = t2_00826.b and (x.a = y.a and x.b = y.b));
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.b = t2_00826.b and 1);
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.b = t2_00826.b and 1);
SELECT 'cross split conjunction';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a >= 1 and t2_00826.b = 1;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a >= 1 and t2_00826.b = 1;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT 'comma';
SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a;
......@@ -69,30 +37,22 @@ SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b;
SELECT 'cross';
SET allow_experimental_cross_to_join_conversion = 0; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a;
SET allow_experimental_cross_to_join_conversion = 1; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a;
ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a;
SELECT 'cross nullable';
SET allow_experimental_cross_to_join_conversion = 0; ANALYZE SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a;
SET allow_experimental_cross_to_join_conversion = 1; ANALYZE SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a;
ANALYZE SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a;
SELECT 'cross nullable vs not nullable';
SET allow_experimental_cross_to_join_conversion = 0; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b;
SET allow_experimental_cross_to_join_conversion = 1; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b;
ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b;
SELECT 'cross self';
SET allow_experimental_cross_to_join_conversion = 0; ANALYZE SELECT * FROM t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b;
SET allow_experimental_cross_to_join_conversion = 1; ANALYZE SELECT * FROM t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b;
ANALYZE SELECT * FROM t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b;
SELECT 'cross one table expr';
SET allow_experimental_cross_to_join_conversion = 0; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t1_00826.b;
SET allow_experimental_cross_to_join_conversion = 1; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t1_00826.b;
ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t1_00826.b;
SELECT 'cross multiple ands';
SET allow_experimental_cross_to_join_conversion = 0; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b;
SET allow_experimental_cross_to_join_conversion = 1; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b;
ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b;
SELECT 'cross and inside and';
SET allow_experimental_cross_to_join_conversion = 0; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b));
SET allow_experimental_cross_to_join_conversion = 1; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b));
ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b));
SELECT 'cross split conjunction';
SET allow_experimental_cross_to_join_conversion = 0; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a >= 1 and t2_00826.b > 0;
SET allow_experimental_cross_to_join_conversion = 1; ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a >= 1 and t2_00826.b > 0;
ANALYZE SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a >= 1 and t2_00826.b > 0;
DROP TABLE t1_00826;
DROP TABLE t2_00826;
DROP TABLE IF EXISTS minmax_idx;
DROP TABLE IF EXISTS minmax_idx2;
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE minmax_idx
(
......
......@@ -3,7 +3,6 @@ DROP TABLE IF EXISTS test.minmax_idx_r;
DROP TABLE IF EXISTS test.minmax_idx2;
DROP TABLE IF EXISTS test.minmax_idx2_r;
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE test.minmax_idx
(
......
#!/usr/bin/env bash
CLICKHOUSE_CLIENT_OPT="--allow_experimental_data_skipping_indices=1"
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
......@@ -9,7 +8,6 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE minmax_idx
(
u64 UInt64,
......
DROP TABLE IF EXISTS minmax_idx1;
DROP TABLE IF EXISTS minmax_idx2;
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE minmax_idx1
(
......
#!/usr/bin/env bash
CLICKHOUSE_CLIENT_OPT="--allow_experimental_data_skipping_indices=1"
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
......@@ -8,7 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE set_idx
(
u64 UInt64,
......
SET joined_subquery_requires_alias = 0;
set allow_experimental_multiple_joins_emulation = 0;
set allow_experimental_cross_to_join_conversion = 0;
select ax, c from (select [1,2] ax, 0 c) array join ax join (select 0 c) using(c);
select ax, c from (select [3,4] ax, 0 c) join (select 0 c) using(c) array join ax;
select ax, c from (select [5,6] ax, 0 c) s1 join system.one s2 ON s1.c = s2.dummy array join ax;
set allow_experimental_multiple_joins_emulation = 1;
set allow_experimental_cross_to_join_conversion = 1;
select ax, c from (select [1,2] ax, 0 c) array join ax join (select 0 c) using(c);
select ax, c from (select [3,4] ax, 0 c) join (select 0 c) using(c) array join ax;
select ax, c from (select [5,6] ax, 0 c) s1 join system.one s2 ON s1.c = s2.dummy array join ax;
......
......@@ -6,7 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE set_idx
(
u64 UInt64,
......
SET allow_experimental_data_skipping_indices=1;
drop table if exists nullable_set_index;
create table nullable_set_index (a UInt64, b Nullable(String), INDEX b_index b TYPE set(0) GRANULARITY 8192) engine = MergeTree order by a;
......
SET allow_experimental_data_skipping_indices=1;
drop table if exists null_lc_set_index;
......
......@@ -10,7 +10,6 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx2;"
# NGRAM BF
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE bloom_filter_idx
(
k UInt64,
......@@ -21,7 +20,6 @@ ORDER BY k
SETTINGS index_granularity = 2;"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE bloom_filter_idx2
(
k UInt64,
......@@ -105,7 +103,6 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM bloom_filter_idx WHERE (s, lower(s)) I
# TOKEN BF
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE bloom_filter_idx3
(
k UInt64,
......@@ -144,7 +141,6 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE bloom_filter_idx3"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx_na;"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE bloom_filter_idx_na
(
na Array(Array(String)),
......
......@@ -8,7 +8,6 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices=1;
CREATE TABLE test.minmax_idx
(
u64 UInt64,
......
......@@ -35,7 +35,6 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;"
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices=1;
ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;"
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;"
......
......@@ -8,7 +8,6 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices=1;
CREATE TABLE test.minmax_idx
(
u64 UInt64,
......
......@@ -8,5 +8,5 @@ set -e
for sequence in 1 10 100 1000 10000 100000 1000000 10000000 100000000 1000000000; do \
rate=`echo "1 $sequence" | awk '{printf("%0.9f\n",$1/$2)}'`
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.bloom_filter_idx";
$CLICKHOUSE_CLIENT --allow_experimental_data_skipping_indices=1 --query="CREATE TABLE test.bloom_filter_idx ( u64 UInt64, i32 Int32, f64 Float64, d Decimal(10, 2), s String, e Enum8('a' = 1, 'b' = 2, 'c' = 3), dt Date, INDEX bloom_filter_a i32 TYPE bloom_filter($rate) GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.bloom_filter_idx ( u64 UInt64, i32 Int32, f64 Float64, d Decimal(10, 2), s String, e Enum8('a' = 1, 'b' = 2, 'c' = 3), dt Date, INDEX bloom_filter_a i32 TYPE bloom_filter($rate) GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192"
done
DROP TABLE IF EXISTS min_max_with_nullable_string;
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE min_max_with_nullable_string (
t DateTime,
......
SET allow_experimental_data_skipping_indices = 1;
DROP TABLE IF EXISTS test.single_column_bloom_filter;
......
SET send_logs_level = 'none';
SET allow_experimental_data_skipping_indices = 1;
DROP TABLE IF EXISTS mt_with_pk;
......
......@@ -7,7 +7,6 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx;"
# NGRAM BF
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE bloom_filter_idx
(
k UInt64,
......
......@@ -6,7 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE set_idx
(
k UInt64,
......
SET allow_experimental_data_skipping_indices = 1;
DROP TABLE IF EXISTS indexed_table;
......
......@@ -9,7 +9,6 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions2;"
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices=1;
CREATE TABLE test.indices_mutaions1
(
u64 UInt64,
......
SET allow_experimental_data_skipping_indices = 1;
DROP TABLE IF EXISTS test.set_index_not;
......
SET allow_experimental_data_skipping_indices = 1;
DROP TABLE IF EXISTS bloom_filter;
......
SET allow_experimental_data_skipping_indices = 1;
DROP TABLE IF EXISTS test.set_array;
......
SET allow_experimental_data_skipping_indices=1;
DROP TABLE IF EXISTS test_vertical_merge;
CREATE TABLE test_vertical_merge (
......
SET allow_experimental_data_skipping_indices=1;
DROP TABLE IF EXISTS bad_skip_idx;
......
SET allow_experimental_data_skipping_indices=1;
CREATE TABLE foo (key int, INDEX i1 key TYPE minmax GRANULARITY 1) Engine=MergeTree() ORDER BY key;
CREATE TABLE as_foo AS foo;
CREATE TABLE dist (key int, INDEX i1 key TYPE minmax GRANULARITY 1) Engine=Distributed(test_shard_localhost, currentDatabase(), 'foo'); -- { serverError 36 }
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册