diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 92af0576129..961633a6bb9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -705,6 +705,10 @@ 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(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_try_infer_integers, true, "Try to infer numbers from string fields while schema inference in text formats", 0) \ + M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ + M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \ M(Bool, output_format_protobuf_nullables_with_google_wrappers, false, "When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized", 0) \ M(UInt64, input_format_csv_skip_first_lines, 0, "Skip specified number of lines at the beginning of data in CSV format", 0) \ diff --git a/src/DataTypes/transformTypesRecursively.cpp b/src/DataTypes/transformTypesRecursively.cpp new file mode 100644 index 00000000000..3544c7e477d --- /dev/null +++ b/src/DataTypes/transformTypesRecursively.cpp @@ -0,0 +1,178 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +void transformTypesRecursively(DataTypes & types, std::function transform_simple_types, std::function transform_complex_types) +{ + { + /// Arrays + bool have_array = false; + bool all_arrays = true; + DataTypes nested_types; + for (const auto & type : types) + { + if (const DataTypeArray * type_array = typeid_cast(type.get())) + { + have_array = true; + nested_types.push_back(type_array->getNestedType()); + } + else + all_arrays = false; + } + + if (have_array) + { + if (all_arrays) + { + transformTypesRecursively(nested_types, transform_simple_types, transform_complex_types); + for (size_t i = 0; i != types.size(); ++i) + types[i] = std::make_shared(nested_types[i]); + } + + if (transform_complex_types) + transform_complex_types(types); + + return; + } + } + + { + /// Tuples + bool have_tuple = false; + bool all_tuples = true; + size_t tuple_size = 0; + + std::vector nested_types; + + for (const auto & type : types) + { + if (const DataTypeTuple * type_tuple = typeid_cast(type.get())) + { + if (!have_tuple) + { + tuple_size = type_tuple->getElements().size(); + nested_types.resize(tuple_size); + for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx) + nested_types[elem_idx].reserve(types.size()); + } + else if (tuple_size != type_tuple->getElements().size()) + return; + + have_tuple = true; + + for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx) + nested_types[elem_idx].emplace_back(type_tuple->getElements()[elem_idx]); + } + else + all_tuples = false; + } + + if (have_tuple) + { + if (all_tuples) + { + std::vector transposed_nested_types(types.size()); + for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx) + { + transformTypesRecursively(nested_types[elem_idx], transform_simple_types, transform_complex_types); + for (size_t i = 0; i != types.size(); ++i) + transposed_nested_types[i].push_back(nested_types[elem_idx][i]); + } + + for (size_t i = 0; i != types.size(); ++i) + types[i] = std::make_shared(transposed_nested_types[i]); + } + + if (transform_complex_types) + transform_complex_types(types); + + return; + } + } + + { + /// Maps + bool have_maps = false; + bool all_maps = true; + DataTypes key_types; + DataTypes value_types; + key_types.reserve(types.size()); + value_types.reserve(types.size()); + + for (const auto & type : types) + { + if (const DataTypeMap * type_map = typeid_cast(type.get())) + { + have_maps = true; + key_types.emplace_back(type_map->getKeyType()); + value_types.emplace_back(type_map->getValueType()); + } + else + all_maps = false; + } + + if (have_maps) + { + if (all_maps) + { + transformTypesRecursively(key_types, transform_simple_types, transform_complex_types); + transformTypesRecursively(value_types, transform_simple_types, transform_complex_types); + + for (size_t i = 0; i != types.size(); ++i) + types[i] = std::make_shared(key_types[i], value_types[i]); + } + + if (transform_complex_types) + transform_complex_types(types); + + return; + } + } + + { + /// Nullable + bool have_nullable = false; + std::vector is_nullable; + is_nullable.reserve(types.size()); + DataTypes nested_types; + nested_types.reserve(types.size()); + for (const auto & type : types) + { + if (const DataTypeNullable * type_nullable = typeid_cast(type.get())) + { + have_nullable = true; + is_nullable.push_back(1); + nested_types.push_back(type_nullable->getNestedType()); + } + else + { + is_nullable.push_back(0); + nested_types.push_back(type); + } + } + + if (have_nullable) + { + transformTypesRecursively(nested_types, transform_simple_types, transform_complex_types); + for (size_t i = 0; i != types.size(); ++i) + { + if (is_nullable[i]) + types[i] = makeNullable(nested_types[i]); + else + types[i] = nested_types[i]; + } + + return; + } + } + + transform_simple_types(types); +} + +} diff --git a/src/DataTypes/transformTypesRecursively.h b/src/DataTypes/transformTypesRecursively.h new file mode 100644 index 00000000000..5cb8f095494 --- /dev/null +++ b/src/DataTypes/transformTypesRecursively.h @@ -0,0 +1,17 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Function that applies custom transformation functions to provided types recursively. +/// Implementation is similar to function getLeastSuperType: +/// If all types are Array/Map/Tuple/Nullable, this function will be called to nested types. +/// If not all types are the same complex type (Array/Map/Tuple), this function won't be called to nested types. +/// Function transform_simple_types will be applied to resulting simple types after all recursive calls. +/// Function transform_complex_types will be applied to complex types (Array/Map/Tuple) after recursive call to their nested types. +void transformTypesRecursively(DataTypes & types, std::function transform_simple_types, std::function transform_complex_types); + +} diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 5aab8909a0c..a5c4dd1dd47 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -9,8 +9,12 @@ #include #include #include -#include +#include +#include #include +#include +#include +#include #include #include #include @@ -255,7 +259,220 @@ String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule e return readByEscapingRule(buf, escaping_rule, format_settings); } -static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) +void transformInferredTypesIfNeededImpl(DataTypes & types, const FormatSettings & settings, bool is_json, const std::unordered_set * 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(); + } + } + } + + /// 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(); + } + } + else if (have_dates && have_datetimes) + { + for (auto & type : data_types) + { + if (isDate(type)) + type = std::make_shared(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) + { + 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) && (!numbers_parsed_from_json_strings || numbers_parsed_from_json_strings->contains(type.get()))) + type = std::make_shared(); + } + } + } + + 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(); + else + type = std::make_shared(); + } + } + } + } + }; + + 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("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 * 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]); +} + +DataTypePtr tryInferDateOrDateTime(const std::string_view & field, const FormatSettings & settings) +{ + if (settings.try_infer_dates) + { + ReadBufferFromString buf(field); + DayNum tmp; + if (tryReadDateText(tmp, buf) && buf.eof()) + return makeNullable(std::make_shared()); + } + + if (settings.try_infer_datetimes) + { + ReadBufferFromString buf(field); + DateTime64 tmp; + if (tryReadDateTime64Text(tmp, 9, buf) && buf.eof()) + return makeNullable(std::make_shared(9)); + } + + return nullptr; +} + +static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBufferFromString & buf, const FormatSettings & settings) { if (buf.eof()) return nullptr; @@ -279,7 +496,7 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) else first = false; - auto nested_type = determineDataTypeForSingleFieldImpl(buf); + auto nested_type = determineDataTypeForSingleFieldImpl(buf, settings); if (!nested_type) return nullptr; @@ -294,6 +511,8 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) if (nested_types.empty()) return std::make_shared(std::make_shared()); + transformInferredTypesIfNeeded(nested_types, settings); + auto least_supertype = tryGetLeastSupertype(nested_types); if (!least_supertype) return nullptr; @@ -320,7 +539,7 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) else first = false; - auto nested_type = determineDataTypeForSingleFieldImpl(buf); + auto nested_type = determineDataTypeForSingleFieldImpl(buf, settings); if (!nested_type) return nullptr; @@ -355,7 +574,7 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) else first = false; - auto key_type = determineDataTypeForSingleFieldImpl(buf); + auto key_type = determineDataTypeForSingleFieldImpl(buf, settings); if (!key_type) return nullptr; @@ -366,7 +585,7 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) return nullptr; skipWhitespaceIfAny(buf); - auto value_type = determineDataTypeForSingleFieldImpl(buf); + auto value_type = determineDataTypeForSingleFieldImpl(buf, settings); if (!value_type) return nullptr; @@ -382,6 +601,9 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) if (key_types.empty()) return std::make_shared(std::make_shared(), std::make_shared()); + transformInferredTypesIfNeeded(key_types, settings); + transformInferredTypesIfNeeded(value_types, settings); + auto key_least_supertype = tryGetLeastSupertype(key_types); auto value_least_supertype = tryGetLeastSupertype(value_types); @@ -398,9 +620,11 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) 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()) @@ -409,6 +633,7 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) if (*buf.position() == '\'') break; + field.push_back(*buf.position()); if (*buf.position() == '\\') ++buf.position(); } @@ -417,6 +642,9 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) return nullptr; ++buf.position(); + if (auto type = tryInferDateOrDateTime(field, settings)) + return type; + return std::make_shared(); } @@ -430,15 +658,29 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) /// 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(); + + buf.position() = float_end_pos; + } + return std::make_shared(); + } return nullptr; } -static DataTypePtr determineDataTypeForSingleField(ReadBuffer & buf) +static DataTypePtr determineDataTypeForSingleField(ReadBufferFromString & buf, const FormatSettings & settings) { - return makeNullableRecursivelyAndCheckForNothing(determineDataTypeForSingleFieldImpl(buf)); + return makeNullableRecursivelyAndCheckForNothing(determineDataTypeForSingleFieldImpl(buf, settings)); } DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule) @@ -448,11 +690,11 @@ DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSe case FormatSettings::EscapingRule::Quoted: { ReadBufferFromString buf(field); - auto type = determineDataTypeForSingleField(buf); + auto type = determineDataTypeForSingleField(buf, format_settings); return buf.eof() ? type : nullptr; } case FormatSettings::EscapingRule::JSON: - return JSONUtils::getDataTypeFromField(field); + return JSONUtils::getDataTypeFromField(field, format_settings); case FormatSettings::EscapingRule::CSV: { if (!format_settings.csv.input_format_use_best_effort_in_schema_inference) @@ -466,9 +708,13 @@ DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSe if (field.size() > 1 && ((field.front() == '\'' && field.back() == '\'') || (field.front() == '"' && field.back() == '"'))) { - ReadBufferFromString buf(std::string_view(field.data() + 1, field.size() - 2)); + auto data = std::string_view(field.data() + 1, field.size() - 2); + if (auto date_type = tryInferDateOrDateTime(data, format_settings)) + return date_type; + + ReadBufferFromString buf(data); /// Try to determine the type of value inside quotes - auto type = determineDataTypeForSingleField(buf); + auto type = determineDataTypeForSingleField(buf, format_settings); if (!type) return nullptr; @@ -481,6 +727,14 @@ DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSe } /// 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()); + } + ReadBufferFromString buf(field); Float64 tmp; if (tryReadFloatText(tmp, buf) && buf.eof()) @@ -500,8 +754,11 @@ DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSe if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation) return DataTypeFactory::instance().get("Nullable(Bool)"); + if (auto date_type = tryInferDateOrDateTime(field, format_settings)) + return date_type; + ReadBufferFromString buf(field); - auto type = determineDataTypeForSingleField(buf); + auto type = determineDataTypeForSingleField(buf, format_settings); if (!buf.eof()) return makeNullable(std::make_shared()); diff --git a/src/Formats/EscapingRuleUtils.h b/src/Formats/EscapingRuleUtils.h index 1ce04a8d1b7..ad4ce65a430 100644 --- a/src/Formats/EscapingRuleUtils.h +++ b/src/Formats/EscapingRuleUtils.h @@ -60,4 +60,21 @@ DataTypes determineDataTypesByEscapingRule(const std::vector & fields, c DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule); DataTypes getDefaultDataTypeForEscapingRules(const std::vector & 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 * numbers_parsed_from_json_strings = nullptr); +void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings); + } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 50d2ee12082..962a57bb512 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -94,6 +94,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; format_settings.json.read_bools_as_numbers = settings.input_format_json_read_bools_as_numbers; + format_settings.json.try_infer_numbers_from_strings = settings.input_format_json_try_infer_numbers_from_strings; format_settings.null_as_default = settings.input_format_null_as_default; format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros; format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; @@ -165,6 +166,9 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.sql_insert.table_name = settings.output_format_sql_insert_table_name; format_settings.sql_insert.use_replace = settings.output_format_sql_insert_use_replace; format_settings.sql_insert.quote_names = settings.output_format_sql_insert_quote_names; + format_settings.try_infer_integers = settings.input_format_try_infer_integers; + format_settings.try_infer_dates = settings.input_format_try_infer_dates; + format_settings.try_infer_datetimes = settings.input_format_try_infer_datetimes; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index b7c55d11beb..c240d9bcb01 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -38,6 +38,9 @@ struct FormatSettings UInt64 max_rows_to_read_for_schema_inference = 100; String column_names_for_schema_inference; + bool try_infer_integers = false; + bool try_infer_dates = false; + bool try_infer_datetimes = false; enum class DateTimeInputFormat { @@ -142,6 +145,7 @@ struct FormatSettings bool named_tuples_as_objects = false; bool serialize_as_strings = false; bool read_bools_as_numbers = true; + bool try_infer_numbers_from_strings = false; } json; struct diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 7c94b122096..895833abf4d 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -121,7 +122,7 @@ namespace JSONUtils } template - DataTypePtr getDataTypeFromFieldImpl(const Element & field) + DataTypePtr getDataTypeFromFieldImpl(const Element & field, const FormatSettings & settings, std::unordered_set & numbers_parsed_from_json_strings) { if (field.isNull()) return nullptr; @@ -129,11 +130,48 @@ namespace JSONUtils if (field.isBool()) return DataTypeFactory::instance().get("Nullable(Bool)"); - if (field.isInt64() || field.isUInt64() || field.isDouble()) + if (field.isInt64() || field.isUInt64()) + { + if (settings.try_infer_integers) + return makeNullable(std::make_shared()); + + return makeNullable(std::make_shared()); + } + + if (field.isDouble()) return makeNullable(std::make_shared()); 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()); + + ReadBufferFromString buf(field.getString()); + + if (settings.try_infer_integers) + { + Int64 tmp_int; + if (tryReadIntText(tmp_int, buf) && buf.eof()) + { + auto type = std::make_shared(); + numbers_parsed_from_json_strings.insert(type.get()); + return makeNullable(type); + } + } + + Float64 tmp; + if (tryReadFloatText(tmp, buf) && buf.eof()) + { + auto type = std::make_shared(); + numbers_parsed_from_json_strings.insert(type.get()); + return makeNullable(type); + } + return makeNullable(std::make_shared()); + } if (field.isArray()) { @@ -145,20 +183,32 @@ namespace JSONUtils DataTypes nested_data_types; /// If this array contains fields with different types we will treat it as Tuple. - bool is_tuple = false; + bool are_types_the_same = true; for (const auto element : array) { - auto type = getDataTypeFromFieldImpl(element); + auto type = getDataTypeFromFieldImpl(element, settings, numbers_parsed_from_json_strings); if (!type) return nullptr; - if (!nested_data_types.empty() && type->getName() != nested_data_types.back()->getName()) - is_tuple = true; + 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 (is_tuple) + 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(nested_data_types); return std::make_shared(nested_data_types.back()); @@ -167,38 +217,35 @@ namespace JSONUtils if (field.isObject()) { auto object = field.getObject(); - DataTypePtr value_type; - bool is_object = false; + DataTypes value_types; + bool have_object_value = false; for (const auto key_value_pair : object) { - auto type = getDataTypeFromFieldImpl(key_value_pair.second); + auto type = getDataTypeFromFieldImpl(key_value_pair.second, settings, numbers_parsed_from_json_strings); if (!type) continue; if (isObject(type)) { - is_object = true; + have_object_value = true; break; } - if (!value_type) - { - value_type = type; - } - else if (!value_type->equals(*type)) - { - is_object = true; - break; - } + value_types.push_back(type); } - if (is_object) + 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 (have_object_value || !are_types_equal) return std::make_shared("json", true); - if (value_type) - return std::make_shared(std::make_shared(), value_type); - - return nullptr; + return std::make_shared(std::make_shared(), value_types[0]); } throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type"}; @@ -215,18 +262,19 @@ namespace JSONUtils #endif } - DataTypePtr getDataTypeFromField(const String & field) + 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); - return getDataTypeFromFieldImpl(element); + std::unordered_set numbers_parsed_from_json_strings; + return getDataTypeFromFieldImpl(element, settings, numbers_parsed_from_json_strings); } template - static DataTypes determineColumnDataTypesFromJSONEachRowDataImpl(ReadBuffer & in, bool /*json_strings*/, Extractor & extractor) + static DataTypes determineColumnDataTypesFromJSONEachRowDataImpl(ReadBuffer & in, const FormatSettings & settings, bool /*json_strings*/, Extractor & extractor) { String line = readJSONEachRowLineIntoStringImpl(in); auto [parser, element] = getJSONParserAndElement(); @@ -238,8 +286,9 @@ namespace JSONUtils DataTypes data_types; data_types.reserve(fields.size()); + std::unordered_set numbers_parsed_from_json_strings; for (const auto & field : fields) - data_types.push_back(getDataTypeFromFieldImpl(field)); + 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? @@ -284,11 +333,11 @@ namespace JSONUtils std::vector column_names; }; - NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, bool json_strings) + NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings) { JSONEachRowFieldsExtractor extractor; auto data_types - = determineColumnDataTypesFromJSONEachRowDataImpl(in, json_strings, extractor); + = determineColumnDataTypesFromJSONEachRowDataImpl(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]); @@ -313,10 +362,10 @@ namespace JSONUtils } }; - DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, bool json_strings) + DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings) { JSONCompactEachRowFieldsExtractor extractor; - return determineColumnDataTypesFromJSONEachRowDataImpl(in, json_strings, extractor); + return determineColumnDataTypesFromJSONEachRowDataImpl(in, settings, json_strings, extractor); } diff --git a/src/Formats/JSONUtils.h b/src/Formats/JSONUtils.h index f2aba3cbcb5..b4ab6a29c93 100644 --- a/src/Formats/JSONUtils.h +++ b/src/Formats/JSONUtils.h @@ -22,16 +22,16 @@ namespace JSONUtils /// 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); + 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, bool json_strings); + NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings); /// 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, bool json_strings); + DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, const FormatSettings & settings, bool json_strings); bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf); diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 058f9b7059b..5ae21340864 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -88,8 +88,13 @@ ColumnsDescription readSchemaFromFormat( catch (...) { auto exception_message = getCurrentExceptionMessage(false); - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file: {}. You can specify the structure manually", format_name, exception_message); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually", + format_name, + exception_message); } + ++iterations; if (is_eof) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 5a6ebf00660..3df9ea70e34 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -17,35 +18,38 @@ namespace ErrorCodes void chooseResultColumnType( DataTypePtr & type, - const DataTypePtr & new_type, - CommonDataTypeChecker common_type_checker, + DataTypePtr & new_type, + std::function transform_types_if_needed, const DataTypePtr & default_type, const String & column_name, size_t row) { if (!type) + { type = new_type; + return; + } + + if (!new_type || type->equals(*new_type)) + return; + + transform_types_if_needed(type, new_type); + if (type->equals(*new_type)) + return; /// If the new type and the previous type for this column are different, /// we will use default type if we have it or throw an exception. - if (new_type && !type->equals(*new_type)) + if (default_type) + type = default_type; + else { - DataTypePtr common_type; - if (common_type_checker) - common_type = common_type_checker(type, new_type); - - if (common_type) - type = common_type; - else if (default_type) - type = default_type; - else - throw Exception( - ErrorCodes::TYPE_MISMATCH, - "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", - type->getName(), - column_name, - row, - new_type->getName()); + throw Exception( + ErrorCodes::TYPE_MISMATCH, + "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", + type->getName(), + column_name, + row, + new_type->getName()); } } @@ -63,8 +67,8 @@ void checkResultColumnTypeAndAppend(NamesAndTypesList & result, DataTypePtr & ty result.emplace_back(name, type); } -IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings) - : ISchemaReader(in_) +IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : ISchemaReader(in_), format_settings(format_settings_) { if (!format_settings.column_names_for_schema_inference.empty()) { @@ -79,14 +83,14 @@ IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & form } } -IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_) - : IRowSchemaReader(in_, format_settings) +IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, DataTypePtr default_type_) + : IRowSchemaReader(in_, format_settings_) { default_type = default_type_; } -IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, const DataTypes & default_types_) - : IRowSchemaReader(in_, format_settings) +IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, const DataTypes & default_types_) + : IRowSchemaReader(in_, format_settings_) { default_types = default_types_; } @@ -116,7 +120,8 @@ NamesAndTypesList IRowSchemaReader::readSchema() if (!new_data_types[i]) continue; - chooseResultColumnType(data_types[i], new_data_types[i], common_type_checker, getDefaultType(i), std::to_string(i + 1), rows_read); + 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); } } @@ -156,8 +161,13 @@ DataTypePtr IRowSchemaReader::getDefaultType(size_t column) const return nullptr; } -IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, DataTypePtr default_type_) - : ISchemaReader(in_), default_type(default_type_) +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_) + : ISchemaReader(in_), format_settings(format_settings_), default_type(default_type_) { } @@ -181,6 +191,7 @@ 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); @@ -188,7 +199,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() /// We reached eof. break; - for (const auto & [name, new_type] : new_names_and_types) + for (auto & [name, new_type] : new_names_and_types) { auto it = names_to_types.find(name); /// If we didn't see this column before, just add it. @@ -200,7 +211,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() } auto & type = it->second; - chooseResultColumnType(type, new_type, common_type_checker, default_type, name, rows_read); + chooseResultColumnType(type, new_type, transform_types_if_needed, default_type, name, rows_read); } } @@ -219,4 +230,9 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() return result; } +void IRowWithNamesSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) +{ + transformInferredTypesIfNeeded(type, new_type, format_settings); +} + } diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index 00987540d04..02c42495b2a 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -53,8 +53,6 @@ public: NamesAndTypesList readSchema() override; - void setCommonTypeChecker(CommonDataTypeChecker checker) { common_type_checker = checker; } - protected: /// Read one row and determine types of columns in it. /// Return types in the same order in which the values were in the row. @@ -67,6 +65,10 @@ protected: void setMaxRowsToRead(size_t max_rows) override { max_rows_to_read = max_rows; } size_t getNumRowsRead() const override { return rows_read; } + FormatSettings format_settings; + + virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t column_idx); + private: DataTypePtr getDefaultType(size_t column) const; @@ -74,7 +76,6 @@ private: size_t rows_read = 0; DataTypePtr default_type; DataTypes default_types; - CommonDataTypeChecker common_type_checker; std::vector column_names; }; @@ -86,12 +87,10 @@ private: class IRowWithNamesSchemaReader : public ISchemaReader { public: - IRowWithNamesSchemaReader(ReadBuffer & in_, DataTypePtr default_type_ = nullptr); + IRowWithNamesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, DataTypePtr default_type_ = nullptr); NamesAndTypesList readSchema() override; bool hasStrictOrderOfColumns() const override { return false; } - void setCommonTypeChecker(CommonDataTypeChecker checker) { common_type_checker = checker; } - protected: /// Read one row and determine types of columns in it. /// Return list with names and types. @@ -102,11 +101,14 @@ protected: void setMaxRowsToRead(size_t max_rows) override { max_rows_to_read = max_rows; } size_t getNumRowsRead() const override { return rows_read; } + virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type); + + FormatSettings format_settings; + private: size_t max_rows_to_read; size_t rows_read = 0; DataTypePtr default_type; - CommonDataTypeChecker common_type_checker; }; /// Base class for schema inference for formats that don't need any data to @@ -122,8 +124,8 @@ public: void chooseResultColumnType( DataTypePtr & type, - const DataTypePtr & new_type, - CommonDataTypeChecker common_type_checker, + DataTypePtr & new_type, + std::function transform_types_if_needed, const DataTypePtr & default_type, const String & column_name, size_t row); diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 61488a94ccd..e5397ca0757 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -318,6 +318,11 @@ DataTypes CustomSeparatedSchemaReader::readRowAndGetDataTypes() return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule()); } +void CustomSeparatedSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) +{ + transformInferredTypesIfNeeded(type, new_type, format_settings, reader.getEscapingRule()); +} + void registerInputFormatCustomSeparated(FormatFactory & factory) { for (bool ignore_spaces : {false, true}) diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h index d9e62a1b8e9..c7e332b983f 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -97,6 +97,8 @@ public: private: DataTypes readRowAndGetDataTypes() override; + void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) override; + PeekableReadBuffer buf; CustomSeparatedFormatReader reader; bool first_row = true; diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp index cdde87f2cf6..7f9fbddd554 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -181,13 +182,14 @@ JSONColumnsSchemaReaderBase::JSONColumnsSchemaReaderBase( { } -void JSONColumnsSchemaReaderBase::chooseResulType(DataTypePtr & type, const DataTypePtr & new_type, const String & column_name, size_t row) const +void JSONColumnsSchemaReaderBase::chooseResulType(DataTypePtr & type, DataTypePtr & new_type, const String & column_name, size_t row) const { - auto common_type_checker = [&](const DataTypePtr & first, const DataTypePtr & second) + auto convert_types_if_needed = [&](DataTypePtr & first, DataTypePtr & second) { - return JSONUtils::getCommonTypeForJSONFormats(first, second, format_settings.json.read_bools_as_numbers); + DataTypes types = {first, second}; + transformInferredJSONTypesIfNeeded(types, format_settings); }; - chooseResultColumnType(type, new_type, common_type_checker, nullptr, column_name, row); + chooseResultColumnType(type, new_type, convert_types_if_needed, nullptr, column_name, row); } NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema() @@ -260,7 +262,7 @@ DataTypePtr JSONColumnsSchemaReaderBase::readColumnAndGetDataType(const String & } readJSONField(field, in); - DataTypePtr field_type = JSONUtils::getDataTypeFromField(field); + DataTypePtr field_type = JSONUtils::getDataTypeFromField(field, format_settings); chooseResulType(column_type, field_type, column_name, rows_read); ++rows_read; } diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h index ac746a2e2d1..6769e60be22 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h @@ -83,7 +83,7 @@ private: 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, const DataTypePtr & new_type, const String & column_name, size_t row) const; + void chooseResulType(DataTypePtr & type, DataTypePtr & new_type, const String & column_name, size_t row) const; const FormatSettings format_settings; std::unique_ptr reader; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 1bc5223a712..8ea379beae5 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -187,11 +188,6 @@ JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader( : FormatWithNamesAndTypesSchemaReader(in_, format_settings_, with_names_, with_types_, &reader) , reader(in_, yield_strings_, format_settings_) { - bool allow_bools_as_numbers = format_settings_.json.read_bools_as_numbers; - setCommonTypeChecker([allow_bools_as_numbers](const DataTypePtr & first, const DataTypePtr & second) - { - return JSONUtils::getCommonTypeForJSONFormats(first, second, allow_bools_as_numbers); - }); } DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypes() @@ -210,7 +206,12 @@ DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypes() if (in.eof()) return {}; - return JSONUtils::readRowAndGetDataTypesForJSONCompactEachRow(in, reader.yieldStrings()); + return JSONUtils::readRowAndGetDataTypesForJSONCompactEachRow(in, format_settings, reader.yieldStrings()); +} + +void JSONCompactEachRowRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) +{ + transformInferredJSONTypesIfNeeded(type, new_type, format_settings); } void registerInputFormatJSONCompactEachRow(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 79c76214774..7be9ba9289b 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -80,6 +80,8 @@ public: private: DataTypes readRowAndGetDataTypes() override; + void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) override; + JSONCompactEachRowFormatReader reader; bool first_row = true; }; diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 3bcea8a8843..7cb62f9bf83 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -306,18 +307,12 @@ void JSONEachRowRowInputFormat::readSuffix() assertEOF(*in); } -JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings) - : IRowWithNamesSchemaReader(in_) +JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings_) + : IRowWithNamesSchemaReader(in_, format_settings_) , json_strings(json_strings_) { - bool allow_bools_as_numbers = format_settings.json.read_bools_as_numbers; - setCommonTypeChecker([allow_bools_as_numbers](const DataTypePtr & first, const DataTypePtr & second) - { - return JSONUtils::getCommonTypeForJSONFormats(first, second, allow_bools_as_numbers); - }); } - NamesAndTypesList JSONEachRowSchemaReader::readRowAndGetNamesAndDataTypes(bool & eof) { if (first_row) @@ -350,7 +345,12 @@ NamesAndTypesList JSONEachRowSchemaReader::readRowAndGetNamesAndDataTypes(bool & return {}; } - return JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, json_strings); + return JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, json_strings); +} + +void JSONEachRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) +{ + transformInferredJSONTypesIfNeeded(type, new_type, format_settings); } void registerInputFormatJSONEachRow(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index 1673d55b9fd..e6db45bb056 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -92,6 +92,7 @@ public: private: NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) override; + void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; bool json_strings; bool first_row = true; diff --git a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp index 8e787edf8ab..8e1beb8ec89 100644 --- a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp @@ -402,7 +402,7 @@ void MySQLDumpRowInputFormat::skipField() } MySQLDumpSchemaReader::MySQLDumpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) - : IRowSchemaReader(in_, format_settings_), format_settings(format_settings_), table_name(format_settings_.mysql_dump.table_name) + : IRowSchemaReader(in_, format_settings_), table_name(format_settings_.mysql_dump.table_name) { } diff --git a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h index 2457f3d4762..6be20550e49 100644 --- a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h +++ b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h @@ -35,7 +35,6 @@ private: NamesAndTypesList readSchema() override; DataTypes readRowAndGetDataTypes() override; - const FormatSettings format_settings; String table_name; }; diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index d92f65f33d1..c6150863bd4 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -133,7 +133,6 @@ RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & buf, format_settings_, getDefaultDataTypeForEscapingRule(format_settings_.regexp.escaping_rule)) - , format_settings(format_settings_) , field_extractor(format_settings) , buf(in_) { @@ -157,6 +156,12 @@ DataTypes RegexpSchemaReader::readRowAndGetDataTypes() return data_types; } +void RegexpSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) +{ + transformInferredTypesIfNeeded(type, new_type, format_settings, format_settings.regexp.escaping_rule); +} + + void registerInputFormatRegexp(FormatFactory & factory) { factory.registerInputFormat("Regexp", []( diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/src/Processors/Formats/Impl/RegexpRowInputFormat.h index 3cc6a3192fd..7fbb3fc320f 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -81,8 +81,10 @@ public: private: DataTypes readRowAndGetDataTypes() override; + void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t) override; + + using EscapingRule = FormatSettings::EscapingRule; - const FormatSettings format_settings; RegexpFieldExtractor field_extractor; PeekableReadBuffer buf; }; diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index fe2c0c5ecdd..7393a1d6ce6 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -214,8 +214,7 @@ void TSKVRowInputFormat::resetParser() } TSKVSchemaReader::TSKVSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) - : IRowWithNamesSchemaReader(in_, getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::Escaped)) - , format_settings(format_settings_) + : IRowWithNamesSchemaReader(in_, format_settings_, getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::Escaped)) { } diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.h b/src/Processors/Formats/Impl/TSKVRowInputFormat.h index bf8580bc6b7..5130ee5e827 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.h +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.h @@ -61,7 +61,6 @@ public: private: NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) override; - const FormatSettings format_settings; bool first_row = true; }; diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index df4d49b172c..6e8bba89d8c 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -458,7 +458,6 @@ TemplateSchemaReader::TemplateSchemaReader( , buf(in_) , format(format_) , row_format(row_format_) - , format_settings(format_settings_) , format_reader(buf, ignore_spaces_, format, row_format, row_between_delimiter, format_settings) { setColumnNames(row_format.column_names); @@ -494,6 +493,11 @@ DataTypes TemplateSchemaReader::readRowAndGetDataTypes() return data_types; } +void TemplateSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t column_idx) +{ + transformInferredTypesIfNeeded(type, new_type, format_settings, row_format.escaping_rules[column_idx]); +} + static ParsedTemplateFormatString fillResultSetFormat(const FormatSettings & settings) { ParsedTemplateFormatString resultset_format; diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/src/Processors/Formats/Impl/TemplateRowInputFormat.h index ab7043f057e..740683ad95d 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.h +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -121,10 +121,11 @@ public: DataTypes readRowAndGetDataTypes() override; private: + void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t column_idx) override; + PeekableReadBuffer buf; const ParsedTemplateFormatString format; const ParsedTemplateFormatString row_format; - FormatSettings format_settings; TemplateFormatReader format_reader; bool first_row = true; }; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 41f77f8bbf2..49b758b78c4 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -567,7 +567,7 @@ void ValuesBlockInputFormat::setReadBuffer(ReadBuffer & in_) } ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) - : IRowSchemaReader(buf, format_settings_), buf(in_), format_settings(format_settings_) + : IRowSchemaReader(buf, format_settings_), buf(in_) { } diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index 9653e431b4e..bf243c54bd7 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -103,7 +103,6 @@ private: DataTypes readRowAndGetDataTypes() override; PeekableReadBuffer buf; - const FormatSettings format_settings; ParserExpression parser; bool first_row = true; bool end_of_data = false; diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index a3dcbe914bb..9ff227e5dab 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -229,12 +229,12 @@ void RowInputFormatWithNamesAndTypes::setReadBuffer(ReadBuffer & in_) FormatWithNamesAndTypesSchemaReader::FormatWithNamesAndTypesSchemaReader( ReadBuffer & in_, - const FormatSettings & format_settings, + const FormatSettings & format_settings_, bool with_names_, bool with_types_, FormatWithNamesAndTypesReader * format_reader_, DataTypePtr default_type_) - : IRowSchemaReader(in_, format_settings, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_) + : IRowSchemaReader(in_, format_settings_, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_) { } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 9fc8b2083df..0afc0146846 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -123,7 +123,7 @@ class FormatWithNamesAndTypesSchemaReader : public IRowSchemaReader public: FormatWithNamesAndTypesSchemaReader( ReadBuffer & in, - const FormatSettings & format_settings, + const FormatSettings & format_settings_, bool with_names_, bool with_types_, FormatWithNamesAndTypesReader * format_reader_, diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 0490c0c1f0d..a9e6cbda67c 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -537,7 +537,7 @@ def test_schema_inference_with_globs(started_cluster): ) result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/data*.jsoncompacteachrow')") - assert result.strip() == "c1\tNullable(Float64)" + assert result.strip() == "c1\tNullable(Int64)" result = node1.query( f"select * from hdfs('hdfs://hdfs1:9000/data*.jsoncompacteachrow')" diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 7d22cb24a5c..a09821137c4 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1307,7 +1307,7 @@ def test_schema_inference_from_globs(started_cluster): result = instance.query( f"desc url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{url_filename}')" ) - assert result.strip() == "c1\tNullable(Float64)" + assert result.strip() == "c1\tNullable(Int64)" result = instance.query( f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{url_filename}')" @@ -1317,7 +1317,7 @@ def test_schema_inference_from_globs(started_cluster): result = instance.query( f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test*.jsoncompacteachrow')" ) - assert result.strip() == "c1\tNullable(Float64)" + assert result.strip() == "c1\tNullable(Int64)" result = instance.query( f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test*.jsoncompacteachrow')" diff --git a/tests/queries/0_stateless/01825_type_json_schema_inference.reference b/tests/queries/0_stateless/01825_type_json_schema_inference.reference index a2089ea3366..22dfdee96a7 100644 --- a/tests/queries/0_stateless/01825_type_json_schema_inference.reference +++ b/tests/queries/0_stateless/01825_type_json_schema_inference.reference @@ -3,6 +3,6 @@ Tuple(k1 Nullable(Int8), k2 Tuple(k3 Nullable(String), k4 Nested(k5 Nullable(Int8), k6 Nullable(Int8))), some Nullable(Int8)) {"id":"1","obj":"aaa","s":"foo"} {"id":"2","obj":"bbb","s":"bar"} -{"map":{"k1":1,"k2":2},"obj":{"k1":1,"k2.k3":2},"map_type":"Map(String, Nullable(Float64))","obj_type":"Object(Nullable('json'))"} -{"obj":{"k1":1,"k2":2},"map":{"k1":"1","k2":"2"}} -Tuple(k1 Float64, k2 Float64) +{"map":{"k1":"1","k2":"2"},"obj":{"k1":1,"k2.k3":2},"map_type":"Map(String, Nullable(Int64))","obj_type":"Object(Nullable('json'))"} +{"obj":{"k1":"1","k2":"2"},"map":{"k1":"1","k2":"2"}} +Tuple(k1 Int64, k2 Int64) diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference index 2d7dd5caca7..954e1813157 100644 --- a/tests/queries/0_stateless/02149_schema_inference.reference +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -1,22 +1,22 @@ TSV -c1 Nullable(Float64) +c1 Nullable(Int64) c2 Nullable(String) -c3 Array(Nullable(Float64)) -c4 Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64)) +c3 Array(Nullable(Int64)) +c4 Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) 42 Some string [1,2,3,4] (1,2,3) 42 abcd [] (4,5,6) TSVWithNames -number Nullable(Float64) +number Nullable(Int64) string Nullable(String) -array Array(Nullable(Float64)) -tuple Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64)) +array Array(Nullable(Int64)) +tuple Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) 42 Some string [1,2,3,4] (1,2,3) 42 abcd [] (4,5,6) CSV -c1 Nullable(Float64) +c1 Nullable(Int64) c2 Nullable(String) c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) -c4 Array(Nullable(Float64)) +c4 Array(Nullable(Int64)) \N Some string [([1,2.3],'String'),([],NULL)] [1,NULL,3] 42 \N [([1,2.3],'String'),([3],'abcd')] [4,5,6] c1 Nullable(String) @@ -28,54 +28,54 @@ c2 Nullable(String) \N [NULL, NULL] \N [] CSVWithNames -a Nullable(Float64) +a Nullable(Int64) b Nullable(String) c Array(Tuple(Array(Nullable(Float64)), Nullable(String))) -d Array(Nullable(Float64)) +d Array(Nullable(Int64)) \N Some string [([1,2.3],'String'),([],NULL)] [1,NULL,3] 42 \N [([1,2.3],'String'),([3],'abcd')] [4,5,6] JSONCompactEachRow c1 Nullable(Float64) -c2 Array(Tuple(Nullable(Float64), Nullable(String))) -c3 Map(String, Nullable(Float64)) +c2 Array(Tuple(Nullable(Int64), Nullable(String))) +c3 Map(String, Nullable(Int64)) c4 Nullable(Bool) 42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true -c1 Nullable(Float64) -c2 Array(Tuple(Nullable(Float64), Nullable(String))) -c3 Map(String, Nullable(Float64)) +c1 Nullable(Int64) +c2 Array(Tuple(Nullable(Int64), Nullable(String))) +c3 Map(String, Nullable(Int64)) c4 Nullable(Bool) \N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N 32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} true JSONCompactEachRowWithNames a Nullable(Float64) -b Array(Tuple(Nullable(Float64), Nullable(String))) -c Map(String, Nullable(Float64)) +b Array(Tuple(Nullable(Int64), Nullable(String))) +c Map(String, Nullable(Int64)) d Nullable(Bool) 42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true JSONEachRow a Nullable(Float64) -b Array(Tuple(Nullable(Float64), Nullable(String))) -c Map(String, Nullable(Float64)) +b Array(Tuple(Nullable(Int64), Nullable(String))) +c Map(String, Nullable(Int64)) d Nullable(Bool) 42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true -a Nullable(Float64) -b Array(Tuple(Nullable(Float64), Nullable(String))) -c Map(String, Nullable(Float64)) +a Nullable(Int64) +b Array(Tuple(Nullable(Int64), Nullable(String))) +c Map(String, Nullable(Int64)) d Nullable(Bool) \N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N 32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} true -a Nullable(Float64) +a Nullable(Int64) b Nullable(String) -c Array(Nullable(Float64)) +c Array(Nullable(Int64)) 1 s1 [] 2 \N [2] \N \N [] \N \N [] \N \N [3] TSKV -a Nullable(Float64) +a Nullable(Int64) b Nullable(String) -c Array(Nullable(Float64)) +c Array(Nullable(Int64)) 1 s1 [] 2 } [2] \N \N [] @@ -84,77 +84,77 @@ c Array(Nullable(Float64)) Values c1 Nullable(Float64) c2 Nullable(String) -c3 Array(Nullable(Float64)) -c4 Tuple(Nullable(Float64), Nullable(String)) -c5 Tuple(Array(Nullable(Float64)), Array(Tuple(Nullable(Float64), Nullable(String)))) +c3 Array(Nullable(Int64)) +c4 Tuple(Nullable(Int64), Nullable(String)) +c5 Tuple(Array(Nullable(Int64)), Array(Tuple(Nullable(Int64), Nullable(String)))) 42.42 Some string [1,2,3] (1,'2') ([1,2],[(3,'4'),(5,'6')]) c1 Nullable(Float64) c2 Nullable(String) -c3 Array(Nullable(Float64)) -c4 Tuple(Nullable(Float64), Nullable(Float64)) -c5 Tuple(Array(Nullable(Float64)), Array(Tuple(Nullable(Float64), Nullable(String)))) +c3 Array(Nullable(Int64)) +c4 Tuple(Nullable(Int64), Nullable(Int64)) +c5 Tuple(Array(Nullable(Int64)), Array(Tuple(Nullable(Int64), Nullable(String)))) 42.42 \N [1,NULL,3] (1,NULL) ([1,2],[(3,'4'),(5,'6')]) \N Some string [10] (1,2) ([],[]) Regexp -c1 Nullable(Float64) +c1 Nullable(Int64) c2 Nullable(String) c3 Nullable(String) 42 Some string 1 [([1, 2, 3], String 1), ([], String 1)] 2 Some string 2 [([4, 5, 6], String 2), ([], String 2)] 312 Some string 3 [([1, 2, 3], String 2), ([], String 2)] -c1 Nullable(Float64) +c1 Nullable(Int64) c2 Nullable(String) -c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 42 Some string 1 [([1,2,3],'String 1'),([],'String 1')] 3 Some string 2 [([3,5,1],'String 2'),([],'String 2')] 244 Some string 3 [([],'String 3'),([],'String 3')] -c1 Nullable(Float64) +c1 Nullable(Int64) c2 Nullable(String) -c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 42 Some string 1 [([1,2,3],'String 1'),([],'String 1')] 2 Some string 2 [([],'String 2'),([],'String 2')] 43 Some string 3 [([1,5,3],'String 3'),([],'String 3')] -c1 Nullable(Float64) +c1 Nullable(Int64) c2 Nullable(String) -c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 52 Some string 2 [([],'String 2'),([1],'String 2')] 24 Some string 3 [([1,2,3],'String 3'),([1],'String 3')] CustomSeparated c1 Nullable(Float64) c2 Nullable(String) -c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] c1 Nullable(Float64) c2 Nullable(String) -c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] c1 Nullable(Float64) c2 Nullable(String) -c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] Template column_1 Nullable(Float64) column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +column_3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] column_1 Nullable(Float64) column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +column_3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] column_1 Nullable(Float64) column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +column_3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] diff --git a/tests/queries/0_stateless/02188_table_function_format.reference b/tests/queries/0_stateless/02188_table_function_format.reference index 403a4044544..5d7febda187 100644 --- a/tests/queries/0_stateless/02188_table_function_format.reference +++ b/tests/queries/0_stateless/02188_table_function_format.reference @@ -39,9 +39,9 @@ World 123 Hello 111 World 123 1 2 [1,2,3] [['abc'],[],['d','e']] -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Array(Nullable(Float64)) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Array(Nullable(Int64)) c4 Array(Array(Nullable(String))) Hello 111 World 123 @@ -49,4 +49,4 @@ Hello 111 Hello 131 World 123 a Nullable(String) -b Nullable(Float64) +b Nullable(Int64) diff --git a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference index 6920aa16198..4a4389f638e 100644 --- a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference +++ b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference @@ -1,4 +1,4 @@ -x Nullable(Float64) +x Nullable(Int64) 0 1 2 @@ -9,7 +9,7 @@ x Nullable(Float64) 7 8 9 -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) 1 2 3 diff --git a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.reference b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.reference index f32b0eb8a92..effc3644b41 100644 --- a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.reference +++ b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.reference @@ -1,3 +1,3 @@ -CREATE TABLE default.test\n(\n `x` Nullable(Float64),\n `y` Nullable(String)\n)\nENGINE = File(\'JSONEachRow\', \'data.jsonl\') +CREATE TABLE default.test\n(\n `x` Nullable(Int64),\n `y` Nullable(String)\n)\nENGINE = File(\'JSONEachRow\', \'data.jsonl\') OK OK diff --git a/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference b/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference index d0ced74f8f6..0f8ac77ff74 100644 --- a/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference +++ b/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference @@ -1,6 +1,6 @@ -a Nullable(Float64) +a Nullable(Int64) b Nullable(String) -c Array(Nullable(Float64)) +c Array(Nullable(Int64)) 1 s1 [] 2 } [2] \N \N [] diff --git a/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.reference b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.reference index d237caf630f..7a124346b9f 100644 --- a/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.reference +++ b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.reference @@ -1,8 +1,8 @@ x Nullable(String) -y Nullable(Float64) +y Nullable(Int64) x Nullable(String) -y Nullable(Float64) +y Nullable(Int64) x Nullable(String) -y Nullable(Float64) +y Nullable(Int64) x Nullable(String) -y Nullable(Float64) +y Nullable(Int64) diff --git a/tests/queries/0_stateless/02244_hdfs_cluster.reference b/tests/queries/0_stateless/02244_hdfs_cluster.reference index 4bf4799e904..32a9f24388c 100644 --- a/tests/queries/0_stateless/02244_hdfs_cluster.reference +++ b/tests/queries/0_stateless/02244_hdfs_cluster.reference @@ -22,24 +22,24 @@ 1 2 3 4 5 6 7 8 9 -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) c1 UInt32 c2 UInt32 c3 UInt32 c1 UInt32 c2 UInt32 c3 UInt32 -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) c1 UInt32 c2 UInt32 c3 UInt32 diff --git a/tests/queries/0_stateless/02245_s3_schema_desc.reference b/tests/queries/0_stateless/02245_s3_schema_desc.reference index e039680d933..d840a365310 100644 --- a/tests/queries/0_stateless/02245_s3_schema_desc.reference +++ b/tests/queries/0_stateless/02245_s3_schema_desc.reference @@ -1,21 +1,21 @@ -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) c1 UInt64 c2 UInt64 c3 UInt64 -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) c1 UInt64 c2 UInt64 c3 UInt64 -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) c1 UInt64 c2 UInt64 c3 UInt64 diff --git a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference index c245f13fdbe..1c60e40942c 100644 --- a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference +++ b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference @@ -1,8 +1,8 @@ TSV -c1 Nullable(Float64) +c1 Nullable(Int64) c2 Nullable(String) -c3 Array(Nullable(Float64)) -c4 Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64)) +c3 Array(Nullable(Int64)) +c4 Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) 42 Some string [1,2,3,4] (1,2,3) 42 abcd [] (4,5,6) c1 Nullable(String) @@ -70,8 +70,8 @@ c1 Nullable(String) CSV c1 Nullable(String) c2 Nullable(String) -c3 Array(Nullable(Float64)) -c4 Array(Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64))) +c3 Array(Nullable(Int64)) +c4 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) 42 Some string [1,2,3,4] [(1,2,3)] 42\\ abcd [] [(4,5,6)] c1 Nullable(String) @@ -101,7 +101,7 @@ c1 Nullable(String) (1, 2, 3) c1 Nullable(String) 123.123 -c1 Array(Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64))) +c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) [(1,2,3)] -c1 Array(Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64))) +c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) [(1,2,3)] diff --git a/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.reference b/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.reference index 300846c17a0..c7774c8138b 100644 --- a/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.reference +++ b/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.reference @@ -1,32 +1,32 @@ -a Nullable(Float64) +a Nullable(Int64) b Nullable(String) -c Array(Nullable(Float64)) +c Array(Nullable(Int64)) 1 s1 [] 2 } [2] \N \N [] \N \N [] \N \N [3] -b Nullable(Float64) -a Nullable(Float64) -c Nullable(Float64) -e Nullable(Float64) +b Nullable(Int64) +a Nullable(Int64) +c Nullable(Int64) +e Nullable(Int64) 1 \N \N \N \N 2 3 \N \N \N \N \N \N \N \N 3 3 3 1 \N -a Nullable(Float64) +a Nullable(Int64) b Nullable(String) -c Array(Nullable(Float64)) +c Array(Nullable(Int64)) 1 s1 [] 2 \N [2] \N \N [] \N \N [] \N \N [3] -b Nullable(Float64) -a Nullable(Float64) -c Nullable(Float64) -e Nullable(Float64) +b Nullable(Int64) +a Nullable(Int64) +c Nullable(Int64) +e Nullable(Int64) 1 \N \N \N \N 2 3 \N \N \N \N \N diff --git a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference index a7609bdd86b..840e77cb122 100644 --- a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference +++ b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference @@ -7,6 +7,12 @@ x Nullable(Float64) x Nullable(Float64) 1 0.42 +x Array(Nullable(Float64)) +[1,0] +[0.42] +x Array(Array(Nullable(Int64))) +[[1,2,3],[1,0],[1,1,0]] +[[1,2,3]] c1 Nullable(Bool) true false @@ -16,3 +22,9 @@ c1 Nullable(Float64) c1 Nullable(Float64) 1 0.42 +c1 Array(Nullable(Float64)) +[1,0] +[0.42] +c1 Array(Array(Nullable(Int64))) +[[1,2,3],[1,0],[1,1,0]] +[[1,2,3]] diff --git a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh index 10f050ea6d1..1b689aaf577 100755 --- a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh +++ b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh @@ -27,6 +27,16 @@ echo -e '{"x" : true} $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" +echo -e '{"x" : [true, false]} +{"x" : [0.42]}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + +echo -e '{"x" : [[1, 2, 3], [true, false], [1, true, false]]} +{"x" : [[1, 2, 3]]}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + echo -e '[true] [false]' > $DATA_FILE @@ -43,5 +53,14 @@ echo -e '[true] $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" +echo -e '[[true, false]] +[[0.42]]' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" + +echo -e '[[[1, 2, 3], [true, false], [1, true, false]]] +[[[1, 2, 3]]]' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" rm $DATA_FILE diff --git a/tests/queries/0_stateless/02268_json_maps_and_objects.reference b/tests/queries/0_stateless/02268_json_maps_and_objects.reference index cfdc6e7e55c..87fb8949e01 100644 --- a/tests/queries/0_stateless/02268_json_maps_and_objects.reference +++ b/tests/queries/0_stateless/02268_json_maps_and_objects.reference @@ -1,2 +1,5 @@ x Object(Nullable(\'json\')) x Object(Nullable(\'json\')) +x Array(Object(Nullable(\'json\'))) +x Array(Object(Nullable(\'json\'))) +x Tuple(Map(String, Nullable(String)), Map(String, Array(Nullable(Int64))), Array(Nullable(Int64))) diff --git a/tests/queries/0_stateless/02268_json_maps_and_objects.sql b/tests/queries/0_stateless/02268_json_maps_and_objects.sql index 83d8fbaac2d..8a9ede6876c 100644 --- a/tests/queries/0_stateless/02268_json_maps_and_objects.sql +++ b/tests/queries/0_stateless/02268_json_maps_and_objects.sql @@ -1,3 +1,6 @@ -- Tags: no-fasttest desc format(JSONEachRow, '{"x" : {"a" : "Some string"}}, {"x" : {"b" : [1, 2, 3]}}, {"x" : {"c" : {"d" : 10}}}'); desc format(JSONEachRow, '{"x" : {"a" : "Some string"}}, {"x" : {"b" : [1, 2, 3], "c" : {"42" : 42}}}'); +desc format(JSONEachRow, '{"x" : [{"a" : "Some string"}]}, {"x" : [{"b" : [1, 2, 3]}]}'); +desc format(JSONEachRow, '{"x" : [{"a" : "Some string"}, {"b" : [1, 2, 3]}]}'); +desc format(JSONEachRow, '{"x" : [{"a" : "Some string"}, {"b" : [1, 2, 3]}, [1, 2, 3]]}'); diff --git a/tests/queries/0_stateless/02286_mysql_dump_input_format.reference b/tests/queries/0_stateless/02286_mysql_dump_input_format.reference index 25be4b727bc..a736358b9b7 100644 --- a/tests/queries/0_stateless/02286_mysql_dump_input_format.reference +++ b/tests/queries/0_stateless/02286_mysql_dump_input_format.reference @@ -130,8 +130,8 @@ x Nullable(Int32) x Nullable(Int32) 1 dump7 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -139,8 +139,8 @@ c2 Nullable(Float64) 4 \N 5 \N 6 7 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -148,15 +148,15 @@ c2 Nullable(Float64) 4 \N 5 \N 6 7 -c1 Nullable(Float64) +c1 Nullable(Int64) 1 2 3 -c1 Nullable(Float64) +c1 Nullable(Int64) 1 dump8 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -164,8 +164,8 @@ c2 Nullable(Float64) 4 \N 5 \N 6 7 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -174,8 +174,8 @@ c2 Nullable(Float64) 5 \N 6 7 dump9 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -183,8 +183,8 @@ c2 Nullable(Float64) 4 \N 5 \N 6 7 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -193,8 +193,8 @@ c2 Nullable(Float64) 5 \N 6 7 dump10 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -202,8 +202,8 @@ c2 Nullable(Float64) 4 \N 5 \N 6 7 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -212,8 +212,8 @@ c2 Nullable(Float64) 5 \N 6 7 dump11 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -221,8 +221,8 @@ c2 Nullable(Float64) 4 \N 5 \N 6 7 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) 1 \N 2 \N 3 \N @@ -265,8 +265,8 @@ color Nullable(String) price Nullable(Int32) apple red 42 dump14 -x Nullable(Float64) -y Nullable(Float64) +x Nullable(Int64) +y Nullable(Int64) 1 \N 2 \N 3 \N @@ -274,8 +274,8 @@ y Nullable(Float64) 4 \N 5 \N 6 7 -x Nullable(Float64) -y Nullable(Float64) +x Nullable(Int64) +y Nullable(Int64) 1 \N 2 \N 3 \N @@ -283,15 +283,15 @@ y Nullable(Float64) 4 \N 5 \N 6 7 -x Nullable(Float64) +x Nullable(Int64) 1 2 3 -x Nullable(Float64) +x Nullable(Int64) 1 dump15 -x Nullable(Float64) -y Nullable(Float64) +x Nullable(Int64) +y Nullable(Int64) 1 \N 2 \N 3 \N @@ -299,8 +299,8 @@ y Nullable(Float64) 4 \N 5 \N 6 7 -x Nullable(Float64) -y Nullable(Float64) +x Nullable(Int64) +y Nullable(Int64) 1 \N 2 \N 3 \N @@ -308,10 +308,10 @@ y Nullable(Float64) 4 \N 5 \N 6 7 -x Nullable(Float64) +x Nullable(Int64) 1 2 3 -x Nullable(Float64) -y Nullable(Float64) +x Nullable(Int64) +y Nullable(Int64) 1 2 diff --git a/tests/queries/0_stateless/02293_formats_json_columns.reference b/tests/queries/0_stateless/02293_formats_json_columns.reference index da8d080ac05..f59f02ad42b 100644 --- a/tests/queries/0_stateless/02293_formats_json_columns.reference +++ b/tests/queries/0_stateless/02293_formats_json_columns.reference @@ -4,9 +4,9 @@ JSONColumns "b": ["String", "String", "String", "String", "String"], "c": [[[[],"String"],[[],"gnirtS"]], [[[0],"String"],[[0],"gnirtS"]], [[[0,1],"String"],[[0,1],"gnirtS"]], [[[],"String"],[[0,1,2],"gnirtS"]], [[[0],"String"],[[],"gnirtS"]]] } -a Nullable(Float64) +a Nullable(Int64) b Nullable(String) -c Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 0 String [([],'String'),([],'gnirtS')] 1 String [([0],'String'),([0],'gnirtS')] 2 String [([0,1],'String'),([0,1],'gnirtS')] @@ -18,9 +18,9 @@ JSONCompactColumns ["String", "String", "String", "String", "String"], [[[[],"String"],[[],"gnirtS"]], [[[0],"String"],[[0],"gnirtS"]], [[[0,1],"String"],[[0,1],"gnirtS"]], [[[],"String"],[[0,1,2],"gnirtS"]], [[[0],"String"],[[],"gnirtS"]]] ] -c1 Nullable(Float64) +c1 Nullable(Int64) c2 Nullable(String) -c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c3 Array(Tuple(Array(Nullable(Int64)), Nullable(String))) 0 String [([],'String'),([],'gnirtS')] 1 String [([0],'String'),([0],'gnirtS')] 2 String [([0,1],'String'),([0,1],'gnirtS')] @@ -74,9 +74,9 @@ JSONColumnsWithMetadata "bytes_read": 20 } } -b Nullable(Float64) -a Nullable(Float64) -c Nullable(Float64) +b Nullable(Int64) +a Nullable(Int64) +c Nullable(Int64) d Nullable(String) 1 3 \N \N 2 2 \N \N @@ -89,8 +89,8 @@ OK 3 2 1 -c1 Nullable(Float64) -c2 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) c3 Nullable(String) 1 1 \N 2 2 \N diff --git a/tests/queries/0_stateless/02305_schema_inference_with_globs.reference b/tests/queries/0_stateless/02305_schema_inference_with_globs.reference index 9df5d2a264c..defa2133823 100644 --- a/tests/queries/0_stateless/02305_schema_inference_with_globs.reference +++ b/tests/queries/0_stateless/02305_schema_inference_with_globs.reference @@ -1,5 +1,5 @@ 2 4 6 -x Nullable(String) -x Nullable(String) +x Nullable(Int64) +x Nullable(Int64) diff --git a/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.reference b/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.reference index 7d8e0c662cd..4274f5769ee 100644 --- a/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.reference +++ b/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.reference @@ -1,14 +1,14 @@ -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) 0 1 2 1 2 3 2 3 4 3 4 5 4 5 6 -c1 Nullable(Float64) -c2 Nullable(Float64) -c3 Nullable(Float64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) 0 1 2 1 2 3 2 3 4 diff --git a/tests/queries/0_stateless/02325_dates_schema_inference.reference b/tests/queries/0_stateless/02325_dates_schema_inference.reference new file mode 100644 index 00000000000..3ac4ad88f1c --- /dev/null +++ b/tests/queries/0_stateless/02325_dates_schema_inference.reference @@ -0,0 +1,60 @@ +JSONEachRow +x Nullable(Date) +x Nullable(DateTime64(9)) +x Nullable(DateTime64(9)) +x Array(Nullable(Date)) +x Array(Nullable(DateTime64(9))) +x Array(Nullable(DateTime64(9))) +x Map(String, Nullable(DateTime64(9))) +x Array(Nullable(DateTime64(9))) +x Array(Nullable(DateTime64(9))) +x Nullable(DateTime64(9)) +x Array(Nullable(String)) +x Nullable(String) +x Array(Nullable(String)) +x Map(String, Array(Array(Nullable(String)))) +CSV +c1 Nullable(Date) +c1 Nullable(DateTime64(9)) +c1 Nullable(DateTime64(9)) +c1 Array(Nullable(Date)) +c1 Array(Nullable(DateTime64(9))) +c1 Array(Nullable(DateTime64(9))) +c1 Map(String, Nullable(DateTime64(9))) +c1 Array(Nullable(DateTime64(9))) +c1 Array(Nullable(DateTime64(9))) +c1 Nullable(DateTime64(9)) +c1 Array(Nullable(String)) +c1 Nullable(String) +c1 Array(Nullable(String)) +c1 Map(String, Array(Array(Nullable(String)))) +TSV +c1 Nullable(Date) +c1 Nullable(DateTime64(9)) +c1 Nullable(DateTime64(9)) +c1 Array(Nullable(Date)) +c1 Array(Nullable(DateTime64(9))) +c1 Array(Nullable(DateTime64(9))) +c1 Map(String, Nullable(DateTime64(9))) +c1 Array(Nullable(DateTime64(9))) +c1 Array(Nullable(DateTime64(9))) +c1 Nullable(DateTime64(9)) +c1 Array(Nullable(String)) +c1 Nullable(String) +c1 Array(Nullable(String)) +c1 Map(String, Array(Array(Nullable(String)))) +Values +c1 Nullable(Date) +c1 Nullable(DateTime64(9)) +c1 Nullable(DateTime64(9)) +c1 Array(Nullable(Date)) +c1 Array(Nullable(DateTime64(9))) +c1 Array(Nullable(DateTime64(9))) +c1 Map(String, Nullable(DateTime64(9))) +c1 Array(Nullable(DateTime64(9))) +c1 Array(Nullable(DateTime64(9))) +c1 Nullable(DateTime64(9)) +c1 Array(Nullable(String)) +c1 Nullable(String) +c1 Array(Nullable(String)) +c1 Map(String, Array(Array(Nullable(String)))) diff --git a/tests/queries/0_stateless/02325_dates_schema_inference.sql b/tests/queries/0_stateless/02325_dates_schema_inference.sql new file mode 100644 index 00000000000..3534a0eb48f --- /dev/null +++ b/tests/queries/0_stateless/02325_dates_schema_inference.sql @@ -0,0 +1,70 @@ +-- Tags: no-fasttest + +set input_format_try_infer_dates=1; +set input_format_try_infer_datetimes=1; + +select 'JSONEachRow'; +desc format(JSONEachRow, '{"x" : "2020-01-01"}'); +desc format(JSONEachRow, '{"x" : "2020-01-01 00:00:00.00000"}'); +desc format(JSONEachRow, '{"x" : "2020-01-01 00:00:00"}'); +desc format(JSONEachRow, '{"x" : ["2020-01-01", "2020-01-02"]}'); +desc format(JSONEachRow, '{"x" : ["2020-01-01", "2020-01-01 00:00:00"]}'); +desc format(JSONEachRow, '{"x" : ["2020-01-01 00:00:00", "2020-01-01 00:00:00"]}'); +desc format(JSONEachRow, '{"x" : {"date1" : "2020-01-01 00:00:00", "date2" : "2020-01-01"}}'); +desc format(JSONEachRow, '{"x" : ["2020-01-01 00:00:00", "2020-01-01"]}\n{"x" : ["2020-01-01"]}'); +desc format(JSONEachRow, '{"x" : ["2020-01-01 00:00:00"]}\n{"x" : ["2020-01-01"]}'); +desc format(JSONEachRow, '{"x" : "2020-01-01 00:00:00"}\n{"x" : "2020-01-01"}'); +desc format(JSONEachRow, '{"x" : ["2020-01-01 00:00:00", "Some string"]}'); +desc format(JSONEachRow, '{"x" : "2020-01-01 00:00:00"}\n{"x" : "Some string"}'); +desc format(JSONEachRow, '{"x" : ["2020-01-01 00:00:00", "2020-01-01"]}\n{"x" : ["2020-01-01", "Some string"]}'); +desc format(JSONEachRow, '{"x" : {"key1" : [["2020-01-01 00:00:00"]], "key2" : [["2020-01-01"]]}}\n{"x" : {"key1" : [["2020-01-01"]], "key2" : [["Some string"]]}}'); + +select 'CSV'; +desc format(CSV, '"2020-01-01"'); +desc format(CSV, '"2020-01-01 00:00:00.00000"'); +desc format(CSV, '"2020-01-01 00:00:00"'); +desc format(CSV, '"[\'2020-01-01\', \'2020-01-02\']"'); +desc format(CSV, '"[\'2020-01-01\', \'2020-01-01 00:00:00\']"'); +desc format(CSV, '"[\'2020-01-01 00:00:00\', \'2020-01-01 00:00:00\']"'); +desc format(CSV, '"{\'date1\' : \'2020-01-01 00:00:00\', \'date2\' : \'2020-01-01\'}"'); +desc format(CSV, '"[\'2020-01-01 00:00:00\', \'2020-01-01\']"\n"[\'2020-01-01\']"'); +desc format(CSV, '"[\'2020-01-01 00:00:00\']"\n"[\'2020-01-01\']"'); +desc format(CSV, '"2020-01-01 00:00:00"\n"2020-01-01"'); +desc format(CSV, '"[\'2020-01-01 00:00:00\', \'Some string\']"'); +desc format(CSV, '"2020-01-01 00:00:00"\n"Some string"'); +desc format(CSV, '"[\'2020-01-01 00:00:00\', \'2020-01-01\']"\n"[\'2020-01-01\', \'Some string\']"'); +desc format(CSV, '"{\'key1\' : [[\'2020-01-01 00:00:00\']], \'key2\' : [[\'2020-01-01\']]}"\n"{\'key1\' : [[\'2020-01-01\']], \'key2\' : [[\'Some string\']]}"'); + +select 'TSV'; +desc format(TSV, '2020-01-01'); +desc format(TSV, '2020-01-01 00:00:00.00000'); +desc format(TSV, '2020-01-01 00:00:00'); +desc format(TSV, '[\'2020-01-01\', \'2020-01-02\']'); +desc format(TSV, '[\'2020-01-01\', \'2020-01-01 00:00:00\']'); +desc format(TSV, '[\'2020-01-01 00:00:00\', \'2020-01-01 00:00:00\']'); +desc format(TSV, '{\'date1\' : \'2020-01-01 00:00:00\', \'date2\' : \'2020-01-01\'}'); +desc format(TSV, '[\'2020-01-01 00:00:00\', \'2020-01-01\']\n[\'2020-01-01\']'); +desc format(TSV, '[\'2020-01-01 00:00:00\']\n[\'2020-01-01\']'); +desc format(TSV, '2020-01-01 00:00:00\n2020-01-01'); +desc format(TSV, '[\'2020-01-01 00:00:00\', \'Some string\']'); +desc format(TSV, '2020-01-01 00:00:00\nSome string'); +desc format(TSV, '[\'2020-01-01 00:00:00\', \'2020-01-01\']\n[\'2020-01-01\', \'Some string\']'); +desc format(TSV, '{\'key1\' : [[\'2020-01-01 00:00:00\']], \'key2\' : [[\'2020-01-01\']]}\n{\'key1\' : [[\'2020-01-01\']], \'key2\' : [[\'Some string\']]}'); + +select 'Values'; +desc format(Values, '(\'2020-01-01\')'); +desc format(Values, '(\'2020-01-01 00:00:00.00000\')'); +desc format(Values, '(\'2020-01-01 00:00:00\')'); +desc format(Values, '([\'2020-01-01\', \'2020-01-02\'])'); +desc format(Values, '([\'2020-01-01\', \'2020-01-01 00:00:00\'])'); +desc format(Values, '([\'2020-01-01 00:00:00\', \'2020-01-01 00:00:00\'])'); +desc format(Values, '({\'date1\' : \'2020-01-01 00:00:00\', \'date2\' : \'2020-01-01\'})'); +desc format(Values, '([\'2020-01-01 00:00:00\', \'2020-01-01\'])\n([\'2020-01-01\'])'); +desc format(Values, '([\'2020-01-01 00:00:00\']), ([\'2020-01-01\'])'); +desc format(Values, '(\'2020-01-01 00:00:00\')\n(\'2020-01-01\')'); +desc format(Values, '([\'2020-01-01 00:00:00\', \'Some string\'])'); +desc format(Values, '(\'2020-01-01 00:00:00\')\n(\'Some string\')'); +desc format(Values, '([\'2020-01-01 00:00:00\', \'2020-01-01\'])\n([\'2020-01-01\', \'Some string\'])'); +desc format(Values, '({\'key1\' : [[\'2020-01-01 00:00:00\']], \'key2\' : [[\'2020-01-01\']]})\n({\'key1\' : [[\'2020-01-01\']], \'key2\' : [[\'Some string\']]})'); + + diff --git a/tests/queries/0_stateless/02326_numbers_from_json_strings_schema_inference.reference b/tests/queries/0_stateless/02326_numbers_from_json_strings_schema_inference.reference new file mode 100644 index 00000000000..6da939d7839 --- /dev/null +++ b/tests/queries/0_stateless/02326_numbers_from_json_strings_schema_inference.reference @@ -0,0 +1,17 @@ +x Nullable(Int64) +x Array(Nullable(Float64)) +x Map(String, Nullable(Int64)) +x Map(String, Array(Nullable(Int64))) +x Nullable(Int64) +x Array(Nullable(Int64)) +x Map(String, Nullable(Int64)) +x Map(String, Array(Nullable(Int64))) +x Array(Nullable(String)) +x Map(String, Nullable(String)) +x Map(String, Array(Nullable(String))) +x Nullable(String) +x Array(Nullable(String)) +x Map(String, Nullable(String)) +x Map(String, Array(Nullable(String))) +x Tuple(Nullable(Int64), Nullable(String)) +x Object(Nullable(\'json\')) diff --git a/tests/queries/0_stateless/02326_numbers_from_json_strings_schema_inference.sql b/tests/queries/0_stateless/02326_numbers_from_json_strings_schema_inference.sql new file mode 100644 index 00000000000..2012a53c09d --- /dev/null +++ b/tests/queries/0_stateless/02326_numbers_from_json_strings_schema_inference.sql @@ -0,0 +1,21 @@ +-- Tags: no-fasttest + +set input_format_json_try_infer_numbers_from_strings=1; + +desc format(JSONEachRow, '{"x" : "123"}'); +desc format(JSONEachRow, '{"x" : ["123", 123, 12.3]}'); +desc format(JSONEachRow, '{"x" : {"k1" : "123", "k2" : 123}}'); +desc format(JSONEachRow, '{"x" : {"k1" : ["123", "123"], "k2" : [123, 123]}}'); +desc format(JSONEachRow, '{"x" : "123"}\n{"x" : 123}'); +desc format(JSONEachRow, '{"x" : ["123", "456"]}\n{"x" : [123, 456]}'); +desc format(JSONEachRow, '{"x" : {"k1" : "123"}}\n{"x" : {"k2" : 123}}'); +desc format(JSONEachRow, '{"x" : {"k1" : ["123", "123"]}}\n{"x": {"k2" : [123, 123]}}'); +desc format(JSONEachRow, '{"x" : ["123", "Some string"]}'); +desc format(JSONEachRow, '{"x" : {"k1" : "123", "k2" : "Some string"}}'); +desc format(JSONEachRow, '{"x" : {"k1" : ["123", "123"], "k2" : ["Some string"]}}'); +desc format(JSONEachRow, '{"x" : "123"}\n{"x" : "Some string"}'); +desc format(JSONEachRow, '{"x" : ["123", "456"]}\n{"x" : ["Some string"]}'); +desc format(JSONEachRow, '{"x" : {"k1" : "123"}}\n{"x" : {"k2" : "Some string"}}'); +desc format(JSONEachRow, '{"x" : {"k1" : ["123", "123"]}}\n{"x": {"k2" : ["Some string"]}}'); +desc format(JSONEachRow, '{"x" : [123, "Some string"]}'); +desc format(JSONEachRow, '{"x" : {"a" : 123, "b" : "Some string"}}'); diff --git a/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference new file mode 100644 index 00000000000..a1cb9f8e5dc --- /dev/null +++ b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference @@ -0,0 +1,36 @@ +JSONEachRow +x Nullable(Int64) +x Array(Nullable(Int64)) +x Map(String, Array(Nullable(Int64))) +x Map(String, Array(Nullable(Int64))) +x Nullable(Float64) +x Nullable(Float64) +x Array(Nullable(Float64)) +x Map(String, Array(Nullable(Float64))) +CSV +c1 Nullable(Int64) +c1 Array(Nullable(Int64)) +c1 Map(String, Array(Nullable(Int64))) +c1 Map(String, Array(Nullable(Int64))) +c1 Nullable(Float64) +c1 Nullable(Float64) +c1 Array(Nullable(Float64)) +c1 Map(String, Array(Nullable(Float64))) +TSV +c1 Nullable(Int64) +c1 Array(Nullable(Int64)) +c1 Map(String, Array(Nullable(Int64))) +c1 Map(String, Array(Nullable(Int64))) +c1 Nullable(Float64) +c1 Nullable(Float64) +c1 Array(Nullable(Float64)) +c1 Map(String, Array(Nullable(Float64))) +Values +c1 Nullable(Int64) +c1 Array(Nullable(Int64)) +c1 Map(String, Array(Nullable(Int64))) +c1 Map(String, Array(Nullable(Int64))) +c1 Nullable(Float64) +c1 Nullable(Float64) +c1 Array(Nullable(Float64)) +c1 Map(String, Array(Nullable(Float64))) diff --git a/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.sql b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.sql new file mode 100644 index 00000000000..0ceed178865 --- /dev/null +++ b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.sql @@ -0,0 +1,45 @@ +-- Tags: no-fasttest + +set input_format_try_infer_integers=1; + +select 'JSONEachRow'; +desc format(JSONEachRow, '{"x" : 123}'); +desc format(JSONEachRow, '{"x" : [123, 123]}'); +desc format(JSONEachRow, '{"x" : {"a" : [123, 123]}}'); +desc format(JSONEachRow, '{"x" : {"a" : [123, 123]}}\n{"x" : {"b" : [321, 321]}}'); +desc format(JSONEachRow, '{"x" : 123}\n{"x" : 123.123}'); +desc format(JSONEachRow, '{"x" : 123}\n{"x" : 1e2}'); +desc format(JSONEachRow, '{"x" : [123, 123]}\n{"x" : [321.321, 312]}'); +desc format(JSONEachRow, '{"x" : {"a" : [123, 123]}}\n{"x" : {"b" : [321.321, 123]}}'); + +select 'CSV'; +desc format(CSV, '123'); +desc format(CSV, '"[123, 123]"'); +desc format(CSV, '"{\'a\' : [123, 123]}"'); +desc format(CSV, '"{\'a\' : [123, 123]}"\n"{\'b\' : [321, 321]}"'); +desc format(CSV, '123\n123.123'); +desc format(CSV, '122\n1e2'); +desc format(CSV, '"[123, 123]"\n"[321.321, 312]"'); +desc format(CSV, '"{\'a\' : [123, 123]}"\n"{\'b\' : [321.321, 123]}"'); + +select 'TSV'; +desc format(TSV, '123'); +desc format(TSV, '[123, 123]'); +desc format(TSV, '{\'a\' : [123, 123]}'); +desc format(TSV, '{\'a\' : [123, 123]}\n{\'b\' : [321, 321]}'); +desc format(TSV, '123\n123.123'); +desc format(TSV, '122\n1e2'); +desc format(TSV, '[123, 123]\n[321.321, 312]'); +desc format(TSV, '{\'a\' : [123, 123]}\n{\'b\' : [321.321, 123]}'); + +select 'Values'; +desc format(Values, '(123)'); +desc format(Values, '([123, 123])'); +desc format(Values, '({\'a\' : [123, 123]})'); +desc format(Values, '({\'a\' : [123, 123]}), ({\'b\' : [321, 321]})'); +desc format(Values, '(123), (123.123)'); +desc format(Values, '(122), (1e2)'); +desc format(Values, '([123, 123])\n([321.321, 312])'); +desc format(Values, '({\'a\' : [123, 123]}), ({\'b\' : [321.321, 123]})'); + +