mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #5549 from yandex/fix_deduplication_bug
Fix complex bug in deduplication
This commit is contained in:
commit
bc387e865f
@ -319,10 +319,17 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
|
||||
{
|
||||
/// Can happen if there are leftover block nodes with children created by previous server versions.
|
||||
zookeeper->removeRecursive(path);
|
||||
cached_block_stats.erase(first_outdated_block->node);
|
||||
}
|
||||
else if (rc)
|
||||
LOG_WARNING(log,
|
||||
"Error while deleting ZooKeeper path `" << path << "`: " + zkutil::ZooKeeper::error2string(rc) << ", ignoring.");
|
||||
else
|
||||
{
|
||||
/// Successfully removed blocks have to be removed from cache
|
||||
cached_block_stats.erase(first_outdated_block->node);
|
||||
}
|
||||
first_outdated_block++;
|
||||
}
|
||||
|
||||
auto num_nodes_to_delete = timed_blocks.end() - first_outdated_block;
|
||||
@ -340,7 +347,9 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
|
||||
if (zookeeper.tryGetChildren(storage.zookeeper_path + "/blocks", blocks, &stat))
|
||||
throw Exception(storage.zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
|
||||
/// Clear already deleted blocks from the cache, cached_block_ctime should be subset of blocks
|
||||
/// Seems like this code is obsolete, because we delete blocks from cache
|
||||
/// when they are deleted from zookeeper. But we don't know about all (maybe future) places in code
|
||||
/// where they can be removed, so just to be sure that cache would not leak we check it here.
|
||||
{
|
||||
NameSet blocks_set(blocks.begin(), blocks.end());
|
||||
for (auto it = cached_block_stats.begin(); it != cached_block_stats.end();)
|
||||
|
@ -0,0 +1,4 @@
|
||||
3
|
||||
4
|
||||
5
|
||||
5
|
63
dbms/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh
Executable file
63
dbms/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh
Executable file
@ -0,0 +1,63 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
set -e
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS elog;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="
|
||||
CREATE TABLE elog (
|
||||
date Date,
|
||||
engine_id UInt32,
|
||||
referrer String
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/elog', 'test')
|
||||
PARTITION BY date
|
||||
ORDER BY (engine_id)
|
||||
SETTINGS replicated_deduplication_window = 2, cleanup_delay_period=4, cleanup_delay_period_random_add=0;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 1, 'hello')"
|
||||
sleep 1
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')"
|
||||
sleep 1
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 3, 'hello')"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 3 rows
|
||||
|
||||
count=`$CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/elog/blocks'"`
|
||||
|
||||
while [[ $count != 2 ]]
|
||||
do
|
||||
sleep 1
|
||||
count=`$CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/elog/blocks'"`
|
||||
done
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 1, 'hello')"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 4 rows
|
||||
count=`$CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/elog/blocks'"`
|
||||
|
||||
while [[ $count != 2 ]]
|
||||
do
|
||||
sleep 1
|
||||
count=`$CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/elog/blocks'"`
|
||||
done
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 5 rows
|
||||
|
||||
count=`$CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/elog/blocks'"`
|
||||
|
||||
while [[ $count != 2 ]]
|
||||
do
|
||||
sleep 1
|
||||
count=`$CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/elog/blocks'"`
|
||||
done
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # still 5 rows
|
Loading…
Reference in New Issue
Block a user