Merge pull request #35735 from Avogar/allow-read-bools-as-numbers

Allow to infer and parse bools as numbers in JSON input formats
This commit is contained in:
Kruglov Pavel 2022-04-07 13:20:49 +02:00 committed by GitHub
commit f3f8f27db5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 167 additions and 76 deletions

View File

@ -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) \

View File

@ -43,7 +43,7 @@ void SerializationNumber<T>::serializeTextJSON(const IColumn & column, size_t ro
}
template <typename T>
void SerializationNumber<T>::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
void SerializationNumber<T>::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<T>::deserializeTextJSON(IColumn & column, ReadBuffer &
static constexpr bool is_uint8 = std::is_same_v<T, UInt8>;
static constexpr bool is_int8 = std::is_same_v<T, Int8>;
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())

View File

@ -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;

View File

@ -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

View File

@ -10,6 +10,7 @@
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeObject.h>
#include <DataTypes/DataTypeFactory.h>
#include <Common/JSONParsers/SimdJSONParser.h>
#include <Common/JSONParsers/RapidJSONParser.h>
#include <Common/JSONParsers/DummyJSONParser.h>
@ -118,7 +119,7 @@ DataTypePtr getDataTypeFromJSONFieldImpl(const Element & field)
return nullptr;
if (field.isBool())
return makeNullable(std::make_shared<DataTypeUInt8>());
return DataTypeFactory::instance().get("Nullable(Bool)");
if (field.isInt64() || field.isUInt64() || field.isDouble())
return makeNullable(std::make_shared<DataTypeFloat64>());

View File

@ -1,6 +1,7 @@
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/ReadSchemaUtils.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNullable.h>
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;

View File

@ -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<String> 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

View File

@ -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_)
{
}

View File

@ -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_)
{
}

View File

@ -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_)
{
}

View File

@ -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;

View File

@ -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))

View File

@ -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

View File

@ -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