diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 4f3c22aa8a3..f8a1ff7002f 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -521,7 +521,7 @@ DataTypePtr createConcreteEmptyDynamicColumn(const DataTypePtr & type_in_storage throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName()); } -bool hasDynamicSubcolumns(const ColumnsDescription & columns) +bool hasDynamicSubcolumnsDeprecated(const ColumnsDescription & columns) { return std::any_of(columns.begin(), columns.end(), [](const auto & column) diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index d4109b971a4..830f0e4c95e 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -63,7 +63,7 @@ DataTypePtr createConcreteEmptyDynamicColumn(const DataTypePtr & type_in_storage void extendObjectColumns(NamesAndTypesList & columns_list, const ColumnsDescription & object_columns, bool with_subcolumns); /// Checks whether @columns contain any column with dynamic subcolumns. -bool hasDynamicSubcolumns(const ColumnsDescription & columns); +bool hasDynamicSubcolumnsDeprecated(const ColumnsDescription & columns); /// Updates types of objects in @object_columns inplace /// according to types in new_columns. diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 315c2d1bc32..ed4357b6cfd 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -220,7 +220,7 @@ struct FormatSettings bool escape_forward_slashes = true; bool read_named_tuples_as_objects = false; bool use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects = false; - bool write_named_tuples_as_objects = false; + bool write_named_tuples_as_objects = true; bool skip_null_value_in_named_tuples = false; bool defaults_for_missing_elements_in_named_tuple = false; bool ignore_unknown_keys_in_named_tuple = false; diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index ead4b6065ac..0aa6d12e714 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4108,6 +4108,7 @@ private: ColumnStringHelpers::WriteHelper write_helper(assert_cast(*json_string), input_rows_count); auto & write_buffer = write_helper.getWriteBuffer(); FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{}; + format_settings.json.quote_64bit_integers = false; auto serialization = arguments[0].type->getDefaultSerialization(); for (size_t i = 0; i < input_rows_count; ++i) { diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index e88fdeb0379..dc1a8e5a1c3 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -755,7 +755,7 @@ void executeQueryWithParallelReplicasCustomKey( } ColumnsDescriptionByShardNum columns_object; - if (hasDynamicSubcolumns(columns)) + if (hasDynamicSubcolumnsDeprecated(columns)) columns_object = getExtendedObjectsOfRemoteTables(*query_info.cluster, storage_id, columns, context); ClusterProxy::SelectStreamFactory select_stream_factory diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ff0e1d7f5a8..c9fcee498d2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1881,7 +1881,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, validateVirtualColumns(*res); - if (!res->supportsDynamicSubcolumnsDeprecated() && hasDynamicSubcolumns(res->getInMemoryMetadataPtr()->getColumns()) && mode <= LoadingStrictnessLevel::CREATE) + if (!res->supportsDynamicSubcolumnsDeprecated() && hasDynamicSubcolumnsDeprecated(res->getInMemoryMetadataPtr()->getColumns()) && mode <= LoadingStrictnessLevel::CREATE) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column of type Object, " diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index fbca222b1e7..9f41c845510 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1450,9 +1450,8 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const const auto old_data_type = all_columns.getColumn(options, column_name).type; bool new_type_has_deprecated_object = command.data_type->hasDynamicSubcolumnsDeprecated(); - bool old_type_has_deprecated_object = old_data_type->hasDynamicSubcolumnsDeprecated(); - if (new_type_has_deprecated_object || old_type_has_deprecated_object) + if (new_type_has_deprecated_object) throw Exception( ErrorCodes::BAD_ARGUMENTS, "The change of data type {} of column {} to {} is not allowed. It has known bugs", diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b9e674a8eec..286c1581fa9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4104,7 +4104,7 @@ void MergeTreeData::checkPartDynamicColumns(MutableDataPartPtr & part, DataParts const auto & columns = metadata_snapshot->getColumns(); auto virtuals = getVirtualsPtr(); - if (!hasDynamicSubcolumns(columns)) + if (!hasDynamicSubcolumnsDeprecated(columns)) return; const auto & part_columns = part->getColumns(); @@ -8754,8 +8754,11 @@ void MergeTreeData::resetObjectColumnsFromActiveParts(const DataPartsLock & /*lo { auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & columns = metadata_snapshot->getColumns(); - if (!hasDynamicSubcolumns(columns)) + if (!hasDynamicSubcolumnsDeprecated(columns)) + { + object_columns = ColumnsDescription{}; return; + } auto range = getDataPartsStateRange(DataPartState::Active); object_columns = getConcreteObjectColumns(range, columns); @@ -8765,7 +8768,7 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart { auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & columns = metadata_snapshot->getColumns(); - if (!hasDynamicSubcolumns(columns)) + if (!hasDynamicSubcolumnsDeprecated(columns)) return; DB::updateObjectColumns(object_columns, columns, part->getColumns()); diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.cpp b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.cpp index 3df03b55cd4..9e59c086e79 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.cpp @@ -50,7 +50,7 @@ StorageSnapshotPtr StorageFromMergeTreeDataPart::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr /*query_context*/) const { const auto & storage_columns = metadata_snapshot->getColumns(); - if (!hasDynamicSubcolumns(storage_columns)) + if (!hasDynamicSubcolumnsDeprecated(storage_columns)) return std::make_shared(*this, metadata_snapshot); auto data_parts = storage.getDataPartsVectorForInternalUsage(); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4f5a95ab508..337800338d1 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -686,7 +686,7 @@ std::optional StorageDistributed::getOptimizedQueryP static bool requiresObjectColumns(const ColumnsDescription & all_columns, ASTPtr query) { - if (!hasDynamicSubcolumns(all_columns)) + if (!hasDynamicSubcolumnsDeprecated(all_columns)) return false; if (!query) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 71447889d86..e352b16546e 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -180,7 +180,7 @@ StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr & /// rows and bytes counters into the MultiVersion-ed struct, then everything would be consistent. snapshot_data->rows_approx = total_size_rows.load(std::memory_order_relaxed); - if (!hasDynamicSubcolumns(metadata_snapshot->getColumns())) + if (!hasDynamicSubcolumnsDeprecated(metadata_snapshot->getColumns())) return std::make_shared(*this, metadata_snapshot, ColumnsDescription{}, std::move(snapshot_data)); auto object_columns = getConcreteObjectColumns( diff --git a/tests/queries/0_stateless/02725_object_column_alter.sql b/tests/queries/0_stateless/02725_object_column_alter.sql index 0e41070742a..9625e7c9777 100644 --- a/tests/queries/0_stateless/02725_object_column_alter.sql +++ b/tests/queries/0_stateless/02725_object_column_alter.sql @@ -3,13 +3,7 @@ SET allow_experimental_object_type=1; DROP TABLE IF EXISTS t_to; -DROP TABLE IF EXISTS t_from; - CREATE TABLE t_to (id UInt64, value Nullable(String)) ENGINE MergeTree() ORDER BY id; -CREATE TABLE t_from (id UInt64, value Object('json')) ENGINE MergeTree() ORDER BY id; - ALTER TABLE t_to MODIFY COLUMN value Object('json'); -- { serverError BAD_ARGUMENTS } -ALTER TABLE t_from MODIFY COLUMN value Nullable(String); -- { serverError BAD_ARGUMENTS } - DROP TABLE t_to; -DROP TABLE t_from; + diff --git a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.reference b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.reference index 0ae94e68663..bfc10723941 100644 --- a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.reference +++ b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.reference @@ -1,20 +1,20 @@ Map to JSON {"a":"0","b":"1970-01-01","c":[],"d":[{"e":"0"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a":"1","b":"1970-01-02","c":["0"],"d":[{"e":"1"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a":"2","b":"1970-01-03","c":["0","1"],"d":[{"e":"2"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a":"3","b":"1970-01-04","c":["0","1","2"],"d":[{"e":"3"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a":"4","b":"1970-01-05","c":["0","1","2","3"],"d":[{"e":"4"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"1","b":"1970-01-02","c":["0"],"d":[{"e":"1"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(Int64))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"2","b":"1970-01-03","c":["0","1"],"d":[{"e":"2"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(Int64))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"3","b":"1970-01-04","c":["0","1","2"],"d":[{"e":"3"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(Int64))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"4","b":"1970-01-05","c":["0","1","2","3"],"d":[{"e":"4"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(Int64))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} {"a0":"0","b0":"1970-01-01","c0":[],"d0":[{"e0":"0"}]} {'a0':'Int64','b0':'Date','c0':'Array(Nullable(String))','d0':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a1":"1","b1":"1970-01-02","c1":["0"],"d1":[{"e1":"1"}]} {'a1':'Int64','b1':'Date','c1':'Array(Nullable(String))','d1':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a2":"2","b2":"1970-01-03","c2":["0","1"],"d2":[{"e2":"2"}]} {'a2':'Int64','b2':'Date','c2':'Array(Nullable(String))','d2':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a0":"3","b0":"1970-01-04","c0":["0","1","2"],"d0":[{"e0":"3"}]} {'a0':'Int64','b0':'Date','c0':'Array(Nullable(String))','d0':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a1":"4","b1":"1970-01-05","c1":["0","1","2","3"],"d1":[{"e1":"4"}]} {'a1':'Int64','b1':'Date','c1':'Array(Nullable(String))','d1':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a1":"1","b1":"1970-01-02","c1":["0"],"d1":[{"e1":"1"}]} {'a1':'Int64','b1':'Date','c1':'Array(Nullable(Int64))','d1':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a2":"2","b2":"1970-01-03","c2":["0","1"],"d2":[{"e2":"2"}]} {'a2':'Int64','b2':'Date','c2':'Array(Nullable(Int64))','d2':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a0":"3","b0":"1970-01-04","c0":["0","1","2"],"d0":[{"e0":"3"}]} {'a0':'Int64','b0':'Date','c0':'Array(Nullable(Int64))','d0':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a1":"4","b1":"1970-01-05","c1":["0","1","2","3"],"d1":[{"e1":"4"}]} {'a1':'Int64','b1':'Date','c1':'Array(Nullable(Int64))','d1':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} Tuple to JSON {"a":"0","b":"1970-01-01","c":[],"d":[{"e":"0"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a":"1","b":"1970-01-02","c":["0"],"d":[{"e":"1"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a":"2","b":"1970-01-03","c":["0","1"],"d":[{"e":"2"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a":"3","b":"1970-01-04","c":["0","1","2"],"d":[{"e":"3"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} -{"a":"4","b":"1970-01-05","c":["0","1","2","3"],"d":[{"e":"4"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"1","b":"1970-01-02","c":["0"],"d":[{"e":"1"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(Int64))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"2","b":"1970-01-03","c":["0","1"],"d":[{"e":"2"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(Int64))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"3","b":"1970-01-04","c":["0","1","2"],"d":[{"e":"3"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(Int64))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"4","b":"1970-01-05","c":["0","1","2","3"],"d":[{"e":"4"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(Int64))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} Object to JSON {"a":"0","b":"1970-01-01","c":[],"d":{"e":["0"]}} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d.e':'Array(Nullable(Int64))'} {"a":"1","b":"1970-01-02","c":["0"],"d":{"e":["1"]}} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d.e':'Array(Nullable(Int64))'} diff --git a/tests/queries/0_stateless/03270_object_to_json_alter.reference b/tests/queries/0_stateless/03270_object_to_json_alter.reference new file mode 100644 index 00000000000..a04884fb374 --- /dev/null +++ b/tests/queries/0_stateless/03270_object_to_json_alter.reference @@ -0,0 +1,132 @@ +{'a0':['Int64'],'a1':['Int64'],'a10':['Int64'],'a11':['Int64'],'a12':['Int64'],'a13':['Int64'],'a14':['Int64'],'a15':['Int64'],'a16':['Int64'],'a17':['Int64'],'a18':['Int64'],'a19':['Int64'],'a2':['Int64'],'a20':['Int64'],'a21':['Int64'],'a22':['Int64'],'a23':['Int64'],'a24':['Int64'],'a25':['Int64'],'a26':['Int64'],'a27':['Int64'],'a28':['Int64'],'a29':['Int64'],'a3':['Int64'],'a30':['Int64'],'a31':['Int64'],'a32':['Int64'],'a33':['Int64'],'a34':['Int64'],'a35':['Int64'],'a36':['Int64'],'a37':['Int64'],'a38':['Int64'],'a39':['Int64'],'a4':['Int64'],'a40':['Int64'],'a41':['Int64'],'a42':['Int64'],'a43':['Int64'],'a44':['Int64'],'a45':['Int64'],'a46':['Int64'],'a47':['Int64'],'a48':['Int64'],'a49':['Int64'],'a5':['Int64'],'a50':['Int64'],'a51':['Int64'],'a52':['Int64'],'a53':['Int64'],'a54':['Int64'],'a55':['Int64'],'a56':['Int64'],'a57':['Int64'],'a58':['Int64'],'a59':['Int64'],'a6':['Int64'],'a60':['Int64'],'a61':['Int64'],'a62':['Int64'],'a63':['Int64'],'a64':['Int64'],'a65':['Int64'],'a66':['Int64'],'a67':['Int64'],'a68':['Int64'],'a69':['Int64'],'a7':['Int64'],'a70':['Int64'],'a71':['Int64'],'a72':['Int64'],'a73':['Int64'],'a74':['Int64'],'a75':['Int64'],'a76':['Int64'],'a77':['Int64'],'a78':['Int64'],'a79':['Int64'],'a8':['Int64'],'a80':['Int64'],'a81':['Int64'],'a82':['Int64'],'a83':['Int64'],'a84':['Int64'],'a85':['Int64'],'a86':['Int64'],'a87':['Int64'],'a88':['Int64'],'a89':['Int64'],'a9':['Int64'],'a90':['Int64'],'a91':['Int64'],'a92':['Int64'],'a93':['Int64'],'a94':['Int64'],'a95':['Int64'],'a96':['Int64'],'a97':['Int64'],'a98':['Int64'],'a99':['Int64']} +0 +100 +200 +300 +400 +500 +600 +700 +800 +900 +0 +99 +199 +299 +399 +499 +599 +699 +799 +899 +999 +{'a0':['Int64'],'a1':['Int64'],'a10':['Int64'],'a11':['Int64'],'a12':['Int64'],'a13':['Int64'],'a14':['Int64'],'a15':['Int64'],'a16':['Int64'],'a17':['Int64'],'a18':['Int64'],'a19':['Int64'],'a2':['Int64'],'a20':['Int64'],'a21':['Int64'],'a22':['Int64'],'a23':['Int64'],'a24':['Int64'],'a25':['Int64'],'a26':['Int64'],'a27':['Int64'],'a28':['Int64'],'a29':['Int64'],'a3':['Int64'],'a30':['Int64'],'a31':['Int64'],'a32':['Int64'],'a33':['Int64'],'a34':['Int64'],'a35':['Int64'],'a36':['Int64'],'a37':['Int64'],'a38':['Int64'],'a39':['Int64'],'a4':['Int64'],'a40':['Int64'],'a41':['Int64'],'a42':['Int64'],'a43':['Int64'],'a44':['Int64'],'a45':['Int64'],'a46':['Int64'],'a47':['Int64'],'a48':['Int64'],'a49':['Int64'],'a5':['Int64'],'a50':['Int64'],'a51':['Int64'],'a52':['Int64'],'a53':['Int64'],'a54':['Int64'],'a55':['Int64'],'a56':['Int64'],'a57':['Int64'],'a58':['Int64'],'a59':['Int64'],'a6':['Int64'],'a60':['Int64'],'a61':['Int64'],'a62':['Int64'],'a63':['Int64'],'a64':['Int64'],'a65':['Int64'],'a66':['Int64'],'a67':['Int64'],'a68':['Int64'],'a69':['Int64'],'a7':['Int64'],'a70':['Int64'],'a71':['Int64'],'a72':['Int64'],'a73':['Int64'],'a74':['Int64'],'a75':['Int64'],'a76':['Int64'],'a77':['Int64'],'a78':['Int64'],'a79':['Int64'],'a8':['Int64'],'a80':['Int64'],'a81':['Int64'],'a82':['Int64'],'a83':['Int64'],'a84':['Int64'],'a85':['Int64'],'a86':['Int64'],'a87':['Int64'],'a88':['Int64'],'a89':['Int64'],'a9':['Int64'],'a90':['Int64'],'a91':['Int64'],'a92':['Int64'],'a93':['Int64'],'a94':['Int64'],'a95':['Int64'],'a96':['Int64'],'a97':['Int64'],'a98':['Int64'],'a99':['Int64']} +0 +100 +200 +300 +400 +500 +600 +700 +800 +900 +0 +99 +199 +299 +399 +499 +599 +699 +799 +899 +999 +{'a0':['Int64'],'a1':['Int64'],'a10':['Int64'],'a11':['Int64'],'a12':['Int64'],'a13':['Int64'],'a14':['Int64'],'a15':['Int64'],'a16':['Int64'],'a17':['Int64'],'a18':['Int64'],'a19':['Int64'],'a2':['Int64'],'a20':['Int64'],'a21':['Int64'],'a22':['Int64'],'a23':['Int64'],'a24':['Int64'],'a25':['Int64'],'a26':['Int64'],'a27':['Int64'],'a28':['Int64'],'a29':['Int64'],'a3':['Int64'],'a30':['Int64'],'a31':['Int64'],'a32':['Int64'],'a33':['Int64'],'a34':['Int64'],'a35':['Int64'],'a36':['Int64'],'a37':['Int64'],'a38':['Int64'],'a39':['Int64'],'a4':['Int64'],'a40':['Int64'],'a41':['Int64'],'a42':['Int64'],'a43':['Int64'],'a44':['Int64'],'a45':['Int64'],'a46':['Int64'],'a47':['Int64'],'a48':['Int64'],'a49':['Int64'],'a5':['Int64'],'a50':['Int64'],'a51':['Int64'],'a52':['Int64'],'a53':['Int64'],'a54':['Int64'],'a55':['Int64'],'a56':['Int64'],'a57':['Int64'],'a58':['Int64'],'a59':['Int64'],'a6':['Int64'],'a60':['Int64'],'a61':['Int64'],'a62':['Int64'],'a63':['Int64'],'a64':['Int64'],'a65':['Int64'],'a66':['Int64'],'a67':['Int64'],'a68':['Int64'],'a69':['Int64'],'a7':['Int64'],'a70':['Int64'],'a71':['Int64'],'a72':['Int64'],'a73':['Int64'],'a74':['Int64'],'a75':['Int64'],'a76':['Int64'],'a77':['Int64'],'a78':['Int64'],'a79':['Int64'],'a8':['Int64'],'a80':['Int64'],'a81':['Int64'],'a82':['Int64'],'a83':['Int64'],'a84':['Int64'],'a85':['Int64'],'a86':['Int64'],'a87':['Int64'],'a88':['Int64'],'a89':['Int64'],'a9':['Int64'],'a90':['Int64'],'a91':['Int64'],'a92':['Int64'],'a93':['Int64'],'a94':['Int64'],'a95':['Int64'],'a96':['Int64'],'a97':['Int64'],'a98':['Int64'],'a99':['Int64']} +0 +100 +200 +300 +400 +500 +600 +700 +800 +900 +0 +99 +199 +299 +399 +499 +599 +699 +799 +899 +999 +{'a0':['Int64'],'a1':['Int64'],'a10':['Int64'],'a11':['Int64'],'a12':['Int64'],'a13':['Int64'],'a14':['Int64'],'a15':['Int64'],'a16':['Int64'],'a17':['Int64'],'a18':['Int64'],'a19':['Int64'],'a2':['Int64'],'a20':['Int64'],'a21':['Int64'],'a22':['Int64'],'a23':['Int64'],'a24':['Int64'],'a25':['Int64'],'a26':['Int64'],'a27':['Int64'],'a28':['Int64'],'a29':['Int64'],'a3':['Int64'],'a30':['Int64'],'a31':['Int64'],'a32':['Int64'],'a33':['Int64'],'a34':['Int64'],'a35':['Int64'],'a36':['Int64'],'a37':['Int64'],'a38':['Int64'],'a39':['Int64'],'a4':['Int64'],'a40':['Int64'],'a41':['Int64'],'a42':['Int64'],'a43':['Int64'],'a44':['Int64'],'a45':['Int64'],'a46':['Int64'],'a47':['Int64'],'a48':['Int64'],'a49':['Int64'],'a5':['Int64'],'a50':['Int64'],'a51':['Int64'],'a52':['Int64'],'a53':['Int64'],'a54':['Int64'],'a55':['Int64'],'a56':['Int64'],'a57':['Int64'],'a58':['Int64'],'a59':['Int64'],'a6':['Int64'],'a60':['Int64'],'a61':['Int64'],'a62':['Int64'],'a63':['Int64'],'a64':['Int64'],'a65':['Int64'],'a66':['Int64'],'a67':['Int64'],'a68':['Int64'],'a69':['Int64'],'a7':['Int64'],'a70':['Int64'],'a71':['Int64'],'a72':['Int64'],'a73':['Int64'],'a74':['Int64'],'a75':['Int64'],'a76':['Int64'],'a77':['Int64'],'a78':['Int64'],'a79':['Int64'],'a8':['Int64'],'a80':['Int64'],'a81':['Int64'],'a82':['Int64'],'a83':['Int64'],'a84':['Int64'],'a85':['Int64'],'a86':['Int64'],'a87':['Int64'],'a88':['Int64'],'a89':['Int64'],'a9':['Int64'],'a90':['Int64'],'a91':['Int64'],'a92':['Int64'],'a93':['Int64'],'a94':['Int64'],'a95':['Int64'],'a96':['Int64'],'a97':['Int64'],'a98':['Int64'],'a99':['Int64']} +0 +100 +200 +300 +400 +500 +600 +700 +800 +900 +0 +99 +199 +299 +399 +499 +599 +699 +799 +899 +999 +{'a0':['Int64'],'a1':['Int64'],'a10':['Int64'],'a11':['Int64'],'a12':['Int64'],'a13':['Int64'],'a14':['Int64'],'a15':['Int64'],'a16':['Int64'],'a17':['Int64'],'a18':['Int64'],'a19':['Int64'],'a2':['Int64'],'a20':['Int64'],'a21':['Int64'],'a22':['Int64'],'a23':['Int64'],'a24':['Int64'],'a25':['Int64'],'a26':['Int64'],'a27':['Int64'],'a28':['Int64'],'a29':['Int64'],'a3':['Int64'],'a30':['Int64'],'a31':['Int64'],'a32':['Int64'],'a33':['Int64'],'a34':['Int64'],'a35':['Int64'],'a36':['Int64'],'a37':['Int64'],'a38':['Int64'],'a39':['Int64'],'a4':['Int64'],'a40':['Int64'],'a41':['Int64'],'a42':['Int64'],'a43':['Int64'],'a44':['Int64'],'a45':['Int64'],'a46':['Int64'],'a47':['Int64'],'a48':['Int64'],'a49':['Int64'],'a5':['Int64'],'a50':['Int64'],'a51':['Int64'],'a52':['Int64'],'a53':['Int64'],'a54':['Int64'],'a55':['Int64'],'a56':['Int64'],'a57':['Int64'],'a58':['Int64'],'a59':['Int64'],'a6':['Int64'],'a60':['Int64'],'a61':['Int64'],'a62':['Int64'],'a63':['Int64'],'a64':['Int64'],'a65':['Int64'],'a66':['Int64'],'a67':['Int64'],'a68':['Int64'],'a69':['Int64'],'a7':['Int64'],'a70':['Int64'],'a71':['Int64'],'a72':['Int64'],'a73':['Int64'],'a74':['Int64'],'a75':['Int64'],'a76':['Int64'],'a77':['Int64'],'a78':['Int64'],'a79':['Int64'],'a8':['Int64'],'a80':['Int64'],'a81':['Int64'],'a82':['Int64'],'a83':['Int64'],'a84':['Int64'],'a85':['Int64'],'a86':['Int64'],'a87':['Int64'],'a88':['Int64'],'a89':['Int64'],'a9':['Int64'],'a90':['Int64'],'a91':['Int64'],'a92':['Int64'],'a93':['Int64'],'a94':['Int64'],'a95':['Int64'],'a96':['Int64'],'a97':['Int64'],'a98':['Int64'],'a99':['Int64']} +0 +100 +200 +300 +400 +500 +600 +700 +800 +900 +0 +99 +199 +299 +399 +499 +599 +699 +799 +899 +999 +{'a0':['Int64'],'a1':['Int64'],'a10':['Int64'],'a11':['Int64'],'a12':['Int64'],'a13':['Int64'],'a14':['Int64'],'a15':['Int64'],'a16':['Int64'],'a17':['Int64'],'a18':['Int64'],'a19':['Int64'],'a2':['Int64'],'a20':['Int64'],'a21':['Int64'],'a22':['Int64'],'a23':['Int64'],'a24':['Int64'],'a25':['Int64'],'a26':['Int64'],'a27':['Int64'],'a28':['Int64'],'a29':['Int64'],'a3':['Int64'],'a30':['Int64'],'a31':['Int64'],'a32':['Int64'],'a33':['Int64'],'a34':['Int64'],'a35':['Int64'],'a36':['Int64'],'a37':['Int64'],'a38':['Int64'],'a39':['Int64'],'a4':['Int64'],'a40':['Int64'],'a41':['Int64'],'a42':['Int64'],'a43':['Int64'],'a44':['Int64'],'a45':['Int64'],'a46':['Int64'],'a47':['Int64'],'a48':['Int64'],'a49':['Int64'],'a5':['Int64'],'a50':['Int64'],'a51':['Int64'],'a52':['Int64'],'a53':['Int64'],'a54':['Int64'],'a55':['Int64'],'a56':['Int64'],'a57':['Int64'],'a58':['Int64'],'a59':['Int64'],'a6':['Int64'],'a60':['Int64'],'a61':['Int64'],'a62':['Int64'],'a63':['Int64'],'a64':['Int64'],'a65':['Int64'],'a66':['Int64'],'a67':['Int64'],'a68':['Int64'],'a69':['Int64'],'a7':['Int64'],'a70':['Int64'],'a71':['Int64'],'a72':['Int64'],'a73':['Int64'],'a74':['Int64'],'a75':['Int64'],'a76':['Int64'],'a77':['Int64'],'a78':['Int64'],'a79':['Int64'],'a8':['Int64'],'a80':['Int64'],'a81':['Int64'],'a82':['Int64'],'a83':['Int64'],'a84':['Int64'],'a85':['Int64'],'a86':['Int64'],'a87':['Int64'],'a88':['Int64'],'a89':['Int64'],'a9':['Int64'],'a90':['Int64'],'a91':['Int64'],'a92':['Int64'],'a93':['Int64'],'a94':['Int64'],'a95':['Int64'],'a96':['Int64'],'a97':['Int64'],'a98':['Int64'],'a99':['Int64']} +0 +100 +200 +300 +400 +500 +600 +700 +800 +900 +0 +99 +199 +299 +399 +499 +599 +699 +799 +899 +999 diff --git a/tests/queries/0_stateless/03270_object_to_json_alter.sql b/tests/queries/0_stateless/03270_object_to_json_alter.sql new file mode 100644 index 00000000000..61db4d0d33a --- /dev/null +++ b/tests/queries/0_stateless/03270_object_to_json_alter.sql @@ -0,0 +1,56 @@ +set allow_experimental_object_type = 1; +set allow_experimental_json_type = 1; +set max_block_size = 100; +set max_insert_block_size = 100; +set min_insert_block_size_rows = 100; +set output_format_json_quote_64bit_integers = 0; + +drop table if exists test; + +create table test (json Object('json')) engine=Memory; +insert into test select toJSONString(map('a' || number % 100, number)) from numbers(1000); +alter table test modify column json JSON; +select distinctJSONPathsAndTypes(json) from test; +select distinct json.a0 from test order by json.a0.:Int64; +select distinct json.a99 from test order by json.a99.:Int64; +drop table test; + +create table test (json Object('json')) engine=Memory; +insert into test select toJSONString(map('a' || number % 100, number)) from numbers(1000); +alter table test modify column json JSON(max_dynamic_paths=10); +select distinctJSONPathsAndTypes(json) from test; +select distinct json.a0 from test order by json.a0.:Int64; +select distinct json.a99 from test order by json.a99.:Int64; +drop table test; + +create table test (json Object('json')) engine=MergeTree order by tuple() settings min_rows_for_wide_part=10000000, min_bytes_for_wide_part=100000000; +insert into test select toJSONString(map('a' || number % 100, number)) from numbers(1000); +alter table test modify column json JSON; +select distinctJSONPathsAndTypes(json) from test; +select distinct json.a0 from test order by json.a0.:Int64; +select distinct json.a99 from test order by json.a99.:Int64; +drop table test; + +create table test (json Object('json')) engine=MergeTree order by tuple() settings min_rows_for_wide_part=10000000, min_bytes_for_wide_part=100000000; +insert into test select toJSONString(map('a' || number % 100, number)) from numbers(1000); +alter table test modify column json JSON(max_dynamic_paths=10); +select distinctJSONPathsAndTypes(json) from test; +select distinct json.a0 from test order by json.a0.:Int64; +select distinct json.a99 from test order by json.a99.:Int64; +drop table test; + +create table test (json Object('json')) engine=MergeTree order by tuple() settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; +insert into test select toJSONString(map('a' || number % 100, number)) from numbers(1000); +alter table test modify column json JSON(); +select distinctJSONPathsAndTypes(json) from test; +select distinct json.a0 from test order by json.a0.:Int64; +select distinct json.a99 from test order by json.a99.:Int64; +drop table test; + +create table test (json Object('json')) engine=MergeTree order by tuple() settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; +insert into test select toJSONString(map('a' || number % 100, number)) from numbers(1000); +alter table test modify column json JSON(max_dynamic_paths=10); +select distinctJSONPathsAndTypes(json) from test; +select distinct json.a0 from test order by json.a0.:Int64; +select distinct json.a99 from test order by json.a99.:Int64; +drop table test; diff --git a/tests/queries/0_stateless/03271_ghdata_object_to_json_alter.reference b/tests/queries/0_stateless/03271_ghdata_object_to_json_alter.reference new file mode 100644 index 00000000000..ca2fb7e8ff9 --- /dev/null +++ b/tests/queries/0_stateless/03271_ghdata_object_to_json_alter.reference @@ -0,0 +1,12 @@ +5000 +leonardomso/33-js-concepts 3 +ytdl-org/youtube-dl 3 +Bogdanp/neko 2 +bminossi/AllVideoPocsFromHackerOne 2 +disclose/diodata 2 +Commit 182 +chipeo345 119 +phanwi346 114 +Nicholas Piggin 95 +direwolf-github 49 +2 diff --git a/tests/queries/0_stateless/03271_ghdata_object_to_json_alter.sh b/tests/queries/0_stateless/03271_ghdata_object_to_json_alter.sh new file mode 100755 index 00000000000..fc217780aa3 --- /dev/null +++ b/tests/queries/0_stateless/03271_ghdata_object_to_json_alter.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data Object('json')) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_object_type + +cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} \ + --max_memory_usage 10G --query "INSERT INTO ghdata FORMAT JSONAsObject" + +${CLICKHOUSE_CLIENT} -q "ALTER TABLE ghdata MODIFY column data JSON SETTINGS mutations_sync=1" --allow_experimental_json_type 1 + +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ghdata WHERE NOT ignore(*)" + +${CLICKHOUSE_CLIENT} -q \ +"SELECT data.repo.name, count() AS stars FROM ghdata \ + WHERE data.type = 'WatchEvent' GROUP BY data.repo.name ORDER BY stars DESC, data.repo.name LIMIT 5" --allow_suspicious_types_in_group_by=1 --allow_suspicious_types_in_order_by=1 + +${CLICKHOUSE_CLIENT} --enable_analyzer=1 -q \ +"SELECT data.payload.commits[].author.name AS name, count() AS c FROM ghdata \ + ARRAY JOIN data.payload.commits[].author.name \ + GROUP BY name ORDER BY c DESC, name LIMIT 5" --allow_suspicious_types_in_group_by=1 --allow_suspicious_types_in_order_by=1 + +${CLICKHOUSE_CLIENT} -q "SELECT max(data.payload.pull_request.assignees[].size0) FROM ghdata" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata" diff --git a/tests/queries/0_stateless/03272_ghdata_object_to_json_alter_shared_data.reference b/tests/queries/0_stateless/03272_ghdata_object_to_json_alter_shared_data.reference new file mode 100644 index 00000000000..ca2fb7e8ff9 --- /dev/null +++ b/tests/queries/0_stateless/03272_ghdata_object_to_json_alter_shared_data.reference @@ -0,0 +1,12 @@ +5000 +leonardomso/33-js-concepts 3 +ytdl-org/youtube-dl 3 +Bogdanp/neko 2 +bminossi/AllVideoPocsFromHackerOne 2 +disclose/diodata 2 +Commit 182 +chipeo345 119 +phanwi346 114 +Nicholas Piggin 95 +direwolf-github 49 +2 diff --git a/tests/queries/0_stateless/03272_ghdata_object_to_json_alter_shared_data.sh b/tests/queries/0_stateless/03272_ghdata_object_to_json_alter_shared_data.sh new file mode 100755 index 00000000000..fad2833b74e --- /dev/null +++ b/tests/queries/0_stateless/03272_ghdata_object_to_json_alter_shared_data.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-msan, no-tsan, no-asan, long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data Object('json')) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_object_type=1 + +cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} \ + --max_memory_usage 10G --query "INSERT INTO ghdata FORMAT JSONAsObject" + +${CLICKHOUSE_CLIENT} -q "ALTER TABLE ghdata MODIFY column data JSON(max_dynamic_paths=8) SETTINGS mutations_sync=1" --allow_experimental_json_type 1 + +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ghdata WHERE NOT ignore(*)" + +${CLICKHOUSE_CLIENT} -q \ +"SELECT data.repo.name, count() AS stars FROM ghdata \ + WHERE data.type = 'WatchEvent' GROUP BY data.repo.name ORDER BY stars DESC, data.repo.name LIMIT 5" --allow_suspicious_types_in_group_by=1 --allow_suspicious_types_in_order_by=1 + +${CLICKHOUSE_CLIENT} --enable_analyzer=1 -q \ +"SELECT data.payload.commits[].author.name AS name, count() AS c FROM ghdata \ + ARRAY JOIN data.payload.commits[].author.name \ + GROUP BY name ORDER BY c DESC, name LIMIT 5" --allow_suspicious_types_in_group_by=1 --allow_suspicious_types_in_order_by=1 + +${CLICKHOUSE_CLIENT} -q "SELECT max(data.payload.pull_request.assignees[].size0) FROM ghdata" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata"