Merge pull request #58970 from ClickHouse/dont-optimize-trivial-insert-select

Disable `optimize_trivial_insert_select` by default
This commit is contained in:
Alexey Milovidov 2024-07-02 01:33:23 +00:00 committed by GitHub
commit 8a8b6a881b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
35 changed files with 62 additions and 23 deletions

View File

@ -378,7 +378,7 @@ class IColumn;
M(UInt64, http_max_fields, 1000000, "Maximum number of fields in HTTP header", 0) \
M(UInt64, http_max_field_name_size, 128 * 1024, "Maximum length of field name in HTTP header", 0) \
M(UInt64, http_max_field_value_size, 128 * 1024, "Maximum length of field value in HTTP header", 0) \
M(Bool, http_skip_not_found_url_for_globs, true, "Skip url's for globs with HTTP_NOT_FOUND error", 0) \
M(Bool, http_skip_not_found_url_for_globs, true, "Skip URLs for globs with HTTP_NOT_FOUND error", 0) \
M(Bool, http_make_head_request, true, "Allows the execution of a `HEAD` request while reading data from HTTP to retrieve information about the file to be read, such as its size", 0) \
M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \
M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \
@ -657,7 +657,7 @@ class IColumn;
M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \
M(Bool, system_events_show_zero_values, false, "When querying system.events or system.metrics tables, include all metrics, even with zero values.", 0) \
M(MySQLDataTypesSupport, mysql_datatypes_support_level, MySQLDataTypesSupportList{}, "Defines how MySQL types are converted to corresponding ClickHouse types. A comma separated list in any combination of 'decimal', 'datetime64', 'date2Date32' or 'date2String'. decimal: convert NUMERIC and DECIMAL types to Decimal when precision allows it. datetime64: convert DATETIME and TIMESTAMP types to DateTime64 instead of DateTime when precision is not 0. date2Date32: convert DATE to Date32 instead of Date. Takes precedence over date2String. date2String: convert DATE to String instead of Date. Overridden by datetime64.", 0) \
M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \
M(Bool, optimize_trivial_insert_select, false, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \
M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \
M(Bool, enable_global_with_statement, true, "Propagate WITH statements to UNION queries and all subqueries", 0) \
M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \
@ -724,8 +724,8 @@ class IColumn;
M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \
M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \
M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \
M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \
M(Bool, enable_url_encoding, true, " Allows to enable/disable decoding/encoding path in uri in URL table engine", 0) \
M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in the URL table engine", 0) \
M(Bool, enable_url_encoding, true, " Allows to enable/disable decoding/encoding path in URI in the URL table engine", 0) \
M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \
M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \
M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \
@ -857,7 +857,7 @@ class IColumn;
M(Bool, schema_inference_use_cache_for_azure, true, "Use cache in schema inference while using azure table function", 0) \
M(Bool, schema_inference_use_cache_for_hdfs, true, "Use cache in schema inference while using hdfs table function", 0) \
M(Bool, schema_inference_use_cache_for_url, true, "Use cache in schema inference while using url table function", 0) \
M(Bool, schema_inference_cache_require_modification_time_for_url, true, "Use schema from cache for URL with last modification time validation (for urls with Last-Modified header)", 0) \
M(Bool, schema_inference_cache_require_modification_time_for_url, true, "Use schema from cache for URL with last modification time validation (for URLs with Last-Modified header)", 0) \
\
M(String, compatibility, "", "Changes other settings according to provided ClickHouse version. If we know that we changed some behaviour in ClickHouse by changing some settings in some version, this compatibility setting will control these settings", 0) \
\

View File

@ -58,6 +58,7 @@ String ClickHouseVersion::toString() const
static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory::SettingsChanges>> settings_changes_history_initializer =
{
{"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."},
{"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."},
}},
{"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"},
{"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"},

View File

