mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #9070 from urykhy/fix-mv-insert-2621
add option to force deduplicate children
This commit is contained in:
commit
7b511b2d90
@ -396,6 +396,7 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingBool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \
|
||||
\
|
||||
M(SettingBool, partial_revokes, false, "Makes it possible to revoke privileges partially.", 0) \
|
||||
M(SettingBool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \
|
||||
\
|
||||
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
|
||||
\
|
||||
|
@ -29,7 +29,9 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
|
||||
/// If the "root" table deduplactes blocks, there are no need to make deduplication for children
|
||||
/// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks
|
||||
bool disable_deduplication_for_children = !no_destination && storage->supportsDeduplication();
|
||||
bool disable_deduplication_for_children = false;
|
||||
if (!context.getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)
|
||||
disable_deduplication_for_children = !no_destination && storage->supportsDeduplication();
|
||||
|
||||
auto table_id = storage->getStorageID();
|
||||
Dependencies dependencies = context.getDependencies(table_id);
|
||||
@ -130,7 +132,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block)
|
||||
}
|
||||
|
||||
/// Don't process materialized views if this block is duplicate
|
||||
if (replicated_output && replicated_output->lastBlockIsDuplicate())
|
||||
if (!context.getSettingsRef().deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate())
|
||||
return;
|
||||
|
||||
// Insert data into materialized views only after successful insert into main table
|
||||
|
44
dbms/tests/integration/test_force_deduplication/test.py
Normal file
44
dbms/tests/integration/test_force_deduplication/test.py
Normal file
@ -0,0 +1,44 @@
|
||||
# pylint: disable=unused-argument
|
||||
# pylint: disable=redefined-outer-name
|
||||
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.client import QueryRuntimeException
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node = cluster.add_instance('node', with_zookeeper=True)
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
def test_basic(start_cluster):
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node.query(
|
||||
'''
|
||||
CREATE TABLE test (A Int64) ENGINE = ReplicatedMergeTree ('/clickhouse/test/tables/test','1') ORDER BY tuple();
|
||||
CREATE MATERIALIZED VIEW test_mv Engine=ReplicatedMergeTree ('/clickhouse/test/tables/test_mv','1') partition by A order by tuple() AS SELECT A FROM test;
|
||||
SET max_partitions_per_insert_block = 3;
|
||||
INSERT INTO test SELECT number FROM numbers(10);
|
||||
'''
|
||||
)
|
||||
|
||||
node.query("INSERT INTO test SELECT number FROM numbers(10)")
|
||||
assert int(node.query("SELECT count() FROM test")) == 10
|
||||
assert int(node.query("SELECT count() FROM test_mv")) == 0
|
||||
|
||||
node.query(
|
||||
'''
|
||||
SET deduplicate_blocks_in_dependent_materialized_views = 1;
|
||||
INSERT INTO test SELECT number FROM numbers(10);
|
||||
'''
|
||||
)
|
||||
assert int(node.query("SELECT count() FROM test")) == 10
|
||||
assert int(node.query("SELECT count() FROM test_mv")) == 10
|
Loading…
Reference in New Issue
Block a user