Fix adaptive granularity compatibility

This commit is contained in:
alesapin 2020-04-08 12:51:04 +03:00
parent f6f4fb0000
commit ded306f016
3 changed files with 57 additions and 1 deletions

View File

@ -246,6 +246,11 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
createTableIfNotExists();
/// We have to check granularity on other replicas. It it's fixed we
/// must create our new replica with fixed granularity and store this
/// information in /replica/metadata.
other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper();
checkTableStructure(zookeeper_path);
Coordination::Stat metadata_stat;
@ -256,11 +261,14 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
}
else
{
/// In old tables this node may missing or be empty
String replica_metadata;
bool replica_metadata_exists = current_zookeeper->tryGet(replica_path + "/metadata", replica_metadata);
if (!replica_metadata_exists || replica_metadata.empty())
{
/// We have to check shared node granularity before we create ours.
other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper();
ReplicatedMergeTreeTableMetadata current_metadata(*this);
current_zookeeper->createOrUpdate(replica_path + "/metadata", current_metadata.toString(), zkutil::CreateMode::Persistent);
}
@ -291,7 +299,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
createNewZooKeeperNodes();
other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper();
}

View File

@ -0,0 +1,49 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=True)
node2 = cluster.add_instance('node2', with_zookeeper=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_attach_detach(start_cluster):
node1.query("""
CREATE TABLE test (key UInt64)
ENGINE = ReplicatedMergeTree('/clickhouse/test', '1')
ORDER BY tuple()
SETTINGS index_granularity_bytes = 0""")
node1.query("INSERT INTO test VALUES (1), (2)")
node2.query("""
CREATE TABLE test (key UInt64)
ENGINE = ReplicatedMergeTree('/clickhouse/test', '2')
ORDER BY tuple()""")
node2.query("INSERT INTO test VALUES (3), (4)")
node1.query("SYSTEM SYNC REPLICA test")
node2.query("SYSTEM SYNC REPLICA test")
assert node1.query("SELECT COUNT() FROM test") == "4\n"
assert node2.query("SELECT COUNT() FROM test") == "4\n"
node1.query("DETACH TABLE test")
node2.query("DETACH TABLE test")
node1.query("ATTACH TABLE test")
node2.query("ATTACH TABLE test")
assert node1.query("SELECT COUNT() FROM test") == "4\n"
assert node2.query("SELECT COUNT() FROM test") == "4\n"