提交 5279c7ff 编写于 作者: N Nikita Vasilev

setting

上级 787e457d
......@@ -300,6 +300,7 @@ struct Settings
M(SettingBool, allow_experimental_cross_to_join_conversion, false, "Convert CROSS JOIN to INNER JOIN if possible") \
M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.") \
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 for 'mysql' table function.") \
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.")\
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};
......
......@@ -14,6 +14,7 @@ struct IndicesDescription
IndicesDescription() = default;
bool empty() const { return indices.empty(); }
String toString() const;
static IndicesDescription parse(const String & str);
......
......@@ -72,6 +72,7 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int MEMORY_LIMIT_EXCEEDED;
extern const int SYNTAX_ERROR;
extern const int INVALID_PARTITION_VALUE;
......@@ -1051,7 +1052,7 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to)
}
void MergeTreeData::checkAlter(const AlterCommands & commands)
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();
......@@ -1060,6 +1061,11 @@ void MergeTreeData::checkAlter(const AlterCommands & commands)
ASTPtr new_primary_key_ast = primary_key_ast;
commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast);
if (getIndicesDescription().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_forbidden;
......
......@@ -481,7 +481,7 @@ public:
/// - all type conversions can be done.
/// - columns corresponding to primary key, indices, sign, sampling expression and date are not affected.
/// If something is wrong, throws an exception.
void checkAlter(const AlterCommands & commands);
void checkAlter(const AlterCommands & commands, const Context & context);
/// Performs ALTER of the data part, writes the result to temporary files.
/// Returns an object allowing to rename temporary files to permanent files.
......
......@@ -610,6 +610,10 @@ 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.data_path, args.database_name, args.table_name,
......
......@@ -211,7 +211,7 @@ void StorageMergeTree::alter(
auto table_soft_lock = lockDataForAlter();
data.checkAlter(params);
data.checkAlter(params, context);
auto new_columns = data.getColumns();
auto new_indices = data.getIndicesDescription();
......
......@@ -3125,7 +3125,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
if (is_readonly)
throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY);
data.checkAlter(params);
data.checkAlter(params, query_context);
ColumnsDescription new_columns = data.getColumns();
IndicesDescription new_indices = data.getIndicesDescription();
......
DROP TABLE IF EXISTS test.minmax_idx;
DROP TABLE IF EXISTS test.minmax_idx2;
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE test.minmax_idx
(
u64 UInt64,
......
......@@ -3,6 +3,8 @@ 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
(
u64 UInt64,
......
......@@ -3,9 +3,12 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.minmax_idx
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE test.minmax_idx
(
u64 UInt64,
i32 Int32,
......@@ -19,7 +22,7 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE test.minmax_idx
INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
) ENGINE = MergeTree()
ORDER BY u64
SETTINGS index_granularity = 2"
SETTINGS index_granularity = 2;"
$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES
......
DROP TABLE IF EXISTS test.minmax_idx1;
DROP TABLE IF EXISTS test.minmax_idx2;
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE test.minmax_idx1
(
u64 UInt64,
......
......@@ -5,7 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.set_idx;"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.set_idx
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_data_skipping_indices = 1;
CREATE TABLE test.set_idx
(
u64 UInt64,
i32 Int32,
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册