mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #44501 from Avogar/validate-types
Validate data types according to settings.
This commit is contained in:
commit
59263f3ae1
@ -5,6 +5,7 @@
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#define MAX_FIXEDSTRING_SIZE 0xFFFFFF
|
||||
#define MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS 256
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -432,7 +432,7 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo
|
||||
settings.json.read_bools_as_numbers,
|
||||
settings.json.read_objects_as_strings,
|
||||
settings.json.read_numbers_as_strings,
|
||||
settings.json.try_infer_objects);
|
||||
settings.json.allow_object_type);
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
|
@ -103,7 +103,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
|
||||
format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata;
|
||||
format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8;
|
||||
format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name;
|
||||
format_settings.json.try_infer_objects = context->getSettingsRef().allow_experimental_object_type;
|
||||
format_settings.json.allow_object_type = context->getSettingsRef().allow_experimental_object_type;
|
||||
format_settings.null_as_default = settings.input_format_null_as_default;
|
||||
format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros;
|
||||
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
|
||||
|
@ -161,7 +161,7 @@ struct FormatSettings
|
||||
bool try_infer_numbers_from_strings = false;
|
||||
bool validate_types_from_metadata = true;
|
||||
bool validate_utf8 = false;
|
||||
bool try_infer_objects = false;
|
||||
bool allow_object_type = false;
|
||||
} json;
|
||||
|
||||
struct
|
||||
|
@ -366,7 +366,7 @@ namespace
|
||||
transformJSONTuplesAndArraysToArrays(data_types, settings, type_indexes, json_info);
|
||||
|
||||
/// Convert Maps to Objects if needed.
|
||||
if (settings.json.try_infer_objects)
|
||||
if (settings.json.allow_object_type)
|
||||
transformMapsAndObjectsToObjects(data_types, type_indexes);
|
||||
|
||||
if (settings.json.read_objects_as_strings)
|
||||
@ -716,7 +716,7 @@ namespace
|
||||
{
|
||||
if constexpr (is_json)
|
||||
{
|
||||
if (settings.json.try_infer_objects)
|
||||
if (settings.json.allow_object_type)
|
||||
return std::make_shared<DataTypeObject>("json", true);
|
||||
}
|
||||
/// Empty Map is Map(Nothing, Nothing)
|
||||
@ -735,7 +735,7 @@ namespace
|
||||
transformInferredTypesIfNeededImpl<is_json>(value_types, settings, json_info);
|
||||
if (!checkIfTypesAreEqual(value_types))
|
||||
{
|
||||
if (settings.json.try_infer_objects)
|
||||
if (settings.json.allow_object_type)
|
||||
return std::make_shared<DataTypeObject>("json", true);
|
||||
if (settings.json.read_objects_as_strings)
|
||||
return std::make_shared<DataTypeString>();
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
#include <Functions/FunctionsConversion.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -32,10 +33,11 @@ public:
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
explicit CastOverloadResolverImpl(std::optional<Diagnostic> diagnostic_, bool keep_nullable_, bool cast_ipv4_ipv6_default_on_conversion_error_)
|
||||
explicit CastOverloadResolverImpl(std::optional<Diagnostic> diagnostic_, bool keep_nullable_, bool cast_ipv4_ipv6_default_on_conversion_error_, const DataTypeValidationSettings & data_type_validation_settings_)
|
||||
: diagnostic(std::move(diagnostic_))
|
||||
, keep_nullable(keep_nullable_)
|
||||
, cast_ipv4_ipv6_default_on_conversion_error(cast_ipv4_ipv6_default_on_conversion_error_)
|
||||
, data_type_validation_settings(data_type_validation_settings_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -46,13 +48,13 @@ public:
|
||||
if constexpr (internal)
|
||||
return createImpl({}, false /*keep_nullable*/, settings_ref.cast_ipv4_ipv6_default_on_conversion_error);
|
||||
|
||||
return createImpl({}, settings_ref.cast_keep_nullable, settings_ref.cast_ipv4_ipv6_default_on_conversion_error);
|
||||
return createImpl({}, settings_ref.cast_keep_nullable, settings_ref.cast_ipv4_ipv6_default_on_conversion_error, DataTypeValidationSettings(settings_ref));
|
||||
}
|
||||
|
||||
static FunctionOverloadResolverPtr createImpl(std::optional<Diagnostic> diagnostic = {}, bool keep_nullable = false, bool cast_ipv4_ipv6_default_on_conversion_error = false)
|
||||
static FunctionOverloadResolverPtr createImpl(std::optional<Diagnostic> diagnostic = {}, bool keep_nullable = false, bool cast_ipv4_ipv6_default_on_conversion_error = false, const DataTypeValidationSettings & data_type_validation_settings = {})
|
||||
{
|
||||
assert(!internal || !keep_nullable);
|
||||
return std::make_unique<CastOverloadResolverImpl>(std::move(diagnostic), keep_nullable, cast_ipv4_ipv6_default_on_conversion_error);
|
||||
return std::make_unique<CastOverloadResolverImpl>(std::move(diagnostic), keep_nullable, cast_ipv4_ipv6_default_on_conversion_error, data_type_validation_settings);
|
||||
}
|
||||
|
||||
protected:
|
||||
@ -83,6 +85,7 @@ protected:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue<String>());
|
||||
validateDataType(type, data_type_validation_settings);
|
||||
|
||||
if constexpr (cast_type == CastType::accurateOrNull)
|
||||
return makeNullable(type);
|
||||
@ -104,6 +107,7 @@ private:
|
||||
std::optional<Diagnostic> diagnostic;
|
||||
bool keep_nullable;
|
||||
bool cast_ipv4_ipv6_default_on_conversion_error;
|
||||
DataTypeValidationSettings data_type_validation_settings;
|
||||
};
|
||||
|
||||
|
||||
|
@ -76,8 +76,6 @@
|
||||
#include <Functions/UserDefined/UserDefinedSQLFunctionVisitor.h>
|
||||
|
||||
|
||||
#define MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS 256
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -4,6 +4,9 @@
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,9 +15,66 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
|
||||
}
|
||||
|
||||
ColumnsDescription parseColumnsListFromString(const std::string & structure, ContextPtr context)
|
||||
void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings & settings)
|
||||
{
|
||||
if (!settings.allow_suspicious_low_cardinality_types)
|
||||
{
|
||||
if (const auto * lc_type = typeid_cast<const DataTypeLowCardinality *>(type.get()))
|
||||
{
|
||||
if (!isStringOrFixedString(*removeNullable(lc_type->getDictionaryType())))
|
||||
throw Exception(
|
||||
ErrorCodes::SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY,
|
||||
"Creating columns of type {} is prohibited by default due to expected negative impact on performance. "
|
||||
"It can be enabled with the \"allow_suspicious_low_cardinality_types\" setting.",
|
||||
lc_type->getName());
|
||||
}
|
||||
}
|
||||
|
||||
if (!settings.allow_experimental_geo_types)
|
||||
{
|
||||
const auto & type_name = type->getName();
|
||||
if (type_name == "MultiPolygon" || type_name == "Polygon" || type_name == "Ring" || type_name == "Point")
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Cannot create column with type '{}' because experimental geo types are not allowed. Set setting "
|
||||
"allow_experimental_geo_types = 1 in order to allow it", type_name);
|
||||
}
|
||||
}
|
||||
|
||||
if (!settings.allow_experimental_object_type)
|
||||
{
|
||||
if (type->hasDynamicSubcolumns())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Cannot create column with type '{}' because experimental Object type is not allowed. "
|
||||
"Set setting allow_experimental_object_type = 1 in order to allow it", type->getName());
|
||||
}
|
||||
}
|
||||
|
||||
if (!settings.allow_suspicious_fixed_string_types)
|
||||
{
|
||||
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 column with type '{}' because fixed string with size > {} is suspicious. "
|
||||
"Set setting allow_suspicious_fixed_string_types = 1 in order to allow it",
|
||||
type->getName(),
|
||||
MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ColumnsDescription parseColumnsListFromString(const std::string & structure, const ContextPtr & context)
|
||||
{
|
||||
ParserColumnDeclarationList parser(true, true);
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
@ -25,10 +85,14 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, Con
|
||||
if (!columns_list)
|
||||
throw Exception("Could not cast AST to ASTExpressionList", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false);
|
||||
auto columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false);
|
||||
auto validation_settings = DataTypeValidationSettings(context->getSettingsRef());
|
||||
for (const auto & [name, type] : columns.getAll())
|
||||
validateDataType(type, validation_settings);
|
||||
return columns;
|
||||
}
|
||||
|
||||
bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, ContextPtr context)
|
||||
bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, const ContextPtr & context)
|
||||
{
|
||||
ParserColumnDeclarationList parser(true, true);
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
@ -47,6 +111,9 @@ bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescrip
|
||||
try
|
||||
{
|
||||
columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false);
|
||||
auto validation_settings = DataTypeValidationSettings(context->getSettingsRef());
|
||||
for (const auto & [name, type] : columns.getAll())
|
||||
validateDataType(type, validation_settings);
|
||||
return true;
|
||||
}
|
||||
catch (...)
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <string>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -9,9 +10,29 @@ namespace DB
|
||||
|
||||
class Context;
|
||||
|
||||
/// Parses a common argument for table functions such as table structure given in string
|
||||
ColumnsDescription parseColumnsListFromString(const std::string & structure, ContextPtr context);
|
||||
struct DataTypeValidationSettings
|
||||
{
|
||||
DataTypeValidationSettings() = default;
|
||||
|
||||
bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, ContextPtr context);
|
||||
explicit DataTypeValidationSettings(const Settings & settings)
|
||||
: allow_suspicious_low_cardinality_types(settings.allow_suspicious_low_cardinality_types)
|
||||
, allow_experimental_geo_types(settings.allow_experimental_geo_types)
|
||||
, allow_experimental_object_type(settings.allow_experimental_object_type)
|
||||
, allow_suspicious_fixed_string_types(settings.allow_suspicious_fixed_string_types)
|
||||
{
|
||||
}
|
||||
|
||||
bool allow_suspicious_low_cardinality_types = true;
|
||||
bool allow_experimental_geo_types = true;
|
||||
bool allow_experimental_object_type = true;
|
||||
bool allow_suspicious_fixed_string_types = true;
|
||||
};
|
||||
|
||||
void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings & settings);
|
||||
|
||||
/// Parses a common argument for table functions such as table structure given in string
|
||||
ColumnsDescription parseColumnsListFromString(const std::string & structure, const ContextPtr & context);
|
||||
|
||||
bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, const ContextPtr & context);
|
||||
|
||||
}
|
||||
|
@ -12,6 +12,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_)
|
||||
@ -206,6 +207,15 @@ void JSONAsObjectRowInputFormat::readJSONObject(IColumn & column)
|
||||
serializations[0]->deserializeTextJSON(column, *buf, format_settings);
|
||||
}
|
||||
|
||||
JSONAsObjectExternalSchemaReader::JSONAsObjectExternalSchemaReader(const FormatSettings & settings)
|
||||
{
|
||||
if (!settings.json.allow_object_type)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Cannot infer the data structure in JSONAsObject format because experimental Object type is not allowed. Set setting "
|
||||
"allow_experimental_object_type = 1 in order to allow it");
|
||||
}
|
||||
|
||||
void registerInputFormatJSONAsString(FormatFactory & factory)
|
||||
{
|
||||
factory.registerInputFormat("JSONAsString", [](
|
||||
@ -260,9 +270,9 @@ void registerFileSegmentationEngineJSONAsObject(FormatFactory & factory)
|
||||
|
||||
void registerJSONAsObjectSchemaReader(FormatFactory & factory)
|
||||
{
|
||||
factory.registerExternalSchemaReader("JSONAsObject", [](const FormatSettings &)
|
||||
factory.registerExternalSchemaReader("JSONAsObject", [](const FormatSettings & settings)
|
||||
{
|
||||
return std::make_shared<JSONAsObjectExternalSchemaReader>();
|
||||
return std::make_shared<JSONAsObjectExternalSchemaReader>(settings);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -77,6 +77,8 @@ public:
|
||||
class JSONAsObjectExternalSchemaReader : public IExternalSchemaReader
|
||||
{
|
||||
public:
|
||||
JSONAsObjectExternalSchemaReader(const FormatSettings & settings);
|
||||
|
||||
NamesAndTypesList readSchema() override
|
||||
{
|
||||
return {{"json", std::make_shared<DataTypeObject>("json", false)}};
|
||||
|
@ -23,6 +23,6 @@ $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_01543"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT number % 2 ? number: NULL as x from numbers(10) FORMAT Avro" > $USER_FILES_PATH/test_01543.avro
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM file('test_01543.avro', 'Avro', 'x LowCardinality(Nullable(UInt64))')"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM file('test_01543.avro', 'Avro', 'x LowCardinality(Nullable(UInt64))')" --allow_suspicious_low_cardinality_types 1
|
||||
|
||||
rm $USER_FILES_PATH/test_01543.avro
|
||||
|
@ -17,7 +17,7 @@ done
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_files"
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_files (file String, data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON')"
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON')" --allow_experimental_object_type 1
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1"
|
||||
@ -26,7 +26,7 @@ ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE IF EXISTS t_json_files"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \
|
||||
SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON') \
|
||||
ORDER BY _file LIMIT 3" --max_threads 1 --min_insert_block_size_rows 1 --max_insert_block_size 1 --max_block_size 1
|
||||
ORDER BY _file LIMIT 3" --max_threads 1 --min_insert_block_size_rows 1 --max_insert_block_size 1 --max_block_size 1 --allow_experimental_object_type 1
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1"
|
||||
@ -35,7 +35,7 @@ ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE IF EXISTS t_json_files"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \
|
||||
SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON') \
|
||||
WHERE _file IN ('01825_file_1.json', '01825_file_3.json')"
|
||||
WHERE _file IN ('01825_file_1.json', '01825_file_3.json')" --allow_experimental_object_type 1
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1"
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
SET allow_experimental_object_type = 1;
|
||||
SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON);
|
||||
SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON), dummy;
|
||||
SELECT materialize('{"k":"v"}'::JSON) SETTINGS extremes = 1;
|
||||
|
@ -1,4 +1,6 @@
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/30231
|
||||
SET allow_suspicious_low_cardinality_types=1;
|
||||
|
||||
SELECT *
|
||||
FROM (
|
||||
SELECT number,
|
||||
|
@ -22,6 +22,9 @@ SELECT toTypeName(sumCount(v)), sumCount(v) FROM
|
||||
)
|
||||
ORDER BY v
|
||||
);
|
||||
|
||||
SET allow_suspicious_low_cardinality_types=1;
|
||||
|
||||
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
|
||||
(
|
||||
SELECT v FROM
|
||||
|
@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "insert into table function file('arrow.dict', 'Arrow', 'x LowCardinality(UInt64)') select number from numbers(10) settings output_format_arrow_low_cardinality_as_dictionary=1, engine_file_truncate_on_insert=1"
|
||||
$CLICKHOUSE_CLIENT -q "insert into table function file('arrow.dict', 'Arrow', 'x LowCardinality(UInt64)') select number from numbers(10) settings output_format_arrow_low_cardinality_as_dictionary=1, engine_file_truncate_on_insert=1, allow_suspicious_low_cardinality_types=1"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file('arrow.dict', 'Arrow')"
|
||||
|
||||
|
19
tests/queries/0_stateless/02513_validate_data_types.sql
Normal file
19
tests/queries/0_stateless/02513_validate_data_types.sql
Normal file
@ -0,0 +1,19 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
set allow_experimental_object_type=0;
|
||||
select CAST('{"x" : 1}', 'JSON'); -- {serverError ILLEGAL_COLUMN}
|
||||
desc file(nonexist.json, JSONAsObject); -- {serverError ILLEGAL_COLUMN}
|
||||
desc file(nonexist.json, JSONEachRow, 'x JSON'); -- {serverError ILLEGAL_COLUMN}
|
||||
|
||||
set allow_experimental_geo_types=0;
|
||||
select CAST([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]], 'Polygon'); -- {serverError ILLEGAL_COLUMN}
|
||||
desc file(nonexist.json, JSONEachRow, 'pg Polygon'); -- {serverError ILLEGAL_COLUMN}
|
||||
|
||||
set allow_suspicious_low_cardinality_types=0;
|
||||
select CAST(1000000, 'LowCardinality(UInt64)'); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY}
|
||||
desc file(nonexist.json, JSONEachRow, 'lc LowCardinality(UInt64)'); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY}
|
||||
|
||||
set allow_suspicious_fixed_string_types=0;
|
||||
select CAST('', 'FixedString(1000)'); -- {serverError ILLEGAL_COLUMN}
|
||||
desc file(nonexist.json, JSONEachRow, 'fs FixedString(1000)'); -- {serverError ILLEGAL_COLUMN}
|
||||
|
Loading…
Reference in New Issue
Block a user