Merge pull request #41495 from canhld94/ch_fixedstring_len

Add settings allow_suspicious_fixed_string_types
This commit is contained in:
Nikolay Degterinsky 2022-09-21 19:35:34 +02:00 committed by GitHub
commit 49e0a87b8f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 25 additions and 1 deletions

View File

@ -127,6 +127,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Float, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \
\
M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \
M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \
M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \

View File

@ -74,8 +74,11 @@
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/logger_useful.h>
#include <DataTypes/DataTypeFixedString.h>
#define MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS 256
namespace DB
{
@ -810,6 +813,22 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
}
}
}
if (!create.attach && !settings.allow_suspicious_fixed_string_types)
{
for (const auto & [name, type] : properties.columns.getAllPhysical())
{
auto basic_type = removeLowCardinality(removeNullable(type));
if (const auto * fixed_string = typeid_cast<const DataTypeFixedString *>(basic_type.get()))
{
if (fixed_string->getN() > MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS)
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Cannot create table with column '{}' which type is '{}' "
"because fixed string with size > {} is suspicious. "
"Set setting allow_suspicious_fixed_string_types = 1 in order to allow it",
name, type->getName(), MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS);
}
}
}
}
String InterpreterCreateQuery::getTableEngineName(DefaultTableEngine default_table_engine)

View File

@ -0,0 +1,4 @@
CREATE TABLE fixed_string (id UInt64, s FixedString(256)) ENGINE = MergeTree() ORDER BY id;
CREATE TABLE suspicious_fixed_string (id UInt64, s FixedString(257)) ENGINE = MergeTree() ORDER BY id; -- { serverError 44 }
SET allow_suspicious_fixed_string_types = 1;
CREATE TABLE suspicious_fixed_string (id UInt64, s FixedString(257)) ENGINE = MergeTree() ORDER BY id;

View File

@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_LOCAL} --structure 'x LowCardinality(FixedString(2454139))' --input-format Values --output-format TSV --query "SELECT * FROM table" <<< '(1)' | wc -c
${CLICKHOUSE_LOCAL} --allow_suspicious_fixed_string_types 1 --structure 'x LowCardinality(FixedString(2454139))' --input-format Values --output-format TSV --query "SELECT * FROM table" <<< '(1)' | wc -c