Merge pull request #44501 from Avogar/validate-types

Validate data types according to settings.
This commit is contained in:
Kruglov Pavel 2023-01-04 14:48:09 +01:00 committed by GitHub
commit 59263f3ae1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 153 additions and 25 deletions

View File

@ -5,6 +5,7 @@
#include <Common/Exception.h> #include <Common/Exception.h>
#define MAX_FIXEDSTRING_SIZE 0xFFFFFF #define MAX_FIXEDSTRING_SIZE 0xFFFFFF
#define MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS 256
namespace DB namespace DB

View File

@ -432,7 +432,7 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo
settings.json.read_bools_as_numbers, settings.json.read_bools_as_numbers,
settings.json.read_objects_as_strings, settings.json.read_objects_as_strings,
settings.json.read_numbers_as_strings, settings.json.read_numbers_as_strings,
settings.json.try_infer_objects); settings.json.allow_object_type);
break; break;
default: default:
break; break;

View File

@ -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_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.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_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.null_as_default = settings.input_format_null_as_default;
format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros; format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros;
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;

View File

@ -161,7 +161,7 @@ struct FormatSettings
bool try_infer_numbers_from_strings = false; bool try_infer_numbers_from_strings = false;
bool validate_types_from_metadata = true; bool validate_types_from_metadata = true;
bool validate_utf8 = false; bool validate_utf8 = false;
bool try_infer_objects = false; bool allow_object_type = false;
} json; } json;
struct struct

View File

@ -366,7 +366,7 @@ namespace
transformJSONTuplesAndArraysToArrays(data_types, settings, type_indexes, json_info); transformJSONTuplesAndArraysToArrays(data_types, settings, type_indexes, json_info);
/// Convert Maps to Objects if needed. /// Convert Maps to Objects if needed.
if (settings.json.try_infer_objects) if (settings.json.allow_object_type)
transformMapsAndObjectsToObjects(data_types, type_indexes); transformMapsAndObjectsToObjects(data_types, type_indexes);
if (settings.json.read_objects_as_strings) if (settings.json.read_objects_as_strings)
@ -716,7 +716,7 @@ namespace
{ {
if constexpr (is_json) if constexpr (is_json)
{ {
if (settings.json.try_infer_objects) if (settings.json.allow_object_type)
return std::make_shared<DataTypeObject>("json", true); return std::make_shared<DataTypeObject>("json", true);
} }
/// Empty Map is Map(Nothing, Nothing) /// Empty Map is Map(Nothing, Nothing)
@ -735,7 +735,7 @@ namespace
transformInferredTypesIfNeededImpl<is_json>(value_types, settings, json_info); transformInferredTypesIfNeededImpl<is_json>(value_types, settings, json_info);
if (!checkIfTypesAreEqual(value_types)) if (!checkIfTypesAreEqual(value_types))
{ {
if (settings.json.try_infer_objects) if (settings.json.allow_object_type)
return std::make_shared<DataTypeObject>("json", true); return std::make_shared<DataTypeObject>("json", true);
if (settings.json.read_objects_as_strings) if (settings.json.read_objects_as_strings)
return std::make_shared<DataTypeString>(); return std::make_shared<DataTypeString>();

View File

@ -1,5 +1,6 @@
#pragma once #pragma once
#include <Functions/FunctionsConversion.h> #include <Functions/FunctionsConversion.h>
#include <Interpreters/parseColumnsListForTableFunction.h>
namespace DB namespace DB
{ {
@ -32,10 +33,11 @@ public:
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } 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_)) : diagnostic(std::move(diagnostic_))
, keep_nullable(keep_nullable_) , keep_nullable(keep_nullable_)
, cast_ipv4_ipv6_default_on_conversion_error(cast_ipv4_ipv6_default_on_conversion_error_) , 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) if constexpr (internal)
return createImpl({}, false /*keep_nullable*/, settings_ref.cast_ipv4_ipv6_default_on_conversion_error); 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); 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: protected:
@ -83,6 +85,7 @@ protected:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue<String>()); DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue<String>());
validateDataType(type, data_type_validation_settings);
if constexpr (cast_type == CastType::accurateOrNull) if constexpr (cast_type == CastType::accurateOrNull)
return makeNullable(type); return makeNullable(type);
@ -104,6 +107,7 @@ private:
std::optional<Diagnostic> diagnostic; std::optional<Diagnostic> diagnostic;
bool keep_nullable; bool keep_nullable;
bool cast_ipv4_ipv6_default_on_conversion_error; bool cast_ipv4_ipv6_default_on_conversion_error;
DataTypeValidationSettings data_type_validation_settings;
}; };

