mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
ClickHouse 3419 Add setting prefer_localhost_replica (#2832)
* add setting prefer_localhost_replica * add prefer_localhost_replica setting * fix bugs * update setting comment * Add test for prefer_localhost_replica * fix bug
This commit is contained in:
parent
973bdab77f
commit
c019d732c5
@ -201,23 +201,18 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se
|
||||
info.weight = weight;
|
||||
|
||||
if (address.is_local)
|
||||
{
|
||||
info.local_addresses.push_back(address);
|
||||
info.per_replica_pools = {nullptr};
|
||||
}
|
||||
else
|
||||
{
|
||||
ConnectionPoolPtr pool = std::make_shared<ConnectionPool>(
|
||||
settings.distributed_connections_pool_size,
|
||||
address.host_name, address.port,
|
||||
address.default_database, address.user, address.password,
|
||||
ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings).getSaturated(settings.max_execution_time),
|
||||
"server", address.compression, address.secure);
|
||||
|
||||
info.pool = std::make_shared<ConnectionPoolWithFailover>(
|
||||
ConnectionPoolPtrs{pool}, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
info.per_replica_pools = {std::move(pool)};
|
||||
}
|
||||
ConnectionPoolPtr pool = std::make_shared<ConnectionPool>(
|
||||
settings.distributed_connections_pool_size,
|
||||
address.host_name, address.port,
|
||||
address.default_database, address.user, address.password,
|
||||
ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings).getSaturated(settings.max_execution_time),
|
||||
"server", address.compression, address.secure);
|
||||
|
||||
info.pool = std::make_shared<ConnectionPoolWithFailover>(
|
||||
ConnectionPoolPtrs{pool}, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
info.per_replica_pools = {std::move(pool)};
|
||||
|
||||
if (weight)
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size());
|
||||
@ -276,36 +271,25 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se
|
||||
|
||||
Addresses shard_local_addresses;
|
||||
|
||||
ConnectionPoolPtrs remote_replicas_pools;
|
||||
ConnectionPoolPtrs all_replicas_pools;
|
||||
remote_replicas_pools.reserve(replica_addresses.size());
|
||||
all_replicas_pools.reserve(replica_addresses.size());
|
||||
|
||||
for (const auto & replica : replica_addresses)
|
||||
{
|
||||
if (replica.is_local)
|
||||
{
|
||||
shard_local_addresses.push_back(replica);
|
||||
all_replicas_pools.emplace_back(nullptr);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto replica_pool = std::make_shared<ConnectionPool>(
|
||||
settings.distributed_connections_pool_size,
|
||||
replica.host_name, replica.port,
|
||||
replica.default_database, replica.user, replica.password,
|
||||
ConnectionTimeouts::getTCPTimeoutsWithFailover(settings).getSaturated(settings.max_execution_time),
|
||||
"server", replica.compression, replica.secure);
|
||||
auto replica_pool = std::make_shared<ConnectionPool>(
|
||||
settings.distributed_connections_pool_size,
|
||||
replica.host_name, replica.port,
|
||||
replica.default_database, replica.user, replica.password,
|
||||
ConnectionTimeouts::getTCPTimeoutsWithFailover(settings).getSaturated(settings.max_execution_time),
|
||||
"server", replica.compression, replica.secure);
|
||||
|
||||
remote_replicas_pools.emplace_back(replica_pool);
|
||||
all_replicas_pools.emplace_back(replica_pool);
|
||||
}
|
||||
all_replicas_pools.emplace_back(replica_pool);
|
||||
if (replica.is_local)
|
||||
shard_local_addresses.push_back(replica);
|
||||
}
|
||||
|
||||
ConnectionPoolWithFailoverPtr shard_pool;
|
||||
if (!remote_replicas_pools.empty())
|
||||
shard_pool = std::make_shared<ConnectionPoolWithFailover>(
|
||||
std::move(remote_replicas_pools), settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
ConnectionPoolWithFailoverPtr shard_pool = std::make_shared<ConnectionPoolWithFailover>(
|
||||
all_replicas_pools, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
|
||||
if (weight)
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size());
|
||||
@ -341,32 +325,23 @@ Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String
|
||||
|
||||
Addresses shard_local_addresses;
|
||||
ConnectionPoolPtrs all_replicas;
|
||||
ConnectionPoolPtrs remote_replicas;
|
||||
all_replicas.reserve(current.size());
|
||||
remote_replicas.reserve(current.size());
|
||||
|
||||
for (const auto & replica : current)
|
||||
{
|
||||
if (replica.is_local && !treat_local_as_remote)
|
||||
{
|
||||
shard_local_addresses.push_back(replica);
|
||||
all_replicas.emplace_back(nullptr);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto replica_pool = std::make_shared<ConnectionPool>(
|
||||
auto replica_pool = std::make_shared<ConnectionPool>(
|
||||
settings.distributed_connections_pool_size,
|
||||
replica.host_name, replica.port,
|
||||
replica.default_database, replica.user, replica.password,
|
||||
ConnectionTimeouts::getTCPTimeoutsWithFailover(settings).getSaturated(settings.max_execution_time),
|
||||
"server", replica.compression, replica.secure);
|
||||
all_replicas.emplace_back(replica_pool);
|
||||
remote_replicas.emplace_back(replica_pool);
|
||||
}
|
||||
all_replicas.emplace_back(replica_pool);
|
||||
if (replica.is_local && !treat_local_as_remote)
|
||||
shard_local_addresses.push_back(replica);
|
||||
}
|
||||
|
||||
ConnectionPoolWithFailoverPtr shard_pool = std::make_shared<ConnectionPoolWithFailover>(
|
||||
std::move(remote_replicas), settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
all_replicas, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), default_weight, shards_info.size());
|
||||
shards_info.push_back({{}, current_shard_num, default_weight, std::move(shard_local_addresses), std::move(shard_pool),
|
||||
|
@ -99,7 +99,7 @@ public:
|
||||
{
|
||||
public:
|
||||
bool isLocal() const { return !local_addresses.empty(); }
|
||||
bool hasRemoteConnections() const { return pool != nullptr; }
|
||||
bool hasRemoteConnections() const { return local_addresses.size() != per_replica_pools.size(); }
|
||||
size_t getLocalNodeCount() const { return local_addresses.size(); }
|
||||
bool hasInternalReplication() const { return has_internal_replication; }
|
||||
|
||||
|
@ -90,7 +90,9 @@ void SelectStreamFactory::createForShard(
|
||||
res.emplace_back(std::move(stream));
|
||||
};
|
||||
|
||||
if (shard_info.isLocal())
|
||||
const auto & settings = context.getSettingsRef();
|
||||
|
||||
if (settings.prefer_localhost_replica && shard_info.isLocal())
|
||||
{
|
||||
StoragePtr main_table_storage;
|
||||
|
||||
@ -106,22 +108,18 @@ void SelectStreamFactory::createForShard(
|
||||
if (!main_table_storage) /// Table is absent on a local server.
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
|
||||
if (shard_info.pool)
|
||||
if (shard_info.hasRemoteConnections())
|
||||
{
|
||||
LOG_WARNING(
|
||||
&Logger::get("ClusterProxy::SelectStreamFactory"),
|
||||
"There is no table " << main_table.database << "." << main_table.table
|
||||
<< " on local replica of shard " << shard_info.shard_num << ", will try remote replicas.");
|
||||
|
||||
emplace_remote_stream();
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Let it fail the usual way.
|
||||
emplace_local_stream();
|
||||
return;
|
||||
}
|
||||
emplace_local_stream(); /// Let it fail the usual way.
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
const auto * replicated_storage = dynamic_cast<const StorageReplicatedMergeTree *>(main_table_storage.get());
|
||||
@ -133,7 +131,6 @@ void SelectStreamFactory::createForShard(
|
||||
return;
|
||||
}
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries;
|
||||
|
||||
if (!max_allowed_delay)
|
||||
@ -158,7 +155,7 @@ void SelectStreamFactory::createForShard(
|
||||
|
||||
if (!settings.fallback_to_stale_replicas_for_distributed_queries)
|
||||
{
|
||||
if (shard_info.pool)
|
||||
if (shard_info.hasRemoteConnections())
|
||||
{
|
||||
/// If we cannot fallback, then we cannot use local replica. Try our luck with remote replicas.
|
||||
emplace_remote_stream();
|
||||
@ -171,7 +168,7 @@ void SelectStreamFactory::createForShard(
|
||||
ErrorCodes::ALL_REPLICAS_ARE_STALE);
|
||||
}
|
||||
|
||||
if (!shard_info.pool)
|
||||
if (!shard_info.hasRemoteConnections())
|
||||
{
|
||||
/// There are no remote replicas but we are allowed to fall back to stale local replica.
|
||||
emplace_local_stream();
|
||||
|
@ -268,8 +268,10 @@ struct Settings
|
||||
M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \
|
||||
\
|
||||
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \
|
||||
M(SettingBool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query between local and remote ones according to load_balancing") \
|
||||
M(SettingUInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.") \
|
||||
|
||||
|
||||
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
|
||||
TYPE NAME {DEFAULT};
|
||||
|
||||
|
@ -16,5 +16,17 @@
|
||||
</replica>
|
||||
</shard>
|
||||
</test_local_cluster>
|
||||
<shard_with_local_replica>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>node1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</shard_with_local_replica>
|
||||
</remote_servers>
|
||||
</yandex>
|
||||
|
@ -18,6 +18,9 @@ instance_test_inserts_local_cluster = cluster.add_instance(
|
||||
'instance_test_inserts_local_cluster',
|
||||
main_configs=['configs/remote_servers.xml'])
|
||||
|
||||
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
|
||||
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
@ -39,6 +42,20 @@ CREATE TABLE distributed (d Date, x UInt32) ENGINE = Distributed('test_cluster',
|
||||
CREATE TABLE distributed_on_local (d Date, x UInt32) ENGINE = Distributed('test_local_cluster', 'default', 'local')
|
||||
''')
|
||||
|
||||
node1.query('''
|
||||
CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node1', date, id, 8192)
|
||||
''')
|
||||
node2.query('''
|
||||
CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node2', date, id, 8192)
|
||||
''')
|
||||
|
||||
node1.query('''
|
||||
CREATE TABLE distributed (date Date, id UInt32) ENGINE = Distributed('shard_with_local_replica', 'default', 'replicated')
|
||||
''')
|
||||
|
||||
node2.query('''
|
||||
CREATE TABLE distributed (date Date, id UInt32) ENGINE = Distributed('shard_with_local_replica', 'default', 'replicated')
|
||||
''')
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
@ -108,7 +125,7 @@ def test_inserts_batching(started_cluster):
|
||||
# 4. Full batch of inserts after ALTER (that have different block structure).
|
||||
# 5. What was left to insert with (d, x) order before ALTER.
|
||||
expected = '''\
|
||||
20000101_20000101_1_1_0 [1]
|
||||
20000101_20000101_1_1_0 [1]
|
||||
20000101_20000101_2_2_0 [3,4,5]
|
||||
20000101_20000101_3_3_0 [2,7,8]
|
||||
20000101_20000101_4_4_0 [10,11,12]
|
||||
@ -122,3 +139,34 @@ def test_inserts_local(started_cluster):
|
||||
instance.query("INSERT INTO distributed_on_local VALUES ('2000-01-01', 1)")
|
||||
time.sleep(0.5)
|
||||
assert instance.query("SELECT count(*) FROM local").strip() == '1'
|
||||
|
||||
def test_prefer_localhost_replica(started_cluster):
|
||||
test_query = "SELECT * FROM distributed ORDER BY id;"
|
||||
node1.query("INSERT INTO distributed VALUES (toDate('2017-06-17'), 11)")
|
||||
node2.query("INSERT INTO distributed VALUES (toDate('2017-06-17'), 22)")
|
||||
time.sleep(1.0)
|
||||
expected_distributed = '''\
|
||||
2017-06-17 11
|
||||
2017-06-17 22
|
||||
'''
|
||||
assert TSV(node1.query(test_query)) == TSV(expected_distributed)
|
||||
assert TSV(node2.query(test_query)) == TSV(expected_distributed)
|
||||
with PartitionManager() as pm:
|
||||
pm.partition_instances(node1, node2, action='REJECT --reject-with tcp-reset')
|
||||
node1.query("INSERT INTO replicated VALUES (toDate('2017-06-17'), 33)")
|
||||
node2.query("INSERT INTO replicated VALUES (toDate('2017-06-17'), 44)")
|
||||
time.sleep(1.0)
|
||||
expected_from_node2 = '''\
|
||||
2017-06-17 11
|
||||
2017-06-17 22
|
||||
2017-06-17 44
|
||||
'''
|
||||
# Query is sent to node2, as it local and prefer_localhost_replica=1
|
||||
assert TSV(node2.query(test_query)) == TSV(expected_from_node2)
|
||||
expected_from_node1 = '''\
|
||||
2017-06-17 11
|
||||
2017-06-17 22
|
||||
2017-06-17 33
|
||||
'''
|
||||
# Now query is sent to node1, as it higher in order
|
||||
assert TSV(node2.query("SET load_balancing='in_order'; SET prefer_localhost_replica=0;" + test_query)) == TSV(expected_from_node1)
|
Loading…
Reference in New Issue
Block a user