From 4ab3e903821e35558442664c571368f354be30fe Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 21:21:30 +0000 Subject: [PATCH 1/7] Validate types in table function arguments/CAST function arguments/JSONAsObject schema inference --- src/DataTypes/DataTypeFixedString.h | 1 + src/Formats/EscapingRuleUtils.cpp | 4 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Formats/SchemaInferenceUtils.cpp | 6 +- src/Functions/CastOverloadResolver.h | 14 ++-- .../parseColumnsListForTableFunction.cpp | 70 ++++++++++++++++++- .../parseColumnsListForTableFunction.h | 27 ++++++- .../Impl/JSONAsStringRowInputFormat.cpp | 14 +++- .../Formats/Impl/JSONAsStringRowInputFormat.h | 2 + .../02513_validate_data_types.reference | 0 .../0_stateless/02513_validate_data_types.sql | 17 +++++ 12 files changed, 139 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02513_validate_data_types.reference create mode 100644 tests/queries/0_stateless/02513_validate_data_types.sql diff --git a/src/DataTypes/DataTypeFixedString.h b/src/DataTypes/DataTypeFixedString.h index 7c089866b23..437e3f9d3f5 100644 --- a/src/DataTypes/DataTypeFixedString.h +++ b/src/DataTypes/DataTypeFixedString.h @@ -5,6 +5,7 @@ #include #define MAX_FIXEDSTRING_SIZE 0xFFFFFF +#define MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS 256 namespace DB diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index df9c711b432..2a600fb9020 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -421,10 +421,10 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo break; case FormatSettings::EscapingRule::JSON: result += fmt::format( - ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, try_infer_objects={}", + ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, allow_object_type={}", settings.json.try_infer_numbers_from_strings, settings.json.read_bools_as_numbers, - settings.json.try_infer_objects); + settings.json.allow_object_type); break; default: break; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 91f2b0958f6..9385a562971 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -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; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 250601c3bf0..dcdd44edfeb 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -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 diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 29b530966a0..89c69bcc9f6 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -363,7 +363,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) @@ -713,7 +713,7 @@ namespace { if constexpr (is_json) { - if (settings.json.try_infer_objects) + if (settings.json.allow_object_type) return std::make_shared("json", true); } /// Empty Map is Map(Nothing, Nothing) @@ -732,7 +732,7 @@ namespace transformInferredTypesIfNeededImpl(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("json", true); if (settings.json.read_objects_as_strings) return makeNullable(std::make_shared()); diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index 6aa3d97ff0a..ad11c802214 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { @@ -32,10 +33,11 @@ public: ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - explicit CastOverloadResolverImpl(std::optional diagnostic_, bool keep_nullable_, bool cast_ipv4_ipv6_default_on_conversion_error_) + explicit CastOverloadResolverImpl(std::optional 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_) { } @@ -44,15 +46,15 @@ public: const auto & settings_ref = context->getSettingsRef(); 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, DataTypeValidationSettings(settings_ref)); - 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 = {}, bool keep_nullable = false, bool cast_ipv4_ipv6_default_on_conversion_error = false) + static FunctionOverloadResolverPtr createImpl(std::optional 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(std::move(diagnostic), keep_nullable, cast_ipv4_ipv6_default_on_conversion_error); + return std::make_unique(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()); + validateDataType(type, data_type_validation_settings); if constexpr (cast_type == CastType::accurateOrNull) return makeNullable(type); @@ -104,6 +107,7 @@ private: std::optional diagnostic; bool keep_nullable; bool cast_ipv4_ipv6_default_on_conversion_error; + DataTypeValidationSettings data_type_validation_settings; }; diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index ed270c2118b..29466514a04 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -4,6 +4,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -12,9 +15,65 @@ 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(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(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 +84,13 @@ 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); + for (const auto & [name, type] : columns.getAll()) + validateDataType(type, DataTypeValidationSettings(context->getSettingsRef())); + 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 +109,8 @@ bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescrip try { columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); + for (const auto & [name, type] : columns.getAll()) + validateDataType(type, DataTypeValidationSettings(context->getSettingsRef())); return true; } catch (...) diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index e82a32f3d23..26968ff4290 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -2,6 +2,7 @@ #include #include +#include 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 = false; + bool allow_experimental_geo_types = false; + bool allow_experimental_object_type = false; + bool allow_suspicious_fixed_string_types = false; +}; + +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); } diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index d369eedceea..dc4d12d7b18 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -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_) @@ -207,6 +208,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", []( @@ -261,9 +271,9 @@ void registerFileSegmentationEngineJSONAsObject(FormatFactory & factory) void registerJSONAsObjectSchemaReader(FormatFactory & factory) { - factory.registerExternalSchemaReader("JSONAsObject", [](const FormatSettings &) + factory.registerExternalSchemaReader("JSONAsObject", [](const FormatSettings & settings) { - return std::make_shared(); + return std::make_shared(settings); }); } diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h index 438107e73e6..3f2d1998139 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h @@ -77,6 +77,8 @@ public: class JSONAsObjectExternalSchemaReader : public IExternalSchemaReader { public: + JSONAsObjectExternalSchemaReader(const FormatSettings & settings); + NamesAndTypesList readSchema() override { return {{"json", std::make_shared("json", false)}}; diff --git a/tests/queries/0_stateless/02513_validate_data_types.reference b/tests/queries/0_stateless/02513_validate_data_types.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02513_validate_data_types.sql b/tests/queries/0_stateless/02513_validate_data_types.sql new file mode 100644 index 00000000000..e4514c79e8d --- /dev/null +++ b/tests/queries/0_stateless/02513_validate_data_types.sql @@ -0,0 +1,17 @@ +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} + From f4b2266db0cc391211ec0391cd277923d4718340 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 21:22:26 +0000 Subject: [PATCH 2/7] Fix style --- .../parseColumnsListForTableFunction.cpp | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index 29466514a04..909efd35324 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -60,16 +60,17 @@ void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings if (!settings.allow_suspicious_fixed_string_types) { - auto basic_type = removeLowCardinality(removeNullable(type)); - if (const auto * fixed_string = typeid_cast(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); - } + auto basic_type = removeLowCardinality(removeNullable(type)); + if (const auto * fixed_string = typeid_cast(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); + } } } From ba86424922d150d1cab21789e83cbf4b7b638003 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 21:23:12 +0000 Subject: [PATCH 3/7] Better --- src/Interpreters/parseColumnsListForTableFunction.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index 909efd35324..c44259a3ccc 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -86,8 +86,9 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, con throw Exception("Could not cast AST to ASTExpressionList", ErrorCodes::LOGICAL_ERROR); auto columns = InterpreterCreateQuery::getColumnsDescription(*columns_list, context, false); + auto validation_settings = DataTypeValidationSettings(context->getSettingsRef()); for (const auto & [name, type] : columns.getAll()) - validateDataType(type, DataTypeValidationSettings(context->getSettingsRef())); + validateDataType(type, validation_settings); return columns; } @@ -110,8 +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, DataTypeValidationSettings(context->getSettingsRef())); + validateDataType(type, validation_settings); return true; } catch (...) From dfe8e3e02eeb503c65e5199953f2beedf70f6756 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 21:32:22 +0000 Subject: [PATCH 4/7] Remove duplicate define --- src/Interpreters/InterpreterCreateQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 50536b66185..43e933d4400 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -76,8 +76,6 @@ #include -#define MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS 256 - namespace DB { From 41c63152cfb443d09344b70d48d98293f490d352 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 21 Dec 2022 23:47:56 +0000 Subject: [PATCH 5/7] Don't check internal CAST --- src/Functions/CastOverloadResolver.h | 2 +- src/Interpreters/parseColumnsListForTableFunction.h | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index ad11c802214..0bd0bad0d14 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -46,7 +46,7 @@ public: const auto & settings_ref = context->getSettingsRef(); if constexpr (internal) - return createImpl({}, false /*keep_nullable*/, settings_ref.cast_ipv4_ipv6_default_on_conversion_error, DataTypeValidationSettings(settings_ref)); + 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, DataTypeValidationSettings(settings_ref)); } diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index 26968ff4290..97923bcad77 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -22,10 +22,10 @@ struct DataTypeValidationSettings { } - bool allow_suspicious_low_cardinality_types = false; - bool allow_experimental_geo_types = false; - bool allow_experimental_object_type = false; - bool allow_suspicious_fixed_string_types = false; + 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); From 0c74b6c066d22d82791f63bc4515a6d9b20d02df Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 22 Dec 2022 17:40:06 +0000 Subject: [PATCH 6/7] Fix tests --- .../0_stateless/02049_lowcardinality_shortcircuit_crash.sql | 2 ++ tests/queries/0_stateless/02119_sumcount.sql | 3 +++ tests/queries/0_stateless/02513_validate_data_types.sql | 2 ++ 3 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql index 2a837380250..84d64d3166e 100644 --- a/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql +++ b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql @@ -1,4 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/30231 +SET allow_suspicious_low_cardinality_types=1; + SELECT * FROM ( SELECT number, diff --git a/tests/queries/0_stateless/02119_sumcount.sql b/tests/queries/0_stateless/02119_sumcount.sql index 22cb8b657da..86625996f44 100644 --- a/tests/queries/0_stateless/02119_sumcount.sql +++ b/tests/queries/0_stateless/02119_sumcount.sql @@ -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 diff --git a/tests/queries/0_stateless/02513_validate_data_types.sql b/tests/queries/0_stateless/02513_validate_data_types.sql index e4514c79e8d..1235d00ba79 100644 --- a/tests/queries/0_stateless/02513_validate_data_types.sql +++ b/tests/queries/0_stateless/02513_validate_data_types.sql @@ -1,3 +1,5 @@ +-- 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} From 2891d14da1f894bbef4641bff65acc6ce2185e20 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 27 Dec 2022 19:34:09 +0000 Subject: [PATCH 7/7] Fix tests --- .../0_stateless/01543_avro_deserialization_with_lc.sh | 2 +- tests/queries/0_stateless/01825_type_json_multiple_files.sh | 6 +++--- tests/queries/0_stateless/01825_type_json_order_by.sql | 1 + .../queries/0_stateless/02166_arrow_dictionary_inference.sh | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh index df43fd273e8..6f48456f71b 100755 --- a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh +++ b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh @@ -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 diff --git a/tests/queries/0_stateless/01825_type_json_multiple_files.sh b/tests/queries/0_stateless/01825_type_json_multiple_files.sh index 9120568bb17..91e7ffb5edf 100755 --- a/tests/queries/0_stateless/01825_type_json_multiple_files.sh +++ b/tests/queries/0_stateless/01825_type_json_multiple_files.sh @@ -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" diff --git a/tests/queries/0_stateless/01825_type_json_order_by.sql b/tests/queries/0_stateless/01825_type_json_order_by.sql index 1d5768fe0b4..98b2a6105d9 100644 --- a/tests/queries/0_stateless/01825_type_json_order_by.sql +++ b/tests/queries/0_stateless/01825_type_json_order_by.sql @@ -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; diff --git a/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh b/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh index 7d313b571d9..04888f3a1f3 100755 --- a/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh +++ b/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh @@ -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')"