@ -962,7 +962,7 @@ def test_s3_engine_heavy_write_check_mem(
"INSERT INTO s3_test SELECT number, toString(number) FROM numbers(50000000)"
f" SETTINGS "
f" max_memory_usage={2*memory}"
f", max_threads=1" # ParallelFormattingOutputFormat consumption depends on it
", max_threads=1, optimize_trivial_insert_select=1" # ParallelFormattingOutputFormat consumption depends on it
f", s3_max_inflight_parts_for_one_file={in_flight}",
query_id=query_id,
)
@ -1010,9 +1010,10 @@ def test_s3_disk_heavy_write_check_mem(cluster, broken_s3, node_name):
node.query(
"INSERT INTO s3_test SELECT number, toString(number) FROM numbers(50000000)"
f" SETTINGS max_memory_usage={2*memory}"
f", max_insert_block_size=50000000"
f", min_insert_block_size_rows=50000000"
f", min_insert_block_size_bytes=1000000000000",
", max_insert_block_size=50000000"
", min_insert_block_size_rows=50000000"
", min_insert_block_size_bytes=1000000000000"
", optimize_trivial_insert_select=1",
query_id=query_id,
)

View File

@ -2,6 +2,7 @@
set allow_deprecated_syntax_for_merge_tree=1;
set optimize_on_insert = 0;
set optimize_trivial_insert_select = 1;
drop table if exists mult_tab;
create table mult_tab (date Date, value String, version UInt64, sign Int8) engine = VersionedCollapsingMergeTree(date, (date), 8192, sign, version);

View File

@ -17,6 +17,8 @@ select
from system.numbers
limit 10000000;
OPTIMIZE TABLE set_array FINAL;
SET max_rows_to_read = 8192;
select count() from set_array where has(index_array, 333);

View File

@ -1,4 +1,6 @@
SET optimize_trivial_insert_select = 1;
SET check_query_single_value_result = 0;
DROP TABLE IF EXISTS check_query_test;
CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0;

View File

@ -16,8 +16,8 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE s1 (a UInt32, s String) ENGINE = MergeTree O
$CLICKHOUSE_CLIENT -q "CREATE TABLE s2 (a UInt32, s String) ENGINE = MergeTree ORDER BY a PARTITION BY a % 3 SETTINGS min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0"
$CLICKHOUSE_CLIENT -q "CREATE TABLE m (a UInt32, s String) engine = Merge('$CLICKHOUSE_DATABASE', 's[1,2]')"
$CLICKHOUSE_CLIENT -q "INSERT INTO s1 select (number % 20) * 2 as n, toString(number * number) from numbers(100000)"
$CLICKHOUSE_CLIENT -q "INSERT INTO s2 select (number % 20) * 2 + 1 as n, toString(number * number * number) from numbers(100000)"
$CLICKHOUSE_CLIENT --optimize_trivial_insert_select 1 -q "INSERT INTO s1 select (number % 20) * 2 as n, toString(number * number) from numbers(100000)"
$CLICKHOUSE_CLIENT --optimize_trivial_insert_select 1 -q "INSERT INTO s2 select (number % 20) * 2 + 1 as n, toString(number * number * number) from numbers(100000)"
$CLICKHOUSE_CLIENT -q "SELECT '---StorageMerge---'"
$CLICKHOUSE_CLIENT -q "SELECT a FROM m ORDER BY a LIMIT 5"

View File

