Add more tests and documentation, fix existing tests and special build

This commit is contained in:
avogar 2024-05-13 13:43:47 +00:00
parent cd3604f235
commit 4653ec618d
10 changed files with 442 additions and 10 deletions

View File

@ -261,7 +261,7 @@ SELECT d, dynamicType(d), d::Dynamic(max_types=1) as d2, dynamicType(d2) FROM te
└─────────┴────────────────┴─────────┴─────────────────┘
```
## Reading Variant type from the data
## Reading Dynamic type from the data
All text formats (TSV, CSV, CustomSeparated, Values, JSONEachRow, etc) supports reading `Dynamic` type. During data parsing ClickHouse tries to infer the type of each value and use it during insertion to `Dynamic` column.
@ -409,3 +409,87 @@ SELECT d, dynamicType(d) FROM test ORDER by d;
└─────┴────────────────┘
```
## Reaching the limit in number of different data types stored inside Dynamic
`Dynamic` data type can store only limited number of different data types inside. By default, this limit is 32, but you can change it in type declaration using syntax `Dynamic(max_types=N)` where N is between 1 and 255 (due to implementation details, it's impossible to have more than 255 different data types inside Dynamic).
When the limit is reached, all new data types inserted to `Dynamic` column will be casted to `String` and stored as `String` values.
Let's see what happens when the limit is reached in different scenarios.
### Reaching the limit during data parsing
During parsing of `Dynamic` values from the data, when the limit is reached for current block of data, all new values will be inserted as `String` values:
```sql
SELECT d, dynamicType(d) FROM format(JSONEachRow, 'd Dynamic(max_types=3)', '
{"d" : 42}
{"d" : [1, 2, 3]}
{"d" : "Hello, World!"}
{"d" : "2020-01-01"}
{"d" : ["str1", "str2", "str3"]}
{"d" : {"a" : 1, "b" : [1, 2, 3]}}
')
```
```text
┌─d──────────────────────────┬─dynamicType(d)─┐
│ 42 │ Int64 │
│ [1,2,3] │ Array(Int64) │
│ Hello, World! │ String │
│ 2020-01-01 │ String │
│ ["str1", "str2", "str3"] │ String │
│ {"a" : 1, "b" : [1, 2, 3]} │ String │
└────────────────────────────┴────────────────┘
```
As we can see, after inserting 3 different data types `Int64`, `Array(Int64)` and `String` all new types were converted to `String`.
### During merges of data parts in MergeTree table engines
During merge of several data parts in MergeTree table the `Dynamic` column in the resulting data part can reach the limit of different data types inside and won't be able to store all types from source parts.
In this case ClickHouse chooses what types will remain after merge and what types will be casted to `String`. In most cases ClickHouse tries to keep the most frequent types and cast the rarest types to `String`, but it depends on the implementation.
Let's see an example of such merge. First, let's create a table with `Dynamic` column, set the limit of different data types to `3` and insert values with `5` different types:
```sql
CREATE TABLE test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree ORDER BY id;
SYSTEM STOP MERGES test;
INSERT INTO test SELECT number, number FROM numbers(5);
INSERT INTO test SELECT number, range(number) FROM numbers(4);
INSERT INTO test SELECT number, toDate(number) FROM numbers(3);
INSERT INTO test SELECT number, map(number, number) FROM numbers(2);
INSERT INTO test SELECT number, 'str_' || toString(number) FROM numbers(1);
```
Each insert will create a separate data pert with `Dynamic` column containing single type:
```sql
SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) ORDER BY _part;
```
```text
┌─count()─┬─dynamicType(d)──────┬─_part─────┐
│ 5 │ UInt64 │ all_1_1_0 │
│ 4 │ Array(UInt64) │ all_2_2_0 │
│ 3 │ Date │ all_3_3_0 │
│ 2 │ Map(UInt64, UInt64) │ all_4_4_0 │
│ 1 │ String │ all_5_5_0 │
└─────────┴─────────────────────┴───────────┘
```
Now, let's merge all parts into one and see what will happen:
```sql
SYSTEM START MERGES test;
OPTIMIZE TABLE test FINAL;
SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) ORDER BY _part;
```
```text
┌─count()─┬─dynamicType(d)─┬─_part─────┐
│ 5 │ UInt64 │ all_1_5_2 │
│ 6 │ String │ all_1_5_2 │
│ 4 │ Array(UInt64) │ all_1_5_2 │
└─────────┴────────────────┴───────────┘
```
As we can see, ClickHouse kept the most frequent types `UInt64` and `Array(UInt64)` and casted all other types to `String`.

View File

@ -290,6 +290,13 @@ void ColumnDynamic::insertRangeFrom(const DB::IColumn & src_, size_t start, size
/// We cannot combine 2 Variant types as total number of variants exceeds the limit.
/// In this case we will add most frequent variants from this range and insert them as usual,
/// all other variants will be converted to String.
/// TODO: instead of keeping all current variants and just adding new most frequent variants
/// from source columns we can also try to replace rarest existing variants with frequent
/// variants from source column (so we will avoid casting new frequent variants to String
/// and keeping rare existing ones). It will require rewriting of existing data in Variant
/// column but will improve usability of Dynamic column for example during squashing blocks
/// during insert.
const auto & src_variant_column = dynamic_src.getVariantColumn();
/// Calculate ranges for each variant in current range.

View File

@ -12,7 +12,7 @@ class DataTypeDynamic final : public IDataType
public:
static constexpr bool is_parametric = true;
DataTypeDynamic(size_t max_dynamic_types_ = DEFAULT_MAX_DYNAMIC_TYPES);
explicit DataTypeDynamic(size_t max_dynamic_types_ = DEFAULT_MAX_DYNAMIC_TYPES);
TypeIndex getTypeId() const override { return TypeIndex::Dynamic; }
const char * getFamilyName() const override { return "Dynamic"; }

View File

@ -1,12 +1,12 @@
MergeTree compact + horizontal merge
ReplacingMergeTree
100000 UInt64
100000 String
100000 UInt64
50000 UInt64
100000 String
SummingMergeTree
100000 UInt64
100000 String
100000 UInt64
200000 1
50000 String
100000 UInt64
@ -22,8 +22,8 @@ AggregatingMergeTree
100000 1
MergeTree wide + horizontal merge
ReplacingMergeTree
100000 UInt64
100000 String
100000 UInt64
50000 UInt64
100000 String
SummingMergeTree
@ -49,16 +49,16 @@ ReplacingMergeTree
50000 UInt64
100000 String
SummingMergeTree
100000 UInt64
100000 String
100000 UInt64
200000 1
50000 String
100000 UInt64
50000 2
100000 1
AggregatingMergeTree
100000 UInt64
100000 String
100000 UInt64
200000 1
50000 String
100000 UInt64
@ -66,8 +66,8 @@ AggregatingMergeTree
100000 1
MergeTree wide + vertical merge
ReplacingMergeTree
100000 UInt64
100000 String
100000 UInt64
50000 UInt64
100000 String
SummingMergeTree
@ -79,8 +79,8 @@ SummingMergeTree
50000 2
100000 1
AggregatingMergeTree
100000 UInt64
100000 String
100000 UInt64
200000 1
50000 String
100000 UInt64

View File

@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT=
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_merge_tree_settings --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --stacktrace --max_insert_threads 3 --group_by_two_level_threshold 1000000 --group_by_two_level_threshold_bytes 42526602 --distributed_aggregation_memory_efficient 1 --fsync_metadata 1 --output_format_parallel_formatting 0 --input_format_parallel_parsing 0 --min_chunk_bytes_for_parallel_parsing 8125230 --max_read_buffer_size 859505 --prefer_localhost_replica 1 --max_block_size 34577 --max_threads 41 --optimize_append_index 0 --optimize_if_chain_to_multiif 1 --optimize_if_transform_strings_to_enum 1 --optimize_read_in_order 1 --optimize_or_like_chain 0 --optimize_substitute_columns 1 --enable_multiple_prewhere_read_steps 1 --read_in_order_two_level_merge_threshold 99 --optimize_aggregation_in_order 1 --aggregation_in_order_max_block_bytes 27635208 --use_uncompressed_cache 0 --min_bytes_to_use_direct_io 10737418240 --min_bytes_to_use_mmap_io 6451111320 --local_filesystem_read_method pread --remote_filesystem_read_method read --local_filesystem_read_prefetch 1 --filesystem_cache_segments_batch_size 50 --read_from_filesystem_cache_if_exists_otherwise_bypass_cache 0 --throw_on_error_from_cache_on_write_operations 0 --remote_filesystem_read_prefetch 1 --allow_prefetched_read_pool_for_remote_filesystem 0 --filesystem_prefetch_max_memory_usage 64Mi --filesystem_prefetches_limit 10 --filesystem_prefetch_min_bytes_for_single_read_task 16Mi --filesystem_prefetch_step_marks 0 --filesystem_prefetch_step_bytes 100Mi --compile_aggregate_expressions 0 --compile_sort_description 1 --merge_tree_coarse_index_granularity 32 --optimize_distinct_in_order 0 --max_bytes_before_external_sort 10737418240 --max_bytes_before_external_group_by 10737418240 --max_bytes_before_remerge_sort 1374192967 --min_compress_block_size 2152247 --max_compress_block_size 1830907 --merge_tree_compact_parts_min_granules_to_multibuffer_read 79 --optimize_sorting_by_input_stream_properties 1 --http_response_buffer_size 106072 --http_wait_end_of_query True --enable_memory_bound_merging_of_aggregation_results 0 --min_count_to_compile_expression 0 --min_count_to_compile_aggregate_expression 3 --min_count_to_compile_sort_description 3 --session_timezone Africa/Khartoum --prefer_warmed_unmerged_parts_seconds 4 --use_page_cache_for_disks_without_file_cache False --page_cache_inject_eviction True --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0.03 --ratio_of_defaults_for_sparse_serialization 0.9779014012142565 --prefer_fetch_merged_part_size_threshold 4254002758 --vertical_merge_algorithm_min_rows_to_activate 1 --vertical_merge_algorithm_min_columns_to_activate 1 --allow_vertical_merges_from_compact_to_wide_parts 1 --min_merge_bytes_to_use_direct_io 1 --index_granularity_bytes 4982992 --merge_max_block_size 16662 --index_granularity 22872 --min_bytes_for_wide_part 1073741824 --compress_marks 0 --compress_primary_key 0 --marks_compress_block_size 86328 --primary_key_compress_block_size 64101 --replace_long_file_name_to_hash 0 --max_file_name_length 81 --min_bytes_for_full_part_storage 536870912 --compact_parts_max_bytes_to_buffer 480908080 --compact_parts_max_granules_to_buffer 1 --compact_parts_merge_max_bytes_to_prefetch_part 4535313 --cache_populated_by_fetch 0"
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1"
function run()
{
@ -74,3 +74,4 @@ echo "MergeTree wide"
$CH_CLIENT -q "create table test (x UInt64, y UInt64 ) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
run
$CH_CLIENT -q "drop table test;"

View File

@ -0,0 +1,182 @@
MergeTree compact
initial insert
alter add column
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
insert after alter add column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
alter rename column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
insert nested dynamic
3 Array(Dynamic)
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N [] [] []
1 1 \N \N \N \N \N [] [] []
2 2 \N \N \N \N \N [] [] []
3 3 3 \N 3 \N \N [] [] []
4 4 4 \N 4 \N \N [] [] []
5 5 5 \N 5 \N \N [] [] []
6 6 str_6 str_6 \N \N \N [] [] []
7 7 str_7 str_7 \N \N \N [] [] []
8 8 str_8 str_8 \N \N \N [] [] []
9 9 \N \N \N \N \N [] [] []
10 10 \N \N \N \N \N [] [] []
11 11 \N \N \N \N \N [] [] []
12 12 12 \N 12 \N \N [] [] []
13 13 str_13 str_13 \N \N \N [] [] []
14 14 \N \N \N \N \N [] [] []
15 15 [15] \N \N \N \N [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N \N [17] [NULL] [NULL]
alter rename column 2
3 Array(Dynamic)
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N [] [] []
1 1 \N \N \N \N \N [] [] []
2 2 \N \N \N \N \N [] [] []
3 3 3 \N 3 \N \N [] [] []
4 4 4 \N 4 \N \N [] [] []
5 5 5 \N 5 \N \N [] [] []
6 6 str_6 str_6 \N \N \N [] [] []
7 7 str_7 str_7 \N \N \N [] [] []
8 8 str_8 str_8 \N \N \N [] [] []
9 9 \N \N \N \N \N [] [] []
10 10 \N \N \N \N \N [] [] []
11 11 \N \N \N \N \N [] [] []
12 12 12 \N 12 \N \N [] [] []
13 13 str_13 str_13 \N \N \N [] [] []
14 14 \N \N \N \N \N [] [] []
15 15 [15] \N \N \N \N [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N \N [17] [NULL] [NULL]
MergeTree wide
initial insert
alter add column
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
insert after alter add column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
alter rename column 1
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
insert nested dynamic
3 Array(Dynamic)
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N [] [] []
1 1 \N \N \N \N \N [] [] []
2 2 \N \N \N \N \N [] [] []
3 3 3 \N 3 \N \N [] [] []
4 4 4 \N 4 \N \N [] [] []
5 5 5 \N 5 \N \N [] [] []
6 6 str_6 str_6 \N \N \N [] [] []
7 7 str_7 str_7 \N \N \N [] [] []
8 8 str_8 str_8 \N \N \N [] [] []
9 9 \N \N \N \N \N [] [] []
10 10 \N \N \N \N \N [] [] []
11 11 \N \N \N \N \N [] [] []
12 12 12 \N 12 \N \N [] [] []
13 13 str_13 str_13 \N \N \N [] [] []
14 14 \N \N \N \N \N [] [] []
15 15 [15] \N \N \N \N [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N \N [17] [NULL] [NULL]
alter rename column 2
3 Array(Dynamic)
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N [] [] []
1 1 \N \N \N \N \N [] [] []
2 2 \N \N \N \N \N [] [] []
3 3 3 \N 3 \N \N [] [] []
4 4 4 \N 4 \N \N [] [] []
5 5 5 \N 5 \N \N [] [] []
6 6 str_6 str_6 \N \N \N [] [] []
7 7 str_7 str_7 \N \N \N [] [] []
8 8 str_8 str_8 \N \N \N [] [] []
9 9 \N \N \N \N \N [] [] []
10 10 \N \N \N \N \N [] [] []
11 11 \N \N \N \N \N [] [] []
12 12 12 \N 12 \N \N [] [] []
13 13 str_13 str_13 \N \N \N [] [] []
14 14 \N \N \N \N \N [] [] []
15 15 [15] \N \N \N \N [15] [NULL] [NULL]
16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL]
17 17 [17] \N \N \N \N [17] [NULL] [NULL]

View File

@ -0,0 +1,57 @@
#!/usr/bin/env bash
# Tags: long
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# reset --log_comment
CLICKHOUSE_LOG_COMMENT=
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1"
function run()
{
echo "initial insert"
$CH_CLIENT -q "insert into test select number, number from numbers(3)"
echo "alter add column"
$CH_CLIENT -q "alter table test add column d Dynamic settings mutations_sync=1"
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)"
$CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.\`Tuple(a UInt64)\`.a from test order by x"
echo "insert after alter add column 1"
$CH_CLIENT -q "insert into test select number, number, number from numbers(3, 3)"
$CH_CLIENT -q "insert into test select number, number, 'str_' || toString(number) from numbers(6, 3)"
$CH_CLIENT -q "insert into test select number, number, NULL from numbers(9, 3)"
$CH_CLIENT -q "insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3)"
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)"
$CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x"
echo "alter rename column 1"
$CH_CLIENT -q "alter table test rename column d to d1 settings mutations_sync=1"
$CH_CLIENT -q "select count(), dynamicType(d1) from test group by dynamicType(d1) order by count(), dynamicType(d1)"
$CH_CLIENT -q "select x, y, d1, d1.String, d1.UInt64, d1.Date, d1.\`Tuple(a UInt64)\`.a from test order by x"
echo "insert nested dynamic"
$CH_CLIENT -q "insert into test select number, number, [number % 2 ? number : 'str_' || toString(number)]::Array(Dynamic) from numbers(15, 3)"
$CH_CLIENT -q "select count(), dynamicType(d1) from test group by dynamicType(d1) order by count(), dynamicType(d1)"
$CH_CLIENT -q "select x, y, d1, d1.String, d1.UInt64, d1.Date, d1.\`Tuple(a UInt64)\`.a, d1.\`Array(Dynamic)\`.UInt64, d1.\`Array(Dynamic)\`.String, d1.\`Array(Dynamic)\`.Date from test order by x"
echo "alter rename column 2"
$CH_CLIENT -q "alter table test rename column d1 to d2 settings mutations_sync=1"
$CH_CLIENT -q "select count(), dynamicType(d2) from test group by dynamicType(d2) order by count(), dynamicType(d2)"
$CH_CLIENT -q "select x, y, d2, d2.String, d2.UInt64, d2.Date, d2.\`Tuple(a UInt64)\`.a, d2.\`Array(Dynamic)\`.UInt64, d2.\`Array(Dynamic)\`.String, d2.\`Array(Dynamic)\`.Date, from test order by x"
}
$CH_CLIENT -q "drop table if exists test;"
echo "MergeTree compact"
$CH_CLIENT -q "create table test (x UInt64, y UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;"
run
$CH_CLIENT -q "drop table test;"
echo "MergeTree wide"
$CH_CLIENT -q "create table test (x UInt64, y UInt64 ) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
run
$CH_CLIENT -q "drop table test;"

View File

@ -0,0 +1,56 @@
MergeTree compact
initial insert
alter add column
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
insert after alter add column
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
check table
1
MergeTree wide
initial insert
alter add column
3 None
0 0 \N \N \N \N
1 1 \N \N \N \N
2 2 \N \N \N \N
insert after alter add column
4 String
4 UInt64
7 None
0 0 \N \N \N \N \N
1 1 \N \N \N \N \N
2 2 \N \N \N \N \N
3 3 3 \N 3 \N \N
4 4 4 \N 4 \N \N
5 5 5 \N 5 \N \N
6 6 str_6 str_6 \N \N \N
7 7 str_7 str_7 \N \N \N
8 8 str_8 str_8 \N \N \N
9 9 \N \N \N \N \N
10 10 \N \N \N \N \N
11 11 \N \N \N \N \N
12 12 12 \N 12 \N \N
13 13 str_13 str_13 \N \N \N
14 14 \N \N \N \N \N
check table
1

View File

@ -0,0 +1,45 @@
#!/usr/bin/env bash
# Tags: long
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# reset --log_comment
CLICKHOUSE_LOG_COMMENT=
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1"
function run()
{
echo "initial insert"
$CH_CLIENT -q "insert into test select number, number from numbers(3)"
echo "alter add column"
$CH_CLIENT -q "alter table test add column d Dynamic(max_types=3) settings mutations_sync=1"
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)"
$CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.\`Tuple(a UInt64)\`.a from test order by x"
echo "insert after alter add column"
$CH_CLIENT -q "insert into test select number, number, number from numbers(3, 3)"
$CH_CLIENT -q "insert into test select number, number, 'str_' || toString(number) from numbers(6, 3)"
$CH_CLIENT -q "insert into test select number, number, NULL from numbers(9, 3)"
$CH_CLIENT -q "insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3)"
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)"
$CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.Date, d.\`Tuple(a UInt64)\`.a from test order by x"
echo "check table"
$CH_CLIENT -q "check table test"
}
$CH_CLIENT -q "drop table if exists test;"
echo "MergeTree compact"
$CH_CLIENT -q "create table test (x UInt64, y UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;"
run
$CH_CLIENT -q "drop table test;"
echo "MergeTree wide"
$CH_CLIENT -q "create table test (x UInt64, y UInt64 ) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
run
$CH_CLIENT -q "drop table test;"