View File

@ -76,8 +76,6 @@
#include <Functions/UserDefined/UserDefinedSQLFunctionVisitor.h> #include <Functions/UserDefined/UserDefinedSQLFunctionVisitor.h>
#define MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS 256
namespace DB namespace DB
{ {

View File

@ -4,6 +4,9 @@
#include <Interpreters/InterpreterCreateQuery.h> #include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/parseColumnsListForTableFunction.h> #include <Interpreters/parseColumnsListForTableFunction.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB namespace DB
@ -12,9 +15,66 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; 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); ParserColumnDeclarationList parser(true, true);
const Settings & settings = context->getSettingsRef(); const Settings & settings = context->getSettingsRef();
@ -25,10 +85,14 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, Con
if (!columns_list) if (!columns_list)
throw Exception("Could not cast AST to ASTExpressionList", ErrorCodes::LOGICAL_ERROR); 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); ParserColumnDeclarationList parser(true, true);
const Settings & settings = context->getSettingsRef(); const Settings & settings = context->getSettingsRef();
@ -47,6 +111,9 @@ bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescrip
try try
{ {
columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); 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; return true;
} }
catch (...) catch (...)

View File

@ -2,6 +2,7 @@
#include <string> #include <string>
#include <Storages/ColumnsDescription.h> #include <Storages/ColumnsDescription.h>
#include <Core/Settings.h>
namespace DB namespace DB
@ -9,9 +10,29 @@ namespace DB
class Context; class Context;
/// Parses a common argument for table functions such as table structure given in string struct DataTypeValidationSettings
ColumnsDescription parseColumnsListFromString(const std::string & structure, ContextPtr context); {
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);
} }

View File

@ -12,6 +12,7 @@ namespace ErrorCodes
{ {
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int INCORRECT_DATA; extern const int INCORRECT_DATA;
extern const int ILLEGAL_COLUMN;
} }
JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) 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); 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) void registerInputFormatJSONAsString(FormatFactory & factory)
{ {
factory.registerInputFormat("JSONAsString", []( factory.registerInputFormat("JSONAsString", [](
@ -260,9 +270,9 @@ void registerFileSegmentationEngineJSONAsObject(FormatFactory & factory)
void registerJSONAsObjectSchemaReader(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);
}); });
} }

View File

@ -77,6 +77,8 @@ public:
class JSONAsObjectExternalSchemaReader : public IExternalSchemaReader class JSONAsObjectExternalSchemaReader : public IExternalSchemaReader
{ {
public: public:
JSONAsObjectExternalSchemaReader(const FormatSettings & settings);
NamesAndTypesList readSchema() override NamesAndTypesList readSchema() override
{ {
return {{"json", std::make_shared<DataTypeObject>("json", false)}}; return {{"json", std::make_shared<DataTypeObject>("json", false)}};

View File

@ -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 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 rm $USER_FILES_PATH/test_01543.avro

View File

@ -17,7 +17,7 @@ done
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_files" ${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 "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 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" ${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 \ ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \
SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON') \ 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 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" ${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 \ ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \
SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON') \ 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 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" ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1"

View File

@ -1,5 +1,6 @@
-- Tags: no-fasttest -- 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);
SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON), dummy; SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON), dummy;
SELECT materialize('{"k":"v"}'::JSON) SETTINGS extremes = 1; SELECT materialize('{"k":"v"}'::JSON) SETTINGS extremes = 1;

View File

@ -1,4 +1,6 @@
-- https://github.com/ClickHouse/ClickHouse/issues/30231 -- https://github.com/ClickHouse/ClickHouse/issues/30231
SET allow_suspicious_low_cardinality_types=1;
SELECT * SELECT *
FROM ( FROM (
SELECT number, SELECT number,

View File

@ -22,6 +22,9 @@ SELECT toTypeName(sumCount(v)), sumCount(v) FROM
) )
ORDER BY v ORDER BY v
); );
SET allow_suspicious_low_cardinality_types=1;
SELECT toTypeName(sumCount(v)), sumCount(v) FROM SELECT toTypeName(sumCount(v)), sumCount(v) FROM
( (
SELECT v FROM SELECT v FROM

View File

@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh
. "$CURDIR"/../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')" $CLICKHOUSE_CLIENT -q "desc file('arrow.dict', 'Arrow')"

View 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}