@ -25,7 +25,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE table_for_mutations(k UInt32, v1 UInt
${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES table_for_mutations"
${CLICKHOUSE_CLIENT} --query="INSERT INTO table_for_mutations select number, number from numbers(100000)"
${CLICKHOUSE_CLIENT} --optimize_trivial_insert_select 1 --query="INSERT INTO table_for_mutations select number, number from numbers(100000)"
${CLICKHOUSE_CLIENT} --query="SELECT sum(v1) FROM table_for_mutations"
@ -53,7 +53,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE replicated_table_for_mutations(k UInt
${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES replicated_table_for_mutations"
# test relays on part ids, which are non-deterministic with keeper fault injections, so disable it
${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)"
${CLICKHOUSE_CLIENT} --optimize_trivial_insert_select 1 --insert_keeper_fault_injection_probability=0 --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)"
${CLICKHOUSE_CLIENT} --query="SELECT sum(v1) FROM replicated_table_for_mutations"

View File

@ -1,4 +1,5 @@
-- Tags: no-debug, no-parallel, long, no-s3-storage, no-random-settings, no-random-merge-tree-settings
SET optimize_trivial_insert_select = 1;
DROP TABLE IF EXISTS table_with_single_pk;

View File

@ -1,3 +1,4 @@
SET optimize_trivial_insert_select = 1;
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
DROP TABLE IF EXISTS test;

View File

@ -3,6 +3,7 @@ SET allow_deprecated_error_prone_window_functions = 1;
DROP TABLE IF EXISTS t;
CREATE TABLE t (x UInt64) ENGINE = StripeLog;
-- For trivial INSERT SELECT, max_threads is lowered to max_insert_threads and max_block_size is changed to min_insert_block_size_rows.
SET optimize_trivial_insert_select = 1;
INSERT INTO t SELECT * FROM numbers_mt(1000000);
SET max_threads = 1;
-- If data was inserted by more threads, we will probably see data out of order.

View File

@ -10,6 +10,7 @@ PARTITION BY key%2
ORDER BY (key, key/2)
SETTINGS index_granularity=10, index_granularity_bytes='10Mi';
SET optimize_trivial_insert_select = 1;
INSERT INTO data_01551 SELECT number FROM numbers(100000);
SET max_threads=3;
SET merge_tree_min_rows_for_concurrent_read=10000;

View File

@ -24,6 +24,7 @@ enable_vertical_merge_algorithm = 0;
SET max_block_size=900;
-- There are about 900 marks for our settings.
SET optimize_trivial_insert_select = 1;
INSERT INTO adaptive_table SELECT number, if(number > 700, randomPrintableASCII(102400), randomPrintableASCII(1)) FROM numbers(10000);
OPTIMIZE TABLE adaptive_table FINAL;

View File

@ -5,6 +5,7 @@ SET optimize_using_constraints = 1;
SET optimize_move_to_prewhere = 1;
SET optimize_substitute_columns = 1;
SET optimize_append_index = 1;
SET optimize_trivial_insert_select = 1;
DROP TABLE IF EXISTS column_swap_test_test;

View File

@ -1,3 +1,5 @@
SET optimize_trivial_insert_select = 1;
DROP TABLE IF EXISTS t_sparse_distinct;
CREATE TABLE t_sparse_distinct (id UInt32, v UInt64)

View File

@ -1,3 +1,5 @@
SET optimize_trivial_insert_select = 1;
DROP TABLE IF EXISTS t_sparse;
CREATE TABLE t_sparse (id UInt64, u UInt64, s String)

View File

@ -5,6 +5,7 @@ DROP TABLE IF EXISTS t_json;
DROP TABLE IF EXISTS t_map;
SET allow_experimental_object_type = 1;
SET optimize_trivial_insert_select = 1;
CREATE TABLE t_json(id UInt64, obj JSON) ENGINE = MergeTree ORDER BY id;
CREATE TABLE t_map(id UInt64, m Map(String, UInt64)) ENGINE = MergeTree ORDER BY id;

View File

@ -3,6 +3,7 @@
DROP TABLE IF EXISTS t_json_sparse;
SET allow_experimental_object_type = 1;
SET optimize_trivial_insert_select = 1;
CREATE TABLE t_json_sparse (data JSON)
ENGINE = MergeTree ORDER BY tuple()

View File

@ -16,7 +16,7 @@ SELECT
FROM source_null
GROUP BY count_subquery, min_subquery, max_subquery;
SET optimize_trivial_insert_select = 1;
INSERT INTO source SELECT number FROM numbers(2000) SETTINGS min_insert_block_size_rows=1500, max_insert_block_size=1500;
SELECT count() FROM source;

View File

@ -1,5 +1,6 @@
SET max_threads=0;
SET optimize_read_in_order=1;
SET optimize_trivial_insert_select = 1;
SET read_in_order_two_level_merge_threshold=100;
DROP TABLE IF EXISTS t_read_in_order;

View File

@ -28,6 +28,7 @@ create materialized view mv_02231 to buffer_02231 as select
from in_02231
group by key;
set optimize_trivial_insert_select = 1;
insert into in_02231 select * from numbers(10e6) settings max_memory_usage='310Mi', max_threads=1;
drop table buffer_02231;

View File

@ -21,6 +21,7 @@ CREATE TABLE t_random_1
)
ENGINE = GenerateRandom(1, 5, 3);
SET optimize_trivial_insert_select = 1;
INSERT INTO t_1 select rowNumberInAllBlocks(), *, '1984-01-01' from t_random_1 limit 1000000;
OPTIMIZE TABLE t_1 FINAL;

View File

@ -1,5 +1,7 @@
-- Tags: no-random-merge-tree-settings
SET optimize_trivial_insert_select = 1;
drop table if exists test_02381;
create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) SETTINGS compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1;
insert into test_02381 select number, number * 10 from system.numbers limit 1000000;

View File

@ -7,6 +7,7 @@ drop table if exists t_different_dbs;
drop table if exists dist_t;
drop table if exists t;
set optimize_trivial_insert_select = 1;
create table t(a UInt64, b UInt64) engine=MergeTree order by a;
system stop merges t;

View File

@ -1,5 +1,7 @@
{% for index_granularity in [999, 1000, 1001, 9999, 10000, 10001] %}
SET optimize_trivial_insert_select = 1;
DROP TABLE IF EXISTS url_na_log;
CREATE TABLE url_na_log(SiteId UInt32, DateVisit Date, PRIMARY KEY (SiteId))

View File

@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "create table t(ts DateTime64) engine=MergeTree order by ts as select * from numbers_mt(1e6);"
$CLICKHOUSE_CLIENT --optimize_trivial_insert_select 1 -q "create table t(ts DateTime64) engine=MergeTree order by ts as select * from numbers_mt(1e6);"
max_block_size=8192

View File

@ -6,6 +6,7 @@ set max_threads = 16;
set allow_aggregate_partitions_independently = 1;
set force_aggregate_partitions_independently = 1;
set optimize_use_projections = 0;
set optimize_trivial_insert_select = 1;
set allow_prefetched_read_pool_for_remote_filesystem = 0;
set allow_prefetched_read_pool_for_local_filesystem = 0;

View File

@ -20,6 +20,7 @@ CREATE TABLE t0
)
ENGINE = MergeTree ORDER BY (c1, c2) settings min_bytes_for_wide_part = 10485760, min_rows_for_wide_part = 0;
SET optimize_trivial_insert_select = 1;
INSERT INTO t0 SELECT
number,
-number,

