From 2626880f6e0ed23e043d1f5756c6ab9786993426 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 14:28:07 +0000 Subject: [PATCH] Fix special builds and tests --- src/Columns/ColumnObject.cpp | 28 ++-- src/Columns/ColumnObject.h | 2 +- src/DataTypes/DataTypeObject.cpp | 37 ++++-- src/DataTypes/DataTypeObject.h | 2 +- .../Serializations/SerializationJSON.h | 2 +- .../Serializations/SerializationObject.cpp | 2 +- .../Serializations/SerializationObject.h | 2 +- .../Serializations/SerializationSubObject.h | 1 - src/Formats/JSONExtractTree.cpp | 65 ++------- src/Formats/SchemaInferenceUtils.cpp | 123 +++++------------- src/Functions/JSONPaths.cpp | 1 + .../InterpreterShowColumnsQuery.cpp | 1 + src/Parsers/ASTObjectTypeArgument.cpp | 18 +-- .../MergeTree/MergeTreeReaderWide.cpp | 6 + src/Storages/MergeTree/checkDataPart.cpp | 3 + .../01825_new_type_json_10.reference | 3 +- .../0_stateless/01825_new_type_json_10.sql | 4 +- .../0_stateless/01825_new_type_json_11.sh | 8 +- .../0_stateless/01825_new_type_json_12.sh | 8 +- .../0_stateless/01825_new_type_json_13.sh | 4 +- .../0_stateless/01825_new_type_json_6.sh | 6 +- .../0_stateless/01825_new_type_json_btc.sh | 6 +- .../01825_new_type_json_ghdata.reference | 12 ++ .../0_stateless/01825_new_type_json_ghdata.sh | 26 ++++ ...1825_new_type_json_ghdata_insert_select.sh | 4 +- .../01825_new_type_json_nbagames.reference | 14 +- .../01825_new_type_json_nbagames.sh | 6 +- .../0_stateless/01825_type_json_10.reference | 9 -- .../0_stateless/01825_type_json_10.sql | 2 +- .../0_stateless/01825_type_json_ghdata.sh | 2 +- .../0_stateless/01825_type_json_nbagames.sh | 2 +- .../02421_type_json_async_insert.sh | 2 +- ...02482_json_nested_arrays_with_same_keys.sh | 2 +- ...w_columns_called_from_clickhouse.reference | 2 +- 34 files changed, 178 insertions(+), 237 deletions(-) create mode 100644 tests/queries/0_stateless/01825_new_type_json_ghdata.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_ghdata.sh diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index f9a3af601e9..2fb5831188d 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -18,13 +18,13 @@ namespace ErrorCodes namespace { -static const FormatSettings & getFormatSettings() +const FormatSettings & getFormatSettings() { static const FormatSettings settings; return settings; } -static const std::shared_ptr & getDynamicSerialization() +const std::shared_ptr & getDynamicSerialization() { static const std::shared_ptr dynamic_serialization = std::make_shared(); return dynamic_serialization; @@ -394,7 +394,7 @@ void ColumnObject::doInsertFrom(const IColumn & src, size_t n) const auto & src_object_column = assert_cast(src); /// First, insert typed paths, they must be the same for both columns. - for (auto & [path, column] : src_object_column.typed_paths) + for (const auto & [path, column] : src_object_column.typed_paths) typed_paths[path]->insertFrom(*column, n); /// Second, insert dynamic paths and extend them if needed. @@ -428,7 +428,7 @@ void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t l const auto & src_object_column = assert_cast(src); /// First, insert typed paths, they must be the same for both columns. - for (auto & [path, column] : src_object_column.typed_paths) + for (const auto & [path, column] : src_object_column.typed_paths) typed_paths[path]->insertRangeFrom(*column, start, length); /// Second, insert dynamic paths and extend them if needed. @@ -898,9 +898,9 @@ void ColumnObject::ensureOwnership() size_t ColumnObject::byteSize() const { size_t size = 0; - for (auto & [_, column] : typed_paths) + for (const auto & [_, column] : typed_paths) size += column->byteSize(); - for (auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths) size += column->byteSize(); size += shared_data->byteSize(); return size; @@ -909,9 +909,9 @@ size_t ColumnObject::byteSize() const size_t ColumnObject::byteSizeAt(size_t n) const { size_t size = 0; - for (auto & [_, column] : typed_paths) + for (const auto & [_, column] : typed_paths) size += column->byteSizeAt(n); - for (auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths) size += column->byteSizeAt(n); size += shared_data->byteSizeAt(n); return size; @@ -920,9 +920,9 @@ size_t ColumnObject::byteSizeAt(size_t n) const size_t ColumnObject::allocatedBytes() const { size_t size = 0; - for (auto & [_, column] : typed_paths) + for (const auto & [_, column] : typed_paths) size += column->allocatedBytes(); - for (auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths) size += column->allocatedBytes(); size += shared_data->allocatedBytes(); return size; @@ -1040,9 +1040,9 @@ void ColumnObject::finalize() bool ColumnObject::isFinalized() const { bool finalized = true; - for (auto & [_, column] : typed_paths) + for (const auto & [_, column] : typed_paths) finalized &= column->isFinalized(); - for (auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths) finalized &= column->isFinalized(); finalized &= shared_data->isFinalized(); return finalized; @@ -1144,8 +1144,8 @@ size_t ColumnObject::findPathLowerBoundInSharedData(StringRef path, const Column Iterator() = delete; Iterator(const ColumnString * data_, size_t index_) : data(data_), index(index_) {} - Iterator(const Iterator & rhs) : data(rhs.data), index(rhs.index) {} - Iterator & operator=(const Iterator & rhs) { data = rhs.data; index = rhs.index; return *this; } + Iterator(const Iterator & rhs) = default; + Iterator & operator=(const Iterator & rhs) = default; inline Iterator& operator+=(difference_type rhs) { index += rhs; return *this;} inline StringRef operator*() const {return data->getDataAt(index);} diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index b130c7f4468..41739782266 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -188,7 +188,7 @@ public: static void fillPathColumnFromSharedData(IColumn & path_column, StringRef path, const ColumnPtr & shared_data_column, size_t start, size_t end); private: - void insertFromSharedDataAndFillRemainingDynamicPaths(const ColumnObject & src_object_column, std::vector & dynamic_paths_to_shared_data, size_t start, size_t length); + void insertFromSharedDataAndFillRemainingDynamicPaths(const ColumnObject & src_object_column, std::vector & src_dynamic_paths_for_shared_data, size_t start, size_t length); void serializePathAndValueIntoArena(Arena & arena, const char *& begin, StringRef path, StringRef value, StringRef & res) const; /// Map path -> column for paths with explicitly specified types. diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index c9378ab395c..a6c5c201f36 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -196,10 +196,29 @@ MutableColumnPtr DataTypeObject::createColumn() const namespace { +/// It is possible to have nested JSON object inside Dynamic. For example when we have an array of JSON objects. +/// During type inference in parsing in case of creating nested JSON objects, we reduce max_dynamic_paths/max_dynamic_types by factors +/// NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR/NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR. +/// So the type name will actually be JSON(max_dynamic_paths=N, max_dynamic_types=M). But we want the user to be able to query it +/// using json.array.:`Array(JSON)`.some.path without specifying max_dynamic_paths/max_dynamic_types. +/// To support it, we do a trick - we replace JSON name in subcolumn to JSON(max_dynamic_paths=N, max_dynamic_types=M), because we know +/// the exact values of max_dynamic_paths/max_dynamic_types for it. +void replaceJSONTypeNameIfNeeded(String & type_name, size_t max_dynamic_paths, size_t max_dynamic_types) +{ + auto pos = type_name.find("JSON"); + while (pos != String::npos) + { + /// Replace only if we don't already have parameters in JSON type declaration. + if (pos + 4 == type_name.size() || type_name[pos + 4] != '(') + type_name.replace(pos, 4, fmt::format("JSON(max_dynamic_paths={}, max_dynamic_types={})", max_dynamic_paths / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, std::max(max_dynamic_types / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))); + pos = type_name.find("JSON", pos + 4); + } +} + /// JSON subcolumn name with Dynamic type subcolumn looks like this: /// "json.some.path.:`Type_name`.some.subcolumn". /// We back quoted type name during identifier parsing so we can distinguish type subcolumn and path element ":TypeName". -std::pair splitPathAndDynamicTypeSubcolumn(std::string_view subcolumn_name) +std::pair splitPathAndDynamicTypeSubcolumn(std::string_view subcolumn_name, size_t max_dynamic_paths, size_t max_dynamic_types) { /// Try to find dynamic type subcolumn in a form .:`Type`. auto pos = subcolumn_name.find(".:`"); @@ -212,6 +231,8 @@ std::pair splitPathAndDynamicTypeSubcolumn(std::string_view subc if (!tryReadBackQuotedString(dynamic_subcolumn, buf)) return {String(subcolumn_name), ""}; + replaceJSONTypeNameIfNeeded(dynamic_subcolumn, max_dynamic_paths, max_dynamic_types); + /// If there is more data in the buffer - it's subcolumn of a type, append it to the type name. if (!buf.eof()) dynamic_subcolumn += String(buf.position(), buf.available()); @@ -333,7 +354,7 @@ std::unique_ptr DataTypeObject::getDynamicSubcolu } /// Split requested subcolumn to the JSON path and Dynamic type subcolumn. - auto [path, path_subcolumn] = splitPathAndDynamicTypeSubcolumn(subcolumn_name); + auto [path, path_subcolumn] = splitPathAndDynamicTypeSubcolumn(subcolumn_name, max_dynamic_paths, max_dynamic_types); std::unique_ptr res; if (auto it = typed_paths.find(path); it != typed_paths.end()) { @@ -373,18 +394,6 @@ std::unique_ptr DataTypeObject::getDynamicSubcolu /// Get subcolumn for Dynamic type if needed. if (!path_subcolumn.empty()) { - /// It is possible to have nested JSON object inside Dynamic. For example when we have an array of JSON objects. - /// During parsing in case of creating nested JSON objects, we reduce max_dynamic_paths/max_dynamic_types by NESTED_OBJECT_REDUCE_FACTOR factor. - /// So the type name will actually be JSON(max_dynamic_paths=N, max_dynamic_types=M). But we want the user to be able to query it - /// using json.array.:`Array(JSON)`.some.path without specifying max_dynamic_paths/max_dynamic_types. - /// To support it, we do a trick - we replace JSON name in subcolumn to JSON(max_dynamic_paths=N, max_dynamic_types=M), because we know - /// the exact values of max_dynamic_paths/max_dynamic_types for it. - auto pos = path_subcolumn.find("JSON"); - /// We want to replace JSON keyword only in the first subcolumn part before the first dot. - auto first_dot_pos = path_subcolumn.find('.'); - if (pos != path_subcolumn.npos && (first_dot_pos == path_subcolumn.npos || pos < first_dot_pos)) - path_subcolumn.replace(pos, 4, fmt::format("JSON(max_dynamic_paths={}, max_dynamic_types={})", max_dynamic_paths / NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, std::max(max_dynamic_types / NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))); - res = res->type->getSubcolumnData(path_subcolumn, *res, throw_if_null); if (!res) return nullptr; diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 5b76d96e0de..6eea777ed26 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -23,7 +23,7 @@ public: static constexpr size_t NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR = 4; static constexpr size_t NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR = 2; - DataTypeObject( + explicit DataTypeObject( const SchemaFormat & schema_format_, const std::unordered_map & typed_paths_ = {}, const std::unordered_set & paths_to_skip_ = {}, diff --git a/src/DataTypes/Serializations/SerializationJSON.h b/src/DataTypes/Serializations/SerializationJSON.h index aee1413bdd3..934c94527f3 100644 --- a/src/DataTypes/Serializations/SerializationJSON.h +++ b/src/DataTypes/Serializations/SerializationJSON.h @@ -31,7 +31,7 @@ public: void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - virtual void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; + void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 8a9860fd592..7c8c23e8a29 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -48,7 +48,7 @@ bool SerializationObject::shouldSkipPath(const String & path) const if (paths_to_skip.contains(path)) return true; - auto it = std::lower_bound(sorted_typed_paths.begin(), sorted_typed_paths.end(), path); + auto it = std::lower_bound(sorted_paths_to_skip.begin(), sorted_paths_to_skip.end(), path); if (it != sorted_paths_to_skip.end() && it != sorted_paths_to_skip.begin() && path.starts_with(*std::prev(it))) return true; diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index b279fab2603..faf15aa3260 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -101,7 +101,7 @@ private: { String path; - TypedPathSubcolumnCreator(const String & path_) : path(path_) {} + explicit TypedPathSubcolumnCreator(const String & path_) : path(path_) {} DataTypePtr create(const DataTypePtr & prev) const override { return prev; } ColumnPtr create(const ColumnPtr & prev) const override { return prev; } diff --git a/src/DataTypes/Serializations/SerializationSubObject.h b/src/DataTypes/Serializations/SerializationSubObject.h index 8a5582536e7..10973b48957 100644 --- a/src/DataTypes/Serializations/SerializationSubObject.h +++ b/src/DataTypes/Serializations/SerializationSubObject.h @@ -67,7 +67,6 @@ private: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Text/binary serialization is not implemented for object sub-object subcolumn"); } -private: String path_prefix; std::unordered_map typed_paths_serializations; SerializationPtr dynamic_serialization; diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index a8980e785de..640ab1b0acd 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1389,7 +1389,7 @@ template class DynamicNode : public JSONExtractTreeNode { public: - DynamicNode( + explicit DynamicNode( size_t max_dynamic_paths_for_object_ = DataTypeObject::DEFAULT_MAX_SEPARATELY_STORED_PATHS, size_t max_dynamic_types_for_object_ = DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES) : max_dynamic_paths_for_object(max_dynamic_paths_for_object_), max_dynamic_types_for_object(max_dynamic_types_for_object_) @@ -1412,7 +1412,7 @@ public: } auto & variant_column = column_dynamic.getVariantColumn(); - auto & variant_info = column_dynamic.getVariantInfo(); + const auto & variant_info = column_dynamic.getVariantInfo(); /// First, try to insert element into current variants but with no types conversion. /// We want to avoid inferring the type on each row, so if we can insert this element into @@ -1486,20 +1486,20 @@ private: switch (element.type()) { case ElementType::NULL_VALUE: - return getNullType(); + return std::make_shared(std::make_shared()); case ElementType::BOOL: - return getBoolType(); + return DataTypeFactory::instance().get("Bool"); case ElementType::INT64: { - auto type = getInt64Type(); + auto type = std::make_shared(); if (element.getInt64() < 0) json_inference_info.negative_integers.insert(type.get()); return type; } case ElementType::UINT64: - return getUInt64Type(); + return std::make_shared(); case ElementType::DOUBLE: - return getFloat64Type(); + return std::make_shared(); case ElementType::STRING: { auto data = element.getString(); @@ -1516,7 +1516,7 @@ private: } } - return getStringType(); + return std::make_shared(); } case ElementType::ARRAY: { @@ -1527,7 +1527,7 @@ private: types.push_back(elementToDataTypeImpl(value, format_settings, json_inference_info)); if (types.empty()) - return getEmptyArrayType(); + return std::make_shared(std::make_shared()); if (checkIfTypesAreEqual(types)) return std::make_shared(types.back()); @@ -1561,51 +1561,6 @@ private: } } - /// During schema inference we create shared_ptr to the some data types quite a lot. - /// Single creating of such shared_ptr is not expensive, but when it happens on each - /// column on each row, it can be noticeable. - const DataTypePtr & getBoolType() const - { - static const DataTypePtr bool_type = DataTypeFactory::instance().get("Bool"); - return bool_type; - } - - const DataTypePtr & getStringType() const - { - static const DataTypePtr string_type = std::make_shared(); - return string_type; - } - - const DataTypePtr & getInt64Type() const - { - static const DataTypePtr int64_type = std::make_shared(); - return int64_type; - } - - const DataTypePtr & getUInt64Type() const - { - static const DataTypePtr uint64_type = std::make_shared(); - return uint64_type; - } - - const DataTypePtr & getFloat64Type() const - { - static const DataTypePtr float64_type = std::make_shared(); - return float64_type; - } - - const DataTypePtr & getNullType() const - { - static const DataTypePtr null_type = std::make_shared(std::make_shared()); - return null_type; - } - - const DataTypePtr & getEmptyArrayType() const - { - static const DataTypePtr empty_array_type = std::make_shared(std::make_shared()); - return empty_array_type; - } - size_t max_dynamic_paths_for_object; size_t max_dynamic_types_for_object; @@ -1772,7 +1727,7 @@ private: } } /// Try to add a new dynamic path. - else if (auto dynamic_column = column_object.tryToAddNewDynamicPath(current_path)) + else if (auto * dynamic_column = column_object.tryToAddNewDynamicPath(current_path)) { if (!dynamic_node->insertResultToColumn(*dynamic_column, element, insert_settings, format_settings, error)) { diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 441e7d77d24..28d60bf078b 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -36,63 +36,6 @@ namespace ErrorCodes namespace { - /// During schema inference we create shared_ptr to the some data types quite a lot. - /// Single creating of such shared_ptr is not expensive, but when it happens on each - /// column on each row, it can be noticeable. - const DataTypePtr & getBoolType() - { - static const DataTypePtr bool_type = DataTypeFactory::instance().get("Bool"); - return bool_type; - } - - const DataTypePtr & getStringType() - { - static const DataTypePtr string_type = std::make_shared(); - return string_type; - } - - const DataTypePtr & getInt64Type() - { - static const DataTypePtr int64_type = std::make_shared(); - return int64_type; - } - - const DataTypePtr & getUInt64Type() - { - static const DataTypePtr uint64_type = std::make_shared(); - return uint64_type; - } - - const DataTypePtr & getFloat64Type() - { - static const DataTypePtr float64_type = std::make_shared(); - return float64_type; - } - - const DataTypePtr & getDateType() - { - static const DataTypePtr date_type = std::make_shared(); - return date_type; - } - - const DataTypePtr & getDateTime64Type() - { - static const DataTypePtr date_type = std::make_shared(9); - return date_type; - } - - const DataTypePtr & getNullType() - { - static const DataTypePtr null_type = std::make_shared(std::make_shared()); - return null_type; - } - - const DataTypePtr & getEmptyArrayType() - { - static const DataTypePtr empty_array_type = std::make_shared(std::make_shared()); - return empty_array_type; - } - /// Special data type that represents JSON object as a set of paths and their types. /// It supports merging two JSON objects and creating Named Tuple from itself. /// It's used only for schema inference of Named Tuples from JSON objects. @@ -265,7 +208,7 @@ namespace if (leaf_type && !isNothing(removeNullable(leaf_type)) && !nodes.empty()) { if (use_string_type_for_ambiguous_paths) - return getStringType(); + return std::make_shared(); throw Exception( ErrorCodes::INCORRECT_DATA, @@ -331,7 +274,7 @@ namespace bool is_negative = json_info && json_info->negative_integers.contains(type.get()); have_negative_integers |= is_negative; if (!is_negative) - type = getUInt64Type(); + type = std::make_shared(); } } @@ -352,7 +295,7 @@ namespace WhichDataType which(type); if (which.isInt64() || which.isUInt64()) { - auto new_type = getFloat64Type(); + const auto & new_type = std::make_shared(); if (json_info && json_info->numbers_parsed_from_json_strings.erase(type.get())) json_info->numbers_parsed_from_json_strings.insert(new_type.get()); type = new_type; @@ -376,7 +319,7 @@ namespace for (auto & type : data_types) { if (isDate(type) || isDateTime64(type)) - type = getStringType(); + type = std::make_shared(); } type_indexes.erase(TypeIndex::Date); @@ -390,7 +333,7 @@ namespace for (auto & type : data_types) { if (isDate(type)) - type = getDateTime64Type(); + type = std::make_shared(9); } type_indexes.erase(TypeIndex::Date); @@ -412,7 +355,7 @@ namespace if (isNumber(type) && (settings.json.read_numbers_as_strings || !json_info || json_info->numbers_parsed_from_json_strings.contains(type.get()))) - type = getStringType(); + type = std::make_shared(); } updateTypeIndexes(data_types, type_indexes); @@ -435,11 +378,11 @@ namespace if (isBool(type)) { if (have_signed_integers) - type = getInt64Type(); + type = std::make_shared(); else if (have_unsigned_integers) - type = getUInt64Type(); + type = std::make_shared(); else - type = getFloat64Type(); + type = std::make_shared(); } } @@ -456,7 +399,7 @@ namespace for (auto & type : data_types) { if (isBool(type)) - type = getStringType(); + type = std::make_shared(); } type_indexes.erase(TypeIndex::UInt8); @@ -606,7 +549,7 @@ namespace for (auto & type : data_types) { if (isMap(type)) - type = getStringType(); + type = std::make_shared(); } type_indexes.erase(TypeIndex::Map); @@ -856,7 +799,7 @@ namespace /// Empty array has type Array(Nothing) if (nested_types.empty()) - return getEmptyArrayType(); + return std::make_shared(std::make_shared()); if (checkIfTypesAreEqual(nested_types)) return std::make_shared(std::move(nested_types.back())); @@ -969,13 +912,13 @@ namespace /// NOTE: it may break parsing of tryReadFloat() != tryReadIntText() + parsing of '.'/'e' /// But, for now it is true if (tryReadFloat(tmp_float, buf, settings, has_fractional) && has_fractional) - return getFloat64Type(); + return std::make_shared(); Int64 tmp_int; buf.position() = number_start; if (tryReadIntText(tmp_int, buf)) { - auto type = getInt64Type(); + auto type = std::make_shared(); if (json_info && tmp_int < 0) json_info->negative_integers.insert(type.get()); return type; @@ -985,7 +928,7 @@ namespace UInt64 tmp_uint; buf.position() = number_start; if (tryReadIntText(tmp_uint, buf)) - return getUInt64Type(); + return std::make_shared(); return nullptr; } @@ -997,13 +940,13 @@ namespace PeekableReadBufferCheckpoint checkpoint(peekable_buf); if (tryReadFloat(tmp_float, peekable_buf, settings, has_fractional) && has_fractional) - return getFloat64Type(); + return std::make_shared(); peekable_buf.rollbackToCheckpoint(/* drop= */ false); Int64 tmp_int; if (tryReadIntText(tmp_int, peekable_buf)) { - auto type = getInt64Type(); + auto type = std::make_shared(); if (json_info && tmp_int < 0) json_info->negative_integers.insert(type.get()); return type; @@ -1013,11 +956,11 @@ namespace /// In case of Int64 overflow we can try to infer UInt64. UInt64 tmp_uint; if (tryReadIntText(tmp_uint, peekable_buf)) - return getUInt64Type(); + return std::make_shared(); } else if (tryReadFloat(tmp_float, buf, settings, has_fractional)) { - return getFloat64Type(); + return std::make_shared(); } /// This is not a number. @@ -1034,7 +977,7 @@ namespace Int64 tmp_int; if (tryReadIntText(tmp_int, buf) && buf.eof()) { - auto type = getInt64Type(); + auto type = std::make_shared(); if (json_inference_info && tmp_int < 0) json_inference_info->negative_integers.insert(type.get()); return type; @@ -1046,7 +989,7 @@ namespace /// In case of Int64 overflow, try to infer UInt64 UInt64 tmp_uint; if (tryReadIntText(tmp_uint, buf) && buf.eof()) - return getUInt64Type(); + return std::make_shared(); } /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. @@ -1055,7 +998,7 @@ namespace Float64 tmp; bool has_fractional; if (tryReadFloat(tmp, buf, settings, has_fractional) && buf.eof()) - return getFloat64Type(); + return std::make_shared(); return nullptr; } @@ -1079,7 +1022,7 @@ namespace if constexpr (is_json) { if (json_info->is_object_key) - return getStringType(); + return std::make_shared(); } if (auto type = tryInferDateOrDateTimeFromString(field, settings)) @@ -1097,7 +1040,7 @@ namespace } } - return getStringType(); + return std::make_shared(); } bool tryReadJSONObject(ReadBuffer & buf, const FormatSettings & settings, DataTypeJSONPaths::Paths & paths, const std::vector & path, JSONInferenceInfo * json_info, size_t depth) @@ -1254,7 +1197,7 @@ namespace return std::make_shared("json", true); if (settings.json.read_objects_as_strings) - return getStringType(); + return std::make_shared(); transformInferredTypesIfNeededImpl(value_types, settings, json_info); if (!checkIfTypesAreEqual(value_types)) @@ -1320,15 +1263,15 @@ namespace /// Bool if (checkStringCaseInsensitive("true", buf) || checkStringCaseInsensitive("false", buf)) - return getBoolType(); + return DataTypeFactory::instance().get("Bool"); /// Null or NaN if (checkCharCaseInsensitive('n', buf)) { if (checkStringCaseInsensitive("ull", buf)) - return getNullType(); + return std::make_shared(std::make_shared()); else if (checkStringCaseInsensitive("an", buf)) - return getFloat64Type(); + return std::make_shared(); } /// Number @@ -1385,7 +1328,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F if (!remain_nothing_types && isNothing(data_type) && settings.json.infer_incomplete_types_as_strings) { - data_type = getStringType(); + data_type = std::make_shared(); return; } @@ -1402,7 +1345,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F /// If all objects were empty, use type String, so these JSON objects will be read as Strings. if (json_paths->empty() && settings.json.infer_incomplete_types_as_strings) { - data_type = getStringType(); + data_type = std::make_shared(); return; } @@ -1424,7 +1367,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F auto key_type = map_type->getKeyType(); /// If all inferred Maps are empty, use type String, so these JSON objects will be read as Strings. if (isNothing(key_type) && settings.json.infer_incomplete_types_as_strings) - key_type = getStringType(); + key_type = std::make_shared(); auto value_type = map_type->getValueType(); @@ -1501,10 +1444,10 @@ DataTypePtr tryInferJSONNumberFromString(std::string_view field, const FormatSet DataTypePtr tryInferDateOrDateTimeFromString(std::string_view field, const FormatSettings & settings) { if (settings.try_infer_dates && tryInferDate(field)) - return getDateType(); + return std::make_shared(); if (settings.try_infer_datetimes && tryInferDateTime(field, settings)) - return getDateTime64Type(); + return std::make_shared(9); return nullptr; } diff --git a/src/Functions/JSONPaths.cpp b/src/Functions/JSONPaths.cpp index 31699ad1c9c..4f29846319d 100644 --- a/src/Functions/JSONPaths.cpp +++ b/src/Functions/JSONPaths.cpp @@ -165,6 +165,7 @@ private: std::vector sorted_dynamic_and_typed_paths; const auto & typed_path_columns = column_object.getTypedPaths(); const auto & dynamic_path_columns = column_object.getDynamicPaths(); + sorted_dynamic_and_typed_paths.reserve(typed_path_columns.size() + dynamic_path_columns.size()); for (const auto & [path, _] : typed_path_columns) sorted_dynamic_and_typed_paths.push_back(path); for (const auto & [path, _] : dynamic_path_columns) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index d8fff4e6026..b39ce25ab57 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -68,6 +68,7 @@ WITH map( 'Map', 'JSON', 'Tuple', 'JSON', 'Object', 'JSON', + 'JSON', 'JSON', 'String', '{}', 'FixedString', '{}') AS native_to_mysql_mapping, )", diff --git a/src/Parsers/ASTObjectTypeArgument.cpp b/src/Parsers/ASTObjectTypeArgument.cpp index aa5d23d7881..975f0389505 100644 --- a/src/Parsers/ASTObjectTypeArgument.cpp +++ b/src/Parsers/ASTObjectTypeArgument.cpp @@ -35,27 +35,27 @@ ASTPtr ASTObjectTypeArgument::clone() const return res; } -void ASTObjectTypeArgument::formatImpl(const FormatSettings & parameters, FormatState & state, FormatStateStacked frame) const +void ASTObjectTypeArgument::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { if (path_with_type) { - path_with_type->formatImpl(parameters, state, frame); + path_with_type->formatImpl(settings, state, frame); } else if (parameter) { - parameter->formatImpl(parameters, state, frame); + parameter->formatImpl(settings, state, frame); } else if (skip_path) { - std::string indent_str = parameters.one_line ? "" : std::string(4 * frame.indent, ' '); - parameters.ostr << indent_str << "SKIP" << ' '; - skip_path->formatImpl(parameters, state, frame); + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + settings.ostr << indent_str << "SKIP" << ' '; + skip_path->formatImpl(settings, state, frame); } else if (skip_path_regexp) { - std::string indent_str = parameters.one_line ? "" : std::string(4 * frame.indent, ' '); - parameters.ostr << indent_str << "SKIP REGEXP" << ' '; - skip_path_regexp->formatImpl(parameters, state, frame); + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + settings.ostr << indent_str << "SKIP REGEXP" << ' '; + skip_path_regexp->formatImpl(settings, state, frame); } } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index c017d7db786..2c876fade74 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -213,6 +213,9 @@ void MergeTreeReaderWide::addStreams( ISerialization::StreamCallback callback = [&] (const ISerialization::SubstreamPath & substream_path) { + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return; + auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums()); /** If data file is missing then we will not try to open it. @@ -348,6 +351,9 @@ void MergeTreeReaderWide::prefetchForColumn( deserializePrefix(serialization, name_and_type, current_task_last_mark, cache, deserialize_states_cache); auto callback = [&](const ISerialization::SubstreamPath & substream_path) { + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return; + auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums()); if (stream_name && !prefetched_streams.contains(*stream_name)) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 774fd95ebc6..dcdcb7f5800 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -211,6 +211,9 @@ static IMergeTreeDataPart::Checksums checkDataPart( { get_serialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return; + auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(column, substream_path, ".bin", data_part_storage); if (!stream_name) diff --git a/tests/queries/0_stateless/01825_new_type_json_10.reference b/tests/queries/0_stateless/01825_new_type_json_10.reference index 0dffa3c46f9..d70c8210914 100644 --- a/tests/queries/0_stateless/01825_new_type_json_10.reference +++ b/tests/queries/0_stateless/01825_new_type_json_10.reference @@ -1,9 +1,8 @@ ('a.b','Int64') ('a.c','Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))') -('a.c','Array(Nullable(String))') +('d','Int64') ('e','Array(Nullable(Int64))') ('f','Int64') -('d','Int64') {"o":{"a":{"b":"1","c":[{"d":"10","e":["31"]},{"d":"20","e":["63","127"]}]}}} {"o":{"a":{"b":"2","c":[]}}} {"o":{"a":{"b":"3","c":[{"e":["32"],"f":"20"},{"e":["64","128"],"f":"30"}]}}} diff --git a/tests/queries/0_stateless/01825_new_type_json_10.sql b/tests/queries/0_stateless/01825_new_type_json_10.sql index 0e599b0ac31..a313adb4757 100644 --- a/tests/queries/0_stateless/01825_new_type_json_10.sql +++ b/tests/queries/0_stateless/01825_new_type_json_10.sql @@ -8,8 +8,8 @@ CREATE TABLE t_json_10 (o JSON) ENGINE = Memory; INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 1, "c": [{"d": 10, "e": [31]}, {"d": 20, "e": [63, 127]}]}} {"a": {"b": 2, "c": []}} INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 3, "c": [{"f": 20, "e": [32]}, {"f": 30, "e": [64, 128]}]}} {"a": {"b": 4, "c": []}} -SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(o)) FROM t_json_10; -SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(o.a.c.:`Array(JSON)`))) FROM t_json_10; +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(o)) as path FROM t_json_10 order by path; +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(o.a.c.:`Array(JSON)`))) as path FROM t_json_10 order by path; SELECT o FROM t_json_10 ORDER BY o.a.b FORMAT JSONEachRow; SELECT o.a.b, o.a.c.:`Array(JSON)`.d, o.a.c.:`Array(JSON)`.e, o.a.c.:`Array(JSON)`.f FROM t_json_10 ORDER BY o.a.b; diff --git a/tests/queries/0_stateless/01825_new_type_json_11.sh b/tests/queries/0_stateless/01825_new_type_json_11.sh index 2eb6c1768fd..f448b7433ab 100755 --- a/tests/queries/0_stateless/01825_new_type_json_11.sh +++ b/tests/queries/0_stateless/01825_new_type_json_11.sh @@ -53,10 +53,10 @@ cat <&1 | grep -o -m1 "Cannot parse object" $CLICKHOUSE_CLIENT -q "SELECT count() FROM t_json_async_insert" diff --git a/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh b/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh index 99f243833f5..e0648f4df6e 100755 --- a/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh +++ b/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh @@ -21,7 +21,7 @@ echo ' } }' > 02482_object_data.jsonl -$CLICKHOUSE_LOCAL --allow_experimental_object_type=1 -q "select * from file(02482_object_data.jsonl, auto, 'obj Object('json')')" +$CLICKHOUSE_LOCAL --allow_experimental_object_type=1 -q "select * from file(02482_object_data.jsonl, auto, 'obj Object(''json'')')" rm 02482_object_data.jsonl diff --git a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.reference b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.reference index de0f151db7d..cb905d63ca5 100644 --- a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.reference +++ b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.reference @@ -44,7 +44,7 @@ nested.col1 Array(String) NO \N nested.col2 Array(UInt32) NO \N nfs Nullable(FixedString(3)) YES \N ns Nullable(String) YES \N -o Object(\'json\') NO \N +o JSON NO \N p Point NO \N pg Polygon NO \N r Ring NO \N