mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #36207 from Avogar/improve-json-schema-inference
Improve schema inference for json objects
This commit is contained in:
commit
3936b3e800
@ -185,7 +185,7 @@ DataTypePtr getDataTypeFromJSONFieldImpl(const Element & field)
|
||||
}
|
||||
|
||||
if (is_object)
|
||||
return std::make_shared<DataTypeObject>("json", false);
|
||||
return std::make_shared<DataTypeObject>("json", true);
|
||||
|
||||
if (value_type)
|
||||
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), value_type);
|
||||
@ -350,4 +350,38 @@ bool readFieldImpl(ReadBuffer & in, IColumn & column, const DataTypePtr & type,
|
||||
}
|
||||
}
|
||||
|
||||
DataTypePtr getCommonTypeForJSONFormats(const DataTypePtr & first, const DataTypePtr & second, bool allow_bools_as_numbers)
|
||||
{
|
||||
if (allow_bools_as_numbers)
|
||||
{
|
||||
auto not_nullable_first = removeNullable(first);
|
||||
auto not_nullable_second = removeNullable(second);
|
||||
/// Check if we have Bool and Number and if so make the result type Number
|
||||
bool bool_type_presents = isBool(not_nullable_first) || isBool(not_nullable_second);
|
||||
bool number_type_presents = isNumber(not_nullable_first) || isNumber(not_nullable_second);
|
||||
if (bool_type_presents && number_type_presents)
|
||||
{
|
||||
if (isBool(not_nullable_first))
|
||||
return second;
|
||||
return first;
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have Map and Object, make result type Object
|
||||
bool object_type_presents = isObject(first) || isObject(second);
|
||||
bool map_type_presents = isMap(first) || isMap(second);
|
||||
if (object_type_presents && map_type_presents)
|
||||
{
|
||||
if (isObject(first))
|
||||
return first;
|
||||
return second;
|
||||
}
|
||||
|
||||
/// If we have different Maps, make result type Object
|
||||
if (isMap(first) && isMap(second) && !first->equals(*second))
|
||||
return std::make_shared<DataTypeObject>("json", true);
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -32,4 +32,6 @@ bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf);
|
||||
|
||||
bool readFieldImpl(ReadBuffer & in, IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, const String & column_name, const FormatSettings & format_settings, bool yield_strings);
|
||||
|
||||
DataTypePtr getCommonTypeForJSONFormats(const DataTypePtr & first, const DataTypePtr & second, bool allow_bools_as_numbers);
|
||||
|
||||
}
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
namespace DB
|
||||
@ -15,7 +14,7 @@ namespace ErrorCodes
|
||||
static void chooseResultType(
|
||||
DataTypePtr & type,
|
||||
const DataTypePtr & new_type,
|
||||
bool allow_bools_as_numbers,
|
||||
CommonDataTypeChecker common_type_checker,
|
||||
const DataTypePtr & default_type,
|
||||
const String & column_name,
|
||||
size_t row)
|
||||
@ -27,17 +26,12 @@ static void chooseResultType(
|
||||
/// 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;
|
||||
}
|
||||
DataTypePtr common_type;
|
||||
if (common_type_checker)
|
||||
common_type = common_type_checker(type, new_type);
|
||||
|
||||
if (common_type)
|
||||
type = common_type;
|
||||
else if (default_type)
|
||||
type = default_type;
|
||||
else
|
||||
@ -67,8 +61,8 @@ static void checkTypeAndAppend(NamesAndTypesList & result, DataTypePtr & type, c
|
||||
result.emplace_back(name, type);
|
||||
}
|
||||
|
||||
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, bool allow_bools_as_numbers_)
|
||||
: ISchemaReader(in_), max_rows_to_read(format_settings.max_rows_to_read_for_schema_inference), allow_bools_as_numbers(allow_bools_as_numbers_)
|
||||
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings)
|
||||
: ISchemaReader(in_), max_rows_to_read(format_settings.max_rows_to_read_for_schema_inference)
|
||||
{
|
||||
if (!format_settings.column_names_for_schema_inference.empty())
|
||||
{
|
||||
@ -83,14 +77,14 @@ IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & form
|
||||
}
|
||||
}
|
||||
|
||||
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_, bool allow_bools_as_numbers_)
|
||||
: IRowSchemaReader(in_, format_settings, allow_bools_as_numbers_)
|
||||
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_)
|
||||
: IRowSchemaReader(in_, format_settings)
|
||||
{
|
||||
default_type = default_type_;
|
||||
}
|
||||
|
||||
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, const DataTypes & default_types_, bool allow_bools_as_numbers_)
|
||||
: IRowSchemaReader(in_, format_settings, allow_bools_as_numbers_)
|
||||
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, const DataTypes & default_types_)
|
||||
: IRowSchemaReader(in_, format_settings)
|
||||
{
|
||||
default_types = default_types_;
|
||||
}
|
||||
@ -114,7 +108,7 @@ NamesAndTypesList IRowSchemaReader::readSchema()
|
||||
if (!new_data_types[i])
|
||||
continue;
|
||||
|
||||
chooseResultType(data_types[i], new_data_types[i], allow_bools_as_numbers, getDefaultType(i), std::to_string(i + 1), row);
|
||||
chooseResultType(data_types[i], new_data_types[i], common_type_checker, getDefaultType(i), std::to_string(i + 1), row);
|
||||
}
|
||||
}
|
||||
|
||||
@ -154,8 +148,8 @@ DataTypePtr IRowSchemaReader::getDefaultType(size_t column) const
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
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_)
|
||||
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_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -192,7 +186,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
|
||||
}
|
||||
|
||||
auto & type = it->second;
|
||||
chooseResultType(type, new_type, allow_bools_as_numbers, default_type, name, row);
|
||||
chooseResultType(type, new_type, common_type_checker, default_type, name, row);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -28,6 +28,8 @@ protected:
|
||||
ReadBuffer & in;
|
||||
};
|
||||
|
||||
using CommonDataTypeChecker = std::function<DataTypePtr(const DataTypePtr &, const DataTypePtr &)>;
|
||||
|
||||
/// Base class for schema inference for formats that read data row by row.
|
||||
/// It reads data row by row (up to max_rows_to_read), determines types of columns
|
||||
/// for each row and compare them with types from the previous rows. If some column
|
||||
@ -38,12 +40,14 @@ protected:
|
||||
class IRowSchemaReader : public ISchemaReader
|
||||
{
|
||||
public:
|
||||
IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, bool allow_bools_as_numbers_ = false);
|
||||
IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_, bool allow_bools_as_numbers_ = false);
|
||||
IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, const DataTypes & default_types_, bool allow_bools_as_numbers_ = false);
|
||||
IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings);
|
||||
IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_);
|
||||
IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, const DataTypes & default_types_);
|
||||
|
||||
NamesAndTypesList readSchema() override;
|
||||
|
||||
void setCommonTypeChecker(CommonDataTypeChecker checker) { common_type_checker = checker; }
|
||||
|
||||
protected:
|
||||
/// Read one row and determine types of columns in it.
|
||||
/// Return types in the same order in which the values were in the row.
|
||||
@ -59,7 +63,7 @@ private:
|
||||
size_t max_rows_to_read;
|
||||
DataTypePtr default_type;
|
||||
DataTypes default_types;
|
||||
bool allow_bools_as_numbers;
|
||||
CommonDataTypeChecker common_type_checker;
|
||||
std::vector<String> column_names;
|
||||
};
|
||||
|
||||
@ -71,10 +75,12 @@ private:
|
||||
class IRowWithNamesSchemaReader : public ISchemaReader
|
||||
{
|
||||
public:
|
||||
IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr, bool allow_bools_as_numbers_ = false);
|
||||
IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr);
|
||||
NamesAndTypesList readSchema() override;
|
||||
bool hasStrictOrderOfColumns() const override { return false; }
|
||||
|
||||
void setCommonTypeChecker(CommonDataTypeChecker checker) { common_type_checker = checker; }
|
||||
|
||||
protected:
|
||||
/// Read one row and determine types of columns in it.
|
||||
/// Return list with names and types.
|
||||
@ -85,7 +91,7 @@ protected:
|
||||
private:
|
||||
size_t max_rows_to_read;
|
||||
DataTypePtr default_type;
|
||||
bool allow_bools_as_numbers;
|
||||
CommonDataTypeChecker common_type_checker;
|
||||
};
|
||||
|
||||
/// Base class for schema inference for formats that don't need any data to
|
||||
|
@ -183,10 +183,14 @@ 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_, with_names_, with_types_, &reader, nullptr, format_settings_.json.read_bools_as_numbers)
|
||||
: FormatWithNamesAndTypesSchemaReader(in_, format_settings_, with_names_, with_types_, &reader)
|
||||
, reader(in_, yield_strings_, format_settings_)
|
||||
{
|
||||
bool allow_bools_as_numbers = format_settings_.json.read_bools_as_numbers;
|
||||
setCommonTypeChecker([allow_bools_as_numbers](const DataTypePtr & first, const DataTypePtr & second)
|
||||
{
|
||||
return getCommonTypeForJSONFormats(first, second, allow_bools_as_numbers);
|
||||
});
|
||||
}
|
||||
|
||||
DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypes()
|
||||
|
@ -307,10 +307,14 @@ 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, nullptr, format_settings.json.read_bools_as_numbers)
|
||||
: IRowWithNamesSchemaReader(in_, format_settings.max_rows_to_read_for_schema_inference)
|
||||
, json_strings(json_strings_)
|
||||
{
|
||||
bool allow_bools_as_numbers = format_settings.json.read_bools_as_numbers;
|
||||
setCommonTypeChecker([allow_bools_as_numbers](const DataTypePtr & first, const DataTypePtr & second)
|
||||
{
|
||||
return getCommonTypeForJSONFormats(first, second, allow_bools_as_numbers);
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
|
@ -297,9 +297,8 @@ FormatWithNamesAndTypesSchemaReader::FormatWithNamesAndTypesSchemaReader(
|
||||
bool with_names_,
|
||||
bool with_types_,
|
||||
FormatWithNamesAndTypesReader * format_reader_,
|
||||
DataTypePtr default_type_,
|
||||
bool allow_bools_as_numbers_)
|
||||
: IRowSchemaReader(in_, format_settings, default_type_, allow_bools_as_numbers_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_)
|
||||
DataTypePtr default_type_)
|
||||
: IRowSchemaReader(in_, format_settings, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -128,8 +128,7 @@ public:
|
||||
bool with_names_,
|
||||
bool with_types_,
|
||||
FormatWithNamesAndTypesReader * format_reader_,
|
||||
DataTypePtr default_type_ = nullptr,
|
||||
bool allow_bools_as_numbers_ = false);
|
||||
DataTypePtr default_type_ = nullptr);
|
||||
|
||||
NamesAndTypesList readSchema() override;
|
||||
|
||||
|
@ -1,8 +1,8 @@
|
||||
{"id":"1","obj":{"k1":1,"k2":{"k3":"2","k4":[{"k5":3,"k6":0},{"k5":4,"k6":0}]},"some":0},"s":"foo"}
|
||||
{"id":"2","obj":{"k1":0,"k2":{"k3":"str","k4":[{"k5":0,"k6":55}]},"some":42},"s":"bar"}
|
||||
Tuple(k1 Int8, k2 Tuple(k3 String, k4 Nested(k5 Int8, k6 Int8)), some Int8)
|
||||
{"id":"1","obj":{"k1":1,"k2":{"k3":"2","k4":[{"k5":3,"k6":null},{"k5":4,"k6":null}]},"some":null},"s":"foo"}
|
||||
{"id":"2","obj":{"k1":null,"k2":{"k3":"str","k4":[{"k5":null,"k6":55}]},"some":42},"s":"bar"}
|
||||
Tuple(k1 Nullable(Int8), k2 Tuple(k3 Nullable(String), k4 Nested(k5 Nullable(Int8), k6 Nullable(Int8))), some Nullable(Int8))
|
||||
{"id":"1","obj":"aaa","s":"foo"}
|
||||
{"id":"2","obj":"bbb","s":"bar"}
|
||||
{"map":{"k1":1,"k2":2},"obj":{"k1":1,"k2.k3":2},"map_type":"Map(String, Nullable(Float64))","obj_type":"Object('json')"}
|
||||
{"map":{"k1":1,"k2":2},"obj":{"k1":1,"k2.k3":2},"map_type":"Map(String, Nullable(Float64))","obj_type":"Object(Nullable('json'))"}
|
||||
{"obj":{"k1":1,"k2":2},"map":{"k1":"1","k2":"2"}}
|
||||
Tuple(k1 Float64, k2 Float64)
|
||||
|
@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_inference"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (id UInt64, obj JSON, s String) \
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (id UInt64, obj Object(Nullable('json')), s String) \
|
||||
ENGINE = MergeTree ORDER BY id" --allow_experimental_object_type 1
|
||||
|
||||
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}')
|
||||
|
@ -0,0 +1,2 @@
|
||||
x Object(Nullable(\'json\'))
|
||||
x Object(Nullable(\'json\'))
|
@ -0,0 +1,3 @@
|
||||
-- Tags: no-fasttest
|
||||
desc format(JSONEachRow, '{"x" : {"a" : "Some string"}}, {"x" : {"b" : [1, 2, 3]}}, {"x" : {"c" : {"d" : 10}}}');
|
||||
desc format(JSONEachRow, '{"x" : {"a" : "Some string"}}, {"x" : {"b" : [1, 2, 3], "c" : {"42" : 42}}}');
|
Loading…
Reference in New Issue
Block a user