View File

@ -2,6 +2,8 @@
-- Tag: no-fasttest -- requires S3
-- Tag: no-replicated-database -- ALTER MOVE PARTITION TO should not be replicated (will be fixed separatelly)
SET optimize_trivial_insert_select = 1;
CREATE TABLE test_move_partition_throttling (key UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY tuple() SETTINGS storage_policy='local_remote';
INSERT INTO test_move_partition_throttling SELECT number FROM numbers(1e6);
SELECT disk_name, partition, rows FROM system.parts WHERE database = currentDatabase() AND table = 'test_move_partition_throttling' and active;

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# Normal importing, as we only insert 1000 rows, so it should be in memtable
${CLICKHOUSE_CLIENT} --query "CREATE TABLE IF NOT EXISTS rocksdb_worm (key UInt64, value UInt64) ENGINE = EmbeddedRocksDB() PRIMARY KEY key SETTINGS optimize_for_bulk_insert = 0;"
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000);"
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000) SETTINGS optimize_trivial_insert_select = 1;"
${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 0 because all data is still in memtable
${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;"
@ -16,32 +16,32 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE rocksdb_worm MODIFY SETTING optimize_f
# Testing that key serialization is identical w. and w/o bulk sink
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;"
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+2 FROM numbers(1000);" # should override previous keys
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+2 FROM numbers(1000) SETTINGS optimize_trivial_insert_select = 1;" # should override previous keys
${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm WHERE value = key + 2;"
# With bulk insertion, there is no memtable, so a small insert should create a new file
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;"
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000);"
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers(1000) SETTINGS optimize_trivial_insert_select = 1;"
${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 1
${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;"
# Testing insert with multiple sinks and fixed block size
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;"
# Must set both max_threads and max_insert_threads to 2 to make sure there is only two sinks
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_threads = 2, max_insert_threads = 2, max_block_size = 10000, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, insert_deduplication_token = '';"
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_threads = 2, max_insert_threads = 2, max_block_size = 10000, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, insert_deduplication_token = '', optimize_trivial_insert_select = 1;"
${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 2 because default bulk sink size is ~1M rows / SST file
${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;"
# Testing insert with duplicated keys
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;"
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number % 1000, number+1 FROM numbers_mt(1000000) SETTINGS max_block_size = 100000, max_insert_threads = 1;"
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number % 1000, number+1 FROM numbers_mt(1000000) SETTINGS max_block_size = 100000, max_insert_threads = 1, optimize_trivial_insert_select = 1;"
${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;"
${CLICKHOUSE_CLIENT} --query "SELECT * FROM rocksdb_worm WHERE key = 0;" # should be the latest value - 999001
# Testing insert with multiple threads
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;"
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000)" &
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000)" &
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS optimize_trivial_insert_select = 1" &
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS optimize_trivial_insert_select = 1" &
wait
${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;"

View File

@ -1,6 +1,7 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1;
SET optimize_trivial_insert_select = 1;
INSERT INTO test SELECT randomString(1000) FROM numbers(100000);
SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() AND table = 'test';

View File

@ -1,3 +1,5 @@
SET optimize_trivial_insert_select = 1;
drop table if exists x;
create table x (i int, j int, k int) engine MergeTree order by tuple() settings index_granularity=8192, index_granularity_bytes = '10Mi', min_bytes_for_wide_part=0, min_rows_for_wide_part=0, ratio_of_defaults_for_sparse_serialization=1;

View File

@ -12,6 +12,7 @@ system stop distributed sends dist_in;
create table dist_out as data engine=Distributed(test_shard_localhost, currentDatabase(), data);
set prefer_localhost_replica=0;
SET optimize_trivial_insert_select = 1;
insert into dist_in select number/100, number from system.numbers limit 1e6 settings max_memory_usage='20Mi';
system flush distributed dist_in; -- { serverError MEMORY_LIMIT_EXCEEDED }

View File

@ -1,3 +1,5 @@
SET optimize_trivial_insert_select = 1;
create table a (k UInt64, v UInt64, index i (v) type set(100) granularity 2) engine MergeTree order by k settings index_granularity=8192, index_granularity_bytes=1000000000, min_index_granularity_bytes=0;
insert into a select number, intDiv(number, 4096) from numbers(1000000);
select sum(1+ignore(*)) from a where indexHint(v in (20, 40));

View File

@ -1,3 +1,5 @@
SET optimize_trivial_insert_select = 1;
DROP TABLE IF EXISTS move_partition_to_oneself;
CREATE TABLE move_partition_to_oneself (key UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO move_partition_to_oneself SELECT number FROM numbers(1e6);