diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 92d6b55e542..2cbfe97cde5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -638,6 +638,7 @@ class IColumn; M(UInt64, input_format_msgpack_number_of_columns, 0, "The number of columns in inserted MsgPack data. Used for automatic schema inference from data.", 0) \ M(MsgPackUUIDRepresentation, output_format_msgpack_uuid_representation, FormatSettings::MsgPackUUIDRepresentation::EXT, "The way how to output UUID in MsgPack format.", 0) \ M(UInt64, input_format_max_rows_to_read_for_schema_inference, 100, "The maximum rows of data to read for automatic schema inference", 0) \ + M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 4b6b79151bc..14c53dd5956 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -43,7 +43,7 @@ void SerializationNumber::serializeTextJSON(const IColumn & column, size_t ro } template -void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { bool has_quote = false; if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without. @@ -67,7 +67,7 @@ void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & static constexpr bool is_uint8 = std::is_same_v; static constexpr bool is_int8 = std::is_same_v; - if (is_uint8 || is_int8) + if (settings.json.read_bools_as_numbers || is_uint8 || is_int8) { // extra conditions to parse true/false strings into 1/0 if (istr.eof()) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index f9c834cb3de..9dbce146ffa 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -90,6 +90,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.json.named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects; format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; + format_settings.json.read_bools_as_numbers = settings.input_format_json_read_bools_as_numbers; 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 c2448fe741d..608afeb8a2c 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -133,6 +133,7 @@ struct FormatSettings bool escape_forward_slashes = true; bool named_tuples_as_objects = false; bool serialize_as_strings = false; + bool read_bools_as_numbers = true; } json; struct diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index d90b925d753..5e1830655f9 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -118,7 +119,7 @@ DataTypePtr getDataTypeFromJSONFieldImpl(const Element & field) return nullptr; if (field.isBool()) - return makeNullable(std::make_shared()); + return DataTypeFactory::instance().get("Nullable(Bool)"); if (field.isInt64() || field.isUInt64() || field.isDouble()) return makeNullable(std::make_shared()); diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 2da27885b59..7cdb0644a9d 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -10,8 +11,63 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } -IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_) - : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_) +static void chooseResultType( + DataTypePtr & type, + const DataTypePtr & new_type, + bool allow_bools_as_numbers, + const DataTypePtr & default_type, + const String & column_name, + size_t row) +{ + if (!type) + type = new_type; + + /// If the new type and the previous type for this column are different, + /// we will use default type if we have it or throw an exception. + if (new_type && !type->equals(*new_type)) + { + /// Check if we have Bool and Number and if allow_bools_as_numbers + /// is true make the result type Number + auto not_nullable_type = removeNullable(type); + auto not_nullable_new_type = removeNullable(new_type); + bool bool_type_presents = isBool(not_nullable_type) || isBool(not_nullable_new_type); + bool number_type_presents = isNumber(not_nullable_type) || isNumber(not_nullable_new_type); + if (allow_bools_as_numbers && bool_type_presents && number_type_presents) + { + if (isBool(not_nullable_type)) + type = new_type; + } + else if (default_type) + type = default_type; + else + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", + type->getName(), + column_name, + row, + new_type->getName()); + } +} + +static void checkTypeAndAppend(NamesAndTypesList & result, DataTypePtr & type, const String & name, const DataTypePtr & default_type, size_t max_rows_to_read) +{ + if (!type) + { + if (!default_type) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum " + "number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference", + max_rows_to_read); + + type = default_type; + } + result.emplace_back(name, type); +} + +IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_, bool allow_bools_as_numbers_) + : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_), allow_bools_as_numbers(allow_bools_as_numbers_) { } @@ -34,20 +90,7 @@ NamesAndTypesList IRowSchemaReader::readSchema() if (!new_data_types[i]) continue; - /// If we couldn't determine the type of column yet, just set the new type. - if (!data_types[i]) - data_types[i] = new_data_types[i]; - /// If the new type and the previous type for this column are different, - /// we will use default type if we have it or throw an exception. - else if (data_types[i]->getName() != new_data_types[i]->getName()) - { - if (default_type) - data_types[i] = default_type; - else - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", new_data_types[i]->getName(), i + 1, row, data_types[i]->getName()); - } + chooseResultType(data_types[i], new_data_types[i], allow_bools_as_numbers, default_type, std::to_string(i + 1), row); } } @@ -72,25 +115,14 @@ NamesAndTypesList IRowSchemaReader::readSchema() for (size_t i = 0; i != data_types.size(); ++i) { /// Check that we could determine the type of this column. - if (!data_types[i]) - { - if (!default_type) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum " - "number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference", - max_rows_to_read); - - data_types[i] = default_type; - } - result.emplace_back(column_names[i], data_types[i]); + checkTypeAndAppend(result, data_types[i], column_names[i], default_type, max_rows_to_read); } return result; } -IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_) - : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_) +IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_, bool allow_bools_as_numbers_) + : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_), allow_bools_as_numbers(allow_bools_as_numbers_) { } @@ -127,20 +159,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() } auto & type = it->second; - /// If we couldn't determine the type of column yet, just set the new type. - if (!type) - type = new_type; - /// If the new type and the previous type for this column are different, - /// we will use default type if we have it or throw an exception. - else if (new_type && type->getName() != new_type->getName()) - { - if (default_type) - type = default_type; - else - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", type->getName(), name, row, new_type->getName()); - } + chooseResultType(type, new_type, allow_bools_as_numbers, default_type, name, row); } } @@ -153,18 +172,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() { auto & type = names_to_types[name]; /// Check that we could determine the type of this column. - if (!type) - { - if (!default_type) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum " - "number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference", - max_rows_to_read); - - type = default_type; - } - result.emplace_back(name, type); + checkTypeAndAppend(result, type, name, default_type, max_rows_to_read); } return result; diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index 811034cb0b0..7e809d3d963 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -36,7 +36,7 @@ protected: class IRowSchemaReader : public ISchemaReader { public: - IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr); + IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr, bool allow_bools_as_numbers_ = false); NamesAndTypesList readSchema() override; protected: @@ -51,6 +51,7 @@ protected: private: size_t max_rows_to_read; DataTypePtr default_type; + bool allow_bools_as_numbers; std::vector column_names; }; @@ -62,7 +63,7 @@ private: class IRowWithNamesSchemaReader : public ISchemaReader { public: - IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr); + IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr, bool allow_bools_as_numbers_ = false); NamesAndTypesList readSchema() override; bool hasStrictOrderOfColumns() const override { return false; } @@ -76,6 +77,7 @@ protected: private: size_t max_rows_to_read; DataTypePtr default_type; + bool allow_bools_as_numbers; }; /// Base class for schema inference for formats that don't need any data to diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index c087749d8d8..0496e3e41a8 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -182,7 +182,15 @@ bool JSONCompactEachRowFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & } JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_) - : FormatWithNamesAndTypesSchemaReader(in_, format_settings_.max_rows_to_read_for_schema_inference, with_names_, with_types_, &reader), reader(in_, yield_strings_, format_settings_) + : FormatWithNamesAndTypesSchemaReader( + in_, + format_settings_.max_rows_to_read_for_schema_inference, + with_names_, + with_types_, + &reader, + nullptr, + format_settings_.json.read_bools_as_numbers) + , reader(in_, yield_strings_, format_settings_) { } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 7f349db3c6c..2eb5b143107 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -307,7 +307,9 @@ void JSONEachRowRowInputFormat::readSuffix() } JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings) - : IRowWithNamesSchemaReader(in_, format_settings.max_rows_to_read_for_schema_inference), json_strings(json_strings_) + : IRowWithNamesSchemaReader( + in_, format_settings.max_rows_to_read_for_schema_inference, nullptr, format_settings.json.read_bools_as_numbers) + , json_strings(json_strings_) { } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 0735f927c6a..4e67007c943 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -297,8 +297,9 @@ FormatWithNamesAndTypesSchemaReader::FormatWithNamesAndTypesSchemaReader( bool with_names_, bool with_types_, FormatWithNamesAndTypesReader * format_reader_, - DataTypePtr default_type_) - : IRowSchemaReader(in_, max_rows_to_read_, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_) + DataTypePtr default_type_, + bool allow_bools_as_numbers_) + : IRowSchemaReader(in_, max_rows_to_read_, default_type_, allow_bools_as_numbers_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_) { } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 25ffc8d6de2..8fbd426112c 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -128,7 +128,8 @@ public: bool with_names_, bool with_types_, FormatWithNamesAndTypesReader * format_reader_, - DataTypePtr default_type_ = nullptr); + DataTypePtr default_type_ = nullptr, + bool allow_bools_as_numbers_ = false); NamesAndTypesList readSchema() override; diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference index 52139aa12dd..e4a0c3c3602 100644 --- a/tests/queries/0_stateless/02149_schema_inference.reference +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -38,32 +38,32 @@ JSONCompactEachRow c1 Nullable(Float64) c2 Array(Tuple(Nullable(Float64), Nullable(String))) c3 Map(String, Nullable(Float64)) -c4 Nullable(UInt8) -42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1 +c4 Nullable(Bool) +42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true c1 Nullable(Float64) c2 Array(Tuple(Nullable(Float64), Nullable(String))) c3 Map(String, Nullable(Float64)) -c4 Nullable(UInt8) +c4 Nullable(Bool) \N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N -32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 1 +32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} true JSONCompactEachRowWithNames a Nullable(Float64) b Array(Tuple(Nullable(Float64), Nullable(String))) c Map(String, Nullable(Float64)) -d Nullable(UInt8) -42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1 +d Nullable(Bool) +42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true JSONEachRow a Nullable(Float64) b Array(Tuple(Nullable(Float64), Nullable(String))) c Map(String, Nullable(Float64)) -d Nullable(UInt8) -42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1 +d Nullable(Bool) +42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true a Nullable(Float64) b Array(Tuple(Nullable(Float64), Nullable(String))) c Map(String, Nullable(Float64)) -d Nullable(UInt8) +d Nullable(Bool) \N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N -32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 1 +32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} true a Nullable(Float64) b Nullable(String) c Array(Nullable(Float64)) diff --git a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference new file mode 100644 index 00000000000..a7609bdd86b --- /dev/null +++ b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference @@ -0,0 +1,18 @@ +x Nullable(Bool) +true +false +x Nullable(Float64) +42.42 +0 +x Nullable(Float64) +1 +0.42 +c1 Nullable(Bool) +true +false +c1 Nullable(Float64) +42.42 +0 +c1 Nullable(Float64) +1 +0.42 diff --git a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh new file mode 100755 index 00000000000..10f050ea6d1 --- /dev/null +++ b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh @@ -0,0 +1,47 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02247.data +DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME + +touch $DATA_FILE + +echo -e '{"x" : true} +{"x" : false}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + +echo -e '{"x" : 42.42} +{"x" : false}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + +echo -e '{"x" : true} +{"x" : 0.42}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + + +echo -e '[true] +[false]' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" + +echo -e '[42.42] +[false]' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" + +echo -e '[true] +[0.42]' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" + + +rm $DATA_FILE