Merge pull request #9070 from urykhy/fix-mv-insert-2621

add option to force deduplicate children
This commit is contained in:
alexey-milovidov 2020-02-28 03:19:27 +03:00 committed by GitHub
commit 7b511b2d90
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 49 additions and 2 deletions

View File

@ -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. */ \
\

View File

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

View 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