This commit is contained in:
Pavel Kruglov 2024-11-20 15:25:26 -08:00 committed by GitHub
commit 62045ccec2
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
19 changed files with 299 additions and 32 deletions

View File

@ -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)

View File

@ -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.

View File

@ -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;

View File

@ -4108,6 +4108,7 @@ private:
ColumnStringHelpers::WriteHelper write_helper(assert_cast<ColumnString &>(*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)
{

View File

@ -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

View File

@ -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, "

View File

@ -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",

View File

@ -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());

View File

@ -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<StorageSnapshot>(*this, metadata_snapshot);
auto data_parts = storage.getDataPartsVectorForInternalUsage();

View File

@ -686,7 +686,7 @@ std::optional<QueryProcessingStage::Enum> StorageDistributed::getOptimizedQueryP
static bool requiresObjectColumns(const ColumnsDescription & all_columns, ASTPtr query)
{
if (!hasDynamicSubcolumns(all_columns))
if (!hasDynamicSubcolumnsDeprecated(all_columns))
return false;
if (!query)

View File

@ -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<StorageSnapshot>(*this, metadata_snapshot, ColumnsDescription{}, std::move(snapshot_data));
auto object_columns = getConcreteObjectColumns(

View File

@ -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;

View File

@ -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))'}

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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"

View File

@ -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

View File

@ -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"