mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Refactor and improve schema inference for text formats
This commit is contained in:
parent
d4cd53ccea
commit
7375a7d429
@ -3434,6 +3434,13 @@ Use schema from cache for URL with last modification time validation (for urls w
|
||||
|
||||
Default value: `true`.
|
||||
|
||||
## schema_inference_make_columns_nullable {#schema_inference_make_columns_nullable}
|
||||
|
||||
Controls making inferred types `Nullable` in schema inference for formats without information about nullability.
|
||||
The inferred type will be `Nullable` only if column contains `NULL` in a sample that is parsed during schema inference.
|
||||
|
||||
Default value: `false`.
|
||||
|
||||
## use_structure_from_insertion_table_in_table_functions {use_structure_from_insertion_table_in_table_functions}
|
||||
|
||||
Use structure from insertion table instead of schema inference from data.
|
||||
|
@ -759,6 +759,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Arrow", 0) \
|
||||
M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \
|
||||
M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column_name1 column_type1, column_name2 column_type2, ...'", 0) \
|
||||
M(Bool, schema_inference_make_columns_nullable, true, "Controls making inferred types Nullable in schema inference for formats without information about nullability.", 0) \
|
||||
M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \
|
||||
M(Bool, input_format_json_try_infer_numbers_from_strings, true, "Try to infer numbers from string fields while schema inference", 0) \
|
||||
M(Bool, input_format_json_validate_types_from_metadata, true, "For JSON/JSONCompact/JSONColumnsWithMetadata input formats this controls whether format parser should check if data types from input metadata match data types of the corresponding columns from the table", 0) \
|
||||
|
@ -47,6 +47,7 @@ void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &
|
||||
bool have_tuple = false;
|
||||
bool all_tuples = true;
|
||||
size_t tuple_size = 0;
|
||||
bool sizes_are_equal = true;
|
||||
|
||||
std::vector<DataTypes> nested_types;
|
||||
|
||||
@ -62,7 +63,10 @@ void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &
|
||||
nested_types[elem_idx].reserve(types.size());
|
||||
}
|
||||
else if (tuple_size != type_tuple->getElements().size())
|
||||
return;
|
||||
{
|
||||
sizes_are_equal = false;
|
||||
break;
|
||||
}
|
||||
|
||||
have_tuple = true;
|
||||
|
||||
@ -75,7 +79,7 @@ void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &
|
||||
|
||||
if (have_tuple)
|
||||
{
|
||||
if (all_tuples)
|
||||
if (all_tuples && sizes_are_equal)
|
||||
{
|
||||
std::vector<DataTypes> transposed_nested_types(types.size());
|
||||
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
|
||||
@ -168,6 +172,9 @@ void transformTypesRecursively(DataTypes & types, std::function<void(DataTypes &
|
||||
types[i] = nested_types[i];
|
||||
}
|
||||
|
||||
if (transform_complex_types)
|
||||
transform_complex_types(types);
|
||||
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
@ -1,21 +1,11 @@
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeObject.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <DataTypes/transformTypesRecursively.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
@ -261,542 +251,76 @@ String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule e
|
||||
return readByEscapingRule<true>(buf, escaping_rule, format_settings);
|
||||
}
|
||||
|
||||
void transformInferredTypesIfNeededImpl(DataTypes & types, const FormatSettings & settings, bool is_json, const std::unordered_set<const IDataType *> * numbers_parsed_from_json_strings = nullptr)
|
||||
{
|
||||
/// Do nothing if we didn't try to infer something special.
|
||||
if (!settings.try_infer_integers && !settings.try_infer_dates && !settings.try_infer_datetimes && !is_json)
|
||||
return;
|
||||
|
||||
auto transform_simple_types = [&](DataTypes & data_types)
|
||||
{
|
||||
/// If we have floats and integers convert them all to float.
|
||||
if (settings.try_infer_integers)
|
||||
{
|
||||
bool have_floats = false;
|
||||
bool have_integers = false;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
have_floats |= isFloat(type);
|
||||
have_integers |= isInteger(type) && !isBool(type);
|
||||
}
|
||||
|
||||
if (have_floats && have_integers)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isInteger(type))
|
||||
type = std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have only dates and datetimes, convert dates to datetime.
|
||||
/// If we have date/datetimes and smth else, convert them to string, because
|
||||
/// There is a special case when we inferred both Date/DateTime and Int64 from Strings,
|
||||
/// for example: "arr: ["2020-01-01", "2000"]" -> Tuple(Date, Int64),
|
||||
/// so if we have Date/DateTime and smth else (not only String) we should
|
||||
/// convert Date/DateTime back to String, so then we will be able to
|
||||
/// convert Int64 back to String as well.
|
||||
if (settings.try_infer_dates || settings.try_infer_datetimes)
|
||||
{
|
||||
bool have_dates = false;
|
||||
bool have_datetimes = false;
|
||||
bool all_dates_or_datetimes = true;
|
||||
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
have_dates |= isDate(type);
|
||||
have_datetimes |= isDateTime64(type);
|
||||
all_dates_or_datetimes &= isDate(type) || isDateTime64(type);
|
||||
}
|
||||
|
||||
if (!all_dates_or_datetimes && (have_dates || have_datetimes))
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isDate(type) || isDateTime64(type))
|
||||
type = std::make_shared<DataTypeString>();
|
||||
}
|
||||
}
|
||||
else if (have_dates && have_datetimes)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isDate(type))
|
||||
type = std::make_shared<DataTypeDateTime64>(9);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!is_json)
|
||||
return;
|
||||
|
||||
/// Check settings specific for JSON formats.
|
||||
|
||||
/// If we have numbers and strings, convert numbers to strings.
|
||||
if (settings.json.try_infer_numbers_from_strings || settings.json.read_numbers_as_strings)
|
||||
{
|
||||
bool have_strings = false;
|
||||
bool have_numbers = false;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
have_strings |= isString(type);
|
||||
have_numbers |= isNumber(type);
|
||||
}
|
||||
|
||||
if (have_strings && have_numbers)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isNumber(type)
|
||||
&& (settings.json.read_numbers_as_strings || !numbers_parsed_from_json_strings
|
||||
|| numbers_parsed_from_json_strings->contains(type.get())))
|
||||
type = std::make_shared<DataTypeString>();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (settings.json.read_bools_as_numbers)
|
||||
{
|
||||
/// Note that have_floats and have_integers both cannot be
|
||||
/// equal to true as in one of previous checks we convert
|
||||
/// integers to floats if we have both.
|
||||
bool have_floats = false;
|
||||
bool have_integers = false;
|
||||
bool have_bools = false;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
have_floats |= isFloat(type);
|
||||
have_integers |= isInteger(type) && !isBool(type);
|
||||
have_bools |= isBool(type);
|
||||
}
|
||||
|
||||
if (have_bools && (have_integers || have_floats))
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isBool(type))
|
||||
{
|
||||
if (have_integers)
|
||||
type = std::make_shared<DataTypeInt64>();
|
||||
else
|
||||
type = std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
auto transform_complex_types = [&](DataTypes & data_types)
|
||||
{
|
||||
if (!is_json)
|
||||
return;
|
||||
|
||||
bool have_maps = false;
|
||||
bool have_objects = false;
|
||||
bool are_maps_equal = true;
|
||||
DataTypePtr first_map_type;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
if (isMap(type))
|
||||
{
|
||||
if (!have_maps)
|
||||
{
|
||||
first_map_type = type;
|
||||
have_maps = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
are_maps_equal &= type->equals(*first_map_type);
|
||||
}
|
||||
}
|
||||
else if (isObject(type))
|
||||
{
|
||||
have_objects = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (have_maps && (have_objects || !are_maps_equal))
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isMap(type))
|
||||
type = std::make_shared<DataTypeObject>("json", true);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
transformTypesRecursively(types, transform_simple_types, transform_complex_types);
|
||||
}
|
||||
|
||||
void transformInferredTypesIfNeeded(DataTypes & types, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule)
|
||||
{
|
||||
transformInferredTypesIfNeededImpl(types, settings, escaping_rule == FormatSettings::EscapingRule::JSON);
|
||||
}
|
||||
|
||||
void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule)
|
||||
{
|
||||
DataTypes types = {first, second};
|
||||
transformInferredTypesIfNeeded(types, settings, escaping_rule);
|
||||
first = std::move(types[0]);
|
||||
second = std::move(types[1]);
|
||||
}
|
||||
|
||||
void transformInferredJSONTypesIfNeeded(DataTypes & types, const FormatSettings & settings, const std::unordered_set<const IDataType *> * numbers_parsed_from_json_strings)
|
||||
{
|
||||
transformInferredTypesIfNeededImpl(types, settings, true, numbers_parsed_from_json_strings);
|
||||
}
|
||||
|
||||
void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings)
|
||||
{
|
||||
DataTypes types = {first, second};
|
||||
transformInferredJSONTypesIfNeeded(types, settings);
|
||||
first = std::move(types[0]);
|
||||
second = std::move(types[1]);
|
||||
}
|
||||
|
||||
bool tryInferDate(const std::string_view & field)
|
||||
{
|
||||
ReadBufferFromString buf(field);
|
||||
DayNum tmp;
|
||||
return tryReadDateText(tmp, buf) && buf.eof();
|
||||
}
|
||||
|
||||
bool tryInferDateTime(const std::string_view & field, const FormatSettings & settings)
|
||||
{
|
||||
if (field.empty())
|
||||
return false;
|
||||
|
||||
ReadBufferFromString buf(field);
|
||||
Float64 tmp_float;
|
||||
/// Check if it's just a number, and if so, don't try to infer DateTime from it,
|
||||
/// because we can interpret this number as a timestamp and it will lead to
|
||||
/// inferring DateTime instead of simple Int64/Float64 in some cases.
|
||||
if (tryReadFloatText(tmp_float, buf) && buf.eof())
|
||||
return false;
|
||||
|
||||
buf.seek(0, SEEK_SET); /// Return position to the beginning
|
||||
DateTime64 tmp;
|
||||
switch (settings.date_time_input_format)
|
||||
{
|
||||
case FormatSettings::DateTimeInputFormat::Basic:
|
||||
if (tryReadDateTime64Text(tmp, 9, buf) && buf.eof())
|
||||
return true;
|
||||
break;
|
||||
case FormatSettings::DateTimeInputFormat::BestEffort:
|
||||
if (tryParseDateTime64BestEffort(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC")) && buf.eof())
|
||||
return true;
|
||||
break;
|
||||
case FormatSettings::DateTimeInputFormat::BestEffortUS:
|
||||
if (tryParseDateTime64BestEffortUS(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC")) && buf.eof())
|
||||
return true;
|
||||
break;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
DataTypePtr tryInferDateOrDateTime(const std::string_view & field, const FormatSettings & settings)
|
||||
{
|
||||
if (settings.try_infer_dates && tryInferDate(field))
|
||||
return makeNullable(std::make_shared<DataTypeDate>());
|
||||
|
||||
if (settings.try_infer_datetimes && tryInferDateTime(field, settings))
|
||||
return makeNullable(std::make_shared<DataTypeDateTime64>(9));
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBufferFromString & buf, const FormatSettings & settings)
|
||||
{
|
||||
if (buf.eof())
|
||||
return nullptr;
|
||||
|
||||
/// Array
|
||||
if (checkChar('[', buf))
|
||||
{
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
DataTypes nested_types;
|
||||
bool first = true;
|
||||
while (!buf.eof() && *buf.position() != ']')
|
||||
{
|
||||
if (!first)
|
||||
{
|
||||
skipWhitespaceIfAny(buf);
|
||||
if (!checkChar(',', buf))
|
||||
return nullptr;
|
||||
skipWhitespaceIfAny(buf);
|
||||
}
|
||||
else
|
||||
first = false;
|
||||
|
||||
auto nested_type = determineDataTypeForSingleFieldImpl(buf, settings);
|
||||
if (!nested_type)
|
||||
return nullptr;
|
||||
|
||||
nested_types.push_back(nested_type);
|
||||
}
|
||||
|
||||
if (buf.eof())
|
||||
return nullptr;
|
||||
|
||||
++buf.position();
|
||||
|
||||
if (nested_types.empty())
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeNothing>());
|
||||
|
||||
transformInferredTypesIfNeeded(nested_types, settings);
|
||||
|
||||
auto least_supertype = tryGetLeastSupertype(nested_types);
|
||||
if (!least_supertype)
|
||||
return nullptr;
|
||||
|
||||
return std::make_shared<DataTypeArray>(least_supertype);
|
||||
}
|
||||
|
||||
/// Tuple
|
||||
if (checkChar('(', buf))
|
||||
{
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
DataTypes nested_types;
|
||||
bool first = true;
|
||||
while (!buf.eof() && *buf.position() != ')')
|
||||
{
|
||||
if (!first)
|
||||
{
|
||||
skipWhitespaceIfAny(buf);
|
||||
if (!checkChar(',', buf))
|
||||
return nullptr;
|
||||
skipWhitespaceIfAny(buf);
|
||||
}
|
||||
else
|
||||
first = false;
|
||||
|
||||
auto nested_type = determineDataTypeForSingleFieldImpl(buf, settings);
|
||||
if (!nested_type)
|
||||
return nullptr;
|
||||
|
||||
nested_types.push_back(nested_type);
|
||||
}
|
||||
|
||||
if (buf.eof() || nested_types.empty())
|
||||
return nullptr;
|
||||
|
||||
++buf.position();
|
||||
|
||||
return std::make_shared<DataTypeTuple>(nested_types);
|
||||
}
|
||||
|
||||
/// Map
|
||||
if (checkChar('{', buf))
|
||||
{
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
DataTypes key_types;
|
||||
DataTypes value_types;
|
||||
bool first = true;
|
||||
while (!buf.eof() && *buf.position() != '}')
|
||||
{
|
||||
if (!first)
|
||||
{
|
||||
skipWhitespaceIfAny(buf);
|
||||
if (!checkChar(',', buf))
|
||||
return nullptr;
|
||||
skipWhitespaceIfAny(buf);
|
||||
}
|
||||
else
|
||||
first = false;
|
||||
|
||||
auto key_type = determineDataTypeForSingleFieldImpl(buf, settings);
|
||||
if (!key_type)
|
||||
return nullptr;
|
||||
|
||||
key_types.push_back(key_type);
|
||||
|
||||
skipWhitespaceIfAny(buf);
|
||||
if (!checkChar(':', buf))
|
||||
return nullptr;
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
auto value_type = determineDataTypeForSingleFieldImpl(buf, settings);
|
||||
if (!value_type)
|
||||
return nullptr;
|
||||
|
||||
value_types.push_back(value_type);
|
||||
}
|
||||
|
||||
if (buf.eof())
|
||||
return nullptr;
|
||||
|
||||
++buf.position();
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
if (key_types.empty())
|
||||
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeNothing>(), std::make_shared<DataTypeNothing>());
|
||||
|
||||
transformInferredTypesIfNeeded(key_types, settings);
|
||||
transformInferredTypesIfNeeded(value_types, settings);
|
||||
|
||||
auto key_least_supertype = tryGetLeastSupertype(key_types);
|
||||
|
||||
auto value_least_supertype = tryGetLeastSupertype(value_types);
|
||||
if (!key_least_supertype || !value_least_supertype)
|
||||
return nullptr;
|
||||
|
||||
if (!DataTypeMap::checkKeyType(key_least_supertype))
|
||||
return nullptr;
|
||||
|
||||
return std::make_shared<DataTypeMap>(key_least_supertype, value_least_supertype);
|
||||
}
|
||||
|
||||
/// String
|
||||
if (*buf.position() == '\'')
|
||||
{
|
||||
++buf.position();
|
||||
String field;
|
||||
while (!buf.eof())
|
||||
{
|
||||
char * next_pos = find_first_symbols<'\\', '\''>(buf.position(), buf.buffer().end());
|
||||
field.append(buf.position(), next_pos);
|
||||
buf.position() = next_pos;
|
||||
|
||||
if (!buf.hasPendingData())
|
||||
continue;
|
||||
|
||||
if (*buf.position() == '\'')
|
||||
break;
|
||||
|
||||
field.push_back(*buf.position());
|
||||
if (*buf.position() == '\\')
|
||||
++buf.position();
|
||||
}
|
||||
|
||||
if (buf.eof())
|
||||
return nullptr;
|
||||
|
||||
++buf.position();
|
||||
if (auto type = tryInferDateOrDateTime(field, settings))
|
||||
return type;
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
/// Bool
|
||||
if (checkStringCaseInsensitive("true", buf) || checkStringCaseInsensitive("false", buf))
|
||||
return DataTypeFactory::instance().get("Bool");
|
||||
|
||||
/// Null
|
||||
if (checkStringCaseInsensitive("NULL", buf))
|
||||
return std::make_shared<DataTypeNothing>();
|
||||
|
||||
/// Number
|
||||
Float64 tmp;
|
||||
auto * pos_before_float = buf.position();
|
||||
if (tryReadFloatText(tmp, buf))
|
||||
{
|
||||
if (settings.try_infer_integers)
|
||||
{
|
||||
auto * float_end_pos = buf.position();
|
||||
buf.position() = pos_before_float;
|
||||
Int64 tmp_int;
|
||||
if (tryReadIntText(tmp_int, buf) && buf.position() == float_end_pos)
|
||||
return std::make_shared<DataTypeInt64>();
|
||||
|
||||
buf.position() = float_end_pos;
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
static DataTypePtr determineDataTypeForSingleField(ReadBufferFromString & buf, const FormatSettings & settings)
|
||||
{
|
||||
return makeNullableRecursivelyAndCheckForNothing(determineDataTypeForSingleFieldImpl(buf, settings));
|
||||
}
|
||||
|
||||
DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule)
|
||||
DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info)
|
||||
{
|
||||
switch (escaping_rule)
|
||||
{
|
||||
case FormatSettings::EscapingRule::Quoted:
|
||||
{
|
||||
ReadBufferFromString buf(field);
|
||||
auto type = determineDataTypeForSingleField(buf, format_settings);
|
||||
return buf.eof() ? type : nullptr;
|
||||
}
|
||||
return tryInferDataTypeForSingleField(field, format_settings);
|
||||
case FormatSettings::EscapingRule::JSON:
|
||||
return JSONUtils::getDataTypeFromField(field, format_settings);
|
||||
return tryInferDataTypeForSingleJSONField(field, format_settings, json_info);
|
||||
case FormatSettings::EscapingRule::CSV:
|
||||
{
|
||||
if (!format_settings.csv.use_best_effort_in_schema_inference)
|
||||
return makeNullable(std::make_shared<DataTypeString>());
|
||||
return std::make_shared<DataTypeString>();
|
||||
|
||||
if (field.empty() || field == format_settings.csv.null_representation)
|
||||
if (field.empty())
|
||||
return nullptr;
|
||||
|
||||
if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation)
|
||||
return DataTypeFactory::instance().get("Nullable(Bool)");
|
||||
if (field == format_settings.csv.null_representation)
|
||||
return makeNullable(std::make_shared<DataTypeNothing>());
|
||||
|
||||
if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation)
|
||||
return DataTypeFactory::instance().get("Bool");
|
||||
|
||||
/// In CSV complex types are serialized in quotes. If we have quotes, we should try to infer type
|
||||
/// from data inside quotes.
|
||||
if (field.size() > 1 && ((field.front() == '\'' && field.back() == '\'') || (field.front() == '"' && field.back() == '"')))
|
||||
{
|
||||
auto data = std::string_view(field.data() + 1, field.size() - 2);
|
||||
if (auto date_type = tryInferDateOrDateTime(data, format_settings))
|
||||
/// First, try to infer dates and datetimes.
|
||||
if (auto date_type = tryInferDateOrDateTimeFromString(data, format_settings))
|
||||
return date_type;
|
||||
|
||||
ReadBufferFromString buf(data);
|
||||
/// Try to determine the type of value inside quotes
|
||||
auto type = determineDataTypeForSingleField(buf, format_settings);
|
||||
auto type = tryInferDataTypeForSingleField(data, format_settings);
|
||||
|
||||
if (!type)
|
||||
return nullptr;
|
||||
|
||||
/// If it's a number or tuple in quotes or there is some unread data in buffer, we determine it as a string.
|
||||
if (isNumber(removeNullable(type)) || isTuple(type) || !buf.eof())
|
||||
return makeNullable(std::make_shared<DataTypeString>());
|
||||
/// If we couldn't infer any type or it's a number or tuple in quotes, we determine it as a string.
|
||||
if (!type || isNumber(removeNullable(type)) || isTuple(type))
|
||||
return std::make_shared<DataTypeString>();
|
||||
|
||||
return type;
|
||||
}
|
||||
|
||||
/// Case when CSV value is not in quotes. Check if it's a number, and if not, determine it's as a string.
|
||||
if (format_settings.try_infer_integers)
|
||||
{
|
||||
ReadBufferFromString buf(field);
|
||||
Int64 tmp_int;
|
||||
if (tryReadIntText(tmp_int, buf) && buf.eof())
|
||||
return makeNullable(std::make_shared<DataTypeInt64>());
|
||||
}
|
||||
auto type = tryInferNumberFromString(field, format_settings);
|
||||
|
||||
ReadBufferFromString buf(field);
|
||||
Float64 tmp;
|
||||
if (tryReadFloatText(tmp, buf) && buf.eof())
|
||||
return makeNullable(std::make_shared<DataTypeFloat64>());
|
||||
if (!type)
|
||||
return std::make_shared<DataTypeString>();
|
||||
|
||||
return makeNullable(std::make_shared<DataTypeString>());
|
||||
return type;
|
||||
}
|
||||
case FormatSettings::EscapingRule::Raw: [[fallthrough]];
|
||||
case FormatSettings::EscapingRule::Escaped:
|
||||
{
|
||||
if (!format_settings.tsv.use_best_effort_in_schema_inference)
|
||||
return makeNullable(std::make_shared<DataTypeString>());
|
||||
return std::make_shared<DataTypeString>();
|
||||
|
||||
if (field.empty() || field == format_settings.tsv.null_representation)
|
||||
if (field.empty())
|
||||
return nullptr;
|
||||
|
||||
if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation)
|
||||
return DataTypeFactory::instance().get("Nullable(Bool)");
|
||||
if (field == format_settings.tsv.null_representation)
|
||||
return makeNullable(std::make_shared<DataTypeNothing>());
|
||||
|
||||
if (auto date_type = tryInferDateOrDateTime(field, format_settings))
|
||||
if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation)
|
||||
return DataTypeFactory::instance().get("Bool");
|
||||
|
||||
if (auto date_type = tryInferDateOrDateTimeFromString(field, format_settings))
|
||||
return date_type;
|
||||
|
||||
ReadBufferFromString buf(field);
|
||||
auto type = determineDataTypeForSingleField(buf, format_settings);
|
||||
if (!buf.eof())
|
||||
return makeNullable(std::make_shared<DataTypeString>());
|
||||
|
||||
auto type = tryInferDataTypeForSingleField(field, format_settings);
|
||||
if (!type)
|
||||
return std::make_shared<DataTypeString>();
|
||||
return type;
|
||||
}
|
||||
default:
|
||||
@ -804,15 +328,34 @@ DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSe
|
||||
}
|
||||
}
|
||||
|
||||
DataTypes determineDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule)
|
||||
DataTypes tryInferDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info)
|
||||
{
|
||||
DataTypes data_types;
|
||||
data_types.reserve(fields.size());
|
||||
for (const auto & field : fields)
|
||||
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, escaping_rule));
|
||||
data_types.push_back(tryInferDataTypeByEscapingRule(field, format_settings, escaping_rule, json_info));
|
||||
return data_types;
|
||||
}
|
||||
|
||||
void transformInferredTypesByEscapingRuleIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info)
|
||||
{
|
||||
switch (escaping_rule)
|
||||
{
|
||||
case FormatSettings::EscapingRule::JSON:
|
||||
transformInferredJSONTypesIfNeeded(first, second, settings, json_info);
|
||||
break;
|
||||
case FormatSettings::EscapingRule::Escaped: [[fallthrough]];
|
||||
case FormatSettings::EscapingRule::Raw: [[fallthrough]];
|
||||
case FormatSettings::EscapingRule::Quoted: [[fallthrough]];
|
||||
case FormatSettings::EscapingRule::CSV:
|
||||
transformInferredTypesIfNeeded(first, second, settings);
|
||||
break;
|
||||
default:
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot transform inferred types for value with {} escaping rule", escapingRuleToString(escaping_rule));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule)
|
||||
{
|
||||
switch (escaping_rule)
|
||||
@ -820,7 +363,7 @@ DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escap
|
||||
case FormatSettings::EscapingRule::CSV:
|
||||
case FormatSettings::EscapingRule::Escaped:
|
||||
case FormatSettings::EscapingRule::Raw:
|
||||
return makeNullable(std::make_shared<DataTypeString>());
|
||||
return std::make_shared<DataTypeString>();
|
||||
default:
|
||||
return nullptr;
|
||||
}
|
||||
@ -837,9 +380,10 @@ DataTypes getDefaultDataTypeForEscapingRules(const std::vector<FormatSettings::E
|
||||
String getAdditionalFormatInfoForAllRowBasedFormats(const FormatSettings & settings)
|
||||
{
|
||||
return fmt::format(
|
||||
"schema_inference_hints={}, max_rows_to_read_for_schema_inference={}",
|
||||
"schema_inference_hints={}, max_rows_to_read_for_schema_inference={}, schema_inference_make_columns_nullable={}",
|
||||
settings.schema_inference_hints,
|
||||
settings.max_rows_to_read_for_schema_inference);
|
||||
settings.max_rows_to_read_for_schema_inference,
|
||||
settings.schema_inference_make_columns_nullable);
|
||||
}
|
||||
|
||||
String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule)
|
||||
@ -876,7 +420,11 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo
|
||||
settings.csv.tuple_delimiter);
|
||||
break;
|
||||
case FormatSettings::EscapingRule::JSON:
|
||||
result += fmt::format(", try_infer_numbers_from_strings={}, read_bools_as_numbers={}", settings.json.try_infer_numbers_from_strings, settings.json.read_bools_as_numbers);
|
||||
result += fmt::format(
|
||||
", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, try_infer_objects={}",
|
||||
settings.json.try_infer_numbers_from_strings,
|
||||
settings.json.read_bools_as_numbers,
|
||||
settings.json.try_infer_objects);
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
@ -38,45 +39,17 @@ String readFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule es
|
||||
|
||||
/// Try to determine the type of the field written by a specific escaping rule.
|
||||
/// If cannot, return nullptr.
|
||||
/// - For Quoted escaping rule we can interpret a single field as a constant
|
||||
/// expression and get it's type by evaluation this expression.
|
||||
/// - For JSON escaping rule we can use JSON parser to parse a single field
|
||||
/// and then convert JSON type of this field to ClickHouse type.
|
||||
/// - For CSV escaping rule we can do the next:
|
||||
/// - If the field is an unquoted string, then we try to parse it as a number,
|
||||
/// and if we cannot, treat it as a String.
|
||||
/// - If the field is a string in quotes, then we try to use some
|
||||
/// tweaks and heuristics to determine the type inside quotes, and if we can't or
|
||||
/// the result is a number or tuple (we don't parse numbers in quotes and don't
|
||||
/// support tuples in CSV) we treat it as a String.
|
||||
/// - If input_format_csv_use_best_effort_in_schema_inference is disabled, we
|
||||
/// treat everything as a string.
|
||||
/// - For TSV and TSVRaw we try to use some tweaks and heuristics to determine the type
|
||||
/// of value if setting input_format_tsv_use_best_effort_in_schema_inference is enabled,
|
||||
/// otherwise we treat everything as a string.
|
||||
DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule);
|
||||
DataTypes determineDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule);
|
||||
/// See tryInferDataTypeForSingle(JSON)Field in SchemaInferenceUtils.h
|
||||
DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info = nullptr);
|
||||
DataTypes tryInferDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info = nullptr);
|
||||
|
||||
/// Check if we need to transform types inferred from data and transform it if necessary.
|
||||
/// See transformInferred(JSON)TypesIfNeeded in SchemaInferenceUtils.h
|
||||
void transformInferredTypesByEscapingRuleIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule, JSONInferenceInfo * json_info = nullptr);
|
||||
|
||||
DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule);
|
||||
DataTypes getDefaultDataTypeForEscapingRules(const std::vector<FormatSettings::EscapingRule> & escaping_rules);
|
||||
|
||||
/// Try to infer Date or Datetime from string if corresponding settings are enabled.
|
||||
DataTypePtr tryInferDateOrDateTime(const std::string_view & field, const FormatSettings & settings);
|
||||
|
||||
/// Check if we need to transform types inferred from data and transform it if necessary.
|
||||
/// It's used when we try to infer some not ordinary types from another types.
|
||||
/// For example dates from strings, we should check if dates were inferred from all strings
|
||||
/// in the same way and if not, transform inferred dates back to strings.
|
||||
/// For example, if we have array of strings and we tried to infer dates from them,
|
||||
/// to make the result type Array(Date) we should ensure that all strings were
|
||||
/// successfully parsed as dated and if not, convert all dates back to strings and make result type Array(String).
|
||||
void transformInferredTypesIfNeeded(DataTypes & types, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule = FormatSettings::EscapingRule::Escaped);
|
||||
void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule = FormatSettings::EscapingRule::Escaped);
|
||||
|
||||
/// Same as transformInferredTypesIfNeeded but takes into account settings that are special for JSON formats.
|
||||
void transformInferredJSONTypesIfNeeded(DataTypes & types, const FormatSettings & settings, const std::unordered_set<const IDataType *> * numbers_parsed_from_json_strings = nullptr);
|
||||
void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings);
|
||||
|
||||
String getAdditionalFormatInfoForAllRowBasedFormats(const FormatSettings & settings);
|
||||
String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule);
|
||||
|
||||
|
@ -168,6 +168,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
|
||||
format_settings.max_rows_to_read_for_schema_inference = settings.input_format_max_rows_to_read_for_schema_inference;
|
||||
format_settings.column_names_for_schema_inference = settings.column_names_for_schema_inference;
|
||||
format_settings.schema_inference_hints = settings.schema_inference_hints;
|
||||
format_settings.schema_inference_make_columns_nullable = settings.schema_inference_make_columns_nullable;
|
||||
format_settings.mysql_dump.table_name = settings.input_format_mysql_dump_table_name;
|
||||
format_settings.mysql_dump.map_column_names = settings.input_format_mysql_dump_map_column_names;
|
||||
format_settings.sql_insert.max_batch_size = settings.output_format_sql_insert_max_batch_size;
|
||||
|
@ -71,6 +71,8 @@ struct FormatSettings
|
||||
Raw
|
||||
};
|
||||
|
||||
bool schema_inference_make_columns_nullable = true;
|
||||
|
||||
DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple;
|
||||
|
||||
bool input_format_ipv4_default_on_conversion_error = false;
|
||||
|
@ -6,19 +6,13 @@
|
||||
#include <IO/WriteBufferValidUTF8.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#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>
|
||||
|
||||
#include <base/find_symbols.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -122,196 +116,6 @@ namespace JSONUtils
|
||||
return {loadAtPosition(in, memory, pos), number_of_rows};
|
||||
}
|
||||
|
||||
template <const char opening_bracket, const char closing_bracket>
|
||||
static String readJSONEachRowLineIntoStringImpl(ReadBuffer & in)
|
||||
{
|
||||
Memory memory;
|
||||
fileSegmentationEngineJSONEachRowImpl<opening_bracket, closing_bracket>(in, memory, 0, 1, 1);
|
||||
return String(memory.data(), memory.size());
|
||||
}
|
||||
|
||||
template <class Element>
|
||||
DataTypePtr getDataTypeFromFieldImpl(const Element & field, const FormatSettings & settings, std::unordered_set<const IDataType *> & numbers_parsed_from_json_strings)
|
||||
{
|
||||
if (field.isNull())
|
||||
return nullptr;
|
||||
|
||||
if (field.isBool())
|
||||
return DataTypeFactory::instance().get("Nullable(Bool)");
|
||||
|
||||
if (field.isInt64() || field.isUInt64())
|
||||
{
|
||||
if (settings.try_infer_integers)
|
||||
return makeNullable(std::make_shared<DataTypeInt64>());
|
||||
|
||||
return makeNullable(std::make_shared<DataTypeFloat64>());
|
||||
}
|
||||
|
||||
if (field.isDouble())
|
||||
return makeNullable(std::make_shared<DataTypeFloat64>());
|
||||
|
||||
if (field.isString())
|
||||
{
|
||||
if (auto date_type = tryInferDateOrDateTime(field.getString(), settings))
|
||||
return date_type;
|
||||
|
||||
if (!settings.json.try_infer_numbers_from_strings)
|
||||
return makeNullable(std::make_shared<DataTypeString>());
|
||||
|
||||
ReadBufferFromString buf(field.getString());
|
||||
|
||||
if (settings.try_infer_integers)
|
||||
{
|
||||
Int64 tmp_int;
|
||||
if (tryReadIntText(tmp_int, buf) && buf.eof())
|
||||
{
|
||||
auto type = std::make_shared<DataTypeInt64>();
|
||||
numbers_parsed_from_json_strings.insert(type.get());
|
||||
return makeNullable(type);
|
||||
}
|
||||
}
|
||||
|
||||
Float64 tmp;
|
||||
if (tryReadFloatText(tmp, buf) && buf.eof())
|
||||
{
|
||||
auto type = std::make_shared<DataTypeFloat64>();
|
||||
numbers_parsed_from_json_strings.insert(type.get());
|
||||
return makeNullable(type);
|
||||
}
|
||||
|
||||
return makeNullable(std::make_shared<DataTypeString>());
|
||||
}
|
||||
|
||||
if (field.isArray())
|
||||
{
|
||||
auto array = field.getArray();
|
||||
|
||||
/// Return nullptr in case of empty array because we cannot determine nested type.
|
||||
if (array.size() == 0)
|
||||
return nullptr;
|
||||
|
||||
DataTypes nested_data_types;
|
||||
/// If this array contains fields with different types we will treat it as Tuple.
|
||||
bool are_types_the_same = true;
|
||||
for (const auto element : array)
|
||||
{
|
||||
auto type = getDataTypeFromFieldImpl(element, settings, numbers_parsed_from_json_strings);
|
||||
if (!type)
|
||||
return nullptr;
|
||||
|
||||
if (!nested_data_types.empty() && !type->equals(*nested_data_types.back()))
|
||||
are_types_the_same = false;
|
||||
|
||||
nested_data_types.push_back(std::move(type));
|
||||
}
|
||||
|
||||
if (!are_types_the_same)
|
||||
{
|
||||
auto nested_types_copy = nested_data_types;
|
||||
transformInferredJSONTypesIfNeeded(nested_types_copy, settings, &numbers_parsed_from_json_strings);
|
||||
are_types_the_same = true;
|
||||
for (size_t i = 1; i < nested_types_copy.size(); ++i)
|
||||
are_types_the_same &= nested_types_copy[i]->equals(*nested_types_copy[i - 1]);
|
||||
|
||||
if (are_types_the_same)
|
||||
nested_data_types = std::move(nested_types_copy);
|
||||
}
|
||||
|
||||
if (!are_types_the_same)
|
||||
return std::make_shared<DataTypeTuple>(nested_data_types);
|
||||
|
||||
return std::make_shared<DataTypeArray>(nested_data_types.back());
|
||||
}
|
||||
|
||||
if (field.isObject())
|
||||
{
|
||||
auto object = field.getObject();
|
||||
DataTypes value_types;
|
||||
for (const auto key_value_pair : object)
|
||||
{
|
||||
auto type = getDataTypeFromFieldImpl(key_value_pair.second, settings, numbers_parsed_from_json_strings);
|
||||
if (!type)
|
||||
{
|
||||
/// If we couldn't infer nested type and Object type is not enabled,
|
||||
/// we can't determine the type of this JSON field.
|
||||
if (!settings.json.try_infer_objects)
|
||||
return nullptr;
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
if (settings.json.try_infer_objects && isObject(type))
|
||||
return std::make_shared<DataTypeObject>("json", true);
|
||||
|
||||
value_types.push_back(type);
|
||||
}
|
||||
|
||||
if (value_types.empty())
|
||||
return nullptr;
|
||||
|
||||
transformInferredJSONTypesIfNeeded(value_types, settings, &numbers_parsed_from_json_strings);
|
||||
bool are_types_equal = true;
|
||||
for (size_t i = 1; i < value_types.size(); ++i)
|
||||
are_types_equal &= value_types[i]->equals(*value_types[0]);
|
||||
|
||||
if (!are_types_equal)
|
||||
{
|
||||
if (!settings.json.try_infer_objects)
|
||||
return nullptr;
|
||||
return std::make_shared<DataTypeObject>("json", true);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), value_types[0]);
|
||||
}
|
||||
|
||||
throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type"};
|
||||
}
|
||||
|
||||
auto getJSONParserAndElement()
|
||||
{
|
||||
#if USE_SIMDJSON
|
||||
return std::pair<SimdJSONParser, SimdJSONParser::Element>();
|
||||
#elif USE_RAPIDJSON
|
||||
return std::pair<RapidJSONParser, RapidJSONParser::Element>();
|
||||
#else
|
||||
return std::pair<DummyJSONParser, DummyJSONParser::Element>();
|
||||
#endif
|
||||
}
|
||||
|
||||
DataTypePtr getDataTypeFromField(const String & field, const FormatSettings & settings)
|
||||
{
|
||||
auto [parser, element] = getJSONParserAndElement();
|
||||
bool parsed = parser.parse(field, element);
|
||||
if (!parsed)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse JSON object here: {}", field);
|
||||
|
||||
std::unordered_set<const IDataType *> numbers_parsed_from_json_strings;
|
||||
return getDataTypeFromFieldImpl(element, settings, numbers_parsed_from_json_strings);
|
||||
}
|
||||
|
||||
template <class Extractor, const char opening_bracket, const char closing_bracket>
|
||||
static DataTypes determineColumnDataTypesFromJSONEachRowDataImpl(ReadBuffer & in, const FormatSettings & settings, bool /*json_strings*/, Extractor & extractor)
|
||||
{
|
||||
String line = readJSONEachRowLineIntoStringImpl<opening_bracket, closing_bracket>(in);
|
||||
auto [parser, element] = getJSONParserAndElement();
|
||||
bool parsed = parser.parse(line, element);
|
||||
if (!parsed)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse JSON object here: {}", line);
|
||||
|
||||
auto fields = extractor.extract(element);
|
||||
|
||||
DataTypes data_types;
|
||||
data_types.reserve(fields.size());
|
||||
std::unordered_set<const IDataType *> numbers_parsed_from_json_strings;
|
||||
for (const auto & field : fields)
|
||||
data_types.push_back(getDataTypeFromFieldImpl(field, settings, numbers_parsed_from_json_strings));
|
||||
|
||||
/// TODO: For JSONStringsEachRow/JSONCompactStringsEach all types will be strings.
|
||||
/// Should we try to parse data inside strings somehow in this case?
|
||||
|
||||
return data_types;
|
||||
}
|
||||
|
||||
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t max_rows)
|
||||
{
|
||||
return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_bytes, 1, max_rows);
|
||||
@ -323,68 +127,56 @@ namespace JSONUtils
|
||||
return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_bytes, min_rows, max_rows);
|
||||
}
|
||||
|
||||
struct JSONEachRowFieldsExtractor
|
||||
NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, JSONInferenceInfo * inference_info)
|
||||
{
|
||||
template <class Element>
|
||||
std::vector<Element> extract(const Element & element)
|
||||
skipWhitespaceIfAny(in);
|
||||
assertChar('{', in);
|
||||
bool first = true;
|
||||
NamesAndTypesList names_and_types;
|
||||
String field;
|
||||
while (!in.eof() && *in.position() != '}')
|
||||
{
|
||||
/// {..., "<column_name>" : <value>, ...}
|
||||
if (!first)
|
||||
skipComma(in);
|
||||
else
|
||||
first = false;
|
||||
|
||||
if (!element.isObject())
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Root JSON value is not an object");
|
||||
|
||||
auto object = element.getObject();
|
||||
std::vector<Element> fields;
|
||||
fields.reserve(object.size());
|
||||
column_names.reserve(object.size());
|
||||
for (const auto & key_value_pair : object)
|
||||
{
|
||||
column_names.emplace_back(key_value_pair.first);
|
||||
fields.push_back(key_value_pair.second);
|
||||
}
|
||||
|
||||
return fields;
|
||||
auto name = readFieldName(in);
|
||||
auto type = tryInferDataTypeForSingleJSONField(in, settings, inference_info);
|
||||
names_and_types.emplace_back(name, type);
|
||||
}
|
||||
|
||||
std::vector<String> column_names;
|
||||
};
|
||||
if (in.eof())
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF while reading JSON object");
|
||||
|
||||
NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings)
|
||||
{
|
||||
JSONEachRowFieldsExtractor extractor;
|
||||
auto data_types
|
||||
= determineColumnDataTypesFromJSONEachRowDataImpl<JSONEachRowFieldsExtractor, '{', '}'>(in, settings, json_strings, extractor);
|
||||
NamesAndTypesList result;
|
||||
for (size_t i = 0; i != extractor.column_names.size(); ++i)
|
||||
result.emplace_back(extractor.column_names[i], data_types[i]);
|
||||
return result;
|
||||
assertChar('}', in);
|
||||
return names_and_types;
|
||||
}
|
||||
|
||||
struct JSONCompactEachRowFieldsExtractor
|
||||
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, JSONInferenceInfo * inference_info)
|
||||
{
|
||||
template <class Element>
|
||||
std::vector<Element> extract(const Element & element)
|
||||
skipWhitespaceIfAny(in);
|
||||
assertChar('[', in);
|
||||
bool first = true;
|
||||
DataTypes types;
|
||||
String field;
|
||||
while (!in.eof() && *in.position() != ']')
|
||||
{
|
||||
/// [..., <value>, ...]
|
||||
if (!element.isArray())
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Root JSON value is not an array");
|
||||
|
||||
auto array = element.getArray();
|
||||
std::vector<Element> fields;
|
||||
fields.reserve(array.size());
|
||||
for (size_t i = 0; i != array.size(); ++i)
|
||||
fields.push_back(array[i]);
|
||||
return fields;
|
||||
if (!first)
|
||||
skipComma(in);
|
||||
else
|
||||
first = false;
|
||||
auto type = tryInferDataTypeForSingleJSONField(in, settings, inference_info);
|
||||
types.push_back(type);
|
||||
}
|
||||
};
|
||||
|
||||
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings)
|
||||
{
|
||||
JSONCompactEachRowFieldsExtractor extractor;
|
||||
return determineColumnDataTypesFromJSONEachRowDataImpl<JSONCompactEachRowFieldsExtractor, '[', ']'>(in, settings, json_strings, extractor);
|
||||
if (in.eof())
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF while reading JSON array");
|
||||
|
||||
assertChar(']', in);
|
||||
return types;
|
||||
}
|
||||
|
||||
|
||||
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf)
|
||||
{
|
||||
/// For JSONEachRow we can safely skip whitespace characters
|
||||
|
@ -13,24 +13,21 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct JSONInferenceInfo;
|
||||
|
||||
namespace JSONUtils
|
||||
{
|
||||
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t max_rows);
|
||||
std::pair<bool, size_t> fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t min_rows, size_t max_rows);
|
||||
|
||||
/// Parse JSON from string and convert it's type to ClickHouse type. Make the result type always Nullable.
|
||||
/// JSON array with different nested types is treated as Tuple.
|
||||
/// If cannot convert (for example when field contains null), return nullptr.
|
||||
DataTypePtr getDataTypeFromField(const String & field, const FormatSettings & settings);
|
||||
|
||||
/// Read row in JSONEachRow format and try to determine type for each field.
|
||||
/// Return list of names and types.
|
||||
/// If cannot determine the type of some field, return nullptr for it.
|
||||
NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings);
|
||||
NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, JSONInferenceInfo * inference_info);
|
||||
|
||||
/// Read row in JSONCompactEachRow format and try to determine type for each field.
|
||||
/// If cannot determine the type of some field, return nullptr for it.
|
||||
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings);
|
||||
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, JSONInferenceInfo * inference_info);
|
||||
|
||||
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf);
|
||||
|
||||
|
@ -197,69 +197,6 @@ ColumnsDescription readSchemaFromFormat(const String & format_name, const std::o
|
||||
return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out);
|
||||
}
|
||||
|
||||
DataTypePtr makeNullableRecursivelyAndCheckForNothing(DataTypePtr type)
|
||||
{
|
||||
if (!type)
|
||||
return nullptr;
|
||||
|
||||
WhichDataType which(type);
|
||||
|
||||
if (which.isNothing())
|
||||
return nullptr;
|
||||
|
||||
if (which.isNullable())
|
||||
{
|
||||
const auto * nullable_type = assert_cast<const DataTypeNullable *>(type.get());
|
||||
return makeNullableRecursivelyAndCheckForNothing(nullable_type->getNestedType());
|
||||
}
|
||||
|
||||
if (which.isArray())
|
||||
{
|
||||
const auto * array_type = assert_cast<const DataTypeArray *>(type.get());
|
||||
auto nested_type = makeNullableRecursivelyAndCheckForNothing(array_type->getNestedType());
|
||||
return nested_type ? std::make_shared<DataTypeArray>(nested_type) : nullptr;
|
||||
}
|
||||
|
||||
if (which.isTuple())
|
||||
{
|
||||
const auto * tuple_type = assert_cast<const DataTypeTuple *>(type.get());
|
||||
DataTypes nested_types;
|
||||
for (const auto & element : tuple_type->getElements())
|
||||
{
|
||||
auto nested_type = makeNullableRecursivelyAndCheckForNothing(element);
|
||||
if (!nested_type)
|
||||
return nullptr;
|
||||
nested_types.push_back(nested_type);
|
||||
}
|
||||
return std::make_shared<DataTypeTuple>(std::move(nested_types));
|
||||
}
|
||||
|
||||
if (which.isMap())
|
||||
{
|
||||
const auto * map_type = assert_cast<const DataTypeMap *>(type.get());
|
||||
auto key_type = makeNullableRecursivelyAndCheckForNothing(map_type->getKeyType());
|
||||
auto value_type = makeNullableRecursivelyAndCheckForNothing(map_type->getValueType());
|
||||
return key_type && value_type ? std::make_shared<DataTypeMap>(removeNullable(key_type), value_type) : nullptr;
|
||||
}
|
||||
|
||||
if (which.isLowCarnality())
|
||||
{
|
||||
const auto * lc_type = assert_cast<const DataTypeLowCardinality *>(type.get());
|
||||
auto nested_type = makeNullableRecursivelyAndCheckForNothing(lc_type->getDictionaryType());
|
||||
return nested_type ? std::make_shared<DataTypeLowCardinality>(nested_type) : nullptr;
|
||||
}
|
||||
|
||||
return makeNullable(type);
|
||||
}
|
||||
|
||||
NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header)
|
||||
{
|
||||
NamesAndTypesList result;
|
||||
for (auto & [name, type] : header.getNamesAndTypesList())
|
||||
result.emplace_back(name, makeNullableRecursivelyAndCheckForNothing(type));
|
||||
return result;
|
||||
}
|
||||
|
||||
SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context)
|
||||
{
|
||||
return getKeysForSchemaCache({source}, format, format_settings, context).front();
|
||||
|
@ -35,21 +35,7 @@ ColumnsDescription readSchemaFromFormat(
|
||||
ContextPtr & context,
|
||||
std::unique_ptr<ReadBuffer> & buf_out);
|
||||
|
||||
/// Make type Nullable recursively:
|
||||
/// - Type -> Nullable(type)
|
||||
/// - Array(Type) -> Array(Nullable(Type))
|
||||
/// - Tuple(Type1, ..., TypeN) -> Tuple(Nullable(Type1), ..., Nullable(TypeN))
|
||||
/// - Map(KeyType, ValueType) -> Map(KeyType, Nullable(ValueType))
|
||||
/// - LowCardinality(Type) -> LowCardinality(Nullable(Type))
|
||||
/// If type is Nothing or one of the nested types is Nothing, return nullptr.
|
||||
DataTypePtr makeNullableRecursivelyAndCheckForNothing(DataTypePtr type);
|
||||
|
||||
/// Call makeNullableRecursivelyAndCheckForNothing for all types
|
||||
/// in the block and return names and types.
|
||||
NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header);
|
||||
|
||||
SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context);
|
||||
SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context);
|
||||
|
||||
void splitSchemaCacheKey(const String & key, String & source, String & format, String & additional_format_info);
|
||||
}
|
||||
|
966
src/Formats/SchemaInferenceUtils.cpp
Normal file
966
src/Formats/SchemaInferenceUtils.cpp
Normal file
@ -0,0 +1,966 @@
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/transformTypesRecursively.h>
|
||||
#include <DataTypes/DataTypeObject.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static bool checkIfTypesAreEqual(const DataTypes & types)
|
||||
{
|
||||
for (size_t i = 1; i < types.size(); ++i)
|
||||
{
|
||||
if (!types[0]->equals(*types[i]))
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/// If we have both Nothing and non Nothing types, convert all Nothing types to the first non Nothing.
|
||||
/// For example if we have types [Nothing, String, Nothing] we change it to [String, String, String]
|
||||
static void transformNothingSimpleTypes(DataTypes & data_types)
|
||||
{
|
||||
bool have_nothing = false;
|
||||
DataTypePtr not_nothing_type = nullptr;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
if (isNothing(type))
|
||||
have_nothing = true;
|
||||
else if (!not_nothing_type)
|
||||
not_nothing_type = type;
|
||||
}
|
||||
|
||||
if (have_nothing && not_nothing_type)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isNothing(type))
|
||||
type = not_nothing_type;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have both Int64 and Float64 types, convert all Int64 to Float64.
|
||||
static void transformIntegersAndFloatsToFloats(DataTypes & data_types)
|
||||
{
|
||||
bool have_floats = false;
|
||||
bool have_integers = false;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
have_floats |= isFloat(type);
|
||||
have_integers |= isInteger(type) && !isBool(type);
|
||||
}
|
||||
|
||||
if (have_floats && have_integers)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isInteger(type))
|
||||
type = std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have only Date and DateTime types, convert Date to DateTime,
|
||||
/// otherwise, convert all Date and DateTime to String.
|
||||
static void transformDatesAndDateTimes(DataTypes & data_types)
|
||||
{
|
||||
bool have_dates = false;
|
||||
bool have_datetimes = false;
|
||||
bool all_dates_or_datetimes = true;
|
||||
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
have_dates |= isDate(type);
|
||||
have_datetimes |= isDateTime64(type);
|
||||
all_dates_or_datetimes &= isDate(type) || isDateTime64(type);
|
||||
}
|
||||
|
||||
if (!all_dates_or_datetimes && (have_dates || have_datetimes))
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isDate(type) || isDateTime64(type))
|
||||
type = std::make_shared<DataTypeString>();
|
||||
}
|
||||
}
|
||||
else if (have_dates && have_datetimes)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isDate(type))
|
||||
type = std::make_shared<DataTypeDateTime64>(9);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have numbers (Int64/Float64) and String types and numbers were parsed from String,
|
||||
/// convert all numbers to String.
|
||||
static void transformJSONNumbersBackToString(DataTypes & data_types, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
bool have_strings = false;
|
||||
bool have_numbers = false;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
have_strings |= isString(type);
|
||||
have_numbers |= isNumber(type);
|
||||
}
|
||||
|
||||
if (have_strings && have_numbers)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isNumber(type)
|
||||
&& (settings.json.read_numbers_as_strings || !json_info
|
||||
|| json_info->numbers_parsed_from_json_strings.contains(type.get())))
|
||||
type = std::make_shared<DataTypeString>();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have both Bool and number (Int64/Float64) types,
|
||||
/// convert all Bool to Int64/Float64.
|
||||
static void transformBoolsAndNumbersToNumbers(DataTypes & data_types)
|
||||
{
|
||||
bool have_floats = false;
|
||||
bool have_integers = false;
|
||||
bool have_bools = false;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
have_floats |= isFloat(type);
|
||||
have_integers |= isInteger(type) && !isBool(type);
|
||||
have_bools |= isBool(type);
|
||||
}
|
||||
|
||||
if (have_bools && (have_integers || have_floats))
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isBool(type))
|
||||
{
|
||||
if (have_integers)
|
||||
type = std::make_shared<DataTypeInt64>();
|
||||
else
|
||||
type = std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have type Nothing (or Nullable(Nothing) for JSON) and some other non Nothing types,
|
||||
/// convert all Nothing types to the first non Nothing.
|
||||
/// For example, when we have [Nothing, Array(Int64)] it will convert it to [Array(Int64), Array(Int64)]
|
||||
/// (it can happen when transforming complex nested types like [Array(Nothing), Array(Array(Int64))])
|
||||
template <bool is_json>
|
||||
static void transformNothingComplexTypes(DataTypes & data_types)
|
||||
{
|
||||
bool have_nothing = false;
|
||||
DataTypePtr not_nothing_type = nullptr;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
if (isNothing(type) || (is_json && type->onlyNull()))
|
||||
have_nothing = true;
|
||||
else if (!not_nothing_type)
|
||||
not_nothing_type = type;
|
||||
}
|
||||
|
||||
if (have_nothing && not_nothing_type)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isNothing(type) || (is_json && type->onlyNull()))
|
||||
type = not_nothing_type;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have both Nullable and non Nullable types, make all types Nullable
|
||||
static void transformNullableTypes(DataTypes & data_types)
|
||||
{
|
||||
bool have_nullable = false;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
if (type->isNullable())
|
||||
{
|
||||
have_nullable = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (have_nullable)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (type->canBeInsideNullable())
|
||||
type = makeNullable(type);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have Tuple with the same nested types like Tuple(Int64, Int64),
|
||||
/// convert it to Array(Int64). It's used for JSON values.
|
||||
/// For example when we had type Tuple(Int64, Nullable(Nothing)) and we
|
||||
/// transformed it to Tuple(Nullable(Int64), Nullable(Int64)) we will
|
||||
/// also transform it to Array(Nullable(Int64))
|
||||
static void transformTuplesWithEqualNestedTypesToArrays(DataTypes & data_types)
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isTuple(type))
|
||||
{
|
||||
const auto * tuple_type = assert_cast<const DataTypeTuple *>(type.get());
|
||||
if (checkIfTypesAreEqual(tuple_type->getElements()))
|
||||
type = std::make_shared<DataTypeArray>(tuple_type->getElements().back());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <bool is_json>
|
||||
static void transformInferredTypesIfNeededImpl(DataTypes & types, const FormatSettings & settings, JSONInferenceInfo * json_info = nullptr);
|
||||
|
||||
/// If we have Tuple and Array types, try to convert them all to Array
|
||||
/// if there is a common type for all nested types.
|
||||
/// For example, if we have [Tuple(Nullable(Nothing), String), Array(Date), Tuple(Date, String)]
|
||||
/// it will convert them all to Array(String)
|
||||
static void transformJSONTuplesAndArraysToArrays(DataTypes & data_types, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
bool have_arrays = false;
|
||||
bool have_tuples = false;
|
||||
bool tuple_sizes_are_equal = true;
|
||||
size_t tuple_size = 0;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
if (isArray(type))
|
||||
have_arrays = true;
|
||||
else if (isTuple(type))
|
||||
{
|
||||
have_tuples = true;
|
||||
const auto & current_tuple_size = assert_cast<const DataTypeTuple &>(*type).getElements().size();
|
||||
if (!tuple_size)
|
||||
tuple_size = current_tuple_size;
|
||||
else
|
||||
tuple_sizes_are_equal &= current_tuple_size == tuple_size;
|
||||
}
|
||||
}
|
||||
|
||||
if (have_tuples && (have_arrays || !tuple_sizes_are_equal))
|
||||
{
|
||||
DataTypes nested_types;
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isArray(type))
|
||||
nested_types.push_back(assert_cast<const DataTypeArray &>(*type).getNestedType());
|
||||
else
|
||||
{
|
||||
const auto & elements = assert_cast<const DataTypeTuple & >(*type).getElements();
|
||||
for (const auto & element : elements)
|
||||
nested_types.push_back(element);
|
||||
}
|
||||
}
|
||||
|
||||
transformInferredTypesIfNeededImpl<true>(nested_types, settings, json_info);
|
||||
if (checkIfTypesAreEqual(nested_types))
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
type = std::make_shared<DataTypeArray>(nested_types.back());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// If we have Map and Object(JSON) types, convert all Map types to Object(JSON).
|
||||
/// If we have Map types with different value types, convert all Map types to Object(JSON)
|
||||
static void transformMapsAndObjectsToObjects(DataTypes & data_types)
|
||||
{
|
||||
bool have_maps = false;
|
||||
bool have_objects = false;
|
||||
bool maps_are_equal = true;
|
||||
DataTypePtr first_map_type;
|
||||
for (const auto & type : data_types)
|
||||
{
|
||||
if (isMap(type))
|
||||
{
|
||||
if (!have_maps)
|
||||
{
|
||||
first_map_type = type;
|
||||
have_maps = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
maps_are_equal &= type->equals(*first_map_type);
|
||||
}
|
||||
}
|
||||
else if (isObject(type))
|
||||
{
|
||||
have_objects = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (have_maps && (have_objects || !maps_are_equal))
|
||||
{
|
||||
for (auto & type : data_types)
|
||||
{
|
||||
if (isMap(type))
|
||||
type = std::make_shared<DataTypeObject>("json", true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <bool is_json>
|
||||
static void transformInferredTypesIfNeededImpl(DataTypes & types, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
auto transform_simple_types = [&](DataTypes & data_types)
|
||||
{
|
||||
/// Remove all Nothing type if possible.
|
||||
transformNothingSimpleTypes(data_types);
|
||||
|
||||
/// Transform integers to floats if needed.
|
||||
if (settings.try_infer_integers)
|
||||
transformIntegersAndFloatsToFloats(data_types);
|
||||
|
||||
/// Transform Date to DateTime or both to String if needed.
|
||||
if (settings.try_infer_dates || settings.try_infer_datetimes)
|
||||
transformDatesAndDateTimes(data_types);
|
||||
|
||||
if constexpr (!is_json)
|
||||
return;
|
||||
|
||||
/// Check settings specific for JSON formats.
|
||||
|
||||
/// Convert numbers inferred from strings back to strings if needed.
|
||||
if (settings.json.try_infer_numbers_from_strings || settings.json.read_numbers_as_strings)
|
||||
transformJSONNumbersBackToString(data_types, settings, json_info);
|
||||
|
||||
/// Convert Bool to number (Int64/Float64) if needed.
|
||||
if (settings.json.read_bools_as_numbers)
|
||||
transformBoolsAndNumbersToNumbers(data_types);
|
||||
};
|
||||
|
||||
auto transform_complex_types = [&](DataTypes & data_types)
|
||||
{
|
||||
/// Make types Nullable if needed.
|
||||
transformNullableTypes(data_types);
|
||||
|
||||
/// If we have type Nothing, it means that we had empty Array/Map while inference.
|
||||
/// If there is at least one non Nothing type, change all Nothing types to it.
|
||||
transformNothingComplexTypes<is_json>(data_types);
|
||||
|
||||
if constexpr (!is_json)
|
||||
return;
|
||||
|
||||
/// Convert JSON tuples with same nested types to arrays.
|
||||
transformTuplesWithEqualNestedTypesToArrays(data_types);
|
||||
|
||||
/// Convert JSON tuples and arrays to arrays if possible.
|
||||
transformJSONTuplesAndArraysToArrays(data_types, settings, json_info);
|
||||
|
||||
/// Convert Maps to Objects if needed.
|
||||
if (settings.json.try_infer_objects)
|
||||
transformMapsAndObjectsToObjects(data_types);
|
||||
};
|
||||
|
||||
transformTypesRecursively(types, transform_simple_types, transform_complex_types);
|
||||
}
|
||||
|
||||
void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings)
|
||||
{
|
||||
DataTypes types = {first, second};
|
||||
transformInferredTypesIfNeededImpl<false>(types, settings, nullptr);
|
||||
first = types[0];
|
||||
second = types[1];
|
||||
}
|
||||
|
||||
void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
DataTypes types = {first, second};
|
||||
transformInferredTypesIfNeededImpl<true>(types, settings, json_info);
|
||||
first = types[0];
|
||||
second = types[1];
|
||||
}
|
||||
|
||||
void transformJSONTupleToArrayIfPossible(DataTypePtr & data_type, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
if (!isTuple(data_type))
|
||||
return;
|
||||
|
||||
const auto * tuple_type = assert_cast<const DataTypeTuple *>(data_type.get());
|
||||
auto nested_types = tuple_type->getElements();
|
||||
transformInferredTypesIfNeededImpl<true>(nested_types, settings, json_info);
|
||||
if (checkIfTypesAreEqual(nested_types))
|
||||
data_type = std::make_shared<DataTypeArray>(nested_types.back());
|
||||
}
|
||||
|
||||
|
||||
template <bool is_json>
|
||||
static DataTypePtr tryInferDataTypeForSingleFieldImpl(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info);
|
||||
|
||||
static bool tryInferDate(const std::string_view & field)
|
||||
{
|
||||
ReadBufferFromString buf(field);
|
||||
DayNum tmp;
|
||||
return tryReadDateText(tmp, buf) && buf.eof();
|
||||
}
|
||||
|
||||
static bool tryInferDateTime(const std::string_view & field, const FormatSettings & settings)
|
||||
{
|
||||
if (field.empty())
|
||||
return false;
|
||||
|
||||
ReadBufferFromString buf(field);
|
||||
Float64 tmp_float;
|
||||
/// Check if it's just a number, and if so, don't try to infer DateTime from it,
|
||||
/// because we can interpret this number as a timestamp and it will lead to
|
||||
/// inferring DateTime instead of simple Int64/Float64 in some cases.
|
||||
if (tryReadFloatText(tmp_float, buf) && buf.eof())
|
||||
return false;
|
||||
|
||||
buf.seek(0, SEEK_SET); /// Return position to the beginning
|
||||
DateTime64 tmp;
|
||||
switch (settings.date_time_input_format)
|
||||
{
|
||||
case FormatSettings::DateTimeInputFormat::Basic:
|
||||
if (tryReadDateTime64Text(tmp, 9, buf) && buf.eof())
|
||||
return true;
|
||||
break;
|
||||
case FormatSettings::DateTimeInputFormat::BestEffort:
|
||||
if (tryParseDateTime64BestEffort(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC")) && buf.eof())
|
||||
return true;
|
||||
break;
|
||||
case FormatSettings::DateTimeInputFormat::BestEffortUS:
|
||||
if (tryParseDateTime64BestEffortUS(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC")) && buf.eof())
|
||||
return true;
|
||||
break;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
DataTypePtr tryInferDateOrDateTimeFromString(const std::string_view & field, const FormatSettings & settings)
|
||||
{
|
||||
if (settings.try_infer_dates && tryInferDate(field))
|
||||
return std::make_shared<DataTypeDate>();
|
||||
|
||||
if (settings.try_infer_datetimes && tryInferDateTime(field, settings))
|
||||
return std::make_shared<DataTypeDateTime64>(9);
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <bool is_json>
|
||||
static DataTypePtr tryInferArray(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
assertChar('[', buf);
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
DataTypes nested_types;
|
||||
bool first = true;
|
||||
while (!buf.eof() && *buf.position() != ']')
|
||||
{
|
||||
if (!first)
|
||||
{
|
||||
/// Skip field delimiter between array elements.
|
||||
skipWhitespaceIfAny(buf);
|
||||
if (!checkChar(',', buf))
|
||||
return nullptr;
|
||||
skipWhitespaceIfAny(buf);
|
||||
}
|
||||
else
|
||||
first = false;
|
||||
|
||||
auto nested_type = tryInferDataTypeForSingleFieldImpl<is_json>(buf, settings, json_info);
|
||||
/// If we couldn't infer element type, array type also cannot be inferred.
|
||||
if (!nested_type)
|
||||
return nullptr;
|
||||
|
||||
nested_types.push_back(nested_type);
|
||||
|
||||
skipWhitespaceIfAny(buf);
|
||||
}
|
||||
|
||||
/// No ']' at the end of array
|
||||
if (buf.eof())
|
||||
return nullptr;
|
||||
|
||||
assertChar(']', buf);
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
/// Empty array has type Array(Nothing)
|
||||
if (nested_types.empty())
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeNothing>());
|
||||
|
||||
if (checkIfTypesAreEqual(nested_types))
|
||||
return std::make_shared<DataTypeArray>(nested_types.back());
|
||||
|
||||
/// If element types are not equal, we should try to find common type.
|
||||
/// If after transformation element types are still different, we return Tuple for JSON and
|
||||
/// nullptr for other formats (nullptr means we couldn't infer the type).
|
||||
if constexpr (is_json)
|
||||
{
|
||||
/// For JSON if we have not complete types, we should not try to transform them
|
||||
/// and return it as a Tuple.
|
||||
/// For example, if we have types [Float64, Nullable(Nothing), Float64]
|
||||
/// it can be Array(Float64) or Tuple(Float64, <some_type>, Float64) and
|
||||
/// we can't determine which one it is. But we will be able to do it later
|
||||
/// when we will have types from other rows for this column.
|
||||
/// For example, if in the next row we will have types [Nullable(Nothing), String, Float64],
|
||||
/// we can determine the type for this colum as Tuple(Nullable(Float64), Nullable(String), Float64).
|
||||
for (const auto & type : nested_types)
|
||||
{
|
||||
if (!checkIfTypeIsComplete(type))
|
||||
return std::make_shared<DataTypeTuple>(nested_types);
|
||||
}
|
||||
|
||||
auto nested_types_copy = nested_types;
|
||||
transformInferredTypesIfNeededImpl<is_json>(nested_types_copy, settings, json_info);
|
||||
|
||||
if (checkIfTypesAreEqual(nested_types_copy))
|
||||
return std::make_shared<DataTypeArray>(nested_types_copy.back());
|
||||
|
||||
return std::make_shared<DataTypeTuple>(nested_types);
|
||||
}
|
||||
else
|
||||
{
|
||||
transformInferredTypesIfNeededImpl<is_json>(nested_types, settings);
|
||||
if (checkIfTypesAreEqual(nested_types))
|
||||
return std::make_shared<DataTypeArray>(nested_types.back());
|
||||
|
||||
/// We couldn't determine common type for array element.
|
||||
return nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
static DataTypePtr tryInferTuple(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
assertChar('(', buf);
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
DataTypes nested_types;
|
||||
bool first = true;
|
||||
while (!buf.eof() && *buf.position() != ')')
|
||||
{
|
||||
if (!first)
|
||||
{
|
||||
skipWhitespaceIfAny(buf);
|
||||
if (!checkChar(',', buf))
|
||||
return nullptr;
|
||||
skipWhitespaceIfAny(buf);
|
||||
}
|
||||
else
|
||||
first = false;
|
||||
|
||||
auto nested_type = tryInferDataTypeForSingleFieldImpl<false>(buf, settings, json_info);
|
||||
/// If we couldn't infer element type, tuple type also cannot be inferred.
|
||||
if (!nested_type)
|
||||
return nullptr;
|
||||
|
||||
nested_types.push_back(nested_type);
|
||||
skipWhitespaceIfAny(buf);
|
||||
}
|
||||
|
||||
if (buf.eof() || nested_types.empty())
|
||||
return nullptr;
|
||||
|
||||
assertChar(')', buf);
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
return std::make_shared<DataTypeTuple>(nested_types);
|
||||
}
|
||||
|
||||
template <bool check_eof>
|
||||
static DataTypePtr tryInferNumberFromStringBuffer(ReadBufferFromString & buf, const FormatSettings & settings)
|
||||
{
|
||||
if (settings.try_infer_integers)
|
||||
{
|
||||
Int64 tmp_int;
|
||||
if (tryReadIntText(tmp_int, buf) && (!check_eof || buf.eof()))
|
||||
return std::make_shared<DataTypeInt64>();
|
||||
}
|
||||
|
||||
/// We cam safely get back to the start of buffer, because we read from a string and we didn't reach eof.
|
||||
buf.position() = buf.buffer().begin();
|
||||
|
||||
Float64 tmp;
|
||||
if (tryReadFloatText(tmp, buf) && (!check_eof || buf.eof()))
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
static DataTypePtr tryInferNumber(ReadBuffer & buf, const FormatSettings & settings)
|
||||
{
|
||||
/// If we read from String, we can do it in a more efficient way.
|
||||
if (auto * string_buf = dynamic_cast<ReadBufferFromString *>(&buf))
|
||||
return tryInferNumberFromStringBuffer<false>(*string_buf, settings);
|
||||
|
||||
Float64 tmp_float;
|
||||
if (settings.try_infer_integers)
|
||||
{
|
||||
/// We should use PeekableReadBuffer, because we need to
|
||||
/// rollback to the start of number to parse it as integer first
|
||||
/// and then as float.
|
||||
PeekableReadBuffer peekable_buf(buf);
|
||||
PeekableReadBufferCheckpoint checkpoint(peekable_buf);
|
||||
Int64 tmp_int;
|
||||
bool read_int = tryReadIntText(tmp_int, peekable_buf);
|
||||
auto * int_end = peekable_buf.position();
|
||||
peekable_buf.rollbackToCheckpoint(true);
|
||||
if (tryReadFloatText(tmp_float, peekable_buf))
|
||||
{
|
||||
/// Float parsing reads no fewer bytes than integer parsing,
|
||||
/// so position of the buffer is either the same, or further.
|
||||
/// If it's the same, then it's integer.
|
||||
if (read_int && peekable_buf.position() == int_end)
|
||||
return std::make_shared<DataTypeInt64>();
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
}
|
||||
else if (tryReadFloatText(tmp_float, buf))
|
||||
{
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
|
||||
/// This is not a number.
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
DataTypePtr tryInferNumberFromString(const std::string_view & field, const FormatSettings & settings)
|
||||
{
|
||||
ReadBufferFromString buf(field);
|
||||
return tryInferNumberFromStringBuffer<true>(buf, settings);
|
||||
}
|
||||
|
||||
template <bool is_json>
|
||||
static DataTypePtr tryInferString(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
String field;
|
||||
bool ok = true;
|
||||
if constexpr (is_json)
|
||||
ok = tryReadJSONStringInto(field, buf);
|
||||
else
|
||||
ok = tryReadQuotedStringInto(field, buf);
|
||||
|
||||
if (!ok)
|
||||
return nullptr;
|
||||
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
/// If it's object key, we should just return String type.
|
||||
if constexpr (is_json)
|
||||
{
|
||||
if (json_info->is_object_key)
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
if (auto type = tryInferDateOrDateTimeFromString(field, settings))
|
||||
return type;
|
||||
|
||||
if constexpr (is_json)
|
||||
{
|
||||
if (settings.json.try_infer_numbers_from_strings)
|
||||
{
|
||||
auto number_type = tryInferNumberFromString(field, settings);
|
||||
if (number_type)
|
||||
{
|
||||
json_info->numbers_parsed_from_json_strings.insert(number_type.get());
|
||||
return number_type;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
template <bool is_json>
|
||||
static DataTypePtr tryInferMapOrObject(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
assertChar('{', buf);
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
DataTypes key_types;
|
||||
DataTypes value_types;
|
||||
bool first = true;
|
||||
while (!buf.eof() && *buf.position() != '}')
|
||||
{
|
||||
if (!first)
|
||||
{
|
||||
skipWhitespaceIfAny(buf);
|
||||
if (!checkChar(',', buf))
|
||||
return nullptr;
|
||||
skipWhitespaceIfAny(buf);
|
||||
}
|
||||
else
|
||||
first = false;
|
||||
|
||||
DataTypePtr key_type;
|
||||
if constexpr (is_json)
|
||||
{
|
||||
/// For JSON key type must be String.
|
||||
json_info->is_object_key = true;
|
||||
key_type = tryInferString<is_json>(buf, settings, json_info);
|
||||
json_info->is_object_key = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
key_type = tryInferDataTypeForSingleFieldImpl<is_json>(buf, settings, nullptr);
|
||||
}
|
||||
|
||||
/// If we couldn't infer key type, we cannot infer Map/JSON object type.
|
||||
if (!key_type)
|
||||
return nullptr;
|
||||
|
||||
key_types.push_back(key_type);
|
||||
|
||||
skipWhitespaceIfAny(buf);
|
||||
if (!checkChar(':', buf))
|
||||
return nullptr;
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
/// If we couldn't infer element type, Map type also cannot be inferred.
|
||||
auto value_type = tryInferDataTypeForSingleFieldImpl<is_json>(buf, settings, json_info);
|
||||
if (!value_type)
|
||||
return nullptr;
|
||||
|
||||
value_types.push_back(value_type);
|
||||
skipWhitespaceIfAny(buf);
|
||||
}
|
||||
|
||||
if (buf.eof())
|
||||
return nullptr;
|
||||
|
||||
assertChar('}', buf);
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
if (key_types.empty())
|
||||
{
|
||||
if constexpr (is_json)
|
||||
{
|
||||
if (settings.json.try_infer_objects)
|
||||
return std::make_shared<DataTypeObject>("json", true);
|
||||
}
|
||||
/// Empty Map is Map(Nothing, Nothing)
|
||||
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeNothing>(), std::make_shared<DataTypeNothing>());
|
||||
}
|
||||
|
||||
if constexpr (is_json)
|
||||
{
|
||||
/// If it's JSON field and one of value types is JSON Object, return also JSON Object.
|
||||
for (const auto & value_type : value_types)
|
||||
{
|
||||
if (isObject(value_type))
|
||||
return std::make_shared<DataTypeObject>("json", true);
|
||||
}
|
||||
|
||||
transformInferredTypesIfNeededImpl<is_json>(value_types, settings, json_info);
|
||||
if (!checkIfTypesAreEqual(value_types))
|
||||
{
|
||||
if (settings.json.try_infer_objects)
|
||||
return std::make_shared<DataTypeObject>("json", true);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeMap>(key_types.back(), value_types.back());
|
||||
}
|
||||
|
||||
if (!checkIfTypesAreEqual(key_types))
|
||||
transformInferredTypesIfNeededImpl<is_json>(key_types, settings);
|
||||
if (!checkIfTypesAreEqual(value_types))
|
||||
transformInferredTypesIfNeededImpl<is_json>(value_types, settings);
|
||||
|
||||
if (!checkIfTypesAreEqual(key_types) || !checkIfTypesAreEqual(value_types))
|
||||
return nullptr;
|
||||
|
||||
auto key_type = removeNullable(key_types.back());
|
||||
if (!DataTypeMap::checkKeyType(key_type))
|
||||
return nullptr;
|
||||
|
||||
return std::make_shared<DataTypeMap>(key_type, value_types.back());
|
||||
}
|
||||
|
||||
template <bool is_json>
|
||||
static DataTypePtr tryInferDataTypeForSingleFieldImpl(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
skipWhitespaceIfAny(buf);
|
||||
|
||||
if (buf.eof())
|
||||
return nullptr;
|
||||
|
||||
/// Array [field1, field2, ...]
|
||||
if (*buf.position() == '[')
|
||||
return tryInferArray<is_json>(buf, settings, json_info);
|
||||
|
||||
/// Tuple (field1, field2, ...), if format is not JSON
|
||||
if constexpr (!is_json)
|
||||
{
|
||||
if (*buf.position() == '(')
|
||||
return tryInferTuple(buf, settings, json_info);
|
||||
}
|
||||
|
||||
/// Map/Object for JSON { key1 : value1, key2 : value2, ...}
|
||||
if (*buf.position() == '{')
|
||||
return tryInferMapOrObject<is_json>(buf, settings, json_info);
|
||||
|
||||
/// String
|
||||
char quote = is_json ? '"' : '\'';
|
||||
if (*buf.position() == quote)
|
||||
return tryInferString<is_json>(buf, settings, json_info);
|
||||
|
||||
/// Bool
|
||||
if (checkStringCaseInsensitive("true", buf) || checkStringCaseInsensitive("false", buf))
|
||||
return DataTypeFactory::instance().get("Bool");
|
||||
|
||||
/// Null or NaN
|
||||
if (checkCharCaseInsensitive('n', buf))
|
||||
{
|
||||
if (checkStringCaseInsensitive("ull", buf))
|
||||
return makeNullable(std::make_shared<DataTypeNothing>());
|
||||
else if (checkStringCaseInsensitive("an", buf))
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
|
||||
/// Number
|
||||
return tryInferNumber(buf, settings);
|
||||
}
|
||||
|
||||
DataTypePtr tryInferDataTypeForSingleField(ReadBuffer & buf, const FormatSettings & settings)
|
||||
{
|
||||
return tryInferDataTypeForSingleFieldImpl<false>(buf, settings, nullptr);
|
||||
}
|
||||
|
||||
DataTypePtr tryInferDataTypeForSingleField(std::string_view field, const FormatSettings & settings)
|
||||
{
|
||||
ReadBufferFromString buf(field);
|
||||
auto type = tryInferDataTypeForSingleFieldImpl<false>(buf, settings, nullptr);
|
||||
/// Check if there is no unread data in buffer.
|
||||
if (!buf.eof())
|
||||
return nullptr;
|
||||
return type;
|
||||
}
|
||||
|
||||
DataTypePtr tryInferDataTypeForSingleJSONField(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
return tryInferDataTypeForSingleFieldImpl<true>(buf, settings, json_info);
|
||||
}
|
||||
|
||||
DataTypePtr tryInferDataTypeForSingleJSONField(std::string_view field, const FormatSettings & settings, JSONInferenceInfo * json_info)
|
||||
{
|
||||
ReadBufferFromString buf(field);
|
||||
auto type = tryInferDataTypeForSingleFieldImpl<true>(buf, settings, json_info);
|
||||
/// Check if there is no unread data in buffer.
|
||||
if (!buf.eof())
|
||||
return nullptr;
|
||||
return type;
|
||||
}
|
||||
|
||||
DataTypePtr makeNullableRecursively(DataTypePtr type)
|
||||
{
|
||||
if (!type)
|
||||
return nullptr;
|
||||
|
||||
WhichDataType which(type);
|
||||
|
||||
if (which.isNullable())
|
||||
return type;
|
||||
|
||||
if (which.isArray())
|
||||
{
|
||||
const auto * array_type = assert_cast<const DataTypeArray *>(type.get());
|
||||
auto nested_type = makeNullableRecursively(array_type->getNestedType());
|
||||
return nested_type ? std::make_shared<DataTypeArray>(nested_type) : nullptr;
|
||||
}
|
||||
|
||||
if (which.isTuple())
|
||||
{
|
||||
const auto * tuple_type = assert_cast<const DataTypeTuple *>(type.get());
|
||||
DataTypes nested_types;
|
||||
for (const auto & element : tuple_type->getElements())
|
||||
{
|
||||
auto nested_type = makeNullableRecursively(element);
|
||||
if (!nested_type)
|
||||
return nullptr;
|
||||
nested_types.push_back(nested_type);
|
||||
}
|
||||
return std::make_shared<DataTypeTuple>(std::move(nested_types));
|
||||
}
|
||||
|
||||
if (which.isMap())
|
||||
{
|
||||
const auto * map_type = assert_cast<const DataTypeMap *>(type.get());
|
||||
auto key_type = makeNullableRecursively(map_type->getKeyType());
|
||||
auto value_type = makeNullableRecursively(map_type->getValueType());
|
||||
return key_type && value_type ? std::make_shared<DataTypeMap>(removeNullable(key_type), value_type) : nullptr;
|
||||
}
|
||||
|
||||
if (which.isLowCarnality())
|
||||
{
|
||||
const auto * lc_type = assert_cast<const DataTypeLowCardinality *>(type.get());
|
||||
auto nested_type = makeNullableRecursively(lc_type->getDictionaryType());
|
||||
return nested_type ? std::make_shared<DataTypeLowCardinality>(nested_type) : nullptr;
|
||||
}
|
||||
|
||||
return makeNullable(type);
|
||||
}
|
||||
|
||||
NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header)
|
||||
{
|
||||
NamesAndTypesList result;
|
||||
for (auto & [name, type] : header.getNamesAndTypesList())
|
||||
result.emplace_back(name, makeNullableRecursively(type));
|
||||
return result;
|
||||
}
|
||||
|
||||
bool checkIfTypeIsComplete(const DataTypePtr & type)
|
||||
{
|
||||
if (!type)
|
||||
return false;
|
||||
|
||||
WhichDataType which(type);
|
||||
|
||||
if (which.isNothing())
|
||||
return false;
|
||||
|
||||
if (which.isNullable())
|
||||
return checkIfTypeIsComplete(assert_cast<const DataTypeNullable *>(type.get())->getNestedType());
|
||||
|
||||
if (which.isArray())
|
||||
return checkIfTypeIsComplete(assert_cast<const DataTypeArray *>(type.get())->getNestedType());
|
||||
|
||||
if (which.isTuple())
|
||||
{
|
||||
const auto * tuple_type = assert_cast<const DataTypeTuple *>(type.get());
|
||||
for (const auto & element : tuple_type->getElements())
|
||||
{
|
||||
if (!checkIfTypeIsComplete(element))
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
if (which.isMap())
|
||||
{
|
||||
const auto * map_type = assert_cast<const DataTypeMap *>(type.get());
|
||||
if (!checkIfTypeIsComplete(map_type->getKeyType()))
|
||||
return false;
|
||||
return checkIfTypeIsComplete(map_type->getValueType());
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
93
src/Formats/SchemaInferenceUtils.h
Normal file
93
src/Formats/SchemaInferenceUtils.h
Normal file
@ -0,0 +1,93 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Struct with some additional information about inferred types for JSON formats.
|
||||
struct JSONInferenceInfo
|
||||
{
|
||||
/// We store numbers that were parsed from strings.
|
||||
/// It's used in types transformation to change such numbers back to string if needed.
|
||||
std::unordered_set<const IDataType *> numbers_parsed_from_json_strings;
|
||||
/// Indicates if currently we are inferring type for Map/Object key.
|
||||
bool is_object_key = false;
|
||||
};
|
||||
|
||||
/// Try to determine datatype of the value in buffer/string. If cannot, return nullptr.
|
||||
/// In general, it tries to parse a type using the following logic:
|
||||
/// If we see '[', we try to parse an array of values and recursively determine datatype for each element.
|
||||
/// If we see '(', we try to parse a tuple of values and recursively determine datatype for each element.
|
||||
/// If we see '{', we try to parse a Map of keys and values and recursively determine datatype for each key/value.
|
||||
/// If we see a quote '\'', we treat it as a string and read until next quote.
|
||||
/// If we see NULL it returns Nullable(Nothing)
|
||||
/// Otherwise we try to read a number.
|
||||
DataTypePtr tryInferDataTypeForSingleField(ReadBuffer & buf, const FormatSettings & settings);
|
||||
DataTypePtr tryInferDataTypeForSingleField(std::string_view field, const FormatSettings & settings);
|
||||
|
||||
/// The same as tryInferDataTypeForSingleField, but for JSON values.
|
||||
DataTypePtr tryInferDataTypeForSingleJSONField(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info);
|
||||
DataTypePtr tryInferDataTypeForSingleJSONField(std::string_view field, const FormatSettings & settings, JSONInferenceInfo * json_info);
|
||||
|
||||
/// Try to parse Date or DateTime value from a string.
|
||||
DataTypePtr tryInferDateOrDateTimeFromString(const std::string_view & field, const FormatSettings & settings);
|
||||
|
||||
/// Try to parse a number value from a string. By default, it tries to parse Float64,
|
||||
/// but if setting try_infer_integers is enables, it also tries to parse Int64.
|
||||
DataTypePtr tryInferNumberFromString(const std::string_view & field, const FormatSettings & settings);
|
||||
|
||||
/// It takes two types inferred for the same column and tries to transform them to a common type if possible.
|
||||
/// It's also used when we try to infer some not ordinary types from another types.
|
||||
/// Example 1:
|
||||
/// Dates inferred from strings. In this case we should check if dates were inferred from all strings
|
||||
/// in the same way and if not, transform inferred dates back to strings.
|
||||
/// For example, when we have Array(Date) (like `['2020-01-01', '2020-02-02']`) and Array(String) (like `['string', 'abc']`
|
||||
/// we will convert the first type to Array(String).
|
||||
/// Example 2:
|
||||
/// When we have integers and floats for the same value, we should convert all integers to floats.
|
||||
/// For example, when we have Array(Int64) (like `[123, 456]`) and Array(Float64) (like `[42.42, 4.42]`)
|
||||
/// we will convert the first type to Array(Float64)
|
||||
/// Example 3:
|
||||
/// When we have not complete types like Nullable(Nothing), Array(Nullable(Nothing)) or Tuple(UInt64, Nullable(Nothing)),
|
||||
/// we try to complete them using the other type.
|
||||
/// For example, if we have Tuple(UInt64, Nullable(Nothing)) and Tuple(Nullable(Nothing), String) we will convert both
|
||||
/// types to common type Tuple(Nullable(UInt64), Nullable(String))
|
||||
void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings);
|
||||
|
||||
/// The same as transformInferredTypesIfNeeded but uses some specific transformations for JSON.
|
||||
/// Example 1:
|
||||
/// When we have numbers inferred from strings and strings, we convert all such numbers back to string.
|
||||
/// For example, if we have Array(Int64) (like `['123', '456']`) and Array(String) (like `['str', 'abc']`)
|
||||
/// we will convert the first type to Array(String). Note that we collect information about numbers inferred
|
||||
/// from strings in json_info while inference and use it here, so we will know that Array(Int64) contains
|
||||
/// integer inferred from a string.
|
||||
/// Example 2:
|
||||
/// When we have maps with different value types, we convert all types to JSON object type.
|
||||
/// For example, if we have Map(String, UInt64) (like `{"a" : 123}`) and Map(String, String) (like `{"b" : 'abc'}`)
|
||||
/// we will convert both types to Object('JSON').
|
||||
void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, JSONInferenceInfo * json_info);
|
||||
|
||||
/// Check it type is Tuple(...), try to transform nested types to find a common type for them and if all nested types
|
||||
/// are the same after transform, we convert this tuple to an Array with common nested type.
|
||||
/// For example, if we have Tuple(String, Nullable(Nothing)) we will convert it to Array(String).
|
||||
/// It's used when all rows were read and we have Tuple in the result type that can be actually an Array.
|
||||
void transformJSONTupleToArrayIfPossible(DataTypePtr & data_type, const FormatSettings & settings, JSONInferenceInfo * json_info);
|
||||
|
||||
/// Make type Nullable recursively:
|
||||
/// - Type -> Nullable(type)
|
||||
/// - Array(Type) -> Array(Nullable(Type))
|
||||
/// - Tuple(Type1, ..., TypeN) -> Tuple(Nullable(Type1), ..., Nullable(TypeN))
|
||||
/// - Map(KeyType, ValueType) -> Map(KeyType, Nullable(ValueType))
|
||||
/// - LowCardinality(Type) -> LowCardinality(Nullable(Type))
|
||||
DataTypePtr makeNullableRecursively(DataTypePtr type);
|
||||
|
||||
/// Call makeNullableRecursively for all types
|
||||
/// in the block and return names and types.
|
||||
NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header);
|
||||
|
||||
/// Check if type contains Nothing, like Array(Tuple(Nullable(Nothing), String))
|
||||
bool checkIfTypeIsComplete(const DataTypePtr & type);
|
||||
|
||||
}
|
@ -317,12 +317,17 @@ template void readStringUntilEOFInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8
|
||||
/** Parse the escape sequence, which can be simple (one character after backslash) or more complex (multiple characters).
|
||||
* It is assumed that the cursor is located on the `\` symbol
|
||||
*/
|
||||
template <typename Vector>
|
||||
static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf)
|
||||
template <typename Vector, typename ReturnType = void>
|
||||
static ReturnType parseComplexEscapeSequence(Vector & s, ReadBuffer & buf)
|
||||
{
|
||||
++buf.position();
|
||||
if (buf.eof())
|
||||
throw Exception("Cannot parse escape sequence", ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE);
|
||||
{
|
||||
if constexpr (std::is_same_v<ReturnType, void>)
|
||||
throw Exception("Cannot parse escape sequence", ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
char char_after_backslash = *buf.position();
|
||||
|
||||
@ -361,6 +366,8 @@ static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf)
|
||||
s.push_back(decoded_char);
|
||||
++buf.position();
|
||||
}
|
||||
|
||||
return ReturnType(true);
|
||||
}
|
||||
|
||||
|
||||
@ -519,14 +526,18 @@ template void readEscapedStringInto<NullOutput>(NullOutput & s, ReadBuffer & buf
|
||||
* backslash escape sequences are also parsed,
|
||||
* that could be slightly confusing.
|
||||
*/
|
||||
template <char quote, bool enable_sql_style_quoting, typename Vector>
|
||||
static void readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
|
||||
template <char quote, bool enable_sql_style_quoting, typename Vector, typename ReturnType = void>
|
||||
static ReturnType readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
|
||||
{
|
||||
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
|
||||
if (buf.eof() || *buf.position() != quote)
|
||||
{
|
||||
throw ParsingException(ErrorCodes::CANNOT_PARSE_QUOTED_STRING,
|
||||
"Cannot parse quoted string: expected opening quote '{}', got '{}'",
|
||||
std::string{quote}, buf.eof() ? "EOF" : std::string{*buf.position()});
|
||||
if constexpr (throw_exception)
|
||||
throw ParsingException(ErrorCodes::CANNOT_PARSE_QUOTED_STRING,
|
||||
"Cannot parse quoted string: expected opening quote '{}', got '{}'",
|
||||
std::string{quote}, buf.eof() ? "EOF" : std::string{*buf.position()});
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
++buf.position();
|
||||
@ -552,15 +563,26 @@ static void readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
|
||||
continue;
|
||||
}
|
||||
|
||||
return;
|
||||
return ReturnType(true);
|
||||
}
|
||||
|
||||
if (*buf.position() == '\\')
|
||||
parseComplexEscapeSequence(s, buf);
|
||||
{
|
||||
if constexpr (throw_exception)
|
||||
parseComplexEscapeSequence<Vector, ReturnType>(s, buf);
|
||||
else
|
||||
{
|
||||
if (!parseComplexEscapeSequence<Vector, ReturnType>(s, buf))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
throw ParsingException("Cannot parse quoted string: expected closing quote",
|
||||
ErrorCodes::CANNOT_PARSE_QUOTED_STRING);
|
||||
if constexpr (throw_exception)
|
||||
throw ParsingException("Cannot parse quoted string: expected closing quote",
|
||||
ErrorCodes::CANNOT_PARSE_QUOTED_STRING);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
template <bool enable_sql_style_quoting, typename Vector>
|
||||
@ -569,6 +591,14 @@ void readQuotedStringInto(Vector & s, ReadBuffer & buf)
|
||||
readAnyQuotedStringInto<'\'', enable_sql_style_quoting>(s, buf);
|
||||
}
|
||||
|
||||
template <typename Vector>
|
||||
bool tryReadQuotedStringInto(Vector & s, ReadBuffer & buf)
|
||||
{
|
||||
return readAnyQuotedStringInto<'\'', false, Vector, bool>(s, buf);
|
||||
}
|
||||
|
||||
template bool tryReadQuotedStringInto(String & s, ReadBuffer & buf);
|
||||
|
||||
template <bool enable_sql_style_quoting, typename Vector>
|
||||
void readDoubleQuotedStringInto(Vector & s, ReadBuffer & buf)
|
||||
{
|
||||
@ -829,6 +859,7 @@ template void readJSONStringInto<PaddedPODArray<UInt8>, void>(PaddedPODArray<UIn
|
||||
template bool readJSONStringInto<PaddedPODArray<UInt8>, bool>(PaddedPODArray<UInt8> & s, ReadBuffer & buf);
|
||||
template void readJSONStringInto<NullOutput>(NullOutput & s, ReadBuffer & buf);
|
||||
template void readJSONStringInto<String>(String & s, ReadBuffer & buf);
|
||||
template bool readJSONStringInto<String, bool>(String & s, ReadBuffer & buf);
|
||||
|
||||
template <typename Vector, typename ReturnType>
|
||||
ReturnType readJSONObjectPossiblyInvalid(Vector & s, ReadBuffer & buf)
|
||||
@ -1396,6 +1427,39 @@ static void readParsedValueInto(Vector & s, ReadBuffer & buf, ParseFunc parse_fu
|
||||
peekable_buf.position() = end;
|
||||
}
|
||||
|
||||
template <typename Vector>
|
||||
static void readQuotedStringFieldInto(Vector & s, ReadBuffer & buf)
|
||||
{
|
||||
assertChar('\'', buf);
|
||||
s.push_back('\'');
|
||||
while (!buf.eof())
|
||||
{
|
||||
char * next_pos = find_first_symbols<'\\', '\''>(buf.position(), buf.buffer().end());
|
||||
|
||||
s.append(buf.position(), next_pos);
|
||||
buf.position() = next_pos;
|
||||
|
||||
if (!buf.hasPendingData())
|
||||
continue;
|
||||
|
||||
if (*buf.position() == '\'')
|
||||
break;
|
||||
|
||||
s.push_back(*buf.position());
|
||||
if (*buf.position() == '\\')
|
||||
{
|
||||
++buf.position();
|
||||
if (!buf.eof())
|
||||
{
|
||||
s.push_back(*buf.position());
|
||||
++buf.position();
|
||||
}
|
||||
}
|
||||
}
|
||||
++buf.position();
|
||||
s.push_back('\'');
|
||||
}
|
||||
|
||||
template <char opening_bracket, char closing_bracket, typename Vector>
|
||||
static void readQuotedFieldInBracketsInto(Vector & s, ReadBuffer & buf)
|
||||
{
|
||||
@ -1413,20 +1477,19 @@ static void readQuotedFieldInBracketsInto(Vector & s, ReadBuffer & buf)
|
||||
if (!buf.hasPendingData())
|
||||
continue;
|
||||
|
||||
s.push_back(*buf.position());
|
||||
|
||||
if (*buf.position() == '\'')
|
||||
{
|
||||
readQuotedStringInto<false>(s, buf);
|
||||
s.push_back('\'');
|
||||
readQuotedStringFieldInto(s, buf);
|
||||
}
|
||||
else if (*buf.position() == opening_bracket)
|
||||
{
|
||||
s.push_back(opening_bracket);
|
||||
++balance;
|
||||
++buf.position();
|
||||
}
|
||||
else if (*buf.position() == closing_bracket)
|
||||
{
|
||||
s.push_back(closing_bracket);
|
||||
--balance;
|
||||
++buf.position();
|
||||
}
|
||||
@ -1449,11 +1512,7 @@ void readQuotedFieldInto(Vector & s, ReadBuffer & buf)
|
||||
/// - Number: integer, float, decimal.
|
||||
|
||||
if (*buf.position() == '\'')
|
||||
{
|
||||
s.push_back('\'');
|
||||
readQuotedStringInto<false>(s, buf);
|
||||
s.push_back('\'');
|
||||
}
|
||||
readQuotedStringFieldInto(s, buf);
|
||||
else if (*buf.position() == '[')
|
||||
readQuotedFieldInBracketsInto<'[', ']'>(s, buf);
|
||||
else if (*buf.position() == '(')
|
||||
|
@ -605,6 +605,9 @@ bool tryReadJSONStringInto(Vector & s, ReadBuffer & buf)
|
||||
return readJSONStringInto<Vector, bool>(s, buf);
|
||||
}
|
||||
|
||||
template <typename Vector>
|
||||
bool tryReadQuotedStringInto(Vector & s, ReadBuffer & buf);
|
||||
|
||||
/// Reads chunk of data between {} in that way,
|
||||
/// that it has balanced parentheses sequence of {}.
|
||||
/// So, it may form a JSON object, but it can be incorrenct.
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
@ -17,10 +16,11 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
template <class SchemaReader>
|
||||
void chooseResultColumnType(
|
||||
SchemaReader & schema_reader,
|
||||
DataTypePtr & type,
|
||||
DataTypePtr & new_type,
|
||||
std::function<void(DataTypePtr &, DataTypePtr &)> transform_types_if_needed,
|
||||
const DataTypePtr & default_type,
|
||||
const String & column_name,
|
||||
size_t row)
|
||||
@ -34,7 +34,7 @@ void chooseResultColumnType(
|
||||
if (!new_type || type->equals(*new_type))
|
||||
return;
|
||||
|
||||
transform_types_if_needed(type, new_type);
|
||||
schema_reader.transformTypesIfNeeded(type, new_type);
|
||||
if (type->equals(*new_type))
|
||||
return;
|
||||
|
||||
@ -55,9 +55,9 @@ void chooseResultColumnType(
|
||||
}
|
||||
}
|
||||
|
||||
void checkResultColumnTypeAndAppend(NamesAndTypesList & result, DataTypePtr & type, const String & name, const DataTypePtr & default_type, size_t rows_read)
|
||||
void checkResultColumnTypeAndAppend(NamesAndTypesList & result, DataTypePtr & type, const String & name, const FormatSettings & settings, const DataTypePtr & default_type, size_t rows_read)
|
||||
{
|
||||
if (!type)
|
||||
if (!checkIfTypeIsComplete(type))
|
||||
{
|
||||
if (!default_type)
|
||||
throw Exception(
|
||||
@ -69,6 +69,10 @@ void checkResultColumnTypeAndAppend(NamesAndTypesList & result, DataTypePtr & ty
|
||||
|
||||
type = default_type;
|
||||
}
|
||||
|
||||
if (settings.schema_inference_make_columns_nullable)
|
||||
type = makeNullableRecursively(type);
|
||||
|
||||
result.emplace_back(name, type);
|
||||
}
|
||||
|
||||
@ -88,6 +92,11 @@ void IIRowSchemaReader::setContext(ContextPtr & context)
|
||||
}
|
||||
}
|
||||
|
||||
void IIRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
|
||||
{
|
||||
transformInferredTypesIfNeeded(type, new_type, format_settings);
|
||||
}
|
||||
|
||||
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
|
||||
: IIRowSchemaReader(in_, format_settings_), column_names(splitColumnNames(format_settings.column_names_for_schema_inference))
|
||||
{
|
||||
@ -160,23 +169,23 @@ NamesAndTypesList IRowSchemaReader::readSchema()
|
||||
if (new_data_types.size() != data_types.size())
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Rows have different amount of values");
|
||||
|
||||
for (size_t i = 0; i != data_types.size(); ++i)
|
||||
for (field_index = 0; field_index != data_types.size(); ++field_index)
|
||||
{
|
||||
/// Check if we couldn't determine the type of this column in a new row
|
||||
/// or the type for this column was taken from hints.
|
||||
if (!new_data_types[i] || hints.contains(column_names[i]))
|
||||
if (!new_data_types[field_index] || hints.contains(column_names[field_index]))
|
||||
continue;
|
||||
|
||||
auto transform_types_if_needed = [&](DataTypePtr & type, DataTypePtr & new_type){ transformTypesIfNeeded(type, new_type, i); };
|
||||
chooseResultColumnType(data_types[i], new_data_types[i], transform_types_if_needed, getDefaultType(i), std::to_string(i + 1), rows_read);
|
||||
chooseResultColumnType(*this, data_types[field_index], new_data_types[field_index], getDefaultType(field_index), std::to_string(field_index + 1), rows_read);
|
||||
}
|
||||
}
|
||||
|
||||
NamesAndTypesList result;
|
||||
for (size_t i = 0; i != data_types.size(); ++i)
|
||||
for (field_index = 0; field_index != data_types.size(); ++field_index)
|
||||
{
|
||||
transformFinalTypeIfNeeded(data_types[field_index]);
|
||||
/// Check that we could determine the type of this column.
|
||||
checkResultColumnTypeAndAppend(result, data_types[i], column_names[i], getDefaultType(i), rows_read);
|
||||
checkResultColumnTypeAndAppend(result, data_types[field_index], column_names[field_index], format_settings, getDefaultType(field_index), rows_read);
|
||||
}
|
||||
|
||||
return result;
|
||||
@ -208,11 +217,6 @@ DataTypePtr IRowSchemaReader::getDefaultType(size_t column) const
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
void IRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
|
||||
{
|
||||
transformInferredTypesIfNeeded(type, new_type, format_settings);
|
||||
}
|
||||
|
||||
IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, DataTypePtr default_type_)
|
||||
: IIRowSchemaReader(in_, format_settings_, default_type_)
|
||||
{
|
||||
@ -245,7 +249,6 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
|
||||
names_order.push_back(name);
|
||||
}
|
||||
|
||||
auto transform_types_if_needed = [&](DataTypePtr & type, DataTypePtr & new_type){ transformTypesIfNeeded(type, new_type); };
|
||||
for (rows_read = 1; rows_read < max_rows_to_read; ++rows_read)
|
||||
{
|
||||
auto new_names_and_types = readRowAndGetNamesAndDataTypes(eof);
|
||||
@ -277,7 +280,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
|
||||
continue;
|
||||
|
||||
auto & type = it->second;
|
||||
chooseResultColumnType(type, new_type, transform_types_if_needed, default_type, name, rows_read);
|
||||
chooseResultColumnType(*this, type, new_type, default_type, name, rows_read);
|
||||
}
|
||||
}
|
||||
|
||||
@ -289,16 +292,12 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
|
||||
for (auto & name : names_order)
|
||||
{
|
||||
auto & type = names_to_types[name];
|
||||
transformFinalTypeIfNeeded(type);
|
||||
/// Check that we could determine the type of this column.
|
||||
checkResultColumnTypeAndAppend(result, type, name, default_type, rows_read);
|
||||
checkResultColumnTypeAndAppend(result, type, name, format_settings, default_type, rows_read);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
void IRowWithNamesSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
|
||||
{
|
||||
transformInferredTypesIfNeeded(type, new_type, format_settings);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -45,10 +45,14 @@ public:
|
||||
bool needContext() const override { return !hints_str.empty(); }
|
||||
void setContext(ContextPtr & context) override;
|
||||
|
||||
virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type);
|
||||
|
||||
protected:
|
||||
void setMaxRowsToRead(size_t max_rows) override { max_rows_to_read = max_rows; }
|
||||
size_t getNumRowsRead() const override { return rows_read; }
|
||||
|
||||
virtual void transformFinalTypeIfNeeded(DataTypePtr &) {}
|
||||
|
||||
size_t max_rows_to_read;
|
||||
size_t rows_read = 0;
|
||||
DataTypePtr default_type;
|
||||
@ -82,7 +86,7 @@ protected:
|
||||
|
||||
void setColumnNames(const std::vector<String> & names) { column_names = names; }
|
||||
|
||||
virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t index);
|
||||
size_t field_index;
|
||||
|
||||
private:
|
||||
DataTypePtr getDefaultType(size_t column) const;
|
||||
@ -110,8 +114,6 @@ protected:
|
||||
/// If it's impossible to determine the type for some column, return nullptr for it.
|
||||
/// Set eof = true if can't read more data.
|
||||
virtual NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) = 0;
|
||||
|
||||
virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type);
|
||||
};
|
||||
|
||||
/// Base class for schema inference for formats that don't need any data to
|
||||
@ -125,16 +127,17 @@ public:
|
||||
virtual ~IExternalSchemaReader() = default;
|
||||
};
|
||||
|
||||
template <class SchemaReader>
|
||||
void chooseResultColumnType(
|
||||
SchemaReader & schema_reader,
|
||||
DataTypePtr & type,
|
||||
DataTypePtr & new_type,
|
||||
std::function<void(DataTypePtr &, DataTypePtr &)> transform_types_if_needed,
|
||||
const DataTypePtr & default_type,
|
||||
const String & column_name,
|
||||
size_t row);
|
||||
|
||||
void checkResultColumnTypeAndAppend(
|
||||
NamesAndTypesList & result, DataTypePtr & type, const String & name, const DataTypePtr & default_type, size_t rows_read);
|
||||
NamesAndTypesList & result, DataTypePtr & type, const String & name, const FormatSettings & settings, const DataTypePtr & default_type, size_t rows_read);
|
||||
|
||||
Strings splitColumnNames(const String & column_names_str);
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
#if USE_ARROW
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/copyData.h>
|
||||
|
@ -274,15 +274,15 @@ void CSVFormatReader::skipPrefixBeforeHeader()
|
||||
}
|
||||
|
||||
|
||||
CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_)
|
||||
CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_settings_)
|
||||
: FormatWithNamesAndTypesSchemaReader(
|
||||
in_,
|
||||
format_setting_,
|
||||
format_settings_,
|
||||
with_names_,
|
||||
with_types_,
|
||||
&reader,
|
||||
getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::CSV))
|
||||
, reader(in_, format_setting_)
|
||||
, reader(in_, format_settings_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -293,7 +293,7 @@ DataTypes CSVSchemaReader::readRowAndGetDataTypes()
|
||||
return {};
|
||||
|
||||
auto fields = reader.readRow();
|
||||
return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), FormatSettings::EscapingRule::CSV);
|
||||
return tryInferDataTypesByEscapingRule(fields, reader.getFormatSettings(), FormatSettings::EscapingRule::CSV);
|
||||
}
|
||||
|
||||
|
||||
|
@ -75,7 +75,7 @@ public:
|
||||
class CSVSchemaReader : public FormatWithNamesAndTypesSchemaReader
|
||||
{
|
||||
public:
|
||||
CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_);
|
||||
CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_settings_);
|
||||
|
||||
private:
|
||||
DataTypes readRowAndGetDataTypes() override;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/Formats/Impl/CustomSeparatedRowInputFormat.h>
|
||||
#include <Processors/Formats/Impl/TemplateRowInputFormat.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <Formats/registerWithNamesAndTypes.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
@ -328,12 +329,12 @@ DataTypes CustomSeparatedSchemaReader::readRowAndGetDataTypes()
|
||||
first_row = false;
|
||||
|
||||
auto fields = reader.readRow();
|
||||
return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule());
|
||||
return tryInferDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule(), &json_inference_info);
|
||||
}
|
||||
|
||||
void CustomSeparatedSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
|
||||
void CustomSeparatedSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
|
||||
{
|
||||
transformInferredTypesIfNeeded(type, new_type, format_settings, reader.getEscapingRule());
|
||||
transformInferredTypesByEscapingRuleIfNeeded(type, new_type, format_settings, reader.getEscapingRule(), &json_inference_info);
|
||||
}
|
||||
|
||||
void registerInputFormatCustomSeparated(FormatFactory & factory)
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
|
||||
#include <Formats/ParsedTemplateFormatString.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
@ -98,11 +99,12 @@ public:
|
||||
private:
|
||||
DataTypes readRowAndGetDataTypes() override;
|
||||
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) override;
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
|
||||
|
||||
PeekableReadBuffer buf;
|
||||
CustomSeparatedFormatReader reader;
|
||||
bool first_row = true;
|
||||
JSONInferenceInfo json_inference_info;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <base/find_symbols.h>
|
||||
|
||||
@ -175,14 +176,9 @@ JSONColumnsSchemaReaderBase::JSONColumnsSchemaReaderBase(
|
||||
{
|
||||
}
|
||||
|
||||
void JSONColumnsSchemaReaderBase::chooseResulType(DataTypePtr & type, DataTypePtr & new_type, const String & column_name, size_t row) const
|
||||
void JSONColumnsSchemaReaderBase::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
|
||||
{
|
||||
auto convert_types_if_needed = [&](DataTypePtr & first, DataTypePtr & second)
|
||||
{
|
||||
DataTypes types = {first, second};
|
||||
transformInferredJSONTypesIfNeeded(types, format_settings);
|
||||
};
|
||||
chooseResultColumnType(type, new_type, convert_types_if_needed, nullptr, column_name, row);
|
||||
transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info);
|
||||
}
|
||||
|
||||
NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema()
|
||||
@ -222,7 +218,8 @@ NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema()
|
||||
|
||||
rows_in_block = 0;
|
||||
auto column_type = readColumnAndGetDataType(column_name, rows_in_block, format_settings.max_rows_to_read_for_schema_inference - total_rows_read);
|
||||
chooseResulType(names_to_types[column_name], column_type, column_name, total_rows_read + 1);
|
||||
chooseResultColumnType(*this, names_to_types[column_name], column_type, nullptr, column_name, total_rows_read + 1);
|
||||
|
||||
++iteration;
|
||||
}
|
||||
while (!reader->checkChunkEndOrSkipColumnDelimiter());
|
||||
@ -237,8 +234,9 @@ NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema()
|
||||
for (auto & name : names_order)
|
||||
{
|
||||
auto & type = names_to_types[name];
|
||||
transformJSONTupleToArrayIfPossible(type, format_settings, &inference_info);
|
||||
/// Check that we could determine the type of this column.
|
||||
checkResultColumnTypeAndAppend(result, type, name, nullptr, format_settings.max_rows_to_read_for_schema_inference);
|
||||
checkResultColumnTypeAndAppend(result, type, name, format_settings, nullptr, format_settings.max_rows_to_read_for_schema_inference);
|
||||
}
|
||||
|
||||
return result;
|
||||
@ -262,8 +260,8 @@ DataTypePtr JSONColumnsSchemaReaderBase::readColumnAndGetDataType(const String &
|
||||
}
|
||||
|
||||
readJSONField(field, in);
|
||||
DataTypePtr field_type = JSONUtils::getDataTypeFromField(field, format_settings);
|
||||
chooseResulType(column_type, field_type, column_name, rows_read);
|
||||
DataTypePtr field_type = tryInferDataTypeForSingleJSONField(field, format_settings, &inference_info);
|
||||
chooseResultColumnType(*this, column_type, field_type, nullptr, column_name, rows_read);
|
||||
++rows_read;
|
||||
}
|
||||
while (!reader->checkColumnEndOrSkipFieldDelimiter());
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
|
||||
@ -76,18 +77,18 @@ class JSONColumnsSchemaReaderBase : public ISchemaReader
|
||||
public:
|
||||
JSONColumnsSchemaReaderBase(ReadBuffer & in_, const FormatSettings & format_settings_, std::unique_ptr<JSONColumnsReaderBase> reader_);
|
||||
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type);
|
||||
|
||||
private:
|
||||
NamesAndTypesList readSchema() override;
|
||||
|
||||
/// Read whole column in the block (up to max_rows_to_read rows) and extract the data type.
|
||||
DataTypePtr readColumnAndGetDataType(const String & column_name, size_t & rows_read, size_t max_rows_to_read);
|
||||
|
||||
/// Choose result type for column from two inferred types from different rows.
|
||||
void chooseResulType(DataTypePtr & type, DataTypePtr & new_type, const String & column_name, size_t row) const;
|
||||
|
||||
const FormatSettings format_settings;
|
||||
std::unique_ptr<JSONColumnsReaderBase> reader;
|
||||
Names column_names_from_settings;
|
||||
JSONInferenceInfo inference_info;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Formats/verbosePrintString.h>
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <Formats/registerWithNamesAndTypes.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
@ -202,12 +203,17 @@ DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypes()
|
||||
if (in.eof())
|
||||
return {};
|
||||
|
||||
return JSONUtils::readRowAndGetDataTypesForJSONCompactEachRow(in, format_settings, reader.yieldStrings());
|
||||
return JSONUtils::readRowAndGetDataTypesForJSONCompactEachRow(in, format_settings, &inference_info);
|
||||
}
|
||||
|
||||
void JSONCompactEachRowRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
|
||||
void JSONCompactEachRowRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
|
||||
{
|
||||
transformInferredJSONTypesIfNeeded(type, new_type, format_settings);
|
||||
transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info);
|
||||
}
|
||||
|
||||
void JSONCompactEachRowRowSchemaReader::transformFinalTypeIfNeeded(DataTypePtr & type)
|
||||
{
|
||||
transformJSONTupleToArrayIfPossible(type, format_settings, &inference_info);
|
||||
}
|
||||
|
||||
void registerInputFormatJSONCompactEachRow(FormatFactory & factory)
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
|
||||
namespace DB
|
||||
@ -80,10 +81,12 @@ public:
|
||||
private:
|
||||
DataTypes readRowAndGetDataTypes() override;
|
||||
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) override;
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
|
||||
void transformFinalTypeIfNeeded(DataTypePtr & type) override;
|
||||
|
||||
JSONCompactEachRowFormatReader reader;
|
||||
bool first_row = true;
|
||||
JSONInferenceInfo inference_info;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Processors/Formats/Impl/JSONEachRowRowInputFormat.h>
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
@ -300,9 +301,8 @@ void JSONEachRowRowInputFormat::readSuffix()
|
||||
assertEOF(*in);
|
||||
}
|
||||
|
||||
JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings_)
|
||||
JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
|
||||
: IRowWithNamesSchemaReader(in_, format_settings_)
|
||||
, json_strings(json_strings_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -336,12 +336,17 @@ NamesAndTypesList JSONEachRowSchemaReader::readRowAndGetNamesAndDataTypes(bool &
|
||||
return {};
|
||||
}
|
||||
|
||||
return JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, json_strings);
|
||||
return JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, &inference_info);
|
||||
}
|
||||
|
||||
void JSONEachRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
|
||||
{
|
||||
transformInferredJSONTypesIfNeeded(type, new_type, format_settings);
|
||||
transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info);
|
||||
}
|
||||
|
||||
void JSONEachRowSchemaReader::transformFinalTypeIfNeeded(DataTypePtr & type)
|
||||
{
|
||||
transformJSONTupleToArrayIfPossible(type, format_settings, &inference_info);
|
||||
}
|
||||
|
||||
void registerInputFormatJSONEachRow(FormatFactory & factory)
|
||||
@ -391,11 +396,11 @@ void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory
|
||||
|
||||
void registerJSONEachRowSchemaReader(FormatFactory & factory)
|
||||
{
|
||||
auto register_schema_reader = [&](const String & format_name, bool json_strings)
|
||||
auto register_schema_reader = [&](const String & format_name)
|
||||
{
|
||||
factory.registerSchemaReader(format_name, [json_strings](ReadBuffer & buf, const FormatSettings & settings)
|
||||
factory.registerSchemaReader(format_name, [](ReadBuffer & buf, const FormatSettings & settings)
|
||||
{
|
||||
return std::make_unique<JSONEachRowSchemaReader>(buf, json_strings, settings);
|
||||
return std::make_unique<JSONEachRowSchemaReader>(buf, settings);
|
||||
});
|
||||
factory.registerAdditionalInfoForSchemaCacheGetter(format_name, [](const FormatSettings & settings)
|
||||
{
|
||||
@ -403,10 +408,10 @@ void registerJSONEachRowSchemaReader(FormatFactory & factory)
|
||||
});
|
||||
};
|
||||
|
||||
register_schema_reader("JSONEachRow", false);
|
||||
register_schema_reader("JSONLines", false);
|
||||
register_schema_reader("NDJSON", false);
|
||||
register_schema_reader("JSONStringsEachRow", true);
|
||||
register_schema_reader("JSONEachRow");
|
||||
register_schema_reader("JSONLines");
|
||||
register_schema_reader("NDJSON");
|
||||
register_schema_reader("JSONStringsEachRow");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Processors/Formats/IRowInputFormat.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
|
||||
|
||||
@ -94,15 +95,16 @@ protected:
|
||||
class JSONEachRowSchemaReader : public IRowWithNamesSchemaReader
|
||||
{
|
||||
public:
|
||||
JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings, const FormatSettings & format_settings_);
|
||||
JSONEachRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_);
|
||||
|
||||
private:
|
||||
NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) override;
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
|
||||
void transformFinalTypeIfNeeded(DataTypePtr & type) override;
|
||||
|
||||
bool json_strings;
|
||||
bool first_row = true;
|
||||
bool data_in_square_brackets = false;
|
||||
JSONInferenceInfo inference_info;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
namespace DB
|
||||
@ -85,7 +86,7 @@ NamesAndTypesList JSONObjectEachRowSchemaReader::readRowAndGetNamesAndDataTypes(
|
||||
JSONUtils::skipComma(in);
|
||||
|
||||
JSONUtils::readFieldName(in);
|
||||
auto names_and_types = JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, false);
|
||||
auto names_and_types = JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, &inference_info);
|
||||
if (!format_settings.json_object_each_row.column_for_object_name.empty())
|
||||
names_and_types.emplace_front(format_settings.json_object_each_row.column_for_object_name, std::make_shared<DataTypeString>());
|
||||
return names_and_types;
|
||||
@ -93,7 +94,12 @@ NamesAndTypesList JSONObjectEachRowSchemaReader::readRowAndGetNamesAndDataTypes(
|
||||
|
||||
void JSONObjectEachRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
|
||||
{
|
||||
transformInferredJSONTypesIfNeeded(type, new_type, format_settings);
|
||||
transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info);
|
||||
}
|
||||
|
||||
void JSONObjectEachRowSchemaReader::transformFinalTypeIfNeeded(DataTypePtr & type)
|
||||
{
|
||||
transformJSONTupleToArrayIfPossible(type, format_settings, &inference_info);
|
||||
}
|
||||
|
||||
void registerInputFormatJSONObjectEachRow(FormatFactory & factory)
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Processors/Formats/Impl/JSONEachRowRowInputFormat.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
|
||||
|
||||
@ -42,8 +43,10 @@ public:
|
||||
private:
|
||||
NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) override;
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
|
||||
void transformFinalTypeIfNeeded(DataTypePtr & type) override;
|
||||
|
||||
bool first_row = true;
|
||||
JSONInferenceInfo inference_info;
|
||||
};
|
||||
|
||||
std::optional<size_t> getColumnIndexForJSONObjectEachRowObjectName(const Block & header, const FormatSettings & settings);
|
||||
|
@ -435,7 +435,7 @@ DataTypes MySQLDumpSchemaReader::readRowAndGetDataTypes()
|
||||
skipFieldDelimiter(in);
|
||||
|
||||
readQuotedField(value, in);
|
||||
auto type = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Quoted);
|
||||
auto type = tryInferDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Quoted);
|
||||
data_types.push_back(std::move(type));
|
||||
}
|
||||
skipEndOfRow(in, table_name);
|
||||
|
@ -3,7 +3,7 @@
|
||||
#if USE_ORC
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/copyData.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
#if USE_PARQUET
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <arrow/api.h>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Processors/Formats/Impl/RegexpRowInputFormat.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <Formats/newLineSegmentationEngine.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
@ -155,15 +156,15 @@ DataTypes RegexpSchemaReader::readRowAndGetDataTypes()
|
||||
for (size_t i = 0; i != field_extractor.getMatchedFieldsSize(); ++i)
|
||||
{
|
||||
String field(field_extractor.getField(i));
|
||||
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule));
|
||||
data_types.push_back(tryInferDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule, &json_inference_info));
|
||||
}
|
||||
|
||||
return data_types;
|
||||
}
|
||||
|
||||
void RegexpSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
|
||||
void RegexpSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
|
||||
{
|
||||
transformInferredTypesIfNeeded(type, new_type, format_settings, format_settings.regexp.escaping_rule);
|
||||
transformInferredTypesByEscapingRuleIfNeeded(type, new_type, format_settings, format_settings.regexp.escaping_rule, &json_inference_info);
|
||||
}
|
||||
|
||||
|
||||
|
@ -5,12 +5,13 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <Core/Block.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <Processors/Formats/IRowInputFormat.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <Formats/ParsedTemplateFormatString.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -81,12 +82,13 @@ public:
|
||||
private:
|
||||
DataTypes readRowAndGetDataTypes() override;
|
||||
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) override;
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
|
||||
|
||||
|
||||
using EscapingRule = FormatSettings::EscapingRule;
|
||||
RegexpFieldExtractor field_extractor;
|
||||
PeekableReadBuffer buf;
|
||||
JSONInferenceInfo json_inference_info;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -249,7 +249,7 @@ NamesAndTypesList TSKVSchemaReader::readRowAndGetNamesAndDataTypes(bool & eof)
|
||||
if (has_value)
|
||||
{
|
||||
readEscapedString(value, in);
|
||||
names_and_types.emplace_back(std::move(name), determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped));
|
||||
names_and_types.emplace_back(std::move(name), tryInferDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped));
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -268,7 +268,7 @@ DataTypes TabSeparatedSchemaReader::readRowAndGetDataTypes()
|
||||
return {};
|
||||
|
||||
auto fields = reader.readRow();
|
||||
return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule());
|
||||
return tryInferDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule());
|
||||
}
|
||||
|
||||
void registerInputFormatTabSeparated(FormatFactory & factory)
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/verbosePrintString.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
@ -493,16 +494,16 @@ DataTypes TemplateSchemaReader::readRowAndGetDataTypes()
|
||||
format_settings.csv.delimiter = row_format.delimiters[i + 1].empty() ? format_settings.csv.delimiter : row_format.delimiters[i + 1].front();
|
||||
|
||||
field = readFieldByEscapingRule(buf, row_format.escaping_rules[i], format_settings);
|
||||
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, row_format.escaping_rules[i]));
|
||||
data_types.push_back(tryInferDataTypeByEscapingRule(field, format_settings, row_format.escaping_rules[i], &json_inference_info));
|
||||
}
|
||||
|
||||
format_reader.skipRowEndDelimiter();
|
||||
return data_types;
|
||||
}
|
||||
|
||||
void TemplateSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t column_idx)
|
||||
void TemplateSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
|
||||
{
|
||||
transformInferredTypesIfNeeded(type, new_type, format_settings, row_format.escaping_rules[column_idx]);
|
||||
transformInferredTypesByEscapingRuleIfNeeded(type, new_type, format_settings, row_format.escaping_rules[field_index], &json_inference_info);
|
||||
}
|
||||
|
||||
static ParsedTemplateFormatString fillResultSetFormat(const FormatSettings & settings)
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/ParsedTemplateFormatString.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -121,13 +122,14 @@ public:
|
||||
DataTypes readRowAndGetDataTypes() override;
|
||||
|
||||
private:
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t column_idx) override;
|
||||
void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
|
||||
|
||||
PeekableReadBuffer buf;
|
||||
const ParsedTemplateFormatString format;
|
||||
const ParsedTemplateFormatString row_format;
|
||||
TemplateFormatReader format_reader;
|
||||
bool first_row = true;
|
||||
JSONInferenceInfo json_inference_info;
|
||||
};
|
||||
|
||||
bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces);
|
||||
|
@ -599,7 +599,7 @@ DataTypes ValuesSchemaReader::readRowAndGetDataTypes()
|
||||
}
|
||||
|
||||
readQuotedField(value, buf);
|
||||
auto type = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Quoted);
|
||||
auto type = tryInferDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Quoted);
|
||||
data_types.push_back(std::move(type));
|
||||
}
|
||||
|
||||
|
23
tests/performance/schema_inference_text_formats.xml
Normal file
23
tests/performance/schema_inference_text_formats.xml
Normal file
@ -0,0 +1,23 @@
|
||||
<test>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>format</name>
|
||||
<values>
|
||||
<value>TabSeparated</value>
|
||||
<value>CSV</value>
|
||||
<value>Values</value>
|
||||
<value>JSONEachRow</value>
|
||||
<value>JSONCompactEachRow</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
|
||||
<fill_query>INSERT INTO function file(data.{format}) SELECT WatchID, Title, EventTime, RefererCategories, RefererRegions FROM test.hits LIMIT 25000 SETTINGS engine_file_truncate_on_insert=1</fill_query>
|
||||
|
||||
<query>DESC file(data.{format}) SETTINGS schema_inference_use_cache_for_file=0</query>
|
||||
|
||||
<drop_query>INSERT INTO FUNCTION file(data.{format}) SELECT * FROM numbers(0) SETTINGS engine_file_truncate_on_insert=1</drop_query>
|
||||
|
||||
</test>
|
@ -0,0 +1,48 @@
|
||||
JSONEachRow
|
||||
x Array(Nullable(Int64))
|
||||
x Array(Nullable(Int64))
|
||||
x Array(Nullable(Int64))
|
||||
x Array(Nullable(Int64))
|
||||
x Tuple(Nullable(String), Nullable(Int64))
|
||||
x Tuple(Nullable(String), Nullable(Int64))
|
||||
x Map(String, Nullable(Int64))
|
||||
x Map(String, Nullable(Int64))
|
||||
x Array(Nullable(Int64))
|
||||
x Array(Array(Nullable(Int64)))
|
||||
x Array(Map(String, Nullable(Int64)))
|
||||
x Array(Array(Nullable(String)))
|
||||
x Array(Int64)
|
||||
x Array(Nullable(Int64))
|
||||
x Array(Int64)
|
||||
x Array(Nullable(Int64))
|
||||
JSONCompactEachRow
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Tuple(Nullable(String), Nullable(Int64))
|
||||
c1 Tuple(Nullable(String), Nullable(Int64))
|
||||
c1 Map(String, Nullable(Int64))
|
||||
c1 Map(String, Nullable(Int64))
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Array(Array(Nullable(Int64)))
|
||||
c1 Array(Map(String, Nullable(Int64)))
|
||||
c1 Array(Array(Nullable(String)))
|
||||
c1 Array(Int64)
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Array(Int64)
|
||||
c1 Array(Nullable(Int64))
|
||||
CSV
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Map(String, Nullable(Int64))
|
||||
c1 Map(String, Nullable(Int64))
|
||||
c1 Array(Array(Nullable(Int64)))
|
||||
c1 Array(Map(String, Nullable(Int64)))
|
||||
c1 Array(Array(Nullable(String)))
|
||||
c1 Array(Int64)
|
||||
c1 Array(Nullable(Int64))
|
||||
c1 Array(Int64)
|
||||
c1 Array(Nullable(Int64))
|
63
tests/queries/0_stateless/02497_schema_inference_nulls.sql
Normal file
63
tests/queries/0_stateless/02497_schema_inference_nulls.sql
Normal file
@ -0,0 +1,63 @@
|
||||
select 'JSONEachRow';
|
||||
set schema_inference_make_columns_nullable=1;
|
||||
desc format(JSONEachRow, '{"x" : 1234}, {"x" : "String"}') settings input_format_json_try_infer_numbers_from_strings=1; -- { serverError TYPE_MISMATCH }
|
||||
desc format(JSONEachRow, '{"x" : [null, 1]}');
|
||||
desc format(JSONEachRow, '{"x" : [null, 1]}, {"x" : []}');
|
||||
desc format(JSONEachRow, '{"x" : [null, 1]}, {"x" : [null]}');
|
||||
desc format(JSONEachRow, '{"x" : [null, 1]}, {"x" : [1, null]}');
|
||||
desc format(JSONEachRow, '{"x" : [null, 1]}, {"x" : ["abc", 1]}');
|
||||
desc format(JSONEachRow, '{"x" : [null, 1]}, {"x" : ["abc", null]}');
|
||||
desc format(JSONEachRow, '{"x" : {}}, {"x" : {"a" : 1}}');
|
||||
desc format(JSONEachRow, '{"x" : {"a" : null}}, {"x" : {"b" : 1}}');
|
||||
desc format(JSONEachRow, '{"x" : null}, {"x" : [1, 2]}');
|
||||
desc format(JSONEachRow, '{"x" : [[], [null], [1, 2, 3]]}');
|
||||
desc format(JSONEachRow, '{"x" : [{"a" : null}, {"b" : 1}]}');
|
||||
desc format(JSONEachRow, '{"x" : [["2020-01-01", null, "1234"], ["abcd"]]}');
|
||||
|
||||
set schema_inference_make_columns_nullable=0;
|
||||
desc format(JSONEachRow, '{"x" : [1, 2]}');
|
||||
desc format(JSONEachRow, '{"x" : [null, 1]}');
|
||||
desc format(JSONEachRow, '{"x" : [1, 2]}, {"x" : [3]}');
|
||||
desc format(JSONEachRow, '{"x" : [1, 2]}, {"x" : [null]}');
|
||||
|
||||
select 'JSONCompactEachRow';
|
||||
set schema_inference_make_columns_nullable=1;
|
||||
desc format(JSONCompactEachRow, '[1234], ["String"]') settings input_format_json_try_infer_numbers_from_strings=1; -- { serverError TYPE_MISMATCH }
|
||||
desc format(JSONCompactEachRow, '[[null, 1]]');
|
||||
desc format(JSONCompactEachRow, '[[null, 1]], [[]]');
|
||||
desc format(JSONCompactEachRow, '[[null, 1]], [[null]]');
|
||||
desc format(JSONCompactEachRow, '[[null, 1]], [[1, null]]');
|
||||
desc format(JSONCompactEachRow, '[[null, 1]], [["abc", 1]]');
|
||||
desc format(JSONCompactEachRow, '[[null, 1]], [["abc", null]]');
|
||||
desc format(JSONCompactEachRow, '[{}], [{"a" : 1}]');
|
||||
desc format(JSONCompactEachRow, '[{"a" : null}], [{"b" : 1}]');
|
||||
desc format(JSONCompactEachRow, '[null], [[1, 2]]');
|
||||
desc format(JSONCompactEachRow, '[[[], [null], [1, 2, 3]]]');
|
||||
desc format(JSONCompactEachRow, '[[{"a" : null}, {"b" : 1}]]');
|
||||
desc format(JSONCompactEachRow, '[[["2020-01-01", null, "1234"], ["abcd"]]]');
|
||||
|
||||
set schema_inference_make_columns_nullable=0;
|
||||
desc format(JSONCompactEachRow, '[[1, 2]]');
|
||||
desc format(JSONCompactEachRow, '[[null, 1]]');
|
||||
desc format(JSONCompactEachRow, '[[1, 2]], [[3]]');
|
||||
desc format(JSONCompactEachRow, '[[1, 2]], [[null]]');
|
||||
|
||||
|
||||
select 'CSV';
|
||||
set schema_inference_make_columns_nullable=1;
|
||||
desc format(CSV, '"[null, 1]"');
|
||||
desc format(CSV, '"[null, 1]"\n"[]"');
|
||||
desc format(CSV, '"[null, 1]"\n"[null]"');
|
||||
desc format(CSV, '"[null, 1]"\n"[1, null]"');
|
||||
desc format(CSV, '"{}"\n"{\'a\' : 1}"');
|
||||
desc format(CSV, '"{\'a\' : null}"\n"{\'b\' : 1}"');
|
||||
desc format(CSV, '"[[], [null], [1, 2, 3]]"');
|
||||
desc format(CSV, '"[{\'a\' : null}, {\'b\' : 1}]"');
|
||||
desc format(CSV, '"[[\'2020-01-01\', null, \'1234\'], [\'abcd\']]"');
|
||||
|
||||
set schema_inference_make_columns_nullable=0;
|
||||
desc format(CSV, '"[1,2]"');
|
||||
desc format(CSV, '"[NULL, 1]"');
|
||||
desc format(CSV, '"[1, 2]"\n"[3]"');
|
||||
desc format(CSV, '"[1, 2]"\n"[null]"');
|
||||
|
@ -0,0 +1 @@
|
||||
s Nullable(String)
|
@ -0,0 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_LOCAL -q "select randomString(100) as s format JSONEachRow" | $CLICKHOUSE_LOCAL -q "desc test" --table='test' --input-format='JSONEachRow'
|
@ -0,0 +1,2 @@
|
||||
c1 Array(Nullable(String))
|
||||
c1 Nullable(String)
|
@ -0,0 +1,2 @@
|
||||
desc format(CSV, '"[\'abc\\\'\']"');
|
||||
desc format(Values, '(\'abc\\\'\')');
|
Loading…
Reference in New Issue
Block a user