From 2223553fe5fe4106f2b64197a50d27e3dc40f8e2 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 11 May 2023 11:15:07 +0200 Subject: [PATCH 01/37] Introduce quorum field in the /sync key Old keys remain the same for backward compatibility. --- patroni/dcs/__init__.py | 32 ++++++++++++++++++++++++-------- patroni/dcs/etcd3.py | 2 +- patroni/dcs/kubernetes.py | 9 ++++++--- patroni/ha.py | 16 ++++++++-------- tests/test_api.py | 2 -- tests/test_etcd.py | 2 +- tests/test_ha.py | 6 +++--- tests/test_kubernetes.py | 2 +- tests/test_raft.py | 4 ++-- tests/test_sync.py | 2 +- 10 files changed, 47 insertions(+), 30 deletions(-) diff --git a/patroni/dcs/__init__.py b/patroni/dcs/__init__.py index f1429c6f2..3936beafd 100644 --- a/patroni/dcs/__init__.py +++ b/patroni/dcs/__init__.py @@ -393,10 +393,13 @@ class SyncState(NamedTuple): :param index: modification index of a synchronization key in a Configuration Store :param leader: reference to member that was leader :param sync_standby: synchronous standby list (comma delimited) which are last synchronized to leader + :param quorum: if the node from sync_standby list is doing a leader race it should + see at least quorum other nodes from the sync_standby + leader list """ index: Optional[_Version] leader: Optional[str] sync_standby: Optional[str] + quorum: int @staticmethod def from_node(index: Optional[_Version], value: Union[str, Dict[str, Any], None]) -> 'SyncState': @@ -418,13 +421,15 @@ def from_node(index: Optional[_Version], value: Union[str, Dict[str, Any], None] if value and isinstance(value, str): value = json.loads(value) assert isinstance(value, dict) - return SyncState(index, value.get('leader'), value.get('sync_standby')) + leader = value.get('leader') + quorum = value.get('quorum') + return SyncState(index, leader, value.get('sync_standby'), int(quorum) if leader and quorum else 0) except (AssertionError, TypeError, ValueError): return SyncState.empty(index) @staticmethod def empty(index: Optional[_Version] = None) -> 'SyncState': - return SyncState(index, None, None) + return SyncState(index, None, None, 0) @property def is_empty(self) -> bool: @@ -441,10 +446,15 @@ def _str_to_list(value: str) -> List[str]: return list(filter(lambda a: a, [s.strip() for s in value.split(',')])) @property - def members(self) -> List[str]: + def voters(self) -> List[str]: """:returns: sync_standby as list.""" return self._str_to_list(self.sync_standby) if not self.is_empty and self.sync_standby else [] + @property + def members(self) -> List[str]: + """:returns: leader and all voters as list""" + return [] if not self.leader else [self.leader] + self.voters + def matches(self, name: Optional[str], check_leader: bool = False) -> bool: """Checks if node is presented in the /sync state. @@ -452,7 +462,7 @@ def matches(self, name: Optional[str], check_leader: bool = False) -> bool: :param name: name of the node :param check_leader: by default the name is searched in members, check_leader=True will include leader to list :returns: `True` if the /sync key not :func:`is_empty` and a given name is among presented in the sync state - >>> s = SyncState(1, 'foo', 'bar,zoo') + >>> s = SyncState(1, 'foo', 'bar,zoo', 0) >>> s.matches('foo') False >>> s.matches('fOo', True) @@ -1077,25 +1087,31 @@ def delete_cluster(self) -> bool: """Delete cluster from DCS""" @staticmethod - def sync_state(leader: Optional[str], sync_standby: Optional[Collection[str]]) -> Dict[str, Any]: + def sync_state(leader: Optional[str], sync_standby: Optional[Collection[str]], + quorum: Optional[int]) -> Dict[str, Any]: """Build sync_state dict. The sync_standby key being kept for backward compatibility. :param leader: name of the leader node that manages /sync key :param sync_standby: collection of currently known synchronous standby node names + :param quorum: if the node from sync_standby list is doing a leader race it should + see at least quorum other nodes from the sync_standby + leader list :returns: dictionary that later could be serialized to JSON or saved directly to DCS """ - return {'leader': leader, 'sync_standby': ','.join(sorted(sync_standby)) if sync_standby else None} + return {'leader': leader, 'quorum': quorum, + 'sync_standby': ','.join(sorted(sync_standby)) if sync_standby else None} def write_sync_state(self, leader: Optional[str], sync_standby: Optional[Collection[str]], - index: Optional[Any] = None) -> Optional[SyncState]: + quorum: Optional[int], index: Optional[Any] = None) -> Optional[SyncState]: """Write the new synchronous state to DCS. Calls :func:`sync_state` method to build a dict and than calls DCS specific :func:`set_sync_state_value` method. :param leader: name of the leader node that manages /sync key :param sync_standby: collection of currently known synchronous standby node names :param index: for conditional update of the key/object + :param quorum: if the node from sync_standby list is doing a leader race it should + see at least quorum other nodes from the sync_standby + leader list :returns: the new :class:`SyncState` object or None """ - sync_value = self.sync_state(leader, sync_standby) + sync_value = self.sync_state(leader, sync_standby, quorum) ret = self.set_sync_state_value(json.dumps(sync_value, separators=(',', ':')), index) if not isinstance(ret, bool): return SyncState.from_node(ret, sync_value) diff --git a/patroni/dcs/etcd3.py b/patroni/dcs/etcd3.py index bdf5cc336..c7a5973af 100644 --- a/patroni/dcs/etcd3.py +++ b/patroni/dcs/etcd3.py @@ -923,6 +923,6 @@ def watch(self, leader_index: Optional[str], timeout: float) -> bool: return True try: - return super(Etcd3, self).watch(None, timeout) + return super(Etcd3, self).watch(None, timeout + 0.5) finally: self.event.clear() diff --git a/patroni/dcs/kubernetes.py b/patroni/dcs/kubernetes.py index e33a8509f..0bb37c03c 100644 --- a/patroni/dcs/kubernetes.py +++ b/patroni/dcs/kubernetes.py @@ -1313,15 +1313,18 @@ def set_sync_state_value(self, value: str, index: Optional[str] = None) -> bool: raise NotImplementedError # pragma: no cover def write_sync_state(self, leader: Optional[str], sync_standby: Optional[Collection[str]], - index: Optional[str] = None) -> Optional[SyncState]: + quorum: Optional[int], index: Optional[str] = None) -> Optional[SyncState]: """Prepare and write annotations to $SCOPE-sync Endpoint or ConfigMap. :param leader: name of the leader node that manages /sync key :param sync_standby: collection of currently known synchronous standby node names + :param quorum: if the node from sync_standby list is doing a leader race it should + see at least quorum other nodes from the sync_standby + leader list :param index: last known `resource_version` for conditional update of the object :returns: the new :class:`SyncState` object or None """ - sync_state = self.sync_state(leader, sync_standby) + sync_state = self.sync_state(leader, sync_standby, quorum) + sync_state['quorum'] = str(sync_state['quorum']) if sync_state['quorum'] is not None else None ret = self.patch_or_create(self.sync_path, sync_state, index, False) if not isinstance(ret, bool): return SyncState.from_node(ret.metadata.resource_version, sync_state) @@ -1333,7 +1336,7 @@ def delete_sync_state(self, index: Optional[str] = None) -> bool: :param index: last known `resource_version` for conditional update of the object :returns: `True` if "delete" was successful """ - return self.write_sync_state(None, None, index=index) is not None + return self.write_sync_state(None, None, None, index=index) is not None def watch(self, leader_index: Optional[str], timeout: float) -> bool: if self.__do_not_watch: diff --git a/patroni/ha.py b/patroni/ha.py index e1559a279..611d945f7 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -586,17 +586,17 @@ def process_sync_replication(self) -> None: promoting standbys that were guaranteed to be replicating synchronously. """ if self.is_synchronous_mode(): - current = CaseInsensitiveSet(self.cluster.sync.members) picked, allow_promote = self.state_handler.sync_handler.current_state(self.cluster) + voters = CaseInsensitiveSet(self.cluster.sync.voters) - if picked != current: + if picked != voters: sync = self.cluster.sync # update synchronous standby list in dcs temporarily to point to common nodes in current and picked - sync_common = current & allow_promote - if sync_common != current: + sync_common = voters & allow_promote + if sync_common != voters: logger.info("Updating synchronous privilege temporarily from %s to %s", - list(current), list(sync_common)) - sync = self.dcs.write_sync_state(self.state_handler.name, sync_common, index=sync.index) + list(voters), list(sync_common)) + sync = self.dcs.write_sync_state(self.state_handler.name, sync_common, 0, index=sync.index) if not sync: return logger.info('Synchronous replication key updated by someone else.') @@ -614,7 +614,7 @@ def process_sync_replication(self) -> None: time.sleep(2) _, allow_promote = self.state_handler.sync_handler.current_state(self.cluster) if allow_promote and allow_promote != sync_common: - if not self.dcs.write_sync_state(self.state_handler.name, allow_promote, index=sync.index): + if not self.dcs.write_sync_state(self.state_handler.name, allow_promote, 0, index=sync.index): return logger.info("Synchronous replication key updated by someone else") logger.info("Synchronous standby status assigned to %s", list(allow_promote)) else: @@ -727,7 +727,7 @@ def enforce_primary_role(self, message: str, promote_message: str) -> str: if self.is_synchronous_mode(): # Just set ourselves as the authoritative source of truth for now. We don't want to wait for standbys # to connect. We will try finding a synchronous standby in the next cycle. - if not self.dcs.write_sync_state(self.state_handler.name, None, index=self.cluster.sync.index): + if not self.dcs.write_sync_state(self.state_handler.name, None, 0, index=self.cluster.sync.index): # Somebody else updated sync state, it may be due to us losing the lock. To be safe, postpone # promotion until next cycle. TODO: trigger immediate retry of run_cycle return 'Postponing promotion because synchronous replication state was updated by somebody else' diff --git a/tests/test_api.py b/tests/test_api.py index 85c01b49f..2e947bb38 100644 --- a/tests/test_api.py +++ b/tests/test_api.py @@ -187,7 +187,6 @@ class TestRestApiHandler(unittest.TestCase): def test_do_GET(self): MockPatroni.dcs.cluster.last_lsn = 20 - MockPatroni.dcs.cluster.sync.members = [MockPostgresql.name] with patch.object(GlobalConfig, 'is_synchronous_mode', PropertyMock(return_value=True)): MockRestApiServer(RestApiHandler, 'GET /replica') MockRestApiServer(RestApiHandler, 'GET /replica?lag=1M') @@ -206,7 +205,6 @@ def test_do_GET(self): MockRestApiServer(RestApiHandler, 'GET /synchronous') MockRestApiServer(RestApiHandler, 'GET /read-only-sync') with patch.object(RestApiHandler, 'get_postgresql_status', Mock(return_value={'role': 'replica'})): - MockPatroni.dcs.cluster.sync.members = [] MockRestApiServer(RestApiHandler, 'GET /asynchronous') with patch.object(MockHa, 'is_leader', Mock(return_value=True)): MockRestApiServer(RestApiHandler, 'GET /replica') diff --git a/tests/test_etcd.py b/tests/test_etcd.py index 6699127ab..1b0c4c50c 100644 --- a/tests/test_etcd.py +++ b/tests/test_etcd.py @@ -338,7 +338,7 @@ def test_set_ttl(self): self.assertTrue(self.etcd.watch(None, 1)) def test_sync_state(self): - self.assertIsNone(self.etcd.write_sync_state('leader', None)) + self.assertIsNone(self.etcd.write_sync_state('leader', None, 0)) self.assertFalse(self.etcd.delete_sync_state()) def test_set_history_value(self): diff --git a/tests/test_ha.py b/tests/test_ha.py index c97345173..db11706dd 100644 --- a/tests/test_ha.py +++ b/tests/test_ha.py @@ -62,7 +62,7 @@ def get_cluster_initialized_without_leader(leader=False, failover=None, sync=Non 'tags': {'clonefrom': True}, 'scheduled_restart': {'schedule': "2100-01-01 10:53:07.560445+00:00", 'postgres_version': '99.0.0'}}) - syncstate = SyncState(0 if sync else None, sync and sync[0], sync and sync[1]) + syncstate = SyncState(0 if sync else None, sync and sync[0], sync and sync[1], 0) failsafe = {m.name: m.api_url for m in (m1, m2)} if failsafe else None return get_cluster(SYSID, leader, [m1, m2], failover, syncstate, cluster_config, failsafe) @@ -1201,7 +1201,7 @@ def test_sync_replication_become_primary(self): # When we just became primary nobody is sync self.assertEqual(self.ha.enforce_primary_role('msg', 'promote msg'), 'promote msg') mock_set_sync.assert_called_once_with(CaseInsensitiveSet()) - mock_write_sync.assert_called_once_with('leader', None, index=0) + mock_write_sync.assert_called_once_with('leader', None, 0, index=0) mock_set_sync.reset_mock() @@ -1239,7 +1239,7 @@ def test_unhealthy_sync_mode(self): mock_acquire.assert_called_once() mock_follow.assert_not_called() mock_promote.assert_called_once() - mock_write_sync.assert_called_once_with('other', None, index=0) + mock_write_sync.assert_called_once_with('other', None, 0, index=0) def test_disable_sync_when_restarting(self): self.ha.is_synchronous_mode = true diff --git a/tests/test_kubernetes.py b/tests/test_kubernetes.py index bbe551571..61f858d68 100644 --- a/tests/test_kubernetes.py +++ b/tests/test_kubernetes.py @@ -378,7 +378,7 @@ def test_delete_sync_state(self): @patch.object(k8s_client.CoreV1Api, 'patch_namespaced_endpoints', mock_namespaced_kind, create=True) def test_write_sync_state(self): - self.assertIsNotNone(self.k.write_sync_state('a', ['b'], 1)) + self.assertIsNotNone(self.k.write_sync_state('a', ['b'], 0, 1)) @patch.object(k8s_client.CoreV1Api, 'patch_namespaced_pod', mock_namespaced_kind, create=True) @patch.object(k8s_client.CoreV1Api, 'create_namespaced_endpoints', mock_namespaced_kind, create=True) diff --git a/tests/test_raft.py b/tests/test_raft.py index 2029a61e1..bf033e37d 100644 --- a/tests/test_raft.py +++ b/tests/test_raft.py @@ -137,8 +137,8 @@ def test_raft(self): self.assertTrue(raft.initialize()) self.assertTrue(raft.cancel_initialization()) self.assertTrue(raft.set_config_value('{}')) - self.assertTrue(raft.write_sync_state('foo', 'bar')) - self.assertFalse(raft.write_sync_state('foo', 'bar', 1)) + self.assertTrue(raft.write_sync_state('foo', 'bar', 0)) + self.assertFalse(raft.write_sync_state('foo', 'bar', 0, 1)) raft._citus_group = '1' self.assertTrue(raft.manual_failover('foo', 'bar')) raft._citus_group = '0' diff --git a/tests/test_sync.py b/tests/test_sync.py index 0cfba7f82..a93b98f39 100644 --- a/tests/test_sync.py +++ b/tests/test_sync.py @@ -28,7 +28,7 @@ def setUp(self): @patch.object(Postgresql, 'last_operation', Mock(return_value=1)) def test_pick_sync_standby(self): cluster = Cluster(True, None, self.leader, 0, [self.me, self.other, self.leadermem], None, - SyncState(0, self.me.name, self.leadermem.name), None, None, None) + SyncState(0, self.me.name, self.leadermem.name, 0), None, None, None) pg_stat_replication = [ {'pid': 100, 'application_name': self.leadermem.name, 'sync_state': 'sync', 'flush_lsn': 1}, From ea019ba5495f766b68f67dcf961078bcfdb0fdb0 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 11 May 2023 11:20:36 +0200 Subject: [PATCH 02/37] Adapt SyncHandler interfaces for quorum commit --- patroni/config.py | 7 ++- patroni/ha.py | 6 ++- patroni/postgresql/__init__.py | 5 ++ patroni/postgresql/sync.py | 91 ++++++++++++++++++++++++++-------- tests/test_ha.py | 22 +++++--- tests/test_sync.py | 66 +++++++++++++++++++++--- 6 files changed, 158 insertions(+), 39 deletions(-) diff --git a/patroni/config.py b/patroni/config.py index 4d3eea8ed..4486fdd36 100644 --- a/patroni/config.py +++ b/patroni/config.py @@ -77,10 +77,15 @@ def is_paused(self) -> bool: """:returns: `True` if cluster is in maintenance mode.""" return self.check_mode('pause') + @property + def is_quorum_commit_mode(self) -> bool: + """:returns: `True` if quorum commit replication is requested""" + return str(self.get('synchronous_mode')).lower() == 'quorum' + @property def is_synchronous_mode(self) -> bool: """:returns: `True` if synchronous replication is requested.""" - return self.check_mode('synchronous_mode') + return self.check_mode('synchronous_mode') is True or self.is_quorum_commit_mode @property def is_synchronous_mode_strict(self) -> bool: diff --git a/patroni/ha.py b/patroni/ha.py index 611d945f7..68e07bb22 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -586,7 +586,9 @@ def process_sync_replication(self) -> None: promoting standbys that were guaranteed to be replicating synchronously. """ if self.is_synchronous_mode(): - picked, allow_promote = self.state_handler.sync_handler.current_state(self.cluster) + current_state = self.state_handler.sync_handler.current_state(self.cluster) + picked = current_state.active + allow_promote = current_state.sync voters = CaseInsensitiveSet(self.cluster.sync.voters) if picked != voters: @@ -612,7 +614,7 @@ def process_sync_replication(self) -> None: if picked and picked != CaseInsensitiveSet('*') and allow_promote != picked: # Wait for PostgreSQL to enable synchronous mode and see if we can immediately set sync_standby time.sleep(2) - _, allow_promote = self.state_handler.sync_handler.current_state(self.cluster) + allow_promote = self.state_handler.sync_handler.current_state(self.cluster).sync if allow_promote and allow_promote != sync_common: if not self.dcs.write_sync_state(self.state_handler.name, allow_promote, 0, index=sync.index): return logger.info("Synchronous replication key updated by someone else") diff --git a/patroni/postgresql/__init__.py b/patroni/postgresql/__init__.py index 7f7928263..b5245bfbb 100644 --- a/patroni/postgresql/__init__.py +++ b/patroni/postgresql/__init__.py @@ -162,6 +162,11 @@ def wal_name(self) -> str: def lsn_name(self) -> str: return 'lsn' if self._major_version >= 100000 else 'location' + @property + def supports_quorum_commit(self) -> bool: + """:returns: `True` if quorum commit is supported by Postgres.""" + return self._major_version >= 100000 + @property def supports_multiple_sync(self) -> bool: """:returns: `True` if Postgres version supports more than one synchronous node.""" diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index 78e7b4272..edff92c5f 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -3,7 +3,7 @@ import time from copy import deepcopy -from typing import Collection, List, NamedTuple, Tuple, TYPE_CHECKING +from typing import Collection, List, NamedTuple, Optional, Tuple, TYPE_CHECKING from ..collections import CaseInsensitiveDict, CaseInsensitiveSet from ..dcs import Cluster @@ -153,6 +153,14 @@ def parse_sync_standby_names(value: str) -> _SSN: return _SSN(sync_type, has_star, num, members) +class _SyncState(NamedTuple): + sync_type: str + numsync: int + numsync_confirmed: int + sync: CaseInsensitiveSet + active: CaseInsensitiveSet + + class SyncHandler(object): """Class responsible for working with the `synchronous_standby_names`. @@ -196,7 +204,7 @@ def _handle_synchronous_standby_names_change(self) -> None: self._postgresql.query('SELECT pg_catalog.txid_current()') # Ensure some WAL traffic to move replication self._postgresql.reset_cluster_info_state(None) # Reset internal cache to query fresh values - def current_state(self, cluster: Cluster) -> Tuple[CaseInsensitiveSet, CaseInsensitiveSet]: + def current_state(self, cluster: Cluster) -> _SyncState: """Finds best candidates to be the synchronous standbys. Current synchronous standby is always preferred, unless it has disconnected or does not want to be a @@ -209,7 +217,8 @@ def current_state(self, cluster: Cluster) -> Tuple[CaseInsensitiveSet, CaseInsen Please note that it will not also swap sync standbys in case where all replicas are hung. - `synchronous_node_count`: controlls how many nodes should be set as synchronous. - :returns: tuple of candidates :class:`CaseInsensitiveSet` and synchronous standbys :class:`CaseInsensitiveSet`. + :param cluster: current cluster topology from DCS + :returns: current synchronous replication state as a :class:`_SyncState` object """ self._handle_synchronous_standby_names_change() @@ -245,41 +254,79 @@ def current_state(self, cluster: Cluster) -> Tuple[CaseInsensitiveSet, CaseInsen if self._postgresql.supports_multiple_sync else 1 sync_node_maxlag = self._postgresql._global_config.maximum_lag_on_syncnode - candidates = CaseInsensitiveSet() + active = CaseInsensitiveSet() sync_nodes = CaseInsensitiveSet() + numsync_confirmed = 0 # Prefer members without nofailover tag. We are relying on the fact that sorts are guaranteed to be stable. - for pid, app_name, sync_state, replica_lsn, _ in sorted(replica_list, key=lambda x: x[4]): - # if standby name is listed in the /sync key we can count it as synchronous, otherwice - # it becomes really synchronous when sync_state = 'sync' and it is known that it managed to catch up - if app_name not in self._ready_replicas and app_name in self._ssn_data.members and\ - (cluster.sync.matches(app_name) or sync_state == 'sync' and replica_lsn >= self._primary_flush_lsn): - self._ready_replicas[app_name] = pid + for pid, app_name, sync_state, replica_lsn, nofailover in sorted(replica_list, key=lambda x: x[4]): + if app_name not in self._ready_replicas and app_name in self._ssn_data.members: + if self._postgresql._global_config.is_quorum_commit_mode: + # When quorum commit is enabled we can't check against cluster.sync because nodes + # are written there when at least one of them caught up with _primary_flush_lsn. + if replica_lsn >= self._primary_flush_lsn\ + and (sync_state == 'quorum' or (not self._postgresql.supports_quorum_commit + and sync_state in ('sync', 'potential'))): + self._ready_replicas[app_name] = pid + elif cluster.sync.matches(app_name) or sync_state == 'sync' and replica_lsn >= self._primary_flush_lsn: + # if standby name is listed in the /sync key we can count it as synchronous, otherwise it becomes + # "really" synchronous when sync_state = 'sync' and we known that it managed to catch up + self._ready_replicas[app_name] = pid if sync_node_maxlag <= 0 or max_lsn - replica_lsn <= sync_node_maxlag: - candidates.add(app_name) - if sync_state == 'sync' and app_name in self._ready_replicas: - sync_nodes.add(app_name) - if len(candidates) >= sync_node_count: - break - - return candidates, sync_nodes - - def set_synchronous_standby_names(self, sync: Collection[str]) -> None: + if self._postgresql._global_config.is_quorum_commit_mode: + # add nodes with nofailover tag only to get enough "active" nodes + if not nofailover or len(active) < sync_node_count: + if app_name in self._ready_replicas: + numsync_confirmed += 1 + active.add(app_name) + else: + active.add(app_name) + if sync_state == 'sync' and app_name in self._ready_replicas: + sync_nodes.add(app_name) + numsync_confirmed += 1 + if len(active) >= sync_node_count: + break + + if self._postgresql._global_config.is_quorum_commit_mode: + sync_nodes = CaseInsensitiveSet() if self._ssn_data.has_star else self._ssn_data.members + + return _SyncState( + self._ssn_data.sync_type, + 0 if self._ssn_data.has_star else self._ssn_data.num, + numsync_confirmed, + sync_nodes, + active) + + def set_synchronous_standby_names(self, sync: Collection[str], num: Optional[int] = None) -> None: """Constructs and sets "synchronous_standby_names" GUC value. :param sync: set of nodes to sync to + :param num: specifies number of nodes to sync to. The *num* is set only in case if quorum commit is enabled """ - has_asterisk = '*' in sync + # Special case. If sync nodes set is empty but requested num of sync nodes >= 1 + # we want to set synchronous_standby_names to '*' + has_asterisk = '*' in sync or num and num >= 1 and not sync if has_asterisk: sync = ['*'] else: - sync = [quote_ident(x) for x in sync] + sync = [quote_ident(x) for x in sorted(sync)] if self._postgresql.supports_multiple_sync and len(sync) > 1: - sync_param = '{0} ({1})'.format(len(sync), ','.join(sync)) + if num is None: + num = len(sync) + sync_param = ','.join(sync) else: sync_param = next(iter(sync), None) + if TYPE_CHECKING: # pragma: no cover + assert self._postgresql._global_config is not None + + if self._postgresql._global_config.is_quorum_commit_mode and sync or\ + self._postgresql.supports_multiple_sync and len(sync) > 1: + prefix = 'ANY ' if self._postgresql._global_config.is_quorum_commit_mode\ + and self._postgresql.supports_quorum_commit else '' + sync_param = '{0}{1} ({2})'.format(prefix, num, sync_param) + if not (self._postgresql.config.set_synchronous_standby_names(sync_param) and self._postgresql.state == 'running' and self._postgresql.is_leader()) or has_asterisk: return diff --git a/tests/test_ha.py b/tests/test_ha.py index db11706dd..590157843 100644 --- a/tests/test_ha.py +++ b/tests/test_ha.py @@ -17,6 +17,7 @@ from patroni.postgresql.postmaster import PostmasterProcess from patroni.postgresql.rewind import Rewind from patroni.postgresql.slots import SlotsHandler +from patroni.postgresql.sync import _SyncState from patroni.utils import tzutc from patroni.watchdog import Watchdog @@ -1126,8 +1127,9 @@ def test_process_sync_replication(self): self.ha.is_synchronous_mode = true # Test sync standby not touched when picking the same node - self.p.sync_handler.current_state = Mock(return_value=(CaseInsensitiveSet(['other']), - CaseInsensitiveSet(['other']))) + self.p.sync_handler.current_state = Mock(return_value=_SyncState('priority', 1, 1, + CaseInsensitiveSet(['other']), + CaseInsensitiveSet(['other']))) self.ha.cluster = get_cluster_initialized_with_leader(sync=('leader', 'other')) self.ha.run_cycle() mock_set_sync.assert_not_called() @@ -1135,14 +1137,16 @@ def test_process_sync_replication(self): mock_set_sync.reset_mock() # Test sync standby is replaced when switching standbys - self.p.sync_handler.current_state = Mock(return_value=(CaseInsensitiveSet(['other2']), CaseInsensitiveSet())) + self.p.sync_handler.current_state = Mock(return_value=_SyncState('priority', 0, 0, CaseInsensitiveSet(), + CaseInsensitiveSet(['other2']))) self.ha.dcs.write_sync_state = Mock(return_value=SyncState.empty()) self.ha.run_cycle() mock_set_sync.assert_called_once_with(CaseInsensitiveSet(['other2'])) # Test sync standby is replaced when new standby is joined - self.p.sync_handler.current_state = Mock(return_value=(CaseInsensitiveSet(['other2', 'other3']), - CaseInsensitiveSet(['other2']))) + self.p.sync_handler.current_state = Mock(return_value=_SyncState('priority', 1, 1, + CaseInsensitiveSet(['other2']), + CaseInsensitiveSet(['other2', 'other3']))) self.ha.dcs.write_sync_state = Mock(return_value=SyncState.empty()) self.ha.run_cycle() self.assertEqual(mock_set_sync.call_args_list[0][0], (CaseInsensitiveSet(['other2']),)) @@ -1159,8 +1163,9 @@ def test_process_sync_replication(self): self.ha.dcs.write_sync_state = Mock(return_value=SyncState.empty()) self.ha.dcs.get_cluster = Mock(return_value=get_cluster_initialized_with_leader(sync=('leader', 'other'))) # self.ha.cluster = get_cluster_initialized_with_leader(sync=('leader', 'other')) - self.p.sync_handler.current_state = Mock(return_value=(CaseInsensitiveSet(['other2']), - CaseInsensitiveSet(['other2']))) + self.p.sync_handler.current_state = Mock(return_value=_SyncState('priority', 1, 1, + CaseInsensitiveSet(['other2']), + CaseInsensitiveSet(['other2']))) self.ha.run_cycle() self.assertEqual(self.ha.dcs.write_sync_state.call_count, 2) @@ -1182,7 +1187,8 @@ def test_process_sync_replication(self): # Test sync set to '*' when synchronous_mode_strict is enabled mock_set_sync.reset_mock() - self.p.sync_handler.current_state = Mock(return_value=(CaseInsensitiveSet(), CaseInsensitiveSet())) + self.p.sync_handler.current_state = Mock(return_value=_SyncState('priority', 0, 0, CaseInsensitiveSet(), + CaseInsensitiveSet())) with patch('patroni.config.GlobalConfig.is_synchronous_mode_strict', PropertyMock(return_value=True)): self.ha.run_cycle() mock_set_sync.assert_called_once_with(CaseInsensitiveSet('*')) diff --git a/tests/test_sync.py b/tests/test_sync.py index a93b98f39..9a2a9d12e 100644 --- a/tests/test_sync.py +++ b/tests/test_sync.py @@ -38,7 +38,8 @@ def test_pick_sync_standby(self): # sync node is a bit behind of async, but we prefer it anyway with patch.object(Postgresql, "_cluster_info_state_get", side_effect=[self.leadermem.name, 'on', pg_stat_replication]): - self.assertEqual(self.s.current_state(cluster), (CaseInsensitiveSet([self.leadermem.name]), + self.assertEqual(self.s.current_state(cluster), ('priority', 1, 1, + CaseInsensitiveSet([self.leadermem.name]), CaseInsensitiveSet([self.leadermem.name]))) # prefer node with sync_state='potential', even if it is slightly behind of async @@ -46,26 +47,46 @@ def test_pick_sync_standby(self): for r in pg_stat_replication: r['write_lsn'] = r.pop('flush_lsn') with patch.object(Postgresql, "_cluster_info_state_get", side_effect=['', 'remote_write', pg_stat_replication]): - self.assertEqual(self.s.current_state(cluster), (CaseInsensitiveSet([self.leadermem.name]), - CaseInsensitiveSet())) + self.assertEqual(self.s.current_state(cluster), ('off', 0, 0, CaseInsensitiveSet(), + CaseInsensitiveSet([self.leadermem.name]))) # when there are no sync or potential candidates we pick async with the minimal replication lag for i, r in enumerate(pg_stat_replication): r.update(replay_lsn=3 - i, application_name=r['application_name'].upper()) missing = pg_stat_replication.pop(0) with patch.object(Postgresql, "_cluster_info_state_get", side_effect=['', 'remote_apply', pg_stat_replication]): - self.assertEqual(self.s.current_state(cluster), (CaseInsensitiveSet([self.me.name]), CaseInsensitiveSet())) + self.assertEqual(self.s.current_state(cluster), ('off', 0, 0, CaseInsensitiveSet(), + CaseInsensitiveSet([self.me.name]))) # unknown sync node is ignored missing.update(application_name='missing', sync_state='sync') pg_stat_replication.insert(0, missing) with patch.object(Postgresql, "_cluster_info_state_get", side_effect=['', 'remote_apply', pg_stat_replication]): - self.assertEqual(self.s.current_state(cluster), (CaseInsensitiveSet([self.me.name]), CaseInsensitiveSet())) + self.assertEqual(self.s.current_state(cluster), ('off', 0, 0, CaseInsensitiveSet(), + CaseInsensitiveSet([self.me.name]))) # invalid synchronous_standby_names and empty pg_stat_replication with patch.object(Postgresql, "_cluster_info_state_get", side_effect=['a b', 'remote_apply', None]): self.p._major_version = 90400 - self.assertEqual(self.s.current_state(cluster), (CaseInsensitiveSet(), CaseInsensitiveSet())) + self.assertEqual(self.s.current_state(cluster), ('off', 0, 0, CaseInsensitiveSet(), CaseInsensitiveSet())) + + @patch.object(Postgresql, 'last_operation', Mock(return_value=1)) + def test_current_state_quorum(self): + self.p._global_config = GlobalConfig({'synchronous_mode': 'quorum'}) + cluster = Cluster(True, None, self.leader, 0, [self.me, self.other, self.leadermem], None, + SyncState(0, self.me.name, self.leadermem.name, 0), None, None, None) + + pg_stat_replication = [ + {'pid': 100, 'application_name': self.leadermem.name, 'sync_state': 'quorum', 'flush_lsn': 1}, + {'pid': 101, 'application_name': self.other.name, 'sync_state': 'quorum', 'flush_lsn': 2}] + + # sync node is a bit behind of async, but we prefer it anyway + with patch.object(Postgresql, "_cluster_info_state_get", + side_effect=['ANY 1 ({0},"{1}")'.format(self.leadermem.name, self.other.name), + 'on', pg_stat_replication]): + self.assertEqual(self.s.current_state(cluster), + ('quorum', 1, 2, CaseInsensitiveSet([self.other.name, self.leadermem.name]), + CaseInsensitiveSet([self.leadermem.name, self.other.name]))) def test_set_sync_standby(self): def value_in_conf(): @@ -84,6 +105,7 @@ def value_in_conf(): mock_reload.assert_not_called() self.assertEqual(value_in_conf(), "synchronous_standby_names = 'n1'") + mock_reload.reset_mock() self.s.set_synchronous_standby_names(CaseInsensitiveSet(['n1', 'n2'])) mock_reload.assert_called() self.assertEqual(value_in_conf(), "synchronous_standby_names = '2 (n1,n2)'") @@ -98,3 +120,35 @@ def value_in_conf(): self.s.set_synchronous_standby_names(CaseInsensitiveSet('*')) mock_reload.assert_called() self.assertEqual(value_in_conf(), "synchronous_standby_names = '*'") + + self.p._global_config = GlobalConfig({'synchronous_mode': 'quorum'}) + mock_reload.reset_mock() + self.s.set_synchronous_standby_names([], 1) + mock_reload.assert_called() + self.assertEqual(value_in_conf(), "synchronous_standby_names = 'ANY 1 (*)'") + + mock_reload.reset_mock() + self.s.set_synchronous_standby_names(['a', 'b'], 1) + mock_reload.assert_called() + self.assertEqual(value_in_conf(), "synchronous_standby_names = 'ANY 1 (a,b)'") + + mock_reload.reset_mock() + self.s.set_synchronous_standby_names(['a', 'b'], 3) + mock_reload.assert_called() + self.assertEqual(value_in_conf(), "synchronous_standby_names = 'ANY 3 (a,b)'") + + self.p._major_version = 90601 + mock_reload.reset_mock() + self.s.set_synchronous_standby_names([], 1) + mock_reload.assert_called() + self.assertEqual(value_in_conf(), "synchronous_standby_names = '1 (*)'") + + mock_reload.reset_mock() + self.s.set_synchronous_standby_names(['a', 'b'], 1) + mock_reload.assert_called() + self.assertEqual(value_in_conf(), "synchronous_standby_names = '1 (a,b)'") + + mock_reload.reset_mock() + self.s.set_synchronous_standby_names(['a', 'b'], 3) + mock_reload.assert_called() + self.assertEqual(value_in_conf(), "synchronous_standby_names = '3 (a,b)'") From f5f0adba1492f55deecff25cc14a1c8e96edb1ca Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 11 May 2023 11:40:05 +0200 Subject: [PATCH 03/37] Compatibility with future synchronous_mode=quorum --- patroni/ha.py | 241 ++++++++++++++++++++++++++++++++++------------- tests/test_ha.py | 48 +++++++++- 2 files changed, 220 insertions(+), 69 deletions(-) diff --git a/patroni/ha.py b/patroni/ha.py index 68e07bb22..3b4483fc0 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -572,12 +572,33 @@ def is_synchronous_mode(self) -> bool: """:returns: `True` if synchronous replication is requested.""" return self.global_config.is_synchronous_mode + def is_synchronous_mode_active(self) -> bool: + """:returns: `True` is synchronous replication requested is active (/sync key has a valid "leader" field).""" + return self.is_synchronous_mode() and not self.cluster.sync.is_empty + + def is_quorum_commit_mode(self) -> bool: + """:returns: `True` if quorum commit replication is requested and "supported".""" + return self.global_config.is_quorum_commit_mode and self.state_handler.supports_multiple_sync + + def is_quorum_commit_mode_active(self) -> bool: + """:returns: `True` if quorum replication is requested and active (/sync key has a valid "leader" field).""" + return self.is_quorum_commit_mode() and not self.cluster.sync.is_empty + def is_failsafe_mode(self) -> bool: """:returns: `True` if failsafe_mode is enabled in global configuration.""" return self.global_config.check_mode('failsafe_mode') - def process_sync_replication(self) -> None: - """Process synchronous standby beahvior. + def disable_synchronous_replication(self) -> None: + """Cleans up /sync key in DCS if synchronous replication is disabled.""" + if not self.cluster.sync.is_empty and self.dcs.delete_sync_state(index=self.cluster.sync.index): + logger.info("Disabled synchronous replication") + self.state_handler.sync_handler.set_synchronous_standby_names(CaseInsensitiveSet()) + + def _process_quorum_replication(self) -> None: + pass + + def _process_multisync_replication(self) -> None: + """Process synchronous replication state with one or more sync standbys. Synchronous standbys are registered in two places postgresql.conf and DCS. The order of updating them must be right. The invariant that should be kept is that if a node is primary and sync_standby is set in DCS, @@ -585,44 +606,89 @@ def process_sync_replication(self) -> None: and then in DCS. When removing, first remove in DCS, then in postgresql.conf. This is so we only consider promoting standbys that were guaranteed to be replicating synchronously. """ - if self.is_synchronous_mode(): - current_state = self.state_handler.sync_handler.current_state(self.cluster) - picked = current_state.active - allow_promote = current_state.sync - voters = CaseInsensitiveSet(self.cluster.sync.voters) - - if picked != voters: - sync = self.cluster.sync - # update synchronous standby list in dcs temporarily to point to common nodes in current and picked - sync_common = voters & allow_promote - if sync_common != voters: - logger.info("Updating synchronous privilege temporarily from %s to %s", - list(voters), list(sync_common)) - sync = self.dcs.write_sync_state(self.state_handler.name, sync_common, 0, index=sync.index) - if not sync: - return logger.info('Synchronous replication key updated by someone else.') - - # When strict mode and no suitable replication connections put "*" to synchronous_standby_names - if self.global_config.is_synchronous_mode_strict and not picked: - picked = CaseInsensitiveSet('*') - logger.warning("No standbys available!") - - # Update postgresql.conf and wait 2 secs for changes to become active - logger.info("Assigning synchronous standby status to %s", list(picked)) - self.state_handler.sync_handler.set_synchronous_standby_names(picked) - - if picked and picked != CaseInsensitiveSet('*') and allow_promote != picked: - # Wait for PostgreSQL to enable synchronous mode and see if we can immediately set sync_standby - time.sleep(2) - allow_promote = self.state_handler.sync_handler.current_state(self.cluster).sync - if allow_promote and allow_promote != sync_common: - if not self.dcs.write_sync_state(self.state_handler.name, allow_promote, 0, index=sync.index): - return logger.info("Synchronous replication key updated by someone else") - logger.info("Synchronous standby status assigned to %s", list(allow_promote)) + current_state = self.state_handler.sync_handler.current_state(self.cluster) + picked = current_state.active + allow_promote = current_state.sync + voters = CaseInsensitiveSet(self.cluster.sync.voters) + + if picked == voters: + return + + sync = self.cluster.sync + + # update synchronous standby list in dcs temporarily to point to common nodes in current and picked + sync_common = voters & allow_promote + if sync_common != voters: + logger.info("Updating synchronous privilege temporarily from %s to %s", + list(voters), list(sync_common)) + sync = self.dcs.write_sync_state(self.state_handler.name, sync_common, 0, index=sync.index) + if not sync: + return logger.info('Synchronous replication key updated by someone else.') + + # When strict mode and no suitable replication connections put "*" to synchronous_standby_names + if self.global_config.is_synchronous_mode_strict and not picked: + picked = CaseInsensitiveSet('*') + logger.warning("No standbys available!") + + # Update postgresql.conf and wait 2 secs for changes to become active + logger.info("Assigning synchronous standby status to %s", list(picked)) + self.state_handler.sync_handler.set_synchronous_standby_names(picked) + + if picked and picked != CaseInsensitiveSet('*') and allow_promote != picked: + # Wait for PostgreSQL to enable synchronous mode and see if we can immediately set sync_standby + time.sleep(2) + allow_promote = self.state_handler.sync_handler.current_state(self.cluster).sync + + if allow_promote and allow_promote != sync_common: + if self.dcs.write_sync_state(self.state_handler.name, allow_promote, 0, index=sync.index): + logger.info("Synchronous standby status assigned to %s", list(allow_promote)) + else: + logger.info("Synchronous replication key updated by someone else") + + def process_sync_replication(self) -> None: + """Process synchronous replication beahvior on the primary.""" + if self.is_quorum_commit_mode(): + self._process_quorum_replication() + elif self.is_synchronous_mode(): + self._process_multisync_replication() else: - if not self.cluster.sync.is_empty and self.dcs.delete_sync_state(index=self.cluster.sync.index): - logger.info("Disabled synchronous replication") - self.state_handler.sync_handler.set_synchronous_standby_names(CaseInsensitiveSet()) + self.disable_synchronous_replication() + + def process_sync_replication_prepromote(self) -> bool: + """Handle sync replication state before promote. + + If quorum replication is requested and we can keep syncing to enough nodes satisfying the quorum invariant + we can promote immediately and let normal quorum resolver process handle any membership changes later. + Otherwise we will just reset DCS state to ourselves and add replicas as they connect. + + :returns: `True` if on success or `False` if failed to update /sync key in DCS. + """ + if not self.is_synchronous_mode(): + self.disable_synchronous_replication() + return True + + if self.is_quorum_commit_mode_active(): + sync = CaseInsensitiveSet(self.cluster.sync.members) + numsync = len(sync) - self.cluster.sync.quorum - 1 + if self.state_handler.name not in sync: # Node outside voters achieved quorum and got leader + numsync += 1 + else: + sync.discard(self.state_handler.name) + else: + sync = CaseInsensitiveSet() + numsync = self.global_config.min_synchronous_nodes + + if not self.is_quorum_commit_mode() or not self.state_handler.supports_multiple_sync and numsync > 1: + sync = CaseInsensitiveSet() + numsync = self.global_config.min_synchronous_nodes + + # Just set ourselves as the authoritative source of truth for now. We don't want to wait for standbys + # to connect. We will try finding a synchronous standby in the next cycle. + if not self.dcs.write_sync_state(self.state_handler.name, None, 0, index=self.cluster.sync.index): + return False + + self.state_handler.sync_handler.set_synchronous_standby_names(sync, numsync) + return True def is_sync_standby(self, cluster: Cluster) -> bool: """:returns: `True` if the current node is a synchronous standby.""" @@ -726,15 +792,10 @@ def enforce_primary_role(self, message: str, promote_message: str) -> str: self.process_sync_replication() return message else: - if self.is_synchronous_mode(): - # Just set ourselves as the authoritative source of truth for now. We don't want to wait for standbys - # to connect. We will try finding a synchronous standby in the next cycle. - if not self.dcs.write_sync_state(self.state_handler.name, None, 0, index=self.cluster.sync.index): - # Somebody else updated sync state, it may be due to us losing the lock. To be safe, postpone - # promotion until next cycle. TODO: trigger immediate retry of run_cycle - return 'Postponing promotion because synchronous replication state was updated by somebody else' - self.state_handler.sync_handler.set_synchronous_standby_names( - CaseInsensitiveSet('*') if self.global_config.is_synchronous_mode_strict else CaseInsensitiveSet()) + if not self.process_sync_replication_prepromote(): + # Somebody else updated sync state, it may be due to us losing the lock. To be safe, + # postpone promotion until next cycle. TODO: trigger immediate retry of run_cycle. + return 'Postponing promotion because synchronous replication state was updated by somebody else' if self.state_handler.role not in ('master', 'promoted', 'primary'): def on_success(): self._rewind.reset_state() @@ -751,10 +812,14 @@ def before_promote(): return promote_message def fetch_node_status(self, member: Member) -> _MemberStatus: - """This function perform http get request on member.api_url and fetches its status - :returns: `_MemberStatus` object - """ + """This function performs http get request on member.api_url and fetches its status. + + Usually it happens during the leader race and we can't afford wating for a response indefinite time, + therefore the request timeout is hardcoded to 2 seconds, which seems to be a good compromise. + The node which is slow to respond most likely will not be healthy. + :returns: :class:`_MemberStatus` object + """ try: response = self.patroni.request(member, timeout=2, retries=0) data = response.data.decode('utf-8') @@ -815,17 +880,24 @@ def check_failsafe_topology(self) -> bool: return all(results) def is_lagging(self, wal_position: int) -> bool: - """Returns if instance with an wal should consider itself unhealthy to be promoted due to replication lag. + """Checks if node should consider itself unhealthy to be promoted due to replication lag. :param wal_position: Current wal position. - :returns True when node is lagging + :returns `True` when node is lagging """ lag = (self.cluster.last_lsn or 0) - wal_position return lag > self.global_config.maximum_lag_on_failover def _is_healthiest_node(self, members: Collection[Member], check_replication_lag: bool = True) -> bool: - """This method tries to determine whether I am healthy enough to became a new leader candidate or not.""" - + """This method tries to determine whether the current node is healthy enough to became a new leader candidate. + + :param members: the list of nodes to check against + :param check_replication_lag: whether to take the replication lag into account. + If the lag exceeds configured threshold the node disqualifies itself. + :returns: `True` in case if the node is eligible to become the new leader. Since this method is executed + on multiple nodes independently it could happen that many nodes will count themselves as + healthiest because they received/replayed up to the same LSN, but it is totally fine. + """ my_wal_position = self.state_handler.last_operation() if check_replication_lag and self.is_lagging(my_wal_position): logger.info('My wal position exceeds maximum replication lag') @@ -841,8 +913,26 @@ def _is_healthiest_node(self, members: Collection[Member], check_replication_lag logger.info('My timeline %s is behind last known cluster timeline %s', my_timeline, cluster_timeline) return False - # Prepare list of nodes to run check against - members = [m for m in members if m.name != self.state_handler.name and not m.nofailover and m.api_url] + if self.is_quorum_commit_mode_active(): + quorum = self.cluster.sync.quorum + voting_set = CaseInsensitiveSet(self.cluster.sync.members) + else: + quorum = 0 + voting_set = CaseInsensitiveSet() + + # Prepare list of nodes to run check against. If quorum commit is enabled + # we also include members with nofailover tag if they are listed in voters. + members = [m for m in members if m.name != self.state_handler.name + and m.api_url and (not m.nofailover or m.name in voting_set)] + + # If there is a quorum active then at least one of the quorum contains latest commit. A quorum member saying + # their WAL position is not ahead counts as a vote saying we may become new leader. Note that a node doesn't + # have to be a member of the voting set to gather the necessary votes. + + # Regardless of voting, if we observe a node that can become a leader and is ahead, we defer to that node. + # This can lead to failure to act on quorum if there is asymmetric connectivity. + quorum_votes = 0 if self.state_handler.name in voting_set else -1 + nodes_ahead = 0 if members: for st in self.fetch_nodes_statuses(members): @@ -851,14 +941,24 @@ def _is_healthiest_node(self, members: Collection[Member], check_replication_lag logger.warning('Primary (%s) is still alive', st.member.name) return False if my_wal_position < st.wal_position: + nodes_ahead += 1 logger.info('Wal position of %s is ahead of my wal position', st.member.name) # In synchronous mode the former leader might be still accessible and even be ahead of us. # We should not disqualify himself from the leader race in such a situation. - if not self.is_synchronous_mode() or self.cluster.sync.is_empty\ + if not self.is_synchronous_mode_active()\ or not self.cluster.sync.leader_matches(st.member.name): return False logger.info('Ignoring the former leader being ahead of us') - return True + # we want to count votes only from nodes with postgres up and running! + elif st.member.name in voting_set and st.wal_position > 0: + logger.info('Got quorum vote from %s', st.member.name) + quorum_votes += 1 + + # When not in quorum commit we just want to return `True`. + # In quorum commit the former leader is special and counted healthy even when there are no other nodes. + # Otherwise check that the number of votes exceeds the quorum field from the /sync key. + return not self.is_quorum_commit_mode_active() or quorum_votes >= quorum\ + or nodes_ahead == 0 and self.cluster.sync.leader == self.state_handler.name def is_failover_possible(self, members: List[Member], check_synchronous: Optional[bool] = True, cluster_lsn: Optional[int] = 0) -> bool: @@ -872,8 +972,10 @@ def is_failover_possible(self, members: List[Member], check_synchronous: Optiona ret = False cluster_timeline = self.cluster.timeline members = [m for m in members if m.name != self.state_handler.name and not m.nofailover and m.api_url] - if check_synchronous and self.is_synchronous_mode() and not self.cluster.sync.is_empty: - members = [m for m in members if self.cluster.sync.matches(m.name)] + if check_synchronous and self.is_synchronous_mode_active(): + # If quorum commit is requested we want to check all nodes (even not voters), + # because they could get enough votes and reach necessary quorum + 1. + members = [m for m in members if self.is_quorum_commit_mode() or self.cluster.sync.matches(m.name)] if members: for st in self.fetch_nodes_statuses(members): not_allowed_reason = st.failover_limitation() @@ -913,9 +1015,10 @@ def manual_failover_process_no_leader(self) -> Optional[bool]: return None return False - # in synchronous mode when our name is not in the /sync key - # we shouldn't take any action even if the candidate is unhealthy - if self.is_synchronous_mode() and not self.cluster.sync.matches(self.state_handler.name, True): + # in synchronous mode (except quorum commit!) when our name is not in the + # /sync key we shouldn't take any action even if the candidate is unhealthy + if self.is_synchronous_mode() and not self.is_quorum_commit_mode()\ + and not self.cluster.sync.matches(self.state_handler.name, True): return False # find specific node and check that it is healthy @@ -937,7 +1040,7 @@ def manual_failover_process_no_leader(self) -> Optional[bool]: # try to pick some other members to failover and check that they are healthy if failover.leader: if self.state_handler.name == failover.leader: # I was the leader - # exclude me and desired member which is unhealthy (failover.candidate can be None) + # exclude me (leader) and desired member which is unhealthy (failover.candidate can be None) members = [m for m in self.cluster.members if m.name not in (failover.candidate, failover.leader)] if self.is_failover_possible(members): # check that there are healthy members return False @@ -1003,9 +1106,11 @@ def is_healthiest_node(self) -> bool: all_known_members += [RemoteMember(name, {'api_url': url}) for name, url in failsafe_members.items()] all_known_members += self.cluster.members - # When in sync mode, only last known primary and sync standby are allowed to promote automatically. - if self.is_synchronous_mode() and not self.cluster.sync.is_empty: - if not self.cluster.sync.matches(self.state_handler.name, True): + # Special handling if synchronous mode was requested and activated (the leader in /sync is not empty) + if self.is_synchronous_mode_active(): + # In quorum commit mode we allow nodes outside of "voters" to take part in + # the leader race. They just need to get enough votes to `reach quorum + 1`. + if not self.is_quorum_commit_mode() and not self.cluster.sync.matches(self.state_handler.name, True): return False # pick between synchronous candidates so we minimize unnecessary failovers/demotions members = {m.name: m for m in all_known_members if self.cluster.sync.matches(m.name, True)} diff --git a/tests/test_ha.py b/tests/test_ha.py index 590157843..0a1c01d56 100644 --- a/tests/test_ha.py +++ b/tests/test_ha.py @@ -1206,7 +1206,7 @@ def test_sync_replication_become_primary(self): # When we just became primary nobody is sync self.assertEqual(self.ha.enforce_primary_role('msg', 'promote msg'), 'promote msg') - mock_set_sync.assert_called_once_with(CaseInsensitiveSet()) + mock_set_sync.assert_called_once_with(frozenset(), 0) mock_write_sync.assert_called_once_with('leader', None, 0, index=0) mock_set_sync.reset_mock() @@ -1433,3 +1433,49 @@ def test_notify_citus_coordinator(self): self.assertEqual(self.ha.patroni.request.call_args[1]['timeout'], 2) mock_logger.assert_called() self.assertTrue(mock_logger.call_args[0][0].startswith('Request to Citus coordinator')) + + def test_process_sync_replication_prepromote(self): + self.p._major_version = 90500 + self.ha.cluster = get_cluster_initialized_without_leader(sync=('other', self.p.name + ',foo')) + self.ha.cluster.config.data.update({'synchronous_mode': 'quorum'}) + self.ha.global_config = self.ha.patroni.config.get_global_config(self.ha.cluster) + self.p.is_leader = false + self.p.set_role('replica') + mock_write_sync = self.ha.dcs.write_sync_state = Mock(return_value=None) + # Postgres 9.5, write_sync_state to DCS failed + self.assertEqual(self.ha.run_cycle(), + 'Postponing promotion because synchronous replication state was updated by somebody else') + self.assertEqual(self.ha.dcs.write_sync_state.call_count, 1) + self.assertEqual(mock_write_sync.call_args_list[0][0], (self.p.name, None, 0)) + self.assertEqual(mock_write_sync.call_args_list[0][1], {'index': 0}) + + mock_set_sync = self.p.config.set_synchronous_standby_names = Mock() + mock_write_sync = self.ha.dcs.write_sync_state = Mock(return_value=True) + # Postgres 9.5, our name is written to leader of the /sync key, while voters list and ssn is empty + self.assertEqual(self.ha.run_cycle(), 'promoted self to leader by acquiring session lock') + self.assertEqual(self.ha.dcs.write_sync_state.call_count, 1) + self.assertEqual(mock_write_sync.call_args_list[0][0], (self.p.name, None, 0)) + self.assertEqual(mock_write_sync.call_args_list[0][1], {'index': 0}) + self.assertEqual(mock_set_sync.call_count, 1) + self.assertEqual(mock_set_sync.call_args_list[0][0], (None,)) + + self.p._major_version = 90600 + mock_set_sync.reset_mock() + mock_write_sync.reset_mock() + self.p.set_role('replica') + # Postgres 9.6, with quorum commit we avoid updating /sync key and put some nodes to ssn + self.assertEqual(self.ha.run_cycle(), 'promoted self to leader by acquiring session lock') + self.assertEqual(mock_write_sync.call_count, 0) + self.assertEqual(mock_set_sync.call_count, 1) + self.assertEqual(mock_set_sync.call_args_list[0][0], ('2 (foo,other)',)) + + self.p._major_version = 150000 + mock_set_sync.reset_mock() + self.p.set_role('replica') + self.p.name = 'nonsync' + self.ha.fetch_node_status = get_node_status() + # Postgres 15, with quorum commit. Non-sync node promoted we avoid updating /sync key and put some nodes to ssn + self.assertEqual(self.ha.run_cycle(), 'promoted self to leader by acquiring session lock') + self.assertEqual(mock_write_sync.call_count, 0) + self.assertEqual(mock_set_sync.call_count, 1) + self.assertEqual(mock_set_sync.call_args_list[0][0], ('ANY 3 (foo,other,postgresql0)',)) From e97d2f09993032b0f0089805be6ec8ac30f79590 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 11 May 2023 12:18:08 +0200 Subject: [PATCH 04/37] Implement synchronous_mode=quorum --- patroni/config.py | 2 +- patroni/ha.py | 56 ++++- patroni/quorum.py | 305 ++++++++++++++++++++++++++++ tests/test_ha.py | 65 +++++- tests/test_quorum.py | 473 +++++++++++++++++++++++++++++++++++++++++++ 5 files changed, 898 insertions(+), 3 deletions(-) create mode 100644 patroni/quorum.py create mode 100644 tests/test_quorum.py diff --git a/patroni/config.py b/patroni/config.py index 4486fdd36..63a34b7e5 100644 --- a/patroni/config.py +++ b/patroni/config.py @@ -563,7 +563,7 @@ def _build_effective_configuration(self, dynamic_configuration: Dict[str, Any], if 'citus' in config: bootstrap = config.setdefault('bootstrap', {}) dcs = bootstrap.setdefault('dcs', {}) - dcs.setdefault('synchronous_mode', True) + dcs.setdefault('synchronous_mode', 'quorum') updated_fields = ( 'name', diff --git a/patroni/ha.py b/patroni/ha.py index 3b4483fc0..a8cc8e95c 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -20,6 +20,7 @@ from .postgresql.misc import postgres_version_to_int from .postgresql.postmaster import PostmasterProcess from .postgresql.rewind import Rewind +from .quorum import QuorumStateResolver from .utils import polling_loop, tzutc logger = logging.getLogger(__name__) @@ -595,7 +596,60 @@ def disable_synchronous_replication(self) -> None: self.state_handler.sync_handler.set_synchronous_standby_names(CaseInsensitiveSet()) def _process_quorum_replication(self) -> None: - pass + """Process synchronous replication state when quorum commit is requested. + + Synchronous standbys are registered in two places postgresql.conf and DCS. The order of updating them must + keep the invariant that `quorum + sync >= len(set(quorum pool)|set(sync pool))`. This is done using + :class:`QuorumStateResolver` that given a current state and set of desired synchronous nodes and replication + level outputs changes to DCS and synchronous replication in correct order to reach the desired state. + In case any of those steps causes an error we can just bail out and let next iteration rediscover the state + and retry necessary transitions. + """ + min_sync = self.global_config.min_synchronous_nodes + sync_wanted = self.global_config.synchronous_node_count + + sync = self.cluster.sync + leader = sync.leader or self.state_handler.name + if sync.is_empty: + sync = self.dcs.write_sync_state(leader, None, 0, index=sync.index) + if not sync: + return logger.warning("Updating sync state failed") + + while True: + transition = 'break' # we need define transition value if `QuorumStateResolver` produced no changes + sync_state = self.state_handler.sync_handler.current_state(self.cluster) + for transition, leader, num, nodes in QuorumStateResolver(leader=leader, + quorum=sync.quorum, + voters=sync.voters, + numsync=sync_state.numsync, + sync=sync_state.sync, + numsync_confirmed=sync_state.numsync_confirmed, + active=sync_state.active, + sync_wanted=sync_wanted, + leader_wanted=self.state_handler.name): + if transition == 'quorum': + logger.info("Setting leader to %s, quorum to %d of %d (%s)", + leader, num, len(nodes), ", ".join(sorted(nodes))) + sync = self.dcs.write_sync_state(leader, nodes, num, index=sync.index) + if not sync: + return logger.info('Synchronous replication key updated by someone else.') + elif transition == 'sync': + logger.info("Setting synchronous replication to %d of %d (%s)", + num, len(nodes), ", ".join(sorted(nodes))) + # Bump up number of num nodes to meet minimum replication factor. Commits will have to wait until + # we have enough nodes to meet replication target. + if num < min_sync: + logger.warning("Replication factor %d requested, but %d synchronous standbys available." + " Commits will be delayed.", min_sync + 1, num) + num = min_sync + self.state_handler.sync_handler.set_synchronous_standby_names(nodes, num) + if transition != 'restart': + break + # synchronous_standby_names was transitioned from empty to non-empty and it may take + # some time for nodes to become synchronous. In this case we want to restart state machine + # hoping that we can update /sync key earlier than in loop_wait seconds. + time.sleep(1) + self.state_handler.reset_cluster_info_state(None) def _process_multisync_replication(self) -> None: """Process synchronous replication state with one or more sync standbys. diff --git a/patroni/quorum.py b/patroni/quorum.py new file mode 100644 index 000000000..a41597b50 --- /dev/null +++ b/patroni/quorum.py @@ -0,0 +1,305 @@ +import logging + +from typing import Collection, Iterator, Optional, Tuple + +from .collections import CaseInsensitiveSet + +logger = logging.getLogger(__name__) + + +class QuorumError(Exception): + pass + + +class QuorumStateResolver(object): + """Calculates a list of state transition tuples of the form `('sync'/'quorum'/'restart',leader,number,set_of_names)` + + Synchronous replication state is set in two places. PostgreSQL configuration sets how many and which nodes are + needed for a commit to succeed, abbreviated as `numsync` and `sync` set here. DCS contains information about how + many and which nodes need to be interrogated to be sure to see an xlog position containing latest confirmed commit, + abbreviated as `quorum` and `voters` set. Both pairs have the meaning "ANY n OF set". + + The number of nodes needed for commit to succeed, `numsync`, is also called the replication factor. + + To guarantee zero lost transactions on failover we need to keep the invariant that at all times any subset of + nodes that can acknowledge a commit overlaps with any subset of nodes that can achieve quorum to promote a new + leader. Given a desired replication factor and a set of nodes able to participate in sync replication there + is one optimal state satisfying this condition. Given the node set `active`, the optimal state is: + + sync = voters = active + numsync = min(sync_wanted, len(active)) + quorum = len(active) - numsync + + We need to be able to produce a series of state changes that take the system to this desired state from any + other state arbitrary given arbitrary changes is node availability, configuration and interrupted transitions. + + To keep the invariant the rule to follow is that when increasing `numsync` or `quorum`, we need to perform the + increasing operation first. When decreasing either, the decreasing operation needs to be performed later. + + Order of adding or removing nodes from sync and voters depends on the state of synchronous_standby_names: + When adding new nodes: + if sync (synchronous_standby_names) is empty: + add new nodes first to sync and then to voters when numsync_confirmed > 0 + else: + add new nodes first to voters and than to sync + When removing nodes: + if sync (synchronous_standby_names) will become empty after removal: + first remove nodes from voters and than from sync + else: + first remove nodes from sync and than from voters. make voters empty if numsync_confirmed == 0""" + + def __init__(self, leader: str, quorum: int, voters: Collection[str], + numsync: int, sync: Collection[str], numsync_confirmed: int, + active: Collection[str], sync_wanted: int, leader_wanted: str) -> None: + self.leader = leader # The leader according to the `/sync` key + self.quorum = quorum # The number of nodes we need to check when doing leader race + self.voters = CaseInsensitiveSet(voters) # Set of nodes we need to check (both stored in the /sync key) + self.numsync = min(numsync, len(sync)) # The number of sync nodes in synchronous_standby_names + self.sync = CaseInsensitiveSet(sync) # Set of nodes in synchronous_standby_names + # The number of nodes that are confirmed to reach safe LSN after adding them to `synchronous_standby_names`. + # We don't list them because it is known that they are always included into active. + self.numsync_confirmed = numsync_confirmed + self.active = CaseInsensitiveSet(active) # Set of active nodes from `pg_stat_replication` + self.sync_wanted = sync_wanted # The desired number of sync nodes + self.leader_wanted = leader_wanted # The desired leader + + def check_invariants(self) -> None: + """Checks invatiant of synchronous_standby_names and /sync key in DCS. + + :raises `QuorumError`: in case of broken state""" + voters = CaseInsensitiveSet(self.voters | CaseInsensitiveSet([self.leader])) + sync = CaseInsensitiveSet(self.sync | CaseInsensitiveSet([self.leader_wanted])) + + # We need to verify that subset of nodes that can acknowledge a commit overlaps + # with any subset of nodes that can achieve quorum to promote a new leader. + if self.voters and not (len(voters | sync) <= self.quorum + self.numsync + 1): + raise QuorumError("Quorum and sync not guaranteed to overlap: nodes %d >= quorum %d + sync %d" % + (len(voters | sync), self.quorum, self.numsync)) + # unstable cases, we are changing synchronous_standby_names and /sync key + # one after another, hence one set is allowed to be a subset of another + if not (voters.issubset(sync) or sync.issubset(voters)): + raise QuorumError("Mismatched sets: quorum only=%s sync only=%s" % + (voters - sync, sync - voters)) + + def quorum_update(self, quorum: int, voters: CaseInsensitiveSet, leader: Optional[str] = None, + adjust_quorum: Optional[bool] = True) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + """Updates quorum, voters and optionally leader fields. + + :param quorum: the new value for `self.quorum`, could be adjusted depending + on values of `self.numsync_confirmed` and `adjust_quorum` + :param voters: the new value for `self.voters`, could be adjusted if numsync_confirmed == 0 + :param leader: the new value for `self.leader`, optional + :param adjust_quorum: if set to `True` the quorum requirement will be increased by the + difference between `self.numsync` and ``self.numsync_confirmed` + :rtype: Iterator[tuple(type, leader, quorum, voters)] with the new quorum state, + where type could be 'quorum' or 'restart'. The latter means that + quorum could not be updated with the current input data + and the :class:`QuorumStateResolver` should be restarted. + :raises `QuorumError`: in case of invalid data or if invariant after transition could not be satisfied + """ + if quorum < 0: + raise QuorumError("Quorum %d < 0 of (%s)" % (quorum, voters)) + if quorum > 0 and quorum >= len(voters): + raise QuorumError("Quorum %d >= N of (%s)" % (quorum, voters)) + + old_leader = self.leader + if leader is not None: # Change of leader was requested + self.leader = leader + elif self.numsync_confirmed == 0: + # If there are no nodes that known to caught up with the primary we want to reset quorum/votes in /sync key + quorum = 0 + voters = CaseInsensitiveSet() + elif adjust_quorum: + # It could be that the number of nodes that are known to catch up with the primary is below desired numsync. + # We want to increase quorum to guaranty that the sync node will be found during the leader race. + quorum += max(self.numsync - self.numsync_confirmed, 0) + + if (self.leader, quorum, voters) == (old_leader, self.quorum, self.voters): + if self.voters: + return + # If transition produces no change of leader/quorum/voters we want to give a hint to + # the caller to fetch the new state from the database and restart QuorumStateResolver. + yield 'restart', self.leader, self.quorum, self.voters + + self.quorum = quorum + self.voters = voters + self.check_invariants() + logger.debug('quorum %s %s %s', self.leader, self.quorum, self.voters) + yield 'quorum', self.leader, self.quorum, self.voters + + def sync_update(self, numsync: int, sync: CaseInsensitiveSet) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + """Updates numsync and sync fields. + + :param numsync: the new value for `self.numsync` + :param sync: the new value for `self.sync` + :rtype: Iterator[tuple('sync', leader, numsync, sync)] with the new state of synchronous_standby_names + :raises `QuorumError`: in case of invalid data or if invariant after transition could not be satisfied + """ + if numsync < 0: + raise QuorumError("Sync %d < 0 of (%s)" % (numsync, sync)) + if numsync > len(sync): + raise QuorumError("Sync %s > N of (%s)" % (numsync, sync)) + + self.numsync = numsync + self.sync = sync + self.check_invariants() + logger.debug('sync %s %s %s', self.leader, self.numsync, self.sync) + yield 'sync', self.leader, self.numsync, self.sync + + def __iter__(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + transitions = list(self._generate_transitions()) + # Merge 2 transitions of the same type to a single one. This is always safe because skipping the first + # transition is equivalent to no one observing the intermediate state. + for cur_transition, next_transition in zip(transitions, transitions[1:] + [None]): + if next_transition and cur_transition[0] == next_transition[0]: + continue + yield cur_transition + if cur_transition[0] == 'restart': + break + + def _generate_transitions(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + logger.debug("Quorum state: leader %s quorum %s, voters %s, numsync %s, sync %s, " + "numsync_confirmed %s, active %s, sync_wanted %s leader_wanted %s", + self.leader, self.quorum, self.voters, self.numsync, self.sync, + self.numsync_confirmed, self.active, self.sync_wanted, self.leader_wanted) + try: + if self.leader_wanted != self.leader: + voters = (self.voters - CaseInsensitiveSet([self.leader_wanted])) | CaseInsensitiveSet([self.leader]) + if not self.sync: + # If sync is empty we need to update synchronous_standby_names first + numsync = len(voters) - self.quorum + yield from self.sync_update(numsync, CaseInsensitiveSet(voters)) + # If leader changed we need to add the old leader to quorum (voters) + yield from self.quorum_update(self.quorum, CaseInsensitiveSet(voters), self.leader_wanted) + # right after promote there could be no replication connections yet + if not self.sync & self.active: + return # give another loop_wait seconds for replicas to reconnect before removing them from quorum + else: + self.check_invariants() + except QuorumError as e: + logger.warning('%s', e) + yield from self.quorum_update(len(self.sync) - self.numsync, self.sync) + + assert self.leader == self.leader_wanted + + # numsync_confirmed could be 0 after restart/failover, we will calculate it from quorum + if self.numsync_confirmed == 0 and self.sync & self.active: + self.numsync_confirmed = min(len(self.sync & self.active), len(self.voters) - self.quorum) + logger.debug('numsync_confirmed=0, adjusting it to %d', self.numsync_confirmed) + + # Handle non steady state cases + if self.sync < self.voters: + logger.debug("Case 1: synchronous_standby_names subset of DCS state") + # Case 1: quorum is superset of sync nodes. In the middle of changing quorum. + # Evict from quorum dead nodes that are not being synced. + remove_from_quorum = self.voters - (self.sync | self.active) + if remove_from_quorum: + yield from self.quorum_update( + quorum=len(self.voters) - len(remove_from_quorum) - self.numsync, + voters=CaseInsensitiveSet(self.voters - remove_from_quorum), + adjust_quorum=not (self.sync - self.active)) + # Start syncing to nodes that are in quorum and alive + add_to_sync = (self.voters & self.active) - self.sync + if add_to_sync: + yield from self.sync_update(self.numsync, CaseInsensitiveSet(self.sync | add_to_sync)) + elif self.sync > self.voters: + logger.debug("Case 2: synchronous_standby_names superset of DCS state") + # Case 2: sync is superset of quorum nodes. In the middle of changing replication factor. + # Add to quorum voters nodes that are already synced and active + add_to_quorum = (self.sync - self.voters) & self.active + if add_to_quorum: + voters = CaseInsensitiveSet(self.voters | add_to_quorum) + yield from self.quorum_update(len(voters) - self.numsync, voters) + # Remove from sync nodes that are dead + remove_from_sync = self.sync - self.voters + if remove_from_sync: + yield from self.sync_update( + numsync=min(self.numsync, len(self.sync) - len(remove_from_sync)), + sync=CaseInsensitiveSet(self.sync - remove_from_sync)) + + # After handling these two cases quorum and sync must match. + assert self.voters == self.sync + + safety_margin = self.quorum + min(self.numsync, self.numsync_confirmed) - len(self.voters | self.sync) + if safety_margin > 0: # In the middle of changing replication factor. + if self.numsync > self.sync_wanted: + logger.debug('Case 3: replication factor is bigger than needed') + yield from self.sync_update(max(self.sync_wanted, len(self.voters) - self.quorum), self.sync) + else: + logger.debug('Case 4: quorum is bigger than needed') + yield from self.quorum_update(len(self.sync) - self.numsync, self.voters) + else: + safety_margin = self.quorum + self.numsync - len(self.voters | self.sync) + if self.numsync == self.sync_wanted and safety_margin > 0 and self.numsync > self.numsync_confirmed: + yield from self.quorum_update(len(self.sync) - self.numsync, self.voters) + + # We are in a steady state point. Find if desired state is different and act accordingly. + + # If any nodes have gone away, evict them + to_remove = self.sync - self.active + if to_remove and self.sync == to_remove: + logger.debug("Removing nodes: %s", to_remove) + yield from self.quorum_update(0, CaseInsensitiveSet(), adjust_quorum=False) + yield from self.sync_update(0, CaseInsensitiveSet()) + elif to_remove: + logger.debug("Removing nodes: %s", to_remove) + can_reduce_quorum_by = self.quorum + # If we can reduce quorum size try to do so first + if can_reduce_quorum_by: + # Pick nodes to remove by sorted order to provide deterministic behavior for tests + remove = CaseInsensitiveSet(sorted(to_remove, reverse=True)[:can_reduce_quorum_by]) + sync = CaseInsensitiveSet(self.sync - remove) + # when removing nodes from sync we can safely increase numsync if requested + numsync = min(self.sync_wanted, len(sync)) if self.sync_wanted > self.numsync else self.numsync + yield from self.sync_update(numsync, sync) + voters = CaseInsensitiveSet(self.voters - remove) + to_remove &= self.sync + yield from self.quorum_update(len(voters) - self.numsync, voters, + adjust_quorum=not to_remove) + if to_remove: + assert self.quorum == 0 + numsync = self.numsync - len(to_remove) + sync = CaseInsensitiveSet(self.sync - to_remove) + voters = CaseInsensitiveSet(self.voters - to_remove) + sync_decrease = numsync - min(self.sync_wanted, len(sync)) + quorum = min(sync_decrease, len(voters) - 1) if sync_decrease else 0 + yield from self.quorum_update(quorum, voters, adjust_quorum=False) + yield from self.sync_update(numsync, sync) + + # If any new nodes, join them to quorum + to_add = self.active - self.sync + if to_add: + # First get to requested replication factor + logger.debug("Adding nodes: %s", to_add) + sync_wanted = min(self.sync_wanted, len(self.sync | to_add)) + increase_numsync_by = sync_wanted - self.numsync + if increase_numsync_by > 0: + if self.sync: + add = CaseInsensitiveSet(sorted(to_add)[:increase_numsync_by]) + increase_numsync_by = len(add) + else: # there is only the leader + add = to_add # and it is safe to add all nodes at once if sync is empty + yield from self.sync_update(self.numsync + increase_numsync_by, CaseInsensitiveSet(self.sync | add)) + voters = CaseInsensitiveSet(self.voters | add) + yield from self.quorum_update(len(voters) - sync_wanted, voters) + to_add -= self.sync + if to_add: + voters = CaseInsensitiveSet(self.voters | to_add) + yield from self.quorum_update(len(voters) - sync_wanted, voters, + adjust_quorum=sync_wanted > self.numsync_confirmed) + yield from self.sync_update(sync_wanted, CaseInsensitiveSet(self.sync | to_add)) + + # Apply requested replication factor change + sync_increase = min(self.sync_wanted, len(self.sync)) - self.numsync + if sync_increase > 0: + # Increase replication factor + logger.debug("Increasing replication factor to %s", self.numsync + sync_increase) + yield from self.sync_update(self.numsync + sync_increase, self.sync) + yield from self.quorum_update(len(self.voters) - self.numsync, self.voters) + elif sync_increase < 0: + # Reduce replication factor + logger.debug("Reducing replication factor to %s", self.numsync + sync_increase) + if self.quorum - sync_increase < len(self.voters): + yield from self.quorum_update(len(self.voters) - self.numsync - sync_increase, self.voters, + adjust_quorum=self.sync_wanted > self.numsync_confirmed) + yield from self.sync_update(self.numsync + sync_increase, self.sync) diff --git a/tests/test_ha.py b/tests/test_ha.py index 0a1c01d56..5a99f456c 100644 --- a/tests/test_ha.py +++ b/tests/test_ha.py @@ -1102,7 +1102,7 @@ def test_demote_immediate(self, follow): self.ha.demote('immediate') follow.assert_called_once_with(None) - def test_process_sync_replication(self): + def test_process__multisync_replication(self): self.ha.has_lock = true mock_set_sync = self.p.sync_handler.set_synchronous_standby_names = Mock() self.p.name = 'leader' @@ -1479,3 +1479,66 @@ def test_process_sync_replication_prepromote(self): self.assertEqual(mock_write_sync.call_count, 0) self.assertEqual(mock_set_sync.call_count, 1) self.assertEqual(mock_set_sync.call_args_list[0][0], ('ANY 3 (foo,other,postgresql0)',)) + + def test_process_quorum_replication(self): + self.p._major_version = 150000 + self.ha.has_lock = true + mock_set_sync = self.p.config.set_synchronous_standby_names = Mock() + self.p.name = 'leader' + + self.ha.cluster.config.data.update({'synchronous_mode': 'quorum'}) + self.ha.global_config = self.ha.patroni.config.get_global_config(self.ha.cluster) + + mock_write_sync = self.ha.dcs.write_sync_state = Mock(return_value=None) + # Test /sync key is attempted to set and failed when missing or invalid + self.p.sync_handler.current_state = Mock(return_value=_SyncState('quorum', 1, 1, CaseInsensitiveSet(['other']), + CaseInsensitiveSet(['other']))) + self.ha.run_cycle() + self.assertEqual(mock_write_sync.call_count, 1) + self.assertEqual(mock_write_sync.call_args_list[0][0], (self.p.name, None, 0)) + self.assertEqual(mock_write_sync.call_args_list[0][1], {'index': None}) + self.assertEqual(mock_set_sync.call_count, 0) + + mock_write_sync = self.ha.dcs.write_sync_state = Mock(side_effect=[SyncState.empty(), None]) + # Test /sync key is attempted to set and succeed when missing or invalid + with patch.object(SyncState, 'is_empty', Mock(side_effect=[True, False])): + self.ha.run_cycle() + self.assertEqual(mock_write_sync.call_count, 2) + self.assertEqual(mock_write_sync.call_args_list[0][0], (self.p.name, None, 0)) + self.assertEqual(mock_write_sync.call_args_list[0][1], {'index': None}) + self.assertEqual(mock_write_sync.call_args_list[1][0], (self.p.name, CaseInsensitiveSet(['other']), 0)) + self.assertEqual(mock_write_sync.call_args_list[1][1], {'index': None}) + self.assertEqual(mock_set_sync.call_count, 0) + + self.p.sync_handler.current_state = Mock(side_effect=[_SyncState('quorum', 1, 0, CaseInsensitiveSet(['foo']), + CaseInsensitiveSet(['other'])), + _SyncState('quorum', 1, 1, CaseInsensitiveSet(['foo']), + CaseInsensitiveSet(['foo']))]) + mock_write_sync = self.ha.dcs.write_sync_state = Mock(return_value=SyncState(1, 'leader', 'foo', 0)) + self.ha.cluster = get_cluster_initialized_with_leader(sync=('leader', 'foo')) + self.ha.cluster.config.data.update({'synchronous_mode': 'quorum'}) + self.ha.global_config = self.ha.patroni.config.get_global_config(self.ha.cluster) + # Test the sync node is removed from voters, added to ssn + with patch.object(Postgresql, 'synchronous_standby_names', Mock(return_value='other')),\ + patch('time.sleep', Mock()): + self.ha.run_cycle() + self.assertEqual(mock_write_sync.call_count, 1) + self.assertEqual(mock_write_sync.call_args_list[0][0], (self.p.name, CaseInsensitiveSet(), 0)) + self.assertEqual(mock_write_sync.call_args_list[0][1], {'index': 0}) + self.assertEqual(mock_set_sync.call_count, 1) + self.assertEqual(mock_set_sync.call_args_list[0][0], ('ANY 1 (other)',)) + + # Test ANY 1 (*) when synchronous_mode_strict and no nodes available + self.ha.cluster.config.data.update({'synchronous_mode_strict': True}) + self.ha.global_config = self.ha.patroni.config.get_global_config(self.ha.cluster) + self.p.sync_handler.current_state = Mock(return_value=_SyncState('quorum', 1, 0, + CaseInsensitiveSet(['other', 'foo']), + CaseInsensitiveSet())) + mock_write_sync.reset_mock() + mock_set_sync.reset_mock() + self.ha.run_cycle() + self.assertEqual(mock_write_sync.call_count, 1) + self.assertEqual(mock_write_sync.call_args_list[0][0], (self.p.name, CaseInsensitiveSet(), 0)) + self.assertEqual(mock_write_sync.call_args_list[0][1], {'index': 0}) + self.assertEqual(mock_set_sync.call_count, 1) + self.assertEqual(mock_set_sync.call_args_list[0][0], ('ANY 1 (*)',)) diff --git a/tests/test_quorum.py b/tests/test_quorum.py new file mode 100644 index 000000000..8dddb4f1a --- /dev/null +++ b/tests/test_quorum.py @@ -0,0 +1,473 @@ +import unittest + +from typing import List, Set, Tuple + +from patroni.quorum import QuorumStateResolver, QuorumError + + +class QuorumTest(unittest.TestCase): + + def check_state_transitions(self, leader: str, quorum: int, voters: Set[str], numsync: int, sync: Set[str], + numsync_confirmed: int, active: Set[str], sync_wanted: int, leader_wanted: str, + expected: List[Tuple[str, str, int, Set[str]]]) -> None: + kwargs = { + 'leader': leader, 'quorum': quorum, 'voters': voters, + 'numsync': numsync, 'sync': sync, 'numsync_confirmed': numsync_confirmed, + 'active': active, 'sync_wanted': sync_wanted, 'leader_wanted': leader_wanted + } + result = list(QuorumStateResolver(**kwargs)) + self.assertEqual(result, expected) + + # also check interrupted transitions + if len(result) > 0 and result[0][0] != 'restart' and kwargs['leader'] == result[0][1]: + if result[0][0] == 'sync': + kwargs.update(numsync=result[0][2], sync=result[0][3]) + else: + kwargs.update(leader=result[0][1], quorum=result[0][2], voters=result[0][3]) + kwargs['expected'] = expected[1:] + self.check_state_transitions(**kwargs) + + def test_1111(self): + leader = 'a' + + # Add node + self.check_state_transitions(leader=leader, quorum=0, voters=set(), + numsync=0, sync=set(), numsync_confirmed=0, active=set('b'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('sync', leader, 1, set('b')), + ('restart', leader, 0, set()), + ]) + self.check_state_transitions(leader=leader, quorum=0, voters=set(), + numsync=1, sync=set('b'), numsync_confirmed=1, active=set('b'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set('b')) + ]) + + self.check_state_transitions(leader=leader, quorum=0, voters=set(), + numsync=0, sync=set(), numsync_confirmed=0, active=set('bcde'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('sync', leader, 2, set('bcde')), + ('restart', leader, 0, set()), + ]) + self.check_state_transitions(leader=leader, quorum=0, voters=set(), + numsync=2, sync=set('bcde'), numsync_confirmed=1, active=set('bcde'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 3, set('bcde')), + ]) + + def test_1222(self): + """2 node cluster""" + leader = 'a' + + # Active set matches state + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=1, sync=set('b'), numsync_confirmed=1, active=set('b'), + sync_wanted=2, leader_wanted=leader, expected=[]) + + # Add node by increasing quorum + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=1, sync=set('b'), numsync_confirmed=1, active=set('BC'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('quorum', leader, 1, set('bC')), + ('sync', leader, 1, set('bC')), + ]) + + # Add node by increasing sync + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=1, sync=set('b'), numsync_confirmed=1, active=set('bc'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('sync', leader, 2, set('bc')), + ('quorum', leader, 1, set('bc')), + ]) + # Reduce quorum after added node caught up + self.check_state_transitions(leader=leader, quorum=1, voters=set('bc'), + numsync=2, sync=set('bc'), numsync_confirmed=2, active=set('bc'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set('bc')), + ]) + + # Add multiple nodes by increasing both sync and quorum + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=1, sync=set('b'), numsync_confirmed=1, active=set('BCdE'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('sync', leader, 2, set('bC')), + ('quorum', leader, 3, set('bCdE')), + ('sync', leader, 2, set('bCdE')), + ]) + # Reduce quorum after added nodes caught up + self.check_state_transitions(leader=leader, quorum=3, voters=set('bcde'), + numsync=2, sync=set('bcde'), numsync_confirmed=3, active=set('bcde'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 2, set('bcde')), + ]) + + # Primary is alone + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=1, sync=set('b'), numsync_confirmed=0, active=set(), + sync_wanted=1, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set()), + ('sync', leader, 0, set()), + ]) + + # Swap out sync replica + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=1, sync=set('b'), numsync_confirmed=0, active=set('c'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set()), + ('sync', leader, 1, set('c')), + ('restart', leader, 0, set()), + ]) + # Update quorum when added node caught up + self.check_state_transitions(leader=leader, quorum=0, voters=set(), + numsync=1, sync=set('c'), numsync_confirmed=1, active=set('c'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set('c')), + ]) + + def test_1233(self): + """Interrupted transition from 2 node cluster to 3 node fully sync cluster""" + leader = 'a' + + # Node c went away, transition back to 2 node cluster + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=2, sync=set('bc'), numsync_confirmed=1, active=set('b'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('sync', leader, 1, set('b')), + ]) + + # Node c is available transition to larger quorum set, but not yet caught up. + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=2, sync=set('bc'), numsync_confirmed=1, active=set('bc'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 1, set('bc')), + ]) + + # Add in a new node at the same time, but node c didn't caught up yet + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=2, sync=set('bc'), numsync_confirmed=1, active=set('bcd'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 2, set('bcd')), + ('sync', leader, 2, set('bcd')), + ]) + # All sync nodes caught up, reduce quorum + self.check_state_transitions(leader=leader, quorum=2, voters=set('bcd'), + numsync=2, sync=set('bcd'), numsync_confirmed=3, active=set('bcd'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 1, set('bcd')), + ]) + + # Change replication factor at the same time + self.check_state_transitions(leader=leader, quorum=0, voters=set('b'), + numsync=2, sync=set('bc'), numsync_confirmed=1, active=set('bc'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('quorum', leader, 1, set('bc')), + ('sync', leader, 1, set('bc')), + ]) + + def test_2322(self): + """Interrupted transition from 2 node cluster to 3 node cluster with replication factor 2""" + leader = 'a' + + # Node c went away, transition back to 2 node cluster + self.check_state_transitions(leader=leader, quorum=1, voters=set('bc'), + numsync=1, sync=set('b'), numsync_confirmed=1, active=set('b'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set('b')), + ]) + + # Node c is available transition to larger quorum set. + self.check_state_transitions(leader=leader, quorum=1, voters=set('bc'), + numsync=1, sync=set('b'), numsync_confirmed=1, active=set('bc'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('sync', leader, 1, set('bc')), + ]) + + # Add in a new node at the same time + self.check_state_transitions(leader=leader, quorum=1, voters=set('bc'), + numsync=1, sync=set('b'), numsync_confirmed=1, active=set('bcd'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('sync', leader, 1, set('bc')), + ('quorum', leader, 2, set('bcd')), + ('sync', leader, 1, set('bcd')), + ]) + + # Convert to a fully synced cluster + self.check_state_transitions(leader=leader, quorum=1, voters=set('bc'), + numsync=1, sync=set('b'), numsync_confirmed=1, active=set('bc'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('sync', leader, 2, set('bc')), + ]) + # Reduce quorum after all nodes caught up + self.check_state_transitions(leader=leader, quorum=1, voters=set('bc'), + numsync=2, sync=set('bc'), numsync_confirmed=2, active=set('bc'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set('bc')), + ]) + + def test_3535(self): + leader = 'a' + + # remove nodes + self.check_state_transitions(leader=leader, quorum=2, voters=set('bcde'), + numsync=2, sync=set('bcde'), numsync_confirmed=2, active=set('bc'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('sync', leader, 2, set('bc')), + ('quorum', leader, 0, set('bc')), + ]) + self.check_state_transitions(leader=leader, quorum=2, voters=set('bcde'), + numsync=2, sync=set('bcde'), numsync_confirmed=3, active=set('bcd'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('sync', leader, 2, set('bcd')), + ('quorum', leader, 1, set('bcd')), + ]) + + # remove nodes and decrease sync + self.check_state_transitions(leader=leader, quorum=2, voters=set('bcde'), + numsync=2, sync=set('bcde'), numsync_confirmed=2, active=set('bc'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('sync', leader, 2, set('bc')), + ('quorum', leader, 1, set('bc')), + ('sync', leader, 1, set('bc')), + ]) + self.check_state_transitions(leader=leader, quorum=1, voters=set('bcde'), + numsync=3, sync=set('bcde'), numsync_confirmed=2, active=set('bc'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('sync', leader, 3, set('bcd')), + ('quorum', leader, 1, set('bc')), + ('sync', leader, 1, set('bc')), + ]) + + # Increase replication factor and decrease quorum + self.check_state_transitions(leader=leader, quorum=2, voters=set('bcde'), + numsync=2, sync=set('bcde'), numsync_confirmed=2, active=set('bcde'), + sync_wanted=3, leader_wanted=leader, expected=[ + ('sync', leader, 3, set('bcde')), + ]) + # decrease quorum after more nodes caught up + self.check_state_transitions(leader=leader, quorum=2, voters=set('bcde'), + numsync=3, sync=set('bcde'), numsync_confirmed=3, active=set('bcde'), + sync_wanted=3, leader_wanted=leader, expected=[ + ('quorum', leader, 1, set('bcde')), + ]) + + # Add node with decreasing sync and increasing quorum + self.check_state_transitions(leader=leader, quorum=2, voters=set('bcde'), + numsync=2, sync=set('bcde'), numsync_confirmed=2, active=set('bcdef'), + sync_wanted=1, leader_wanted=leader, expected=[ + # increase quorum by 2, 1 for added node and another for reduced sync + ('quorum', leader, 4, set('bcdef')), + # now reduce replication factor to requested value + ('sync', leader, 1, set('bcdef')), + ]) + + # Remove node with increasing sync and decreasing quorum + self.check_state_transitions(leader=leader, quorum=2, voters=set('bcde'), + numsync=2, sync=set('bcde'), numsync_confirmed=2, active=set('bcd'), + sync_wanted=3, leader_wanted=leader, expected=[ + # node e removed from sync wth replication factor increase + ('sync', leader, 3, set('bcd')), + # node e removed from voters with quorum decrease + ('quorum', leader, 1, set('bcd')), + ]) + + def test_remove_nosync_node(self): + leader = 'a' + self.check_state_transitions(leader=leader, quorum=0, voters=set('bc'), + numsync=2, sync=set('bc'), numsync_confirmed=1, active=set('b'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set('b')), + ('sync', leader, 1, set('b')) + ]) + + def test_swap_sync_node(self): + leader = 'a' + self.check_state_transitions(leader=leader, quorum=0, voters=set('bc'), + numsync=2, sync=set('bc'), numsync_confirmed=1, active=set('bd'), + sync_wanted=2, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set('b')), + ('sync', leader, 2, set('bd')), + ('quorum', leader, 1, set('bd')) + ]) + + def test_promotion(self): + # Beginning stat: 'a' in the primary, 1 of bcd in sync + # a fails, c gets quorum votes and promotes + self.check_state_transitions(leader='a', quorum=2, voters=set('bcd'), + numsync=0, sync=set(), numsync_confirmed=0, active=set(), + sync_wanted=1, leader_wanted='c', expected=[ + ('sync', 'a', 1, set('abd')), # set a and b to sync + ('quorum', 'c', 2, set('abd')), # set c as a leader and move a to voters + # and stop because there are no active nodes + ]) + + # next loop, b managed to reconnect + self.check_state_transitions(leader='c', quorum=2, voters=set('abd'), + numsync=1, sync=set('abd'), numsync_confirmed=0, active=set('b'), + sync_wanted=1, leader_wanted='c', expected=[ + ('sync', 'c', 1, set('b')), # remove a from sync as inactive + ('quorum', 'c', 0, set('b')), # remove a from voters and reduce quorum + ]) + + # alternative reality: next loop, no one reconnected + self.check_state_transitions(leader='c', quorum=2, voters=set('abd'), + numsync=1, sync=set('abd'), numsync_confirmed=0, active=set(), + sync_wanted=1, leader_wanted='c', expected=[ + ('quorum', 'c', 0, set()), + ('sync', 'c', 0, set()), + ]) + + def test_nonsync_promotion(self): + # Beginning state: 1 of bc in sync. e.g. (a primary, ssn = ANY 1 (b c)) + # a fails, d sees b and c, knows that it is in sync and decides to promote. + # We include in sync state former primary increasing replication factor + # and let situation resolve. Node d ssn=ANY 1 (b c) + leader = 'd' + self.check_state_transitions(leader='a', quorum=1, voters=set('bc'), + numsync=0, sync=set(), numsync_confirmed=0, active=set(), + sync_wanted=1, leader_wanted=leader, expected=[ + # Set a, b, and c to sync and increase replication factor + ('sync', 'a', 2, set('abc')), + # Set ourselves as the leader and move the old leader to voters + ('quorum', leader, 1, set('abc')), + # and stop because there are no active nodes + ]) + # next loop, b and c managed to reconnect + self.check_state_transitions(leader=leader, quorum=1, voters=set('abc'), + numsync=2, sync=set('abc'), numsync_confirmed=0, active=set('bc'), + sync_wanted=1, leader_wanted=leader, expected=[ + ('sync', leader, 2, set('bc')), # Remove a from being synced to. + ('quorum', leader, 1, set('bc')), # Remove a from quorum + ('sync', leader, 1, set('bc')), # Can now reduce replication factor back + ]) + # alternative reality: next loop, no one reconnected + self.check_state_transitions(leader=leader, quorum=1, voters=set('abc'), + numsync=2, sync=set('abc'), numsync_confirmed=0, active=set(), + sync_wanted=1, leader_wanted=leader, expected=[ + ('quorum', leader, 0, set()), + ('sync', leader, 0, set()), + ]) + + def test_invalid_states(self): + leader = 'a' + + # Main invariant is not satisfied, system is in an unsafe state + resolver = QuorumStateResolver(leader=leader, quorum=0, voters=set('bc'), + numsync=1, sync=set('bc'), numsync_confirmed=1, + active=set('bc'), sync_wanted=1, leader_wanted=leader) + self.assertRaises(QuorumError, resolver.check_invariants) + self.assertEqual(list(resolver), [ + ('quorum', leader, 1, set('bc')) + ]) + + # Quorum and sync states mismatched, somebody other than Patroni modified system state + resolver = QuorumStateResolver(leader=leader, quorum=1, voters=set('bc'), + numsync=2, sync=set('bd'), numsync_confirmed=1, + active=set('bd'), sync_wanted=1, leader_wanted=leader) + self.assertRaises(QuorumError, resolver.check_invariants) + self.assertEqual(list(resolver), [ + ('quorum', leader, 1, set('bd')), + ('sync', leader, 1, set('bd')), + ]) + self.assertTrue(repr(resolver.sync).startswith(' Date: Thu, 11 May 2023 12:18:33 +0200 Subject: [PATCH 05/37] update REST API - Don't return 200 for quorum nodes in GET `/sync` endpoint - Dew `GET /quorum` endpoint, returns 200 for quorum nodes - Show Quorum Standby role for quorum nodes in `patronictl list` --- docs/rest_api.rst | 11 ++++++++--- patroni/api.py | 20 ++++++++++++++------ patroni/utils.py | 5 +++-- tests/test_api.py | 3 +++ 4 files changed, 28 insertions(+), 11 deletions(-) diff --git a/docs/rest_api.rst b/docs/rest_api.rst index 7bde651f5..c7d7e68b2 100644 --- a/docs/rest_api.rst +++ b/docs/rest_api.rst @@ -45,6 +45,8 @@ For all health check ``GET`` requests Patroni returns a JSON document with the s - ``GET /read-only-sync``: like the above endpoint, but also includes the primary. +- ``GET /quorum``: returns HTTP status code **200** only when this Patroni node is listed as a quorum node in ``synchronous_standby_names`` on the primary. + - ``GET /asynchronous`` or ``GET /async``: returns HTTP status code **200** only when the Patroni node is running as an asynchronous standby. @@ -140,9 +142,12 @@ Retrieve the Patroni metrics in Prometheus format through the ``GET /metrics`` e # HELP patroni_replica Value is 1 if this node is a replica, 0 otherwise. # TYPE patroni_replica gauge patroni_replica{scope="batman"} 0 - # HELP patroni_sync_standby Value is 1 if this node is a sync standby replica, 0 otherwise. - # TYPE patroni_sync_standby gauge - patroni_sync_standby{scope="batman"} 0 + # HELP patroni_sync_standby Value is 1 if this node is a sync standby, 0 otherwise. + # TYPE patroni_sync_standby gauge + patroni_sync_standby{scope="batman"} 0 + # HELP patroni_quorum_standby Value is 1 if this node is a quorum standby, 0 otherwise. + # TYPE patroni_quorum_standby gauge + patroni_quorum_standby{scope="batman"} 0 # HELP patroni_xlog_received_location Current location of the received Postgres transaction log, 0 if this node is not a replica. # TYPE patroni_xlog_received_location counter patroni_xlog_received_location{scope="batman"} 0 diff --git a/patroni/api.py b/patroni/api.py index 45b50a4d3..350f5acfa 100644 --- a/patroni/api.py +++ b/patroni/api.py @@ -150,10 +150,13 @@ def do_GET(self, write_status_code_only: Optional[bool] = False) -> None: elif 'health' in path: status_code = 200 if response.get('state') == 'running' else 503 elif cluster: # dcs is available + is_quorum = response.get('quorum_standby') is_synchronous = response.get('sync_standby') if path in ('/sync', '/synchronous') and is_synchronous: status_code = replica_status_code - elif path in ('/async', '/asynchronous') and not is_synchronous: + elif path == '/quorum' and is_quorum: + status_code = replica_status_code + elif path in ('/async', '/asynchronous') and not is_synchronous and not is_quorum: status_code = replica_status_code elif path in ('/read-only-sync', '/read-only-synchronous'): if 200 in (primary_status_code, standby_leader_status_code): @@ -279,10 +282,14 @@ def do_GET_metrics(self) -> None: metrics.append("# TYPE patroni_replica gauge") metrics.append("patroni_replica{0} {1}".format(scope_label, int(postgres['role'] == 'replica'))) - metrics.append("# HELP patroni_sync_standby Value is 1 if this node is a sync standby replica, 0 otherwise.") + metrics.append("# HELP patroni_sync_standby Value is 1 if this node is a sync standby, 0 otherwise.") metrics.append("# TYPE patroni_sync_standby gauge") metrics.append("patroni_sync_standby{0} {1}".format(scope_label, int(postgres.get('sync_standby', False)))) + metrics.append("# HELP patroni_quorum_standby Value is 1 if this node is a quorum standby, 0 otherwise.") + metrics.append("# TYPE patroni_quorum_standby gauge") + metrics.append("patroni_quorum_standby{0} {1}".format(scope_label, int(postgres.get('quorum_standby', False)))) + metrics.append("# HELP patroni_xlog_received_location Current location of the received" " Postgres transaction log, 0 if this node is not a replica.") metrics.append("# TYPE patroni_xlog_received_location counter") @@ -558,16 +565,17 @@ def is_failover_possible(self, cluster: Cluster, leader: str, candidate: str, ac :returns: a string with the error message or `None` if good nodes are found """ - is_synchronous_mode = self.server.patroni.config.get_global_config(cluster).is_synchronous_mode + global_config = self.server.patroni.config.get_global_config(cluster) if leader and (not cluster.leader or cluster.leader.name != leader): return 'leader name does not match' if candidate: - if action == 'switchover' and is_synchronous_mode and not cluster.sync.matches(candidate): + if action == 'switchover' and global_config.is_synchronous_mode\ + and not global_config.is_quorum_commit_mode and not cluster.sync.matches(candidate): return 'candidate name does not match with sync_standby' members = [m for m in cluster.members if m.name == candidate] if not members: return 'candidate does not exists' - elif is_synchronous_mode: + elif global_config.is_synchronous_mode and not global_config.is_quorum_commit_mode: members = [m for m in cluster.members if cluster.sync.matches(m.name)] if not members: return action + ' is not possible: can not find sync_standby' @@ -716,7 +724,7 @@ def get_postgresql_status(self, retry: Optional[bool] = False) -> Dict[str, Any] if result['role'] == 'replica' and global_config.is_synchronous_mode\ and cluster.sync.matches(postgresql.name): - result['sync_standby'] = True + result['quorum_standby' if global_config.is_quorum_commit_mode else 'sync_standby'] = True if row[1] > 0: result['timeline'] = row[1] diff --git a/patroni/utils.py b/patroni/utils.py index f6cfa9191..b6abae0e3 100644 --- a/patroni/utils.py +++ b/patroni/utils.py @@ -735,11 +735,12 @@ def cluster_as_json(cluster: 'Cluster', global_config: Optional['GlobalConfig'] cluster_lsn = cluster.last_lsn or 0 ret: Dict[str, Any] = {'members': []} + sync_role = 'quorum_standby' if global_config.is_quorum_commit_mode else 'sync_standby' for m in cluster.members: if m.name == leader_name: role = 'standby_leader' if global_config.is_standby_cluster else 'leader' - elif cluster.sync.matches(m.name): - role = 'sync_standby' + elif cluster.sync.matches(m.name, global_config.is_quorum_commit_mode): + role = sync_role else: role = 'replica' diff --git a/tests/test_api.py b/tests/test_api.py index 2e947bb38..d4e9c8344 100644 --- a/tests/test_api.py +++ b/tests/test_api.py @@ -204,6 +204,9 @@ def test_do_GET(self): Mock(return_value={'role': 'replica', 'sync_standby': True})): MockRestApiServer(RestApiHandler, 'GET /synchronous') MockRestApiServer(RestApiHandler, 'GET /read-only-sync') + with patch.object(RestApiHandler, 'get_postgresql_status', + Mock(return_value={'role': 'replica', 'quorum_standby': True})): + MockRestApiServer(RestApiHandler, 'GET /quorum') with patch.object(RestApiHandler, 'get_postgresql_status', Mock(return_value={'role': 'replica'})): MockRestApiServer(RestApiHandler, 'GET /asynchronous') with patch.object(MockHa, 'is_leader', Mock(return_value=True)): From 7284416ee39d82c6fe7ba32977ec551fa7bfe42e Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 11 May 2023 12:19:25 +0200 Subject: [PATCH 06/37] Behave tests --- features/quorum_commit.feature | 68 +++++++++++++++++++++++++++++++++ features/steps/quorum_commit.py | 60 +++++++++++++++++++++++++++++ 2 files changed, 128 insertions(+) create mode 100644 features/quorum_commit.feature create mode 100644 features/steps/quorum_commit.py diff --git a/features/quorum_commit.feature b/features/quorum_commit.feature new file mode 100644 index 000000000..2204ab596 --- /dev/null +++ b/features/quorum_commit.feature @@ -0,0 +1,68 @@ +Feature: quorum commit + Check basic workfrlows when quorum commit is enabled + + Scenario: check enable quorum commit and that the only leader promotes after restart + Given I start postgres0 + Then postgres0 is a leader after 10 seconds + And there is a non empty initialize key in DCS after 15 seconds + When I issue a PATCH request to http://127.0.0.1:8008/config with {"ttl": 20, "synchronous_mode": "quorum"} + Then I receive a response code 200 + And sync key in DCS has leader=postgres0 after 20 seconds + And sync key in DCS has quorum=0 after 2 seconds + And synchronous_standby_names on postgres0 is set to "_empty_str_" after 2 seconds + When I shut down postgres0 + And sync key in DCS has leader=postgres0 after 2 seconds + When I start postgres0 + Then postgres0 role is the primary after 10 seconds + When I issue a PATCH request to http://127.0.0.1:8008/config with {"synchronous_mode_strict": true} + Then synchronous_standby_names on postgres0 is set to "ANY 1 (*)" after 10 seconds + + Scenario: check failover with one quorum standby + Given I start postgres1 + Then sync key in DCS has sync_standby=postgres1 after 10 seconds + And synchronous_standby_names on postgres0 is set to "ANY 1 (postgres1)" after 2 seconds + When I shut down postgres0 + Then postgres1 role is the primary after 10 seconds + And sync key in DCS has quorum=0 after 10 seconds + Then synchronous_standby_names on postgres1 is set to "ANY 1 (*)" after 10 seconds + When I start postgres0 + Then sync key in DCS has leader=postgres1 after 10 seconds + Then sync key in DCS has sync_standby=postgres0 after 10 seconds + And synchronous_standby_names on postgres1 is set to "ANY 1 (postgres0)" after 2 seconds + + Scenario: check behavior with three nodes and different replication factor + Given I start postgres2 + Then sync key in DCS has sync_standby=postgres0,postgres2 after 10 seconds + And sync key in DCS has quorum=1 after 2 seconds + And synchronous_standby_names on postgres1 is set to "ANY 1 (postgres0,postgres2)" after 2 seconds + When I issue a PATCH request to http://127.0.0.1:8009/config with {"synchronous_node_count": 2} + Then sync key in DCS has quorum=0 after 10 seconds + And synchronous_standby_names on postgres1 is set to "ANY 2 (postgres0,postgres2)" after 2 seconds + + Scenario: switch from quorum replication to good old multisync and back + Given I issue a PATCH request to http://127.0.0.1:8009/config with {"synchronous_mode": true, "synchronous_node_count": 1} + And I shut down postgres0 + Then synchronous_standby_names on postgres1 is set to "postgres2" after 10 seconds + And sync key in DCS has sync_standby=postgres2 after 10 seconds + Then sync key in DCS has quorum=0 after 2 seconds + When I issue a PATCH request to http://127.0.0.1:8009/config with {"synchronous_mode": "quorum"} + And I start postgres0 + Then synchronous_standby_names on postgres1 is set to "ANY 1 (postgres0,postgres2)" after 10 seconds + And sync key in DCS has sync_standby=postgres0,postgres2 after 10 seconds + Then sync key in DCS has quorum=1 after 2 seconds + + Scenario: REST API and patronictl + Given I run patronictl.py list batman + Then I receive a response returncode 0 + And I receive a response output "Quorum Standby" + And Status code on GET http://127.0.0.1:8008/quorum is 200 after 3 seconds + And Status code on GET http://127.0.0.1:8010/quorum is 200 after 3 seconds + + Scenario: nosync node is removed from voters and synchronous_standby_names + Given I add tag nosync true to postgres2 config + When I issue an empty POST request to http://127.0.0.1:8010/reload + Then I receive a response code 202 + And sync key in DCS has quorum=0 after 10 seconds + And sync key in DCS has sync_standby=postgres0 after 10 seconds + And synchronous_standby_names on postgres1 is set to "ANY 1 (postgres0)" after 2 seconds + And Status code on GET http://127.0.0.1:8010/quorum is 503 after 10 seconds diff --git a/features/steps/quorum_commit.py b/features/steps/quorum_commit.py new file mode 100644 index 000000000..7cedc0d5b --- /dev/null +++ b/features/steps/quorum_commit.py @@ -0,0 +1,60 @@ +import json +import re +import time + +from behave import step, then + + +@step('sync key in DCS has {key:w}={value} after {time_limit:d} seconds') +def check_sync(context, key, value, time_limit): + time_limit *= context.timeout_multiplier + max_time = time.time() + int(time_limit) + dcs_value = None + while time.time() < max_time: + try: + response = json.loads(context.dcs_ctl.query('sync')) + dcs_value = response.get(key) + if key == 'sync_standby' and set((dcs_value or '').split(',')) == set(value.split(',')): + return + elif str(dcs_value) == value: + return + except Exception: + pass + time.sleep(1) + assert False, "sync does not have {0}={1} (found {2}) in dcs after {3} seconds".format(key, value, + dcs_value, time_limit) + + +@then('synchronous_standby_names on {name:2} is set to "{value}" after {time_limit:d} seconds') +def check_synchronous_standby_names(context, name, value, time_limit): + time_limit *= context.timeout_multiplier + max_time = time.time() + int(time_limit) + + if value == '_empty_str_': + value = '' + + if '(' in value: + m = re.match(r'.*(\d+) \(([^)]+)\)', value) + expected_value = set(m.group(2).split()) + expected_num = m.group(1) + else: + expected_value = set([value]) + expected_num = '1' + + while time.time() < max_time: + try: + ssn = context.pctl.query(name, "SHOW synchronous_standby_names").fetchone()[0] + if '(' in ssn: + m = re.match(r'.*(\d+) \(([^)]+)\)', ssn) + db_value = set(m.group(2).split()) + db_num = m.group(1) + else: + db_value = set([ssn]) + db_num = '1' + if expected_value == db_value and expected_num == db_num: + return + except Exception: + pass + time.sleep(1) + assert False, "synchronous_standby_names is not set to '{0}' (found '{1}') after {2} seconds".format(value, ssn, + time_limit) From dbfe844a6fbc792682dc2b090f9dc9f490e71775 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 11 May 2023 12:19:50 +0200 Subject: [PATCH 07/37] Update documentation --- docs/dynamic_configuration.rst | 4 +-- docs/replication_modes.rst | 48 ++++++++++++++++++++++++++++------ 2 files changed, 42 insertions(+), 10 deletions(-) diff --git a/docs/dynamic_configuration.rst b/docs/dynamic_configuration.rst index 85121d7cf..a10fd606e 100644 --- a/docs/dynamic_configuration.rst +++ b/docs/dynamic_configuration.rst @@ -16,7 +16,7 @@ In order to change the dynamic configuration you can use either ``patronictl edi - **max\_timelines\_history**: maximum number of timeline history items kept in DCS. Default value: 0. When set to 0, it keeps the full history in DCS. - **primary\_start\_timeout**: the amount of time a primary is allowed to recover from failures before failover is triggered (in seconds). Default is 300 seconds. When set to 0 failover is done immediately after a crash is detected if possible. When using asynchronous replication a failover can cause lost transactions. Worst case failover time for primary failure is: loop\_wait + primary\_start\_timeout + loop\_wait, unless primary\_start\_timeout is zero, in which case it's just loop\_wait. Set the value according to your durability/availability tradeoff. - **primary\_stop\_timeout**: The number of seconds Patroni is allowed to wait when stopping Postgres and effective only when synchronous_mode is enabled. When set to > 0 and the synchronous_mode is enabled, Patroni sends SIGKILL to the postmaster if the stop operation is running for more than the value set by primary\_stop\_timeout. Set the value according to your durability/availability tradeoff. If the parameter is not set or set <= 0, primary\_stop\_timeout does not apply. -- **synchronous\_mode**: turns on synchronous replication mode. In this mode a replica will be chosen as synchronous and only the latest leader and synchronous replica are able to participate in leader election. Synchronous mode makes sure that successfully committed transactions will not be lost at failover, at the cost of losing availability for writes when Patroni cannot ensure transaction durability. See :ref:`replication modes documentation ` for details. +- **synchronous\_mode**: turns on synchronous replication mode. Possible values: ``off``, ``on``, ``quorum``. In this mode the leader takes care about management of ``synchronous_standby_names`` and and only the last known leader or one of synchronous replicas are allowed to participate in leader race. Synchronous mode makes sure that successfully committed transactions will not be lost at failover, at the cost of losing availability for writes when Patroni cannot ensure transaction durability. See :ref:`replication modes documentation ` for details. - **synchronous\_mode\_strict**: prevents disabling synchronous replication if no synchronous replicas are available, blocking all client writes to the primary. See :ref:`replication modes documentation ` for details. - **failsafe\_mode**: Enables :ref:`DCS Failsafe Mode `. Defaults to `false`. - **postgresql**: @@ -70,4 +70,4 @@ Note: **slots** is a hashmap while **ignore_slots** is an array. For example: plugin: test_decoding - name: ignored_physical_slot_name type: physical - ... \ No newline at end of file + ... diff --git a/docs/replication_modes.rst b/docs/replication_modes.rst index 0ba5f4b16..6ab0d291f 100644 --- a/docs/replication_modes.rst +++ b/docs/replication_modes.rst @@ -6,8 +6,9 @@ Replication modes Patroni uses PostgreSQL streaming replication. For more information about streaming replication, see the `Postgres documentation `__. By default Patroni configures PostgreSQL for asynchronous replication. Choosing your replication schema is dependent on your business considerations. Investigate both async and sync replication, as well as other HA solutions, to determine which solution is best for you. + Asynchronous mode durability ----------------------------- +============================ In asynchronous mode the cluster is allowed to lose some committed transactions to ensure availability. When the primary server fails or becomes unavailable for any other reason Patroni will automatically promote a sufficiently healthy standby to primary. Any transactions that have not been replicated to that standby remain in a "forked timeline" on the primary, and are effectively unrecoverable [1]_. @@ -15,10 +16,11 @@ The amount of transactions that can be lost is controlled via ``maximum_lag_on_f By default, when running leader elections, Patroni does not take into account the current timeline of replicas, what in some cases could be undesirable behavior. You can prevent the node not having the same timeline as a former primary become the new leader by changing the value of ``check_timeline`` parameter to ``true``. + PostgreSQL synchronous replication ----------------------------------- +================================== -You can use Postgres's `synchronous replication `__ with Patroni. Synchronous replication ensures consistency across a cluster by confirming that writes are written to a secondary before returning to the connecting client with a success. The cost of synchronous replication: reduced throughput on writes. This throughput will be entirely based on network performance. +You can use Postgres's `synchronous replication `__ with Patroni. Synchronous replication ensures consistency across a cluster by confirming that writes are written to a secondary before returning to the connecting client with a success. The cost of synchronous replication: increased latency and reduced throughput on writes. This throughput will be entirely based on network performance. In hosted datacenter environments (like AWS, Rackspace, or any network you do not control), synchronous replication significantly increases the variability of write performance. If followers become inaccessible from the leader, the leader effectively becomes read-only. @@ -33,10 +35,11 @@ When using PostgreSQL synchronous replication, use at least three Postgres data Using PostgreSQL synchronous replication does not guarantee zero lost transactions under all circumstances. When the primary and the secondary that is currently acting as a synchronous replica fail simultaneously a third node that might not contain all transactions will be promoted. + .. _synchronous_mode: Synchronous mode ----------------- +================ For use cases where losing committed transactions is not permissible you can turn on Patroni's ``synchronous_mode``. When ``synchronous_mode`` is turned on Patroni will not promote a standby unless it is certain that the standby contains all transactions that may have returned a successful commit status to client [2]_. This means that the system may be unavailable for writes even though some servers are available. System administrators can still use manual failover commands to promote a standby even if it results in transaction loss. @@ -55,16 +58,27 @@ up. You can ensure that a standby never becomes the synchronous standby by setting ``nosync`` tag to true. This is recommended to set for standbys that are behind slow network connections and would cause performance degradation when becoming a synchronous standby. -Synchronous mode can be switched on and off via Patroni REST interface. See :ref:`dynamic configuration ` for instructions. +Synchronous mode can be switched on and off using ``patronictl edit-config`` command or via Patroni REST interface. See :ref:`dynamic configuration ` for instructions. Note: Because of the way synchronous replication is implemented in PostgreSQL it is still possible to lose transactions even when using ``synchronous_mode_strict``. If the PostgreSQL backend is cancelled while waiting to acknowledge replication (as a result of packet cancellation due to client timeout or backend failure) transaction changes become visible for other backends. Such changes are not yet replicated and may be lost in case of standby promotion. + Synchronous Replication Factor ------------------------------- -The parameter ``synchronous_node_count`` is used by Patroni to manage number of synchronous standby databases. It is set to 1 by default. It has no effect when ``synchronous_mode`` is set to off. When enabled, Patroni manages precise number of synchronous standby databases based on parameter ``synchronous_node_count`` and adjusts the state in DCS & synchronous_standby_names as members join and leave. +============================== + +The parameter ``synchronous_node_count`` is used by Patroni to manage number of synchronous standby databases. It is set to 1 by default. It has no effect when ``synchronous_mode`` is set to off. When enabled, Patroni manages precise number of synchronous standby databases based on parameter ``synchronous_node_count`` and adjusts the state in DCS & synchronous_standby_names as members join and leave. If the parameter is set to the value higher than the number of eligible nodes it will be automatically reduced by Patroni down to 1. + + +Maximum lag on synchronous node +=============================== + +By default Patroni sticks to a node that is declared as ``synchronous`` according to the ``pg_stat_replication`` even when there are other nodes ahead of it. It is done to minimize the number of changes of ``synchronous_standby_names``. To change this behavior one may use ``maximum_lag_on_syncnode`` parameter. It controls how much the replica can lag in to be allowed chosen as synchronous. + +Patroni utilizes the max replica LSN if there is more than one standby, otherwise it will use leader's current wal LSN. Default is ``-1``, and Patroni will not take action to swap synchronous unhealthy standby when the value is set to 0 or below. Please set the value high enough so Patroni won't swap synchrounous standbys fequently during high transaction volume. + Synchronous mode implementation -------------------------------- +=============================== When in synchronous mode Patroni maintains synchronization state in the DCS, containing the latest primary and current synchronous standby databases. This state is updated with strict ordering constraints to ensure the following invariants: @@ -79,6 +93,24 @@ Patroni will only assign one or more synchronous standby nodes based on ``synchr On each HA loop iteration Patroni re-evaluates synchronous standby nodes choice. If the current list of synchronous standby nodes are connected and has not requested its synchronous status to be removed it remains picked. Otherwise the cluster member available for sync that is furthest ahead in replication is picked. +.. _quorum_mode: + +Quorum commit mode +================== + +Starting from PostgreSQL v10 Patroni supports quorum-based synchronous replication. + +In this mode Patroni maintains synchronization state in the DCS, containing the latest known primary, number of nodes required for quorum and nodes currently eligible to vote on quorum. In steady state the nodes voting on quorum are the leader and all synchronous standbys. This state is updated with strict ordering constraints with regards to node promotion and ``synchronous_standby_names`` to ensure that at all times any subset of voters that can achieve quorum is contained to have at least one node having the latest successful commit. + +On each iteration of HA loop Patroni re-evaluates synchronous standby choices and quorum based on node availability and requested cluster configuration. In PostgreSQL versions above 9.6 all eligible nodes are added as synchronous standbys as soon as their replication catches up to leader. + +Quorum commit helps to reduce worst case latencies even during normal operation as a higher latency of replicating to one standby can be compensated by other standbys. + +The quorum-based synchronous mode could be enabled by setting ``synchronous_mode`` to ``quorum`` using ``patronictl edit-config`` command or via Patroni REST interface. See :ref:`dynamic configuration ` for instructions. + +Other parameters, like ``synchronous_node_count``, ``maximum_lag_on_syncnode``, and ``synchronous_mode_strict`` continue to work the same way as with ``synchronous_mode=on``. + + .. [1] The data is still there, but recovering it requires a manual recovery effort by data recovery specialists. When Patroni is allowed to rewind with ``use_pg_rewind`` the forked timeline will be automatically erased to rejoin the failed primary with the cluster. .. [2] Clients can change the behavior per transaction using PostgreSQL's ``synchronous_commit`` setting. Transactions with ``synchronous_commit`` values of ``off`` and ``local`` may be lost on fail over, but will not be blocked by replication delays. From 8f60b18f03a210a0dee16496514f5a01f9c19a3f Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Tue, 23 May 2023 14:01:58 +0200 Subject: [PATCH 08/37] Delay _process_quorum_replication by loop_wait seconds after promote It takes some time for existing standbys to start streaming from the new primary and we want to do our best to not empty the /sync key before that. --- patroni/ha.py | 14 +++++++++++++- tests/test_ha.py | 1 + 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/patroni/ha.py b/patroni/ha.py index 147524f70..2b77638a6 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -147,6 +147,7 @@ def __init__(self, patroni: Patroni): self._is_leader_lock = RLock() self._failsafe = Failsafe(patroni.dcs) self._was_paused = False + self._promote_time = 0 self._leader_timeline = None self.recovering = False self._async_response = CriticalTask() @@ -196,6 +197,8 @@ def is_leader(self) -> bool: def set_is_leader(self, value: bool) -> None: with self._is_leader_lock: self._is_leader = time.time() + self.dcs.ttl if value else 0 + if not value: + self._promote_time = 0 def load_cluster_from_dcs(self) -> None: cluster = self.dcs.get_cluster() @@ -702,7 +705,16 @@ def _process_multisync_replication(self) -> None: def process_sync_replication(self) -> None: """Process synchronous replication beahvior on the primary.""" if self.is_quorum_commit_mode(): - self._process_quorum_replication() + # The synchronous_standby_names was adjusted right before promote. + # After that, when postgres has become a primary, we need to reflect this change + # in the /sync key. Further changes of synchronous_standby_names and /sync key should + # be postponed for `loop_wait` seconds, to give a chance to some replicas to start streaming. + # In opposite case the /sync key will end up without synchronous nodes. + if self.state_handler.is_leader(): + if self._promote_time == 0 or time.time() - self._promote_time > self.dcs.loop_wait: + self._process_quorum_replication() + if self._promote_time == 0: + self._promote_time = time.time() elif self.is_synchronous_mode(): self._process_multisync_replication() else: diff --git a/tests/test_ha.py b/tests/test_ha.py index 6a3d97ab9..391282754 100644 --- a/tests/test_ha.py +++ b/tests/test_ha.py @@ -1500,6 +1500,7 @@ def test_process_quorum_replication(self): self.assertEqual(mock_write_sync.call_args_list[0][1], {'version': None}) self.assertEqual(mock_set_sync.call_count, 0) + self.ha._promote_time = 1 mock_write_sync = self.ha.dcs.write_sync_state = Mock(side_effect=[SyncState.empty(), None]) # Test /sync key is attempted to set and succeed when missing or invalid with patch.object(SyncState, 'is_empty', Mock(side_effect=[True, False])): From a5e1c53e98bc3f11a403406c95f0f12ecd00868b Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Tue, 23 May 2023 15:42:56 +0200 Subject: [PATCH 09/37] Fix citus tests. Metadata sync could be slow after coordinator switch --- features/citus.feature | 16 ++++++++-------- features/steps/citus.py | 22 +++++++++++++++++----- 2 files changed, 25 insertions(+), 13 deletions(-) diff --git a/features/citus.feature b/features/citus.feature index 2cf295ebf..e2e51626c 100644 --- a/features/citus.feature +++ b/features/citus.feature @@ -10,20 +10,20 @@ Feature: citus And I start postgres3 in citus group 1 Then replication works from postgres0 to postgres1 after 15 seconds Then replication works from postgres2 to postgres3 after 15 seconds - And postgres0 is registered in the postgres0 as the worker in group 0 - And postgres2 is registered in the postgres0 as the worker in group 1 + And postgres0 is registered in the postgres0 as the worker in group 0 after 5 seconds + And postgres2 is registered in the postgres0 as the worker in group 1 after 5 seconds Scenario: coordinator failover updates pg_dist_node Given I run patronictl.py failover batman --group 0 --candidate postgres1 --force Then postgres1 role is the primary after 10 seconds And replication works from postgres1 to postgres0 after 15 seconds + And postgres1 is registered in the postgres2 as the worker in group 0 after 5 seconds And "sync" key in a group 0 in DCS has sync_standby=postgres0 after 15 seconds - And postgres1 is registered in the postgres2 as the worker in group 0 When I run patronictl.py failover batman --group 0 --candidate postgres0 --force Then postgres0 role is the primary after 10 seconds And replication works from postgres0 to postgres1 after 15 seconds + And postgres0 is registered in the postgres2 as the worker in group 0 after 5 seconds And "sync" key in a group 0 in DCS has sync_standby=postgres1 after 15 seconds - And postgres0 is registered in the postgres2 as the worker in group 0 Scenario: worker switchover doesn't break client queries on the coordinator Given I create a distributed table on postgres0 @@ -32,15 +32,15 @@ Feature: citus Then I receive a response returncode 0 And postgres3 role is the primary after 10 seconds And replication works from postgres3 to postgres2 after 15 seconds + And postgres3 is registered in the postgres0 as the worker in group 1 after 5 seconds And "sync" key in a group 1 in DCS has sync_standby=postgres2 after 15 seconds - And postgres3 is registered in the postgres0 as the worker in group 1 And a thread is still alive When I run patronictl.py switchover batman --group 1 --force Then I receive a response returncode 0 And postgres2 role is the primary after 10 seconds And replication works from postgres2 to postgres3 after 15 seconds + And postgres2 is registered in the postgres0 as the worker in group 1 after 5 seconds And "sync" key in a group 1 in DCS has sync_standby=postgres3 after 15 seconds - And postgres2 is registered in the postgres0 as the worker in group 1 And a thread is still alive When I stop a thread Then a distributed table on postgres0 has expected rows @@ -52,7 +52,7 @@ Feature: citus Then I receive a response returncode 0 And postgres2 role is the primary after 10 seconds And replication works from postgres2 to postgres3 after 15 seconds - And postgres2 is registered in the postgres0 as the worker in group 1 + And postgres2 is registered in the postgres0 as the worker in group 1 after 5 seconds And a thread is still alive When I stop a thread Then a distributed table on postgres0 has expected rows @@ -65,7 +65,7 @@ Feature: citus Then I receive a response returncode 0 And I receive a response output "+ttl: 20" When I sleep for 2 seconds - Then postgres4 is registered in the postgres2 as the worker in group 2 + Then postgres4 is registered in the postgres2 as the worker in group 2 after 5 seconds When I shut down postgres4 Then There is a transaction in progress on postgres0 changing pg_dist_node When I run patronictl.py restart batman postgres2 --group 1 --force diff --git a/features/steps/citus.py b/features/steps/citus.py index d645a504f..09274fc23 100644 --- a/features/steps/citus.py +++ b/features/steps/citus.py @@ -44,12 +44,24 @@ def start_citus(context, name, group): return context.pctl.start(name, custom_config={"citus": {"database": "postgres", "group": int(group)}}) -@step('{name1:w} is registered in the {name2:w} as the worker in group {group:d}') -def check_registration(context, name1, name2, group): +@step('{name1:w} is registered in the {name2:w} as the worker in group {group:d} after {time_limit:d} seconds') +def check_registration(context, name1, name2, group, time_limit): + time_limit *= context.timeout_multiplier + max_time = time.time() + int(time_limit) + pg_dist_value = None worker_port = int(context.pctl.query(name1, "SHOW port").fetchone()[0]) - r = context.pctl.query(name2, "SELECT nodeport FROM pg_catalog.pg_dist_node WHERE groupid = {0}".format(group)) - assert worker_port == r.fetchone()[0],\ - "Worker {0} is not registered in pg_dist_node on the coordinator {1}".format(name1, name2) + while time.time() < max_time: + try: + r = context.pctl.query(name2, + "SELECT nodeport FROM pg_catalog.pg_dist_node WHERE groupid = {0}".format(group)) + pg_dist_value = r.fetchone()[0] + if pg_dist_value == worker_port: + return + except Exception: + pass + time.sleep(1) + assert False, "Worker {0} is not registered in pg_dist_node on the coordinator {1} after {2} seconds"\ + .format(name1, name2, time_limit) @step('I create a distributed table on {name:w}') From e6d251bda060064e959c311a20527c6cd82fd23e Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Tue, 18 Jul 2023 14:16:28 +0200 Subject: [PATCH 10/37] Limit time spent in _process_quorum_replication by loop_wait seconds --- patroni/ha.py | 12 ++++++++++-- tests/test_ha.py | 4 ++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/patroni/ha.py b/patroni/ha.py index 24f8d7f4e..dccd68f2d 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -642,6 +642,8 @@ def _process_quorum_replication(self) -> None: In case any of those steps causes an error we can just bail out and let next iteration rediscover the state and retry necessary transitions. """ + start_time = time.time() + min_sync = self.global_config.min_synchronous_nodes sync_wanted = self.global_config.synchronous_node_count @@ -652,6 +654,9 @@ def _process_quorum_replication(self) -> None: if not sync: return logger.warning("Updating sync state failed") + def _check_timeout(offset: float = 0) -> bool: + return time.time() - start_time + offset >= self.dcs.loop_wait + while True: transition = 'break' # we need define transition value if `QuorumStateResolver` produced no changes sync_state = self.state_handler.sync_handler.current_state(self.cluster) @@ -664,6 +669,9 @@ def _process_quorum_replication(self) -> None: active=sync_state.active, sync_wanted=sync_wanted, leader_wanted=self.state_handler.name): + if _check_timeout(): + return + if transition == 'quorum': logger.info("Setting leader to %s, quorum to %d of %d (%s)", leader, num, len(nodes), ", ".join(sorted(nodes))) @@ -680,8 +688,8 @@ def _process_quorum_replication(self) -> None: " Commits will be delayed.", min_sync + 1, num) num = min_sync self.state_handler.sync_handler.set_synchronous_standby_names(nodes, num) - if transition != 'restart': - break + if transition != 'restart' or _check_timeout(1): + return # synchronous_standby_names was transitioned from empty to non-empty and it may take # some time for nodes to become synchronous. In this case we want to restart state machine # hoping that we can update /sync key earlier than in loop_wait seconds. diff --git a/tests/test_ha.py b/tests/test_ha.py index 473a6db99..116cc5af1 100644 --- a/tests/test_ha.py +++ b/tests/test_ha.py @@ -1555,3 +1555,7 @@ def test_process_quorum_replication(self): self.assertEqual(mock_write_sync.call_args_list[0][1], {'version': 0}) self.assertEqual(mock_set_sync.call_count, 1) self.assertEqual(mock_set_sync.call_args_list[0][0], ('ANY 1 (*)',)) + + # Test that _process_quorum_replication doesn't take longer than loop_wait + with patch('time.time', Mock(side_effect=[30, 60, 90, 120])): + self.ha.process_sync_replication() From 300740c9199f60fc8d36ca022e54bf8e2984b039 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Tue, 18 Jul 2023 15:35:32 +0200 Subject: [PATCH 11/37] Please codacy --- patroni/postgresql/sync.py | 83 ++++++++++++++++++++++---------------- patroni/quorum.py | 77 ++++++++++++++++++++--------------- 2 files changed, 92 insertions(+), 68 deletions(-) diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index df6801b7d..1ceab579f 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -3,7 +3,7 @@ import time from copy import deepcopy -from typing import Collection, List, NamedTuple, Optional, Tuple, TYPE_CHECKING +from typing import Collection, Iterator, List, NamedTuple, Optional, Tuple, TYPE_CHECKING from ..collections import CaseInsensitiveDict, CaseInsensitiveSet from ..dcs import Cluster @@ -137,7 +137,7 @@ def parse_sync_standby_names(value: str) -> _SSN: if len(synclist) == i + 1: # except the last token raise ValueError("Unparseable synchronous_standby_names value %r: Unexpected token %s %r at %d" % (value, a_type, a_value, a_pos)) - elif a_type != 'comma': + if a_type != 'comma': raise ValueError("Unparseable synchronous_standby_names value %r: ""Got token %s %r while" " expecting comma at %d" % (value, a_type, a_value, a_pos)) elif a_type in {'ident', 'first', 'any'}: @@ -209,36 +209,20 @@ def _handle_synchronous_standby_names_change(self) -> None: END;$$""") self._postgresql.reset_cluster_info_state(None) # Reset internal cache to query fresh values - def current_state(self, cluster: Cluster) -> _SyncState: - """Finds best candidates to be the synchronous standbys. - - Current synchronous standby is always preferred, unless it has disconnected or does not want to be a - synchronous standby any longer. - - Standbys are selected based on values from the global configuration: - - `maximum_lag_on_syncnode`: would help swapping unhealthy sync replica in case if it stops - responding (or hung). Please set the value high enough so it won't unncessarily swap sync - standbys during high loads. Any value less or equal of 0 keeps the behavior backward compatible. - Please note that it will not also swap sync standbys in case where all replicas are hung. - - `synchronous_node_count`: controlls how many nodes should be set as synchronous. - - :param cluster: current cluster topology from DCS - :returns: current synchronous replication state as a :class:`_SyncState` object - """ - self._handle_synchronous_standby_names_change() + def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, int, bool]]: + """Yields candidates based on higher replay/remote_write/flush lsn.""" - # Pick candidates based on who has higher replay/remote_write/flush lsn. + # What column from pg_stat_replication we want to sort on? Choose based on ``synchronous_commit`` value. sort_col = { 'remote_apply': 'replay', 'remote_write': 'write' }.get(self._postgresql.synchronous_commit(), 'flush') + '_lsn' - pg_stat_replication = [(r['pid'], r['application_name'], r['sync_state'], r[sort_col]) for r in self._postgresql.pg_stat_replication() if r[sort_col] is not None] members = CaseInsensitiveDict({m.name: m for m in cluster.members}) - replica_list: List[Tuple[int, str, str, int, bool]] = [] + # pg_stat_replication.sync_state has 4 possible states - async, potential, quorum, sync. # That is, alphabetically they are in the reversed order of priority. # Since we are doing reversed sort on (sync_state, lsn) tuples, it helps to keep the result @@ -248,22 +232,13 @@ def current_state(self, cluster: Cluster) -> _SyncState: for pid, app_name, sync_state, replica_lsn in sorted(pg_stat_replication, key=lambda r: r[2:4], reverse=True): member = members.get(app_name) if member and member.is_running and not member.tags.get('nosync', False): - replica_list.append((pid, member.name, sync_state, replica_lsn, bool(member.nofailover))) - - max_lsn = max(replica_list, key=lambda x: x[3])[3]\ - if len(replica_list) > 1 else self._postgresql.last_operation() + yield (pid, member.name, sync_state, replica_lsn, bool(member.nofailover)) + def _process_replica_readiness(self, cluster: Cluster, replica_list: List[Tuple[int, str, str, int, bool]]) -> None: + """Flags replicas as truely "synchronous" when they caught up with "_primary_flush_lsn".""" if TYPE_CHECKING: # pragma: no cover assert self._postgresql.global_config is not None - sync_node_count = self._postgresql.global_config.synchronous_node_count\ - if self._postgresql.supports_multiple_sync else 1 - sync_node_maxlag = self._postgresql.global_config.maximum_lag_on_syncnode - - active = CaseInsensitiveSet() - sync_nodes = CaseInsensitiveSet() - numsync_confirmed = 0 - # Prefer members without nofailover tag. We are relying on the fact that sorts are guaranteed to be stable. - for pid, app_name, sync_state, replica_lsn, nofailover in sorted(replica_list, key=lambda x: x[4]): + for pid, app_name, sync_state, replica_lsn, _ in replica_list: if app_name not in self._ready_replicas and app_name in self._ssn_data.members: if self._postgresql.global_config.is_quorum_commit_mode: # When quorum commit is enabled we can't check against cluster.sync because nodes @@ -277,6 +252,44 @@ def current_state(self, cluster: Cluster) -> _SyncState: # "really" synchronous when sync_state = 'sync' and we known that it managed to catch up self._ready_replicas[app_name] = pid + def current_state(self, cluster: Cluster) -> _SyncState: + """Finds best candidates to be the synchronous standbys. + + Current synchronous standby is always preferred, unless it has disconnected or does not want to be a + synchronous standby any longer. + + Standbys are selected based on values from the global configuration: + - `maximum_lag_on_syncnode`: would help swapping unhealthy sync replica in case if it stops + responding (or hung). Please set the value high enough so it won't unncessarily swap sync + standbys during high loads. Any value less or equal of 0 keeps the behavior backward compatible. + Please note that it will not also swap sync standbys in case where all replicas are hung. + - `synchronous_node_count`: controlls how many nodes should be set as synchronous. + + :param cluster: current cluster topology from DCS + :returns: current synchronous replication state as a :class:`_SyncState` object + """ + self._handle_synchronous_standby_names_change() + + replica_list = list(self._get_replica_list(cluster)) + self._process_replica_readiness(cluster, replica_list) + + active = CaseInsensitiveSet() + sync_nodes = CaseInsensitiveSet() + numsync_confirmed = 0 + + if TYPE_CHECKING: # pragma: no cover + assert self._postgresql.global_config is not None + sync_node_count = self._postgresql.global_config.synchronous_node_count\ + if self._postgresql.supports_multiple_sync else 1 + sync_node_maxlag = self._postgresql.global_config.maximum_lag_on_syncnode + + # When checking *maximum_lag_on_syncnode* we want to compare with the most + # up-to-date replica or with cluster LSN if there is only one replica. + max_lsn = max(replica_list, key=lambda x: x[3])[3]\ + if len(replica_list) > 1 else self._postgresql.last_operation() + + # Prefer members without nofailover tag. We are relying on the fact that sorts are guaranteed to be stable. + for _, app_name, sync_state, replica_lsn, nofailover in sorted(replica_list, key=lambda x: x[4]): if sync_node_maxlag <= 0 or max_lsn - replica_lsn <= sync_node_maxlag: if self._postgresql.global_config.is_quorum_commit_mode: # add nodes with nofailover tag only to get enough "active" nodes diff --git a/patroni/quorum.py b/patroni/quorum.py index a41597b50..2cb4539cf 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -157,37 +157,7 @@ def __iter__(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: if cur_transition[0] == 'restart': break - def _generate_transitions(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: - logger.debug("Quorum state: leader %s quorum %s, voters %s, numsync %s, sync %s, " - "numsync_confirmed %s, active %s, sync_wanted %s leader_wanted %s", - self.leader, self.quorum, self.voters, self.numsync, self.sync, - self.numsync_confirmed, self.active, self.sync_wanted, self.leader_wanted) - try: - if self.leader_wanted != self.leader: - voters = (self.voters - CaseInsensitiveSet([self.leader_wanted])) | CaseInsensitiveSet([self.leader]) - if not self.sync: - # If sync is empty we need to update synchronous_standby_names first - numsync = len(voters) - self.quorum - yield from self.sync_update(numsync, CaseInsensitiveSet(voters)) - # If leader changed we need to add the old leader to quorum (voters) - yield from self.quorum_update(self.quorum, CaseInsensitiveSet(voters), self.leader_wanted) - # right after promote there could be no replication connections yet - if not self.sync & self.active: - return # give another loop_wait seconds for replicas to reconnect before removing them from quorum - else: - self.check_invariants() - except QuorumError as e: - logger.warning('%s', e) - yield from self.quorum_update(len(self.sync) - self.numsync, self.sync) - - assert self.leader == self.leader_wanted - - # numsync_confirmed could be 0 after restart/failover, we will calculate it from quorum - if self.numsync_confirmed == 0 and self.sync & self.active: - self.numsync_confirmed = min(len(self.sync & self.active), len(self.voters) - self.quorum) - logger.debug('numsync_confirmed=0, adjusting it to %d', self.numsync_confirmed) - - # Handle non steady state cases + def __handle_non_steady_cases(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: if self.sync < self.voters: logger.debug("Case 1: synchronous_standby_names subset of DCS state") # Case 1: quorum is superset of sync nodes. In the middle of changing quorum. @@ -233,8 +203,7 @@ def _generate_transitions(self) -> Iterator[Tuple[str, str, int, CaseInsensitive if self.numsync == self.sync_wanted and safety_margin > 0 and self.numsync > self.numsync_confirmed: yield from self.quorum_update(len(self.sync) - self.numsync, self.voters) - # We are in a steady state point. Find if desired state is different and act accordingly. - + def __remove_gone_nodes(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: # If any nodes have gone away, evict them to_remove = self.sync - self.active if to_remove and self.sync == to_remove: @@ -266,6 +235,7 @@ def _generate_transitions(self) -> Iterator[Tuple[str, str, int, CaseInsensitive yield from self.quorum_update(quorum, voters, adjust_quorum=False) yield from self.sync_update(numsync, sync) + def __add_new_nodes(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: # If any new nodes, join them to quorum to_add = self.active - self.sync if to_add: @@ -289,6 +259,7 @@ def _generate_transitions(self) -> Iterator[Tuple[str, str, int, CaseInsensitive adjust_quorum=sync_wanted > self.numsync_confirmed) yield from self.sync_update(sync_wanted, CaseInsensitiveSet(self.sync | to_add)) + def __handle_replication_factor_change(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: # Apply requested replication factor change sync_increase = min(self.sync_wanted, len(self.sync)) - self.numsync if sync_increase > 0: @@ -303,3 +274,43 @@ def _generate_transitions(self) -> Iterator[Tuple[str, str, int, CaseInsensitive yield from self.quorum_update(len(self.voters) - self.numsync - sync_increase, self.voters, adjust_quorum=self.sync_wanted > self.numsync_confirmed) yield from self.sync_update(self.numsync + sync_increase, self.sync) + + def _generate_transitions(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + logger.debug("Quorum state: leader %s quorum %s, voters %s, numsync %s, sync %s, " + "numsync_confirmed %s, active %s, sync_wanted %s leader_wanted %s", + self.leader, self.quorum, self.voters, self.numsync, self.sync, + self.numsync_confirmed, self.active, self.sync_wanted, self.leader_wanted) + try: + if self.leader_wanted != self.leader: + voters = (self.voters - CaseInsensitiveSet([self.leader_wanted])) | CaseInsensitiveSet([self.leader]) + if not self.sync: + # If sync is empty we need to update synchronous_standby_names first + numsync = len(voters) - self.quorum + yield from self.sync_update(numsync, CaseInsensitiveSet(voters)) + # If leader changed we need to add the old leader to quorum (voters) + yield from self.quorum_update(self.quorum, CaseInsensitiveSet(voters), self.leader_wanted) + # right after promote there could be no replication connections yet + if not self.sync & self.active: + return # give another loop_wait seconds for replicas to reconnect before removing them from quorum + else: + self.check_invariants() + except QuorumError as e: + logger.warning('%s', e) + yield from self.quorum_update(len(self.sync) - self.numsync, self.sync) + + assert self.leader == self.leader_wanted + + # numsync_confirmed could be 0 after restart/failover, we will calculate it from quorum + if self.numsync_confirmed == 0 and self.sync & self.active: + self.numsync_confirmed = min(len(self.sync & self.active), len(self.voters) - self.quorum) + logger.debug('numsync_confirmed=0, adjusting it to %d', self.numsync_confirmed) + + yield from self.__handle_non_steady_cases() + + # We are in a steady state point. Find if desired state is different and act accordingly. + + yield from self.__remove_gone_nodes() + + yield from self.__add_new_nodes() + + yield from self.__handle_replication_factor_change() From ad4bea7e565c375c5e324067dd2bac0ad31ec471 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 20 Jul 2023 07:03:22 +0200 Subject: [PATCH 12/37] Apply suggestions from code review Co-authored-by: Matt Baker <93600443+matthbakeredb@users.noreply.github.com> Co-authored-by: Israel --- docs/dynamic_configuration.rst | 2 +- docs/replication_modes.rst | 12 ++++++------ patroni/ha.py | 21 +++++++++++---------- patroni/postgresql/sync.py | 1 + 4 files changed, 19 insertions(+), 17 deletions(-) diff --git a/docs/dynamic_configuration.rst b/docs/dynamic_configuration.rst index f260de3e8..361f0e01b 100644 --- a/docs/dynamic_configuration.rst +++ b/docs/dynamic_configuration.rst @@ -16,7 +16,7 @@ In order to change the dynamic configuration you can use either ``patronictl edi - **max\_timelines\_history**: maximum number of timeline history items kept in DCS. Default value: 0. When set to 0, it keeps the full history in DCS. - **primary\_start\_timeout**: the amount of time a primary is allowed to recover from failures before failover is triggered (in seconds). Default is 300 seconds. When set to 0 failover is done immediately after a crash is detected if possible. When using asynchronous replication a failover can cause lost transactions. Worst case failover time for primary failure is: loop\_wait + primary\_start\_timeout + loop\_wait, unless primary\_start\_timeout is zero, in which case it's just loop\_wait. Set the value according to your durability/availability tradeoff. - **primary\_stop\_timeout**: The number of seconds Patroni is allowed to wait when stopping Postgres and effective only when synchronous_mode is enabled. When set to > 0 and the synchronous_mode is enabled, Patroni sends SIGKILL to the postmaster if the stop operation is running for more than the value set by primary\_stop\_timeout. Set the value according to your durability/availability tradeoff. If the parameter is not set or set <= 0, primary\_stop\_timeout does not apply. -- **synchronous\_mode**: turns on synchronous replication mode. Possible values: ``off``, ``on``, ``quorum``. In this mode the leader takes care about management of ``synchronous_standby_names`` and and only the last known leader or one of synchronous replicas are allowed to participate in leader race. Synchronous mode makes sure that successfully committed transactions will not be lost at failover, at the cost of losing availability for writes when Patroni cannot ensure transaction durability. See :ref:`replication modes documentation ` for details. +- **synchronous\_mode**: turns on synchronous replication mode. Possible values: ``off``, ``on``, ``quorum``. In this mode the leader takes care of management of ``synchronous_standby_names``, and only the last known leader, or one of synchronous replicas, are allowed to participate in leader race. Synchronous mode makes sure that successfully committed transactions will not be lost at failover, at the cost of losing availability for writes when Patroni cannot ensure transaction durability. See :ref:`replication modes documentation ` for details. - **synchronous\_mode\_strict**: prevents disabling synchronous replication if no synchronous replicas are available, blocking all client writes to the primary. See :ref:`replication modes documentation ` for details. - **failsafe\_mode**: Enables :ref:`DCS Failsafe Mode `. Defaults to `false`. - **postgresql**: diff --git a/docs/replication_modes.rst b/docs/replication_modes.rst index 6ab0d291f..ee7fd827d 100644 --- a/docs/replication_modes.rst +++ b/docs/replication_modes.rst @@ -66,15 +66,15 @@ Note: Because of the way synchronous replication is implemented in PostgreSQL it Synchronous Replication Factor ============================== -The parameter ``synchronous_node_count`` is used by Patroni to manage number of synchronous standby databases. It is set to 1 by default. It has no effect when ``synchronous_mode`` is set to off. When enabled, Patroni manages precise number of synchronous standby databases based on parameter ``synchronous_node_count`` and adjusts the state in DCS & synchronous_standby_names as members join and leave. If the parameter is set to the value higher than the number of eligible nodes it will be automatically reduced by Patroni down to 1. +The parameter ``synchronous_node_count`` is used by Patroni to manage number of synchronous standby databases. It is set to 1 by default. It has no effect when ``synchronous_mode`` is set to off. When enabled, Patroni manages precise number of synchronous standby databases based on parameter ``synchronous_node_count`` and adjusts the state in DCS & ``synchronous_standby_names`` in PostgreSQL as members join and leave. If the parameter is set to a value higher than the number of eligible nodes it will be automatically reduced by Patroni down to 1. Maximum lag on synchronous node =============================== -By default Patroni sticks to a node that is declared as ``synchronous`` according to the ``pg_stat_replication`` even when there are other nodes ahead of it. It is done to minimize the number of changes of ``synchronous_standby_names``. To change this behavior one may use ``maximum_lag_on_syncnode`` parameter. It controls how much the replica can lag in to be allowed chosen as synchronous. +By default Patroni sticks to a node that is declared as ``synchronous``, according to the setting ``pg_stat_replication``, even when there are other nodes ahead of it. This is done to minimize the number of changes of ``synchronous_standby_names``. To change this behavior one may use ``maximum_lag_on_syncnode`` parameter. It controls how much lag the replica can have to still be considered as "synchronous". -Patroni utilizes the max replica LSN if there is more than one standby, otherwise it will use leader's current wal LSN. Default is ``-1``, and Patroni will not take action to swap synchronous unhealthy standby when the value is set to 0 or below. Please set the value high enough so Patroni won't swap synchrounous standbys fequently during high transaction volume. +Patroni utilizes the max replica LSN if there is more than one standby, otherwise it will use leader's current wal LSN. The default is ``-1``, and Patroni will not take action to swap a synchronous unhealthy standby when the value is set to 0 or below. Please set the value high enough so that Patroni won't swap synchronous standbys frequently during high transaction volume. Synchronous mode implementation @@ -100,11 +100,11 @@ Quorum commit mode Starting from PostgreSQL v10 Patroni supports quorum-based synchronous replication. -In this mode Patroni maintains synchronization state in the DCS, containing the latest known primary, number of nodes required for quorum and nodes currently eligible to vote on quorum. In steady state the nodes voting on quorum are the leader and all synchronous standbys. This state is updated with strict ordering constraints with regards to node promotion and ``synchronous_standby_names`` to ensure that at all times any subset of voters that can achieve quorum is contained to have at least one node having the latest successful commit. +In this mode, Patroni maintains synchronization state in the DCS, containing the latest known primary, the number of nodes required for quorum and the nodes currently eligible to vote on quorum. In steady state, the nodes voting on quorum are the leader and all synchronous standbys. This state is updated with strict ordering constraints, with regards to node promotion and ``synchronous_standby_names``, to ensure that at all times any subset of voters that can achieve quorum includes at least one node with the latest successful commit. -On each iteration of HA loop Patroni re-evaluates synchronous standby choices and quorum based on node availability and requested cluster configuration. In PostgreSQL versions above 9.6 all eligible nodes are added as synchronous standbys as soon as their replication catches up to leader. +On each iteration of HA loop, Patroni re-evaluates synchronous standby choices and quorum, based on node availability and requested cluster configuration. In PostgreSQL versions above 9.6 all eligible nodes are added as synchronous standbys as soon as their replication catches up to leader. -Quorum commit helps to reduce worst case latencies even during normal operation as a higher latency of replicating to one standby can be compensated by other standbys. +Quorum commit helps to reduce worst case latencies, even during normal operation, as a higher latency of replicating to one standby can be compensated by other standbys. The quorum-based synchronous mode could be enabled by setting ``synchronous_mode`` to ``quorum`` using ``patronictl edit-config`` command or via Patroni REST interface. See :ref:`dynamic configuration ` for instructions. diff --git a/patroni/ha.py b/patroni/ha.py index dccd68f2d..9975e2dd7 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -745,7 +745,7 @@ def _process_multisync_replication(self) -> None: logger.info("Synchronous replication key updated by someone else") def process_sync_replication(self) -> None: - """Process synchronous replication beahvior on the primary.""" + """Process synchronous replication behavior on the primary.""" if self.is_quorum_commit_mode(): # The synchronous_standby_names was adjusted right before promote. # After that, when postgres has become a primary, we need to reflect this change @@ -920,11 +920,11 @@ def before_promote(): return promote_message def fetch_node_status(self, member: Member) -> _MemberStatus: - """This function performs http get request on member.api_url and fetches its status. + """Perform http get request on member.api_url to fetch its status. - Usually it happens during the leader race and we can't afford wating for a response indefinite time, - therefore the request timeout is hardcoded to 2 seconds, which seems to be a good compromise. - The node which is slow to respond most likely will not be healthy. + Usually this happens during the leader race and we can't afford to wait an indefinite time + for a response, therefore the request timeout is hardcoded to 2 seconds, which seems to be a + good compromise. The node which is slow to respond is most likely unhealthy. :returns: :class:`_MemberStatus` object """ @@ -987,7 +987,7 @@ def check_failsafe_topology(self) -> bool: return all(results) def is_lagging(self, wal_position: int) -> bool: - """Checks if node should consider itself unhealthy to be promoted due to replication lag. + """Check if node should consider itself unhealthy to be promoted due to replication lag. :param wal_position: Current wal position. :returns `True` when node is lagging @@ -996,14 +996,15 @@ def is_lagging(self, wal_position: int) -> bool: return lag > self.global_config.maximum_lag_on_failover def _is_healthiest_node(self, members: Collection[Member], check_replication_lag: bool = True) -> bool: - """This method tries to determine whether the current node is healthy enough to became a new leader candidate. + """Determine whether the current node is healthy enough to become a new leader candidate. :param members: the list of nodes to check against :param check_replication_lag: whether to take the replication lag into account. If the lag exceeds configured threshold the node disqualifies itself. - :returns: `True` in case if the node is eligible to become the new leader. Since this method is executed - on multiple nodes independently it could happen that many nodes will count themselves as - healthiest because they received/replayed up to the same LSN, but it is totally fine. + :returns: `True` if the node is eligible to become the new leader. Since this method is executed + on multiple nodes independently it is possible that multiple nodes could count + themselves as the healthiest because they received/replayed up to the same LSN, + but this is totally fine. """ my_wal_position = self.state_handler.last_operation() if check_replication_lag and self.is_lagging(my_wal_position): diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index 1ceab579f..921b47657 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -266,6 +266,7 @@ def current_state(self, cluster: Cluster) -> _SyncState: - `synchronous_node_count`: controlls how many nodes should be set as synchronous. :param cluster: current cluster topology from DCS + :returns: current synchronous replication state as a :class:`_SyncState` object """ self._handle_synchronous_standby_names_change() From 893e460695faf8fed5154d852ad8c4a5d7336d67 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 20 Jul 2023 08:00:52 +0200 Subject: [PATCH 13/37] Address review feedback --- docs/rest_api.rst | 2 ++ features/steps/citus.py | 3 +-- features/steps/quorum_commit.py | 27 ++++++++++----------- patroni/api.py | 12 ++++++++- patroni/ha.py | 6 ++--- patroni/postgresql/sync.py | 43 +++++++++++++++++++++++++++------ tests/test_api.py | 2 ++ 7 files changed, 68 insertions(+), 27 deletions(-) diff --git a/docs/rest_api.rst b/docs/rest_api.rst index d7a6824d5..5310f73ec 100644 --- a/docs/rest_api.rst +++ b/docs/rest_api.rst @@ -47,6 +47,8 @@ For all health check ``GET`` requests Patroni returns a JSON document with the s - ``GET /quorum``: returns HTTP status code **200** only when this Patroni node is listed as a quorum node in ``synchronous_standby_names`` on the primary. +- ``GET /read-only-quorum``: like the above endpoint, but also includes the primary. + - ``GET /asynchronous`` or ``GET /async``: returns HTTP status code **200** only when the Patroni node is running as an asynchronous standby. diff --git a/features/steps/citus.py b/features/steps/citus.py index 84186e8fe..1af70a302 100644 --- a/features/steps/citus.py +++ b/features/steps/citus.py @@ -61,8 +61,7 @@ def check_registration(context, name1, name2, role, group, time_limit): except Exception: pass time.sleep(1) - assert False, "Worker {0} is not registered in pg_dist_node on the coordinator {1} after {2} seconds"\ - .format(name1, name2, time_limit) + assert False, "Node {0} is not registered in pg_dist_node on the node {1}".format(name1, name2) @step('I create a distributed table on {name:w}') diff --git a/features/steps/quorum_commit.py b/features/steps/quorum_commit.py index 7cedc0d5b..19a887a7b 100644 --- a/features/steps/quorum_commit.py +++ b/features/steps/quorum_commit.py @@ -25,6 +25,17 @@ def check_sync(context, key, value, time_limit): dcs_value, time_limit) +def _parse_synchronous_standby_names(value): + if '(' in value: + m = re.match(r'.*(\d+) \(([^)]+)\)', value) + expected_value = set(m.group(2).split()) + expected_num = m.group(1) + else: + expected_value = set([value]) + expected_num = '1' + return expected_num, expected_value + + @then('synchronous_standby_names on {name:2} is set to "{value}" after {time_limit:d} seconds') def check_synchronous_standby_names(context, name, value, time_limit): time_limit *= context.timeout_multiplier @@ -33,24 +44,12 @@ def check_synchronous_standby_names(context, name, value, time_limit): if value == '_empty_str_': value = '' - if '(' in value: - m = re.match(r'.*(\d+) \(([^)]+)\)', value) - expected_value = set(m.group(2).split()) - expected_num = m.group(1) - else: - expected_value = set([value]) - expected_num = '1' + expected_num, expected_value = _parse_synchronous_standby_names(value) while time.time() < max_time: try: ssn = context.pctl.query(name, "SHOW synchronous_standby_names").fetchone()[0] - if '(' in ssn: - m = re.match(r'.*(\d+) \(([^)]+)\)', ssn) - db_value = set(m.group(2).split()) - db_num = m.group(1) - else: - db_value = set([ssn]) - db_num = '1' + db_num, db_value = _parse_synchronous_standby_names(ssn) if expected_value == db_value and expected_num == db_num: return except Exception: diff --git a/patroni/api.py b/patroni/api.py index fe03b7f4b..5da815d5c 100644 --- a/patroni/api.py +++ b/patroni/api.py @@ -219,6 +219,10 @@ def do_GET(self, write_status_code_only: bool = False) -> None: * HTTP status ``200``: if up and running as a standby and without ``noloadbalance`` tag. * ``/read-only``: * HTTP status ``200``: if up and running and without ``noloadbalance`` tag. + * ``/quorum``: + * HTTP status ``200``: if up and running as a quorum synchronous standby. + * ``/read-only-quorum``: + * HTTP status ``200``: if up and running as a quorum synchronous standby or primary. * ``/synchronous`` or ``/sync``: * HTTP status ``200``: if up and running as a synchronous standby. * ``/read-only-sync``: @@ -290,7 +294,7 @@ def do_GET(self, write_status_code_only: bool = False) -> None: ignore_tags = True elif 'replica' in path: status_code = replica_status_code - elif 'read-only' in path and 'sync' not in path: + elif 'read-only' in path and 'sync' not in path and 'quorum' not in path: status_code = 200 if 200 in (primary_status_code, standby_leader_status_code) else replica_status_code elif 'health' in path: status_code = 200 if response.get('state') == 'running' else 503 @@ -303,6 +307,11 @@ def do_GET(self, write_status_code_only: bool = False) -> None: status_code = replica_status_code elif path in ('/async', '/asynchronous') and not is_synchronous and not is_quorum: status_code = replica_status_code + elif path == '/read-only-quorum': + if 200 in (primary_status_code, standby_leader_status_code): + status_code = 200 + elif is_quorum: + status_code = replica_status_code elif path in ('/read-only-sync', '/read-only-synchronous'): if 200 in (primary_status_code, standby_leader_status_code): status_code = 200 @@ -455,6 +464,7 @@ def do_GET_metrics(self) -> None: * ``patroni_standby_leader``: ``1`` if standby leader node, else ``0``; * ``patroni_replica``: ``1`` if a replica, else ``0``; * ``patroni_sync_standby``: ``1`` if a sync replica, else ``0``; + * ``patroni_quorum_standby``: ``1`` if a quorum sync replica, else ``0``; * ``patroni_xlog_received_location``: ``pg_wal_lsn_diff(pg_last_wal_receive_lsn(), '0/0')``; * ``patroni_xlog_replayed_location``: ``pg_wal_lsn_diff(pg_last_wal_replay_lsn(), '0/0)``; * ``patroni_xlog_replayed_timestamp``: ``pg_last_xact_replay_timestamp``; diff --git a/patroni/ha.py b/patroni/ha.py index 9975e2dd7..d7afc0862 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -922,8 +922,8 @@ def before_promote(): def fetch_node_status(self, member: Member) -> _MemberStatus: """Perform http get request on member.api_url to fetch its status. - Usually this happens during the leader race and we can't afford to wait an indefinite time - for a response, therefore the request timeout is hardcoded to 2 seconds, which seems to be a + Usually this happens during the leader race and we can't afford to wait an indefinite time + for a response, therefore the request timeout is hardcoded to 2 seconds, which seems to be a good compromise. The node which is slow to respond is most likely unhealthy. :returns: :class:`_MemberStatus` object @@ -1002,7 +1002,7 @@ def _is_healthiest_node(self, members: Collection[Member], check_replication_lag :param check_replication_lag: whether to take the replication lag into account. If the lag exceeds configured threshold the node disqualifies itself. :returns: `True` if the node is eligible to become the new leader. Since this method is executed - on multiple nodes independently it is possible that multiple nodes could count + on multiple nodes independently it is possible that multiple nodes could count themselves as the healthiest because they received/replayed up to the same LSN, but this is totally fine. """ diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index 921b47657..1b9fda105 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -154,6 +154,18 @@ def parse_sync_standby_names(value: str) -> _SSN: class _SyncState(NamedTuple): + """Class representing the current synchronous state. + + :ivar sync_type: possible values: 'off', 'priority', 'quorum' + :ivar numsync: how many nodes are required to be synchronous (according to ``synchronous_standby_names``). + Is ``0`` in case if synchronous_standby_names value is invalid or has ``*``. + :ivar numsync_confirmed: how many nodes are known to be synchronous according to the ``pg_stat_replication`` + view. Only nodes that caught up with the ``SyncHandler._primary_flush_lsn` are counted. + :ivar sync: collection of synchronous node names. In case of quorum commit all nodes listed + in ``synchronous_standby_names`` or nodes that are confirmed to be synchronous according + to the `pg_stat_replication` view. + :ivar active: collection of node names that are streaming and have no restrictions to become synchronous. + """ sync_type: str numsync: int numsync_confirmed: int @@ -210,7 +222,22 @@ def _handle_synchronous_standby_names_change(self) -> None: self._postgresql.reset_cluster_info_state(None) # Reset internal cache to query fresh values def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, int, bool]]: - """Yields candidates based on higher replay/remote_write/flush lsn.""" + """Yields candidates based on higher replay/remote_write/flush lsn. + + .. note:: + Tuples are reverse ordered by sync_state and LSN fields so nodes that already synchronous or having + higher LSN values are preferred. + + :param cluster: current cluster topology from DCS. + + :yields: tuples composed of: + + * pid - a PID of walsender process + * member name - matches with the application_name + * sync_state - one of ("async", "potential", "quorum", "sync") + * LSN - write_lsn, flush_lsn, or replica_lsn, depending on the value of ``synchronous_commit`` GUC + * nofailover - whether the member has ``nofailover`` tag set + """ # What column from pg_stat_replication we want to sort on? Choose based on ``synchronous_commit`` value. sort_col = { @@ -259,14 +286,16 @@ def current_state(self, cluster: Cluster) -> _SyncState: synchronous standby any longer. Standbys are selected based on values from the global configuration: - - `maximum_lag_on_syncnode`: would help swapping unhealthy sync replica in case if it stops - responding (or hung). Please set the value high enough so it won't unncessarily swap sync - standbys during high loads. Any value less or equal of 0 keeps the behavior backward compatible. - Please note that it will not also swap sync standbys in case where all replicas are hung. - - `synchronous_node_count`: controlls how many nodes should be set as synchronous. + + - `maximum_lag_on_syncnode`: would help swapping unhealthy sync replica in case if it stops + responding (or hung). Please set the value high enough so it won't unncessarily swap sync + standbys during high loads. Any value less or equal of 0 keeps the behavior backward compatible. + Please note that it will not also swap sync standbys in case where all replicas are hung. + + - `synchronous_node_count`: controlls how many nodes should be set as synchronous. :param cluster: current cluster topology from DCS - + :returns: current synchronous replication state as a :class:`_SyncState` object """ self._handle_synchronous_standby_names_change() diff --git a/tests/test_api.py b/tests/test_api.py index 075e2029e..ba62dd53b 100644 --- a/tests/test_api.py +++ b/tests/test_api.py @@ -212,11 +212,13 @@ def test_do_GET(self): with patch.object(RestApiHandler, 'get_postgresql_status', Mock(return_value={'role': 'replica', 'quorum_standby': True})): MockRestApiServer(RestApiHandler, 'GET /quorum') + MockRestApiServer(RestApiHandler, 'GET /read-only-quorum') with patch.object(RestApiHandler, 'get_postgresql_status', Mock(return_value={'role': 'replica'})): MockRestApiServer(RestApiHandler, 'GET /asynchronous') with patch.object(MockHa, 'is_leader', Mock(return_value=True)): MockRestApiServer(RestApiHandler, 'GET /replica') MockRestApiServer(RestApiHandler, 'GET /read-only-sync') + MockRestApiServer(RestApiHandler, 'GET /read-only-quorum') with patch.object(GlobalConfig, 'is_standby_cluster', Mock(return_value=True)): MockRestApiServer(RestApiHandler, 'GET /standby_leader') MockPatroni.dcs.cluster = None From 2e9b6b21dec0c119d1b2278fa56d9f57f4a7df09 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Fri, 21 Jul 2023 11:38:33 +0200 Subject: [PATCH 14/37] Apply suggestions from code review Co-authored-by: Israel --- docs/replication_modes.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/replication_modes.rst b/docs/replication_modes.rst index ee7fd827d..3e9cb825b 100644 --- a/docs/replication_modes.rst +++ b/docs/replication_modes.rst @@ -72,9 +72,9 @@ The parameter ``synchronous_node_count`` is used by Patroni to manage number of Maximum lag on synchronous node =============================== -By default Patroni sticks to a node that is declared as ``synchronous``, according to the setting ``pg_stat_replication``, even when there are other nodes ahead of it. This is done to minimize the number of changes of ``synchronous_standby_names``. To change this behavior one may use ``maximum_lag_on_syncnode`` parameter. It controls how much lag the replica can have to still be considered as "synchronous". +By default Patroni sticks to a node that is declared as ``synchronous``, according to the ``pg_stat_replication`` view, even when there are other nodes ahead of it. This is done to minimize the number of changes of ``synchronous_standby_names``. To change this behavior one may use ``maximum_lag_on_syncnode`` parameter. It controls how much lag the replica can have to still be considered as "synchronous". -Patroni utilizes the max replica LSN if there is more than one standby, otherwise it will use leader's current wal LSN. The default is ``-1``, and Patroni will not take action to swap a synchronous unhealthy standby when the value is set to 0 or below. Please set the value high enough so that Patroni won't swap synchronous standbys frequently during high transaction volume. +Patroni utilizes the max replica LSN if there is more than one standby, otherwise it will use leader's current wal LSN. The default is ``-1``, and Patroni will not take action to swap a synchronous unhealthy standby when the value is set to ``0`` or less. Please set the value high enough so that Patroni won't swap synchronous standbys frequently during high transaction volume. Synchronous mode implementation From 7114a07fa63dd30caf0062cf8af9239c4fcb5a4e Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Fri, 21 Jul 2023 13:24:09 +0200 Subject: [PATCH 15/37] remove unrelated change --- patroni/dcs/etcd3.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/patroni/dcs/etcd3.py b/patroni/dcs/etcd3.py index 9a2e2f38a..32d003591 100644 --- a/patroni/dcs/etcd3.py +++ b/patroni/dcs/etcd3.py @@ -939,6 +939,6 @@ def watch(self, leader_version: Optional[str], timeout: float) -> bool: timeout += 0.5 try: - return super(Etcd3, self).watch(None, timeout + 0.5) + return super(Etcd3, self).watch(None, timeout) finally: self.event.clear() From 5dfe5e0e5a478f3b99f79ab276de0ec095bb0272 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Tue, 25 Jul 2023 08:43:16 +0200 Subject: [PATCH 16/37] Revert unwanted change --- features/citus.feature | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/features/citus.feature b/features/citus.feature index 6acb572a2..35ccebbef 100644 --- a/features/citus.feature +++ b/features/citus.feature @@ -20,7 +20,7 @@ Feature: citus And replication works from postgres1 to postgres0 after 15 seconds And postgres1 is registered in the postgres2 as the primary in group 0 after 5 seconds And "sync" key in a group 0 in DCS has sync_standby=postgres0 after 15 seconds - When I run patronictl.py failover batman --group 0 --candidate postgres0 --force + When I run patronictl.py switchover batman --group 0 --candidate postgres0 --force Then postgres0 role is the primary after 10 seconds And replication works from postgres0 to postgres1 after 15 seconds And postgres0 is registered in the postgres2 as the primary in group 0 after 5 seconds From 3ee0238e98a9ae53dd6701ca73b0f8863c9066af Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Tue, 25 Jul 2023 10:00:24 +0200 Subject: [PATCH 17/37] Add more examples of sync and quorum modes --- docs/replication_modes.rst | 77 +++++++++++++++++++++++++++++++++++--- 1 file changed, 71 insertions(+), 6 deletions(-) diff --git a/docs/replication_modes.rst b/docs/replication_modes.rst index 3e9cb825b..b50a972e9 100644 --- a/docs/replication_modes.rst +++ b/docs/replication_modes.rst @@ -66,13 +66,13 @@ Note: Because of the way synchronous replication is implemented in PostgreSQL it Synchronous Replication Factor ============================== -The parameter ``synchronous_node_count`` is used by Patroni to manage number of synchronous standby databases. It is set to 1 by default. It has no effect when ``synchronous_mode`` is set to off. When enabled, Patroni manages precise number of synchronous standby databases based on parameter ``synchronous_node_count`` and adjusts the state in DCS & ``synchronous_standby_names`` in PostgreSQL as members join and leave. If the parameter is set to a value higher than the number of eligible nodes it will be automatically reduced by Patroni down to 1. +The parameter ``synchronous_node_count`` is used by Patroni to manage number of synchronous standby databases. It is set to 1 by default. It has no effect when ``synchronous_mode`` is set to off. When enabled, Patroni manages precise number of synchronous standby databases based on parameter ``synchronous_node_count`` and adjusts the state in DCS & ``synchronous_standby_names`` in PostgreSQL as members join and leave. If the parameter is set to a value higher than the number of eligible nodes it will be automatically reduced by Patroni down. Maximum lag on synchronous node =============================== -By default Patroni sticks to a node that is declared as ``synchronous``, according to the ``pg_stat_replication`` view, even when there are other nodes ahead of it. This is done to minimize the number of changes of ``synchronous_standby_names``. To change this behavior one may use ``maximum_lag_on_syncnode`` parameter. It controls how much lag the replica can have to still be considered as "synchronous". +By default Patroni sticks to nodes that are declared as ``synchronous``, according to the ``pg_stat_replication`` view, even when there are other nodes ahead of it. This is done to minimize the number of changes of ``synchronous_standby_names``. To change this behavior one may use ``maximum_lag_on_syncnode`` parameter. It controls how much lag the replica can have to still be considered as "synchronous". Patroni utilizes the max replica LSN if there is more than one standby, otherwise it will use leader's current wal LSN. The default is ``-1``, and Patroni will not take action to swap a synchronous unhealthy standby when the value is set to ``0`` or less. Please set the value high enough so that Patroni won't swap synchronous standbys frequently during high transaction volume. @@ -80,17 +80,49 @@ Patroni utilizes the max replica LSN if there is more than one standby, otherwis Synchronous mode implementation =============================== -When in synchronous mode Patroni maintains synchronization state in the DCS, containing the latest primary and current synchronous standby databases. This state is updated with strict ordering constraints to ensure the following invariants: +When in synchronous mode Patroni maintains synchronization state in the DCS (``/sync`` key), containing the latest primary and current synchronous standby databases. This state is updated with strict ordering constraints to ensure the following invariants: - A node must be marked as the latest leader whenever it can accept write transactions. Patroni crashing or PostgreSQL not shutting down can cause violations of this invariant. -- A node must be set as the synchronous standby in PostgreSQL as long as it is published as the synchronous standby. +- A node must be set as the synchronous standby in PostgreSQL as long as it is published as the synchronous standby in the ``/sync`` key in DCS.. - A node that is not the leader or current synchronous standby is not allowed to promote itself automatically. Patroni will only assign one or more synchronous standby nodes based on ``synchronous_node_count`` parameter to ``synchronous_standby_names``. -On each HA loop iteration Patroni re-evaluates synchronous standby nodes choice. If the current list of synchronous standby nodes are connected and has not requested its synchronous status to be removed it remains picked. Otherwise the cluster member available for sync that is furthest ahead in replication is picked. +On each HA loop iteration Patroni re-evaluates synchronous standby nodes choice. If the current list of synchronous standby nodes are connected and has not requested its synchronous status to be removed it remains picked. Otherwise the cluster members available for sync that are furthest ahead in replication are picked. + +Example: +--------- + +``/config`` key in DCS +^^^^^^^^^^^^^^^^^^^^^^ + +.. code-block:: YAML + + synchronous_mode: on + synchronous_node_count: 2 + ... + +``/sync`` key in DCS +^^^^^^^^^^^^^^^^^^^^ + +.. code-block:: JSON + + { + "leader": "node0", + "sync_standby": "node1,node2" + } + +postgresql.conf +^^^^^^^^^^^^^^^ + +.. code-block:: INI + + synchronous_standby_names = 'FIRST 2 (node1,node2)' + + +In the above examples only nodes ``node1`` and ``node2`` are known to be synchronous and allowed to be automatically promoted if the primary (``node0``) fails. .. _quorum_mode: @@ -100,7 +132,7 @@ Quorum commit mode Starting from PostgreSQL v10 Patroni supports quorum-based synchronous replication. -In this mode, Patroni maintains synchronization state in the DCS, containing the latest known primary, the number of nodes required for quorum and the nodes currently eligible to vote on quorum. In steady state, the nodes voting on quorum are the leader and all synchronous standbys. This state is updated with strict ordering constraints, with regards to node promotion and ``synchronous_standby_names``, to ensure that at all times any subset of voters that can achieve quorum includes at least one node with the latest successful commit. +In this mode, Patroni maintains synchronization state in the DCS, containing the latest known primary, the number of nodes required for quorum, and the nodes currently eligible to vote on quorum. In steady state, the nodes voting on quorum are the leader and all synchronous standbys. This state is updated with strict ordering constraints, with regards to node promotion and ``synchronous_standby_names``, to ensure that at all times any subset of voters that can achieve quorum includes at least one node with the latest successful commit. On each iteration of HA loop, Patroni re-evaluates synchronous standby choices and quorum, based on node availability and requested cluster configuration. In PostgreSQL versions above 9.6 all eligible nodes are added as synchronous standbys as soon as their replication catches up to leader. @@ -110,6 +142,39 @@ The quorum-based synchronous mode could be enabled by setting ``synchronous_mode Other parameters, like ``synchronous_node_count``, ``maximum_lag_on_syncnode``, and ``synchronous_mode_strict`` continue to work the same way as with ``synchronous_mode=on``. +Example: +--------- + +``/config`` key in DCS +^^^^^^^^^^^^^^^^^^^^^^ + +.. code-block:: YAML + + synchronous_mode: quorum + synchronous_node_count: 2 + ... + +``/sync`` key in DCS +^^^^^^^^^^^^^^^^^^^^ + +.. code-block:: JSON + + { + "leader": "node0", + "sync_standby": "node1,node2,node3", + "quorum": 1 + } + +postgresql.conf +^^^^^^^^^^^^^^^ + +.. code-block:: INI + + synchronous_standby_names = 'ANY 2 (node1,node2,node3)' + + +If the primary (``node0``) failed, in the above example two of the ``node1``, ``node2``, ``node3`` will have the latest transaction received, but we don't know which ones. To figure out whether the node ``node1`` has received the latest transaction, we needs to compare its LSN with the LSN on **at least** one node (``quorum=1`` in the ``/sync`` key) of ``node2`` and ``node3``. If ``node1`` isn't behind of at least one of them, we can guaranty that there will be no user visible data loss if ``node1`` is promoted. + .. [1] The data is still there, but recovering it requires a manual recovery effort by data recovery specialists. When Patroni is allowed to rewind with ``use_pg_rewind`` the forked timeline will be automatically erased to rejoin the failed primary with the cluster. From bea97d116b11a844ca53447a9b185d34b3839b73 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Wed, 26 Jul 2023 07:36:42 +0200 Subject: [PATCH 18/37] Apply suggestions from code review Co-authored-by: Israel --- docs/replication_modes.rst | 4 ++-- patroni/postgresql/sync.py | 18 +++++++++--------- patroni/utils.py | 2 +- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/replication_modes.rst b/docs/replication_modes.rst index b50a972e9..676b9b5fe 100644 --- a/docs/replication_modes.rst +++ b/docs/replication_modes.rst @@ -66,7 +66,7 @@ Note: Because of the way synchronous replication is implemented in PostgreSQL it Synchronous Replication Factor ============================== -The parameter ``synchronous_node_count`` is used by Patroni to manage number of synchronous standby databases. It is set to 1 by default. It has no effect when ``synchronous_mode`` is set to off. When enabled, Patroni manages precise number of synchronous standby databases based on parameter ``synchronous_node_count`` and adjusts the state in DCS & ``synchronous_standby_names`` in PostgreSQL as members join and leave. If the parameter is set to a value higher than the number of eligible nodes it will be automatically reduced by Patroni down. +The parameter ``synchronous_node_count`` is used by Patroni to manage the number of synchronous standby databases. It is set to ``1`` by default. It has no effect when ``synchronous_mode`` is set to ``off``. When enabled, Patroni manages the precise number of synchronous standby databases based on parameter ``synchronous_node_count`` and adjusts the state in DCS & ``synchronous_standby_names`` in PostgreSQL as members join and leave. If the parameter is set to a value higher than the number of eligible nodes it will be automatically reduced by Patroni. Maximum lag on synchronous node @@ -173,7 +173,7 @@ postgresql.conf synchronous_standby_names = 'ANY 2 (node1,node2,node3)' -If the primary (``node0``) failed, in the above example two of the ``node1``, ``node2``, ``node3`` will have the latest transaction received, but we don't know which ones. To figure out whether the node ``node1`` has received the latest transaction, we needs to compare its LSN with the LSN on **at least** one node (``quorum=1`` in the ``/sync`` key) of ``node2`` and ``node3``. If ``node1`` isn't behind of at least one of them, we can guaranty that there will be no user visible data loss if ``node1`` is promoted. +If the primary (``node0``) failed, in the above example two of the ``node1``, ``node2``, ``node3`` will have the latest transaction received, but we don't know which ones. To figure out whether the node ``node1`` has received the latest transaction, we need to compare its LSN with the LSN on **at least** one node (``quorum=1`` in the ``/sync`` key) among ``node2`` and ``node3``. If ``node1`` isn't behind of at least one of them, we can guarantee that there will be no user visible data loss if ``node1`` is promoted. .. [1] The data is still there, but recovering it requires a manual recovery effort by data recovery specialists. When Patroni is allowed to rewind with ``use_pg_rewind`` the forked timeline will be automatically erased to rejoin the failed primary with the cluster. diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index 1b9fda105..8716ad8cf 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -158,7 +158,7 @@ class _SyncState(NamedTuple): :ivar sync_type: possible values: 'off', 'priority', 'quorum' :ivar numsync: how many nodes are required to be synchronous (according to ``synchronous_standby_names``). - Is ``0`` in case if synchronous_standby_names value is invalid or has ``*``. + Is ``0`` in case if ``synchronous_standby_names`` value is invalid or has ``*``. :ivar numsync_confirmed: how many nodes are known to be synchronous according to the ``pg_stat_replication`` view. Only nodes that caught up with the ``SyncHandler._primary_flush_lsn` are counted. :ivar sync: collection of synchronous node names. In case of quorum commit all nodes listed @@ -222,21 +222,21 @@ def _handle_synchronous_standby_names_change(self) -> None: self._postgresql.reset_cluster_info_state(None) # Reset internal cache to query fresh values def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, int, bool]]: - """Yields candidates based on higher replay/remote_write/flush lsn. + """Yields candidates based on higher replay/write/flush LSN. .. note:: - Tuples are reverse ordered by sync_state and LSN fields so nodes that already synchronous or having + Tuples are reverse ordered by ``sync_state`` and LSN fields so nodes that already synchronous or having higher LSN values are preferred. :param cluster: current cluster topology from DCS. :yields: tuples composed of: - * pid - a PID of walsender process - * member name - matches with the application_name - * sync_state - one of ("async", "potential", "quorum", "sync") - * LSN - write_lsn, flush_lsn, or replica_lsn, depending on the value of ``synchronous_commit`` GUC - * nofailover - whether the member has ``nofailover`` tag set + * ``pid`` - PID of the walsender process + * ``member name`` - matches with the ``application_name``` + * ``sync_state`` - one of (``async``, ``potential``, ``quorum``, ``sync``) + * ``LSN`` - write_lsn, flush_lsn, or replica_lsn, depending on the value of ``synchronous_commit`` GUC + * ``nofailover`` - whether the member has ``nofailover`` tag set """ # What column from pg_stat_replication we want to sort on? Choose based on ``synchronous_commit`` value. @@ -262,7 +262,7 @@ def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, i yield (pid, member.name, sync_state, replica_lsn, bool(member.nofailover)) def _process_replica_readiness(self, cluster: Cluster, replica_list: List[Tuple[int, str, str, int, bool]]) -> None: - """Flags replicas as truely "synchronous" when they caught up with "_primary_flush_lsn".""" + """Flags replicas as truly "synchronous" when they caught up with "_primary_flush_lsn".""" if TYPE_CHECKING: # pragma: no cover assert self._postgresql.global_config is not None for pid, app_name, sync_state, replica_lsn, _ in replica_list: diff --git a/patroni/utils.py b/patroni/utils.py index f976df3a6..3916484a5 100644 --- a/patroni/utils.py +++ b/patroni/utils.py @@ -769,7 +769,7 @@ def cluster_as_json(cluster: 'Cluster', global_config: Optional['GlobalConfig'] for m in cluster.members: if m.name == leader_name: role = 'standby_leader' if global_config.is_standby_cluster else 'leader' - elif cluster.sync.matches(m.name, global_config.is_quorum_commit_mode): + elif cluster.sync.matches(m.name): role = sync_role else: role = 'replica' From 3bf7095eadf15ae0228d62a6c445638b9b24ec46 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Wed, 26 Jul 2023 07:56:32 +0200 Subject: [PATCH 19/37] Address review feedback --- patroni/postgresql/sync.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index 8716ad8cf..f37a8febd 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -226,7 +226,8 @@ def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, i .. note:: Tuples are reverse ordered by ``sync_state`` and LSN fields so nodes that already synchronous or having - higher LSN values are preferred. + higher LSN values are preferred. Replicas that are streaming, but don't have ``running`` ``state`` + or tagged with ``nofailover`` tag in DCS are skipped. :param cluster: current cluster topology from DCS. @@ -262,7 +263,12 @@ def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, i yield (pid, member.name, sync_state, replica_lsn, bool(member.nofailover)) def _process_replica_readiness(self, cluster: Cluster, replica_list: List[Tuple[int, str, str, int, bool]]) -> None: - """Flags replicas as truly "synchronous" when they caught up with "_primary_flush_lsn".""" + """Flags replicas as truly "synchronous" when they caught up with "_primary_flush_lsn". + + :param cluster: current cluster topology from DCS + :param replica_list: the list of tuples returned from :func:``_get_replica_list`` method + (represents replication connections) that we want to evaluate. + """ if TYPE_CHECKING: # pragma: no cover assert self._postgresql.global_config is not None for pid, app_name, sync_state, replica_lsn, _ in replica_list: From a48ef036ea40b6e69f438f02339a36f96b67157f Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Wed, 26 Jul 2023 07:58:29 +0200 Subject: [PATCH 20/37] Please pyright --- patroni/quorum.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/patroni/quorum.py b/patroni/quorum.py index 2cb4539cf..5b97b1fbf 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -151,7 +151,7 @@ def __iter__(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: # Merge 2 transitions of the same type to a single one. This is always safe because skipping the first # transition is equivalent to no one observing the intermediate state. for cur_transition, next_transition in zip(transitions, transitions[1:] + [None]): - if next_transition and cur_transition[0] == next_transition[0]: + if isinstance(next_transition, tuple) and cur_transition[0] == next_transition[0]: continue yield cur_transition if cur_transition[0] == 'restart': From e2805fddb412403588a836155e39a50d655411ba Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Mon, 31 Jul 2023 07:07:35 +0200 Subject: [PATCH 21/37] Apply suggestions from code review Co-authored-by: Matt Baker <93600443+matthbakeredb@users.noreply.github.com> --- features/steps/quorum_commit.py | 1 + patroni/ha.py | 4 ++-- patroni/postgresql/sync.py | 25 +++++++++++++------------ 3 files changed, 16 insertions(+), 14 deletions(-) diff --git a/features/steps/quorum_commit.py b/features/steps/quorum_commit.py index 19a887a7b..48fc4b623 100644 --- a/features/steps/quorum_commit.py +++ b/features/steps/quorum_commit.py @@ -46,6 +46,7 @@ def check_synchronous_standby_names(context, name, value, time_limit): expected_num, expected_value = _parse_synchronous_standby_names(value) + ssn = None while time.time() < max_time: try: ssn = context.pctl.query(name, "SHOW synchronous_standby_names").fetchone()[0] diff --git a/patroni/ha.py b/patroni/ha.py index 8e32429a2..d4a6b6de6 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -766,9 +766,9 @@ def process_sync_replication(self) -> None: def process_sync_replication_prepromote(self) -> bool: """Handle sync replication state before promote. - If quorum replication is requested and we can keep syncing to enough nodes satisfying the quorum invariant + If quorum replication is requested, and we can keep syncing to enough nodes satisfying the quorum invariant we can promote immediately and let normal quorum resolver process handle any membership changes later. - Otherwise we will just reset DCS state to ourselves and add replicas as they connect. + Otherwise, we will just reset DCS state to ourselves and add replicas as they connect. :returns: `True` if on success or `False` if failed to update /sync key in DCS. """ diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index f37a8febd..96cdf8fbf 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -225,9 +225,9 @@ def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, i """Yields candidates based on higher replay/write/flush LSN. .. note:: - Tuples are reverse ordered by ``sync_state`` and LSN fields so nodes that already synchronous or having - higher LSN values are preferred. Replicas that are streaming, but don't have ``running`` ``state`` - or tagged with ``nofailover`` tag in DCS are skipped. + Tuples are ordered by ``sync_state`` and LSN fields in reverse, so nodes that are already synchronous or + have higher LSN values are preferred. Replicas that are streaming, but don't have a ``running`` ``state`` + or are tagged with ``nofailover`` tag in DCS, are skipped. :param cluster: current cluster topology from DCS. @@ -236,7 +236,8 @@ def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, i * ``pid`` - PID of the walsender process * ``member name`` - matches with the ``application_name``` * ``sync_state`` - one of (``async``, ``potential``, ``quorum``, ``sync``) - * ``LSN`` - write_lsn, flush_lsn, or replica_lsn, depending on the value of ``synchronous_commit`` GUC + * ``LSN`` - ``write_lsn``, ``flush_lsn``, or ``replica_lsn``, depending on the value of + ``synchronous_commit`` GUC * ``nofailover`` - whether the member has ``nofailover`` tag set """ @@ -260,10 +261,10 @@ def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, i for pid, app_name, sync_state, replica_lsn in sorted(pg_stat_replication, key=lambda r: r[2:4], reverse=True): member = members.get(app_name) if member and member.is_running and not member.tags.get('nosync', False): - yield (pid, member.name, sync_state, replica_lsn, bool(member.nofailover)) + yield pid, member.name, sync_state, replica_lsn, bool(member.nofailover) def _process_replica_readiness(self, cluster: Cluster, replica_list: List[Tuple[int, str, str, int, bool]]) -> None: - """Flags replicas as truly "synchronous" when they caught up with "_primary_flush_lsn". + """Flags replicas as truly "synchronous" when they have caught up with "_primary_flush_lsn". :param cluster: current cluster topology from DCS :param replica_list: the list of tuples returned from :func:``_get_replica_list`` method @@ -293,12 +294,12 @@ def current_state(self, cluster: Cluster) -> _SyncState: Standbys are selected based on values from the global configuration: - - `maximum_lag_on_syncnode`: would help swapping unhealthy sync replica in case if it stops - responding (or hung). Please set the value high enough so it won't unncessarily swap sync - standbys during high loads. Any value less or equal of 0 keeps the behavior backward compatible. - Please note that it will not also swap sync standbys in case where all replicas are hung. + - `maximum_lag_on_syncnode`: would help swapping unhealthy sync replica in case it stops + responding (or hung). Please set the value high enough, so it won't unnecessarily swap sync + standbys during high loads. Any value less or equal to 0 keeps the behavior backwards compatible. + Please note that it will also not swap sync standbys when all replicas are hung. - - `synchronous_node_count`: controlls how many nodes should be set as synchronous. + - `synchronous_node_count`: controls how many nodes should be set as synchronous. :param cluster: current cluster topology from DCS @@ -379,7 +380,7 @@ def set_synchronous_standby_names(self, sync: Collection[str], num: Optional[int self._postgresql.supports_multiple_sync and len(sync) > 1: prefix = 'ANY ' if self._postgresql.global_config.is_quorum_commit_mode\ and self._postgresql.supports_quorum_commit else '' - sync_param = '{0}{1} ({2})'.format(prefix, num, sync_param) + sync_param = f'{prefix}{num} ({sync_param})' if not (self._postgresql.config.set_synchronous_standby_names(sync_param) and self._postgresql.state == 'running' and self._postgresql.is_leader()) or has_asterisk: From 538d621fed799851251bd2806aef77508b363101 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Mon, 31 Jul 2023 07:12:54 +0200 Subject: [PATCH 22/37] Address review feedback --- patroni/ha.py | 10 +++++----- patroni/postgresql/sync.py | 2 +- tests/test_ha.py | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/patroni/ha.py b/patroni/ha.py index d4a6b6de6..4b21073b8 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -147,7 +147,7 @@ def __init__(self, patroni: Patroni): self._is_leader_lock = RLock() self._failsafe = Failsafe(patroni.dcs) self._was_paused = False - self._promote_time = 0 + self._promote_timestamp = 0 self._leader_timeline = None self.recovering = False self._async_response = CriticalTask() @@ -197,7 +197,7 @@ def set_is_leader(self, value: bool) -> None: with self._is_leader_lock: self._is_leader = time.time() + self.dcs.ttl if value else 0 if not value: - self._promote_time = 0 + self._promote_timestamp = 0 def load_cluster_from_dcs(self) -> None: cluster = self.dcs.get_cluster() @@ -754,10 +754,10 @@ def process_sync_replication(self) -> None: # be postponed for `loop_wait` seconds, to give a chance to some replicas to start streaming. # In opposite case the /sync key will end up without synchronous nodes. if self.state_handler.is_leader(): - if self._promote_time == 0 or time.time() - self._promote_time > self.dcs.loop_wait: + if self._promote_timestamp == 0 or time.time() - self._promote_timestamp > self.dcs.loop_wait: self._process_quorum_replication() - if self._promote_time == 0: - self._promote_time = time.time() + if self._promote_timestamp == 0: + self._promote_timestamp = time.time() elif self.is_synchronous_mode(): self._process_multisync_replication() else: diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index 96cdf8fbf..b82f69990 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -236,7 +236,7 @@ def _get_replica_list(self, cluster: Cluster) -> Iterator[Tuple[int, str, str, i * ``pid`` - PID of the walsender process * ``member name`` - matches with the ``application_name``` * ``sync_state`` - one of (``async``, ``potential``, ``quorum``, ``sync``) - * ``LSN`` - ``write_lsn``, ``flush_lsn``, or ``replica_lsn``, depending on the value of + * ``LSN`` - ``write_lsn``, ``flush_lsn``, or ``replica_lsn``, depending on the value of ``synchronous_commit`` GUC * ``nofailover`` - whether the member has ``nofailover`` tag set """ diff --git a/tests/test_ha.py b/tests/test_ha.py index c78c3e9ab..bea622b88 100644 --- a/tests/test_ha.py +++ b/tests/test_ha.py @@ -1516,7 +1516,7 @@ def test_process_quorum_replication(self): self.assertEqual(mock_write_sync.call_args_list[0][1], {'version': None}) self.assertEqual(mock_set_sync.call_count, 0) - self.ha._promote_time = 1 + self.ha._promote_timestamp = 1 mock_write_sync = self.ha.dcs.write_sync_state = Mock(side_effect=[SyncState.empty(), None]) # Test /sync key is attempted to set and succeed when missing or invalid with patch.object(SyncState, 'is_empty', Mock(side_effect=[True, False])): From c7fbd3572be32e146625f53cc47419d6caea8d12 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 17 Aug 2023 09:47:53 +0200 Subject: [PATCH 23/37] Apply suggestions from code review Co-authored-by: Israel --- patroni/postgresql/sync.py | 14 +++++----- patroni/quorum.py | 54 ++++++++++++++++++++------------------ 2 files changed, 35 insertions(+), 33 deletions(-) diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index 7a0668947..815afe564 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -158,11 +158,11 @@ class _SyncState(NamedTuple): :ivar sync_type: possible values: 'off', 'priority', 'quorum' :ivar numsync: how many nodes are required to be synchronous (according to ``synchronous_standby_names``). - Is ``0`` in case if ``synchronous_standby_names`` value is invalid or has ``*``. + Is ``0`` if ``synchronous_standby_names`` value is invalid or contains ``*``. :ivar numsync_confirmed: how many nodes are known to be synchronous according to the ``pg_stat_replication`` view. Only nodes that caught up with the ``SyncHandler._primary_flush_lsn` are counted. :ivar sync: collection of synchronous node names. In case of quorum commit all nodes listed - in ``synchronous_standby_names`` or nodes that are confirmed to be synchronous according + in ``synchronous_standby_names``, otherwise nodes that are confirmed to be synchronous according to the `pg_stat_replication` view. :ivar active: collection of node names that are streaming and have no restrictions to become synchronous. """ @@ -236,7 +236,7 @@ def __init__(self, postgresql: 'Postgresql', cluster: Cluster) -> None: # Prefer replicas that are in state ``sync`` and with higher values of ``write``/``flush``/``replay`` LSN. self.sort(key=lambda r: (r.nofailover, r.sync_state, r.lsn), reverse=True) - # When checking *maximum_lag_on_syncnode* we want to compare with the most + # When checking ``maximum_lag_on_syncnode`` we want to compare with the most # up-to-date replica or with cluster LSN if there is only one replica. self.max_lsn = max(self, key=lambda x: x.lsn).lsn if len(self) > 1 else postgresql.last_operation() @@ -324,12 +324,12 @@ def current_state(self, cluster: Cluster) -> _SyncState: Standbys are selected based on values from the global configuration: - - `maximum_lag_on_syncnode`: would help swapping unhealthy sync replica in case it stops + - ``maximum_lag_on_syncnode``: would help swapping unhealthy sync replica in case it stops responding (or hung). Please set the value high enough, so it won't unnecessarily swap sync - standbys during high loads. Any value less or equal to 0 keeps the behavior backwards compatible. + standbys during high loads. Any value less or equal to ``0`` keeps the behavior backwards compatible. Please note that it will also not swap sync standbys when all replicas are hung. - - `synchronous_node_count`: controls how many nodes should be set as synchronous. + - ``synchronous_node_count``: controls how many nodes should be set as synchronous. :param cluster: current cluster topology from DCS @@ -378,7 +378,7 @@ def current_state(self, cluster: Cluster) -> _SyncState: active) def set_synchronous_standby_names(self, sync: Collection[str], num: Optional[int] = None) -> None: - """Constructs and sets "synchronous_standby_names" GUC value. + """Constructs and sets ``synchronous_standby_names`` GUC value. :param sync: set of nodes to sync to :param num: specifies number of nodes to sync to. The *num* is set only in case if quorum commit is enabled diff --git a/patroni/quorum.py b/patroni/quorum.py index 5b97b1fbf..c724c69b4 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -15,38 +15,38 @@ class QuorumStateResolver(object): """Calculates a list of state transition tuples of the form `('sync'/'quorum'/'restart',leader,number,set_of_names)` Synchronous replication state is set in two places. PostgreSQL configuration sets how many and which nodes are - needed for a commit to succeed, abbreviated as `numsync` and `sync` set here. DCS contains information about how - many and which nodes need to be interrogated to be sure to see an xlog position containing latest confirmed commit, - abbreviated as `quorum` and `voters` set. Both pairs have the meaning "ANY n OF set". + needed for a commit to succeed, abbreviated as ``numsync`` and ``sync`` set here. DCS contains information about how + many and which nodes need to be interrogated to be sure to see an wal position containing latest confirmed commit, + abbreviated as ``quorum`` and ``voters`` set. Both pairs have the meaning "ANY n OF set". - The number of nodes needed for commit to succeed, `numsync`, is also called the replication factor. + The number of nodes needed for commit to succeed, ``numsync``, is also called the replication factor. - To guarantee zero lost transactions on failover we need to keep the invariant that at all times any subset of + To guarantee zero transaction loss on failover we need to keep the invariant that at all times any subset of nodes that can acknowledge a commit overlaps with any subset of nodes that can achieve quorum to promote a new leader. Given a desired replication factor and a set of nodes able to participate in sync replication there - is one optimal state satisfying this condition. Given the node set `active`, the optimal state is: + is one optimal state satisfying this condition. Given the node set ``active``, the optimal state is: sync = voters = active numsync = min(sync_wanted, len(active)) quorum = len(active) - numsync We need to be able to produce a series of state changes that take the system to this desired state from any - other state arbitrary given arbitrary changes is node availability, configuration and interrupted transitions. + other arbitrary state given arbitrary changes is node availability, configuration and interrupted transitions. - To keep the invariant the rule to follow is that when increasing `numsync` or `quorum`, we need to perform the + To keep the invariant the rule to follow is that when increasing ``numsync`` or ``quorum``, we need to perform the increasing operation first. When decreasing either, the decreasing operation needs to be performed later. - Order of adding or removing nodes from sync and voters depends on the state of synchronous_standby_names: + Order of adding or removing nodes from ``sync`` and ``voters`` depends on the state of ``synchronous_standby_names``: When adding new nodes: - if sync (synchronous_standby_names) is empty: - add new nodes first to sync and then to voters when numsync_confirmed > 0 + if ``sync`` (``synchronous_standby_names``) is empty: + add new nodes first to ``sync`` and then to ``voters`` when ``numsync_confirmed`` > ``0`` else: - add new nodes first to voters and than to sync + add new nodes first to ``voters`` and then to ``sync`` When removing nodes: - if sync (synchronous_standby_names) will become empty after removal: - first remove nodes from voters and than from sync + if ``sync`` (``synchronous_standby_names``) will become empty after removal: + first remove nodes from ``voters`` and then from ``sync`` else: - first remove nodes from sync and than from voters. make voters empty if numsync_confirmed == 0""" + first remove nodes from ``sync`` and then from ``voters``. Make ``voters`` empty if ``numsync_confirmed`` == ``0``""" def __init__(self, leader: str, quorum: int, voters: Collection[str], numsync: int, sync: Collection[str], numsync_confirmed: int, @@ -64,9 +64,10 @@ def __init__(self, leader: str, quorum: int, voters: Collection[str], self.leader_wanted = leader_wanted # The desired leader def check_invariants(self) -> None: - """Checks invatiant of synchronous_standby_names and /sync key in DCS. + """Checks invatiant of ``synchronous_standby_names`` and ``/sync`` key in DCS. - :raises `QuorumError`: in case of broken state""" + :raises: + :exc:`QuorumError`: in case of broken state""" voters = CaseInsensitiveSet(self.voters | CaseInsensitiveSet([self.leader])) sync = CaseInsensitiveSet(self.sync | CaseInsensitiveSet([self.leader_wanted])) @@ -85,17 +86,18 @@ def quorum_update(self, quorum: int, voters: CaseInsensitiveSet, leader: Optiona adjust_quorum: Optional[bool] = True) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: """Updates quorum, voters and optionally leader fields. - :param quorum: the new value for `self.quorum`, could be adjusted depending - on values of `self.numsync_confirmed` and `adjust_quorum` - :param voters: the new value for `self.voters`, could be adjusted if numsync_confirmed == 0 - :param leader: the new value for `self.leader`, optional - :param adjust_quorum: if set to `True` the quorum requirement will be increased by the - difference between `self.numsync` and ``self.numsync_confirmed` - :rtype: Iterator[tuple(type, leader, quorum, voters)] with the new quorum state, - where type could be 'quorum' or 'restart'. The latter means that + :param quorum: the new value for :attr:`quorum`, could be adjusted depending + on values of :attr:`numsync_confirmed` and *adjust_quorum* + :param voters: the new value for :attr:`voters`, could be adjusted if :attr:`numsync_confirmed` == ``0`` + :param leader: the new value for :attr:`leader`, optional + :param adjust_quorum: if set to ``True`` the quorum requirement will be increased by the + difference between :attr:`numsync` and :attr:`numsync_confirmed` + :yields: the new quorum state, + where type could be ``quorum`` or ``restart``. The latter means that quorum could not be updated with the current input data and the :class:`QuorumStateResolver` should be restarted. - :raises `QuorumError`: in case of invalid data or if invariant after transition could not be satisfied + :raises: + :exc:`QuorumError`: in case of invalid data or if invariant after transition could not be satisfied """ if quorum < 0: raise QuorumError("Quorum %d < 0 of (%s)" % (quorum, voters)) From ef8aa21a771023598a1cb792153c61f9577166f8 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 17 Aug 2023 10:26:15 +0200 Subject: [PATCH 24/37] Address code review feedback --- patroni/postgresql/sync.py | 4 ++-- patroni/quorum.py | 13 ++++++++----- patroni/utils.py | 2 +- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index e5814221f..8864a1661 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -234,10 +234,10 @@ def __init__(self, postgresql: 'Postgresql', cluster: Cluster) -> None: row['sync_state'], row[sort_col], bool(member.nofailover))) # Prefer replicas that are in state ``sync`` and with higher values of ``write``/``flush``/``replay`` LSN. - self.sort(key=lambda r: (r.nofailover, r.sync_state, r.lsn), reverse=True) + self.sort(key=lambda r: (r.sync_state, r.lsn), reverse=True) # When checking ``maximum_lag_on_syncnode`` we want to compare with the most - # up-to-date replica or with cluster LSN if there is only one replica. + # up-to-date replica otherwise with cluster LSN if there is only one replica. self.max_lsn = max(self, key=lambda x: x.lsn).lsn if len(self) > 1 else postgresql.last_operation() diff --git a/patroni/quorum.py b/patroni/quorum.py index c724c69b4..e4e5eb55b 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -36,17 +36,20 @@ class QuorumStateResolver(object): To keep the invariant the rule to follow is that when increasing ``numsync`` or ``quorum``, we need to perform the increasing operation first. When decreasing either, the decreasing operation needs to be performed later. - Order of adding or removing nodes from ``sync`` and ``voters`` depends on the state of ``synchronous_standby_names``: + Order of adding or removing nodes from ``sync`` and ``voters`` depends on the state of + ``synchronous_standby_names``. When adding new nodes: if ``sync`` (``synchronous_standby_names``) is empty: - add new nodes first to ``sync`` and then to ``voters`` when ``numsync_confirmed`` > ``0`` + add new nodes first to ``sync`` and then to ``voters`` when ``numsync_confirmed`` > ``0``. else: - add new nodes first to ``voters`` and then to ``sync`` + add new nodes first to ``voters`` and then to ``sync``. When removing nodes: if ``sync`` (``synchronous_standby_names``) will become empty after removal: - first remove nodes from ``voters`` and then from ``sync`` + first remove nodes from ``voters`` and then from ``sync``. else: - first remove nodes from ``sync`` and then from ``voters``. Make ``voters`` empty if ``numsync_confirmed`` == ``0``""" + first remove nodes from ``sync`` and then from ``voters``. + Make ``voters`` empty if ``numsync_confirmed`` == ``0``. + """ def __init__(self, leader: str, quorum: int, voters: Collection[str], numsync: int, sync: Collection[str], numsync_confirmed: int, diff --git a/patroni/utils.py b/patroni/utils.py index da48b3797..ad04b7fae 100644 --- a/patroni/utils.py +++ b/patroni/utils.py @@ -753,7 +753,7 @@ def cluster_as_json(cluster: 'Cluster', global_config: Optional['GlobalConfig'] * ``members``: list of members in the cluster. Each value is a :class:`dict` that may have the following keys: * ``name``: the name of the host (unique in the cluster). The ``members`` list is sorted by this key; - * ``role``: ``leader``, ``standby_leader``, ``sync_standby``, or ``replica``; + * ``role``: ``leader``, ``standby_leader``, ``quorum_standby``, ``sync_standby``, or ``replica``; * ``state``: ``stopping``, ``stopped``, ``stop failed``, ``crashed``, ``running``, ``starting``, ``start failed``, ``restarting``, ``restart failed``, ``initializing new cluster``, ``initdb failed``, ``running custom bootstrap script``, ``custom bootstrap failed``, or ``creating replica``; From ce7fce3f2527bcb234639503e30548950485ea4c Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 17 Aug 2023 13:14:08 +0200 Subject: [PATCH 25/37] Please sphinx --- patroni/quorum.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/patroni/quorum.py b/patroni/quorum.py index e4e5eb55b..40d338ddc 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -27,7 +27,9 @@ class QuorumStateResolver(object): is one optimal state satisfying this condition. Given the node set ``active``, the optimal state is: sync = voters = active + numsync = min(sync_wanted, len(active)) + quorum = len(active) - numsync We need to be able to produce a series of state changes that take the system to this desired state from any @@ -39,11 +41,14 @@ class QuorumStateResolver(object): Order of adding or removing nodes from ``sync`` and ``voters`` depends on the state of ``synchronous_standby_names``. When adding new nodes: + if ``sync`` (``synchronous_standby_names``) is empty: add new nodes first to ``sync`` and then to ``voters`` when ``numsync_confirmed`` > ``0``. else: add new nodes first to ``voters`` and then to ``sync``. + When removing nodes: + if ``sync`` (``synchronous_standby_names``) will become empty after removal: first remove nodes from ``voters`` and then from ``sync``. else: From 4d26435244ca8817f8537845b49a45a84630af8c Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Wed, 23 Aug 2023 08:28:01 +0200 Subject: [PATCH 26/37] Address review feedback --- patroni/api.py | 1 + patroni/quorum.py | 203 ++++++++++++++++++++++++++++++++-------------- 2 files changed, 143 insertions(+), 61 deletions(-) diff --git a/patroni/api.py b/patroni/api.py index 474a17998..1ac98696f 100644 --- a/patroni/api.py +++ b/patroni/api.py @@ -1248,6 +1248,7 @@ def get_postgresql_status(self, retry: bool = False) -> Dict[str, Any]: * ``paused``: ``pg_is_wal_replay_paused()``; * ``sync_standby``: ``True`` if replication mode is synchronous and this is a sync standby; + * ``quorum_standby``: ``True`` if replication mode is quorum and this is a quorum standby; * ``timeline``: PostgreSQL primary node timeline; * ``replication``: :class:`list` of :class:`dict` entries, one for each replication connection. Each entry contains the following keys: diff --git a/patroni/quorum.py b/patroni/quorum.py index 40d338ddc..086c5d528 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -1,14 +1,38 @@ import logging -from typing import Collection, Iterator, Optional, Tuple +from typing import Collection, Iterator, NamedTuple, Optional from .collections import CaseInsensitiveSet logger = logging.getLogger(__name__) +class Transition(NamedTuple): + """Object describing transition of ``/sync`` or ``synchronous_standby_names`` to the new state. + + .. note:: + Object attributes represent the new state. + + :ivar transition_type: possible values: + + * ``sync`` - indicates that we needed to update ``synchronous_standby_names``. + * ``quorum`` - indicates that we need to update ``/sync`` key in DCS. + * ``restart`` - caller should stop iterating over transitions and restart :class:`QuorumStateResolver`. + :ivar leader: the new value of the ``leader`` field in the ``/sync`` key. + :ivar num: the new value of the synchronous nodes in ``synchronous_standby_names`` or value of the ``quorum`` + field in the ``/sync`` key for *transition_type* values ``sync`` and ``quorum`` respectively. + :ivar names: the new value of node names listed in ``synchronous_standby_names`` or value of ``voters`` + field in the ``/sync`` key for *transition_type* values ``sync`` and ``quorum`` respectively. + """ + + transition_type: str + leader: str + num: int + names: CaseInsensitiveSet + + class QuorumError(Exception): - pass + """Exception indicating that the quorum state is broken.""" class QuorumStateResolver(object): @@ -54,22 +78,52 @@ class QuorumStateResolver(object): else: first remove nodes from ``sync`` and then from ``voters``. Make ``voters`` empty if ``numsync_confirmed`` == ``0``. + + :ivar leader: name of the leader, according to the ``/sync`` key. + :ivar quorum: ``quorum`` value from the ``/sync`` key, the minimal number of nodes we need see + when doing the leader race. + :ivar voters: ``sync_standby`` value from the ``/sync`` key, set of node names we will be + running the leader race against. + :ivar numsync: the number of synchronous nodes from the ``synchronous_standby_names``. + :ivar sync: set of node names listed in the ``synchronous_standby_names``. + :ivar numsync_confirmed: the number of nodes that are confirmed to reach "safe" LSN after they were added to the + ``synchronous_standby_names``. + :ivar active: set of node names that are replicating from the primary (according to ``pg_stat_replication``) + and are eligible to be listed in ``synchronous_standby_names``. + :ivar sync_wanted: desired number of synchronous nodes (``synchronous_node_count`` from the global configuration). + :ivar leader_wanted: the desired leader (could be different from the *leader* right after a failover). """ def __init__(self, leader: str, quorum: int, voters: Collection[str], numsync: int, sync: Collection[str], numsync_confirmed: int, active: Collection[str], sync_wanted: int, leader_wanted: str) -> None: - self.leader = leader # The leader according to the `/sync` key - self.quorum = quorum # The number of nodes we need to check when doing leader race - self.voters = CaseInsensitiveSet(voters) # Set of nodes we need to check (both stored in the /sync key) - self.numsync = min(numsync, len(sync)) # The number of sync nodes in synchronous_standby_names - self.sync = CaseInsensitiveSet(sync) # Set of nodes in synchronous_standby_names - # The number of nodes that are confirmed to reach safe LSN after adding them to `synchronous_standby_names`. - # We don't list them because it is known that they are always included into active. + """Instantiate :class:``QuorumStateResolver`` based on input parameters. + + :param leader: name of the leader, according to the ``/sync`` key. + :param quorum: ``quorum`` value from the ``/sync`` key, the minimal number of nodes we need see + when doing the leader race. + :param voters: ``sync_standby`` value from the ``/sync`` key, set of node names we will be + running the leader race against. + :param numsync: the number of synchronous nodes from the ``synchronous_standby_names``. + :param sync: Set of node names listed in the ``synchronous_standby_names``. + :param numsync_confirmed: the number of nodes that are confirmed to reach "safe" LSN after + they were added to the ``synchronous_standby_names``. + :param active: set of node names that are replicating from the primary (according to ``pg_stat_replication``) + and are eligible to be listed in ``synchronous_standby_names``. + :param sync_wanted: desired number of synchronous nodes + (``synchronous_node_count`` from the global configuration). + :param leader_wanted: the desired leader (could be different from the *leader* right after a failover). + + """ + self.leader = leader + self.quorum = quorum + self.voters = CaseInsensitiveSet(voters) + self.numsync = min(numsync, len(sync)) # numsync can't be bigger than number of listed synchronous nodes. + self.sync = CaseInsensitiveSet(sync) self.numsync_confirmed = numsync_confirmed - self.active = CaseInsensitiveSet(active) # Set of active nodes from `pg_stat_replication` - self.sync_wanted = sync_wanted # The desired number of sync nodes - self.leader_wanted = leader_wanted # The desired leader + self.active = CaseInsensitiveSet(active) + self.sync_wanted = sync_wanted + self.leader_wanted = leader_wanted def check_invariants(self) -> None: """Checks invatiant of ``synchronous_standby_names`` and ``/sync`` key in DCS. @@ -91,26 +145,25 @@ def check_invariants(self) -> None: (voters - sync, sync - voters)) def quorum_update(self, quorum: int, voters: CaseInsensitiveSet, leader: Optional[str] = None, - adjust_quorum: Optional[bool] = True) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + adjust_quorum: Optional[bool] = True) -> Iterator[Transition]: """Updates quorum, voters and optionally leader fields. :param quorum: the new value for :attr:`quorum`, could be adjusted depending - on values of :attr:`numsync_confirmed` and *adjust_quorum* - :param voters: the new value for :attr:`voters`, could be adjusted if :attr:`numsync_confirmed` == ``0`` - :param leader: the new value for :attr:`leader`, optional + on values of :attr:`numsync_confirmed` and *adjust_quorum*. + :param voters: the new value for :attr:`voters`, could be adjusted if :attr:`numsync_confirmed` == ``0``. + :param leader: the new value for :attr:`leader`, optional. :param adjust_quorum: if set to ``True`` the quorum requirement will be increased by the - difference between :attr:`numsync` and :attr:`numsync_confirmed` - :yields: the new quorum state, - where type could be ``quorum`` or ``restart``. The latter means that - quorum could not be updated with the current input data - and the :class:`QuorumStateResolver` should be restarted. + difference between :attr:`numsync` and :attr:`numsync_confirmed`. + + :yields: the new state of the ``/sync`` key as a :class:`Transition` object. + :raises: - :exc:`QuorumError`: in case of invalid data or if invariant after transition could not be satisfied + :exc:`QuorumError` in case of invalid data or if the invariant after transition could not be satisfied. """ if quorum < 0: - raise QuorumError("Quorum %d < 0 of (%s)" % (quorum, voters)) + raise QuorumError(f'Quorum {quorum} < 0 of ({voters})') if quorum > 0 and quorum >= len(voters): - raise QuorumError("Quorum %d >= N of (%s)" % (quorum, voters)) + raise QuorumError(f'Quorum {quorum} >= N of ({voters})') old_leader = self.leader if leader is not None: # Change of leader was requested @@ -129,66 +182,80 @@ def quorum_update(self, quorum: int, voters: CaseInsensitiveSet, leader: Optiona return # If transition produces no change of leader/quorum/voters we want to give a hint to # the caller to fetch the new state from the database and restart QuorumStateResolver. - yield 'restart', self.leader, self.quorum, self.voters + yield Transition('restart', self.leader, self.quorum, self.voters) self.quorum = quorum self.voters = voters self.check_invariants() logger.debug('quorum %s %s %s', self.leader, self.quorum, self.voters) - yield 'quorum', self.leader, self.quorum, self.voters + yield Transition('quorum', self.leader, self.quorum, self.voters) + + def sync_update(self, numsync: int, sync: CaseInsensitiveSet) -> Iterator[Transition]: + """Updates :attr:`numsync` and :attr:`sync` fields. + + :param numsync: the new value for :attr:`numsync`. + :param sync: the new value for :attr:`sync`: - def sync_update(self, numsync: int, sync: CaseInsensitiveSet) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: - """Updates numsync and sync fields. + :yields: the new state of ``synchronous_standby_names`` as a :class:`Transition` object. - :param numsync: the new value for `self.numsync` - :param sync: the new value for `self.sync` - :rtype: Iterator[tuple('sync', leader, numsync, sync)] with the new state of synchronous_standby_names - :raises `QuorumError`: in case of invalid data or if invariant after transition could not be satisfied + :raises: + :exc:`QuorumError` in case of invalid data or if invariant after transition could not be satisfied """ if numsync < 0: - raise QuorumError("Sync %d < 0 of (%s)" % (numsync, sync)) + raise QuorumError(f'Sync {numsync} < 0 of ({sync})') if numsync > len(sync): - raise QuorumError("Sync %s > N of (%s)" % (numsync, sync)) + raise QuorumError(f'Sync {numsync} > N of ({sync})') self.numsync = numsync self.sync = sync self.check_invariants() logger.debug('sync %s %s %s', self.leader, self.numsync, self.sync) - yield 'sync', self.leader, self.numsync, self.sync + yield Transition('sync', self.leader, self.numsync, self.sync) + + def __iter__(self) -> Iterator[Transition]: + """Merge two transitions of the same type to a single one. - def __iter__(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + .. note:: + This is always safe because skipping the first transition is equivalent + to no one observing the intermediate state. + + :yields: transitions as :class:`Transition` objects. + """ transitions = list(self._generate_transitions()) - # Merge 2 transitions of the same type to a single one. This is always safe because skipping the first - # transition is equivalent to no one observing the intermediate state. for cur_transition, next_transition in zip(transitions, transitions[1:] + [None]): - if isinstance(next_transition, tuple) and cur_transition[0] == next_transition[0]: + if isinstance(next_transition, Transition) \ + and cur_transition.transition_type == next_transition.transition_type: continue yield cur_transition - if cur_transition[0] == 'restart': + if cur_transition.transition_type == 'restart': break - def __handle_non_steady_cases(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + def __handle_non_steady_cases(self) -> Iterator[Transition]: + """Handle cases when set of transitions produces on previous run was interrupted. + + :yields: transitions as :class:`Transition` objects. + """ if self.sync < self.voters: logger.debug("Case 1: synchronous_standby_names subset of DCS state") - # Case 1: quorum is superset of sync nodes. In the middle of changing quorum. - # Evict from quorum dead nodes that are not being synced. - remove_from_quorum = self.voters - (self.sync | self.active) - if remove_from_quorum: + # Case 1: voters is superset of sync nodes. In the middle of changing voters (quorum). + # Evict dead nodes from voters that are not being synced. + remove_from_voters = self.voters - (self.sync | self.active) + if remove_from_voters: yield from self.quorum_update( - quorum=len(self.voters) - len(remove_from_quorum) - self.numsync, - voters=CaseInsensitiveSet(self.voters - remove_from_quorum), + quorum=len(self.voters) - len(remove_from_voters) - self.numsync, + voters=CaseInsensitiveSet(self.voters - remove_from_voters), adjust_quorum=not (self.sync - self.active)) - # Start syncing to nodes that are in quorum and alive + # Start syncing to nodes that are in voters and alive add_to_sync = (self.voters & self.active) - self.sync if add_to_sync: yield from self.sync_update(self.numsync, CaseInsensitiveSet(self.sync | add_to_sync)) elif self.sync > self.voters: logger.debug("Case 2: synchronous_standby_names superset of DCS state") - # Case 2: sync is superset of quorum nodes. In the middle of changing replication factor. - # Add to quorum voters nodes that are already synced and active - add_to_quorum = (self.sync - self.voters) & self.active - if add_to_quorum: - voters = CaseInsensitiveSet(self.voters | add_to_quorum) + # Case 2: sync is superset of voters nodes. In the middle of changing replication factor (sync). + # Add to voters nodes that are already synced and active + add_to_voters = (self.sync - self.voters) & self.active + if add_to_voters: + voters = CaseInsensitiveSet(self.voters | add_to_voters) yield from self.quorum_update(len(voters) - self.numsync, voters) # Remove from sync nodes that are dead remove_from_sync = self.sync - self.voters @@ -197,7 +264,7 @@ def __handle_non_steady_cases(self) -> Iterator[Tuple[str, str, int, CaseInsensi numsync=min(self.numsync, len(self.sync) - len(remove_from_sync)), sync=CaseInsensitiveSet(self.sync - remove_from_sync)) - # After handling these two cases quorum and sync must match. + # After handling these two cases voters and sync must match. assert self.voters == self.sync safety_margin = self.quorum + min(self.numsync, self.numsync_confirmed) - len(self.voters | self.sync) @@ -213,8 +280,11 @@ def __handle_non_steady_cases(self) -> Iterator[Tuple[str, str, int, CaseInsensi if self.numsync == self.sync_wanted and safety_margin > 0 and self.numsync > self.numsync_confirmed: yield from self.quorum_update(len(self.sync) - self.numsync, self.voters) - def __remove_gone_nodes(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: - # If any nodes have gone away, evict them + def __remove_gone_nodes(self) -> Iterator[Transition]: + """Remove inactive nodes from ``synchronous_standby_names`` and from ``/sync`` key. + + :yields: transitions as :class:`Transition` objects. + """ to_remove = self.sync - self.active if to_remove and self.sync == to_remove: logger.debug("Removing nodes: %s", to_remove) @@ -245,8 +315,11 @@ def __remove_gone_nodes(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSe yield from self.quorum_update(quorum, voters, adjust_quorum=False) yield from self.sync_update(numsync, sync) - def __add_new_nodes(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: - # If any new nodes, join them to quorum + def __add_new_nodes(self) -> Iterator[Transition]: + """Add new active nodes to ``synchronous_standby_names`` and to ``/sync`` key. + + :yields: transitions as :class:`Transition` objects. + """ to_add = self.active - self.sync if to_add: # First get to requested replication factor @@ -269,7 +342,11 @@ def __add_new_nodes(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: adjust_quorum=sync_wanted > self.numsync_confirmed) yield from self.sync_update(sync_wanted, CaseInsensitiveSet(self.sync | to_add)) - def __handle_replication_factor_change(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + def __handle_replication_factor_change(self) -> Iterator[Transition]: + """Handle change of the replication factor (:attr:`sync_wanted`, aka ``synchronous_node_count``). + + :yields: transitions as :class:`Transition` objects. + """ # Apply requested replication factor change sync_increase = min(self.sync_wanted, len(self.sync)) - self.numsync if sync_increase > 0: @@ -285,13 +362,17 @@ def __handle_replication_factor_change(self) -> Iterator[Tuple[str, str, int, Ca adjust_quorum=self.sync_wanted > self.numsync_confirmed) yield from self.sync_update(self.numsync + sync_increase, self.sync) - def _generate_transitions(self) -> Iterator[Tuple[str, str, int, CaseInsensitiveSet]]: + def _generate_transitions(self) -> Iterator[Transition]: + """Produce a set of changes to safely transition from the current state to the desired. + + :yields: transitions as :class:`Transition` objects. + """ logger.debug("Quorum state: leader %s quorum %s, voters %s, numsync %s, sync %s, " "numsync_confirmed %s, active %s, sync_wanted %s leader_wanted %s", self.leader, self.quorum, self.voters, self.numsync, self.sync, self.numsync_confirmed, self.active, self.sync_wanted, self.leader_wanted) try: - if self.leader_wanted != self.leader: + if self.leader_wanted != self.leader: # failover voters = (self.voters - CaseInsensitiveSet([self.leader_wanted])) | CaseInsensitiveSet([self.leader]) if not self.sync: # If sync is empty we need to update synchronous_standby_names first From 79b409847d0d55ddbc9495625e068171b717c349 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 24 Aug 2023 11:14:19 +0200 Subject: [PATCH 27/37] Apply suggestions from code review Co-authored-by: Israel --- patroni/postgresql/__init__.py | 2 +- patroni/postgresql/sync.py | 6 ++--- patroni/quorum.py | 45 ++++++++++++++++++++++------------ 3 files changed, 34 insertions(+), 19 deletions(-) diff --git a/patroni/postgresql/__init__.py b/patroni/postgresql/__init__.py index 35ba7ad54..668d2d117 100644 --- a/patroni/postgresql/__init__.py +++ b/patroni/postgresql/__init__.py @@ -170,7 +170,7 @@ def lsn_name(self) -> str: @property def supports_quorum_commit(self) -> bool: - """:returns: `True` if quorum commit is supported by Postgres.""" + """``True`` if quorum commit is supported by Postgres.""" return self._major_version >= 100000 @property diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index d2d41bfd9..b503af43c 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -156,14 +156,14 @@ def parse_sync_standby_names(value: str) -> _SSN: class _SyncState(NamedTuple): """Class representing the current synchronous state. - :ivar sync_type: possible values: 'off', 'priority', 'quorum' + :ivar sync_type: possible values: ``off``, ``priority``, ``quorum`` :ivar numsync: how many nodes are required to be synchronous (according to ``synchronous_standby_names``). Is ``0`` if ``synchronous_standby_names`` value is invalid or contains ``*``. :ivar numsync_confirmed: how many nodes are known to be synchronous according to the ``pg_stat_replication`` - view. Only nodes that caught up with the ``SyncHandler._primary_flush_lsn` are counted. + view. Only nodes that caught up with the :attr:`SyncHandler._primary_flush_lsn` are counted. :ivar sync: collection of synchronous node names. In case of quorum commit all nodes listed in ``synchronous_standby_names``, otherwise nodes that are confirmed to be synchronous according - to the `pg_stat_replication` view. + to the ``pg_stat_replication`` view. :ivar active: collection of node names that are streaming and have no restrictions to become synchronous. """ sync_type: str diff --git a/patroni/quorum.py b/patroni/quorum.py index 086c5d528..2d75f0c08 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -19,10 +19,10 @@ class Transition(NamedTuple): * ``quorum`` - indicates that we need to update ``/sync`` key in DCS. * ``restart`` - caller should stop iterating over transitions and restart :class:`QuorumStateResolver`. :ivar leader: the new value of the ``leader`` field in the ``/sync`` key. - :ivar num: the new value of the synchronous nodes in ``synchronous_standby_names`` or value of the ``quorum`` - field in the ``/sync`` key for *transition_type* values ``sync`` and ``quorum`` respectively. + :ivar num: the new value of the synchronous nodes count in ``synchronous_standby_names`` or value of the ``quorum`` + field in the ``/sync`` key for :attr:`transition_type` values ``sync`` and ``quorum`` respectively. :ivar names: the new value of node names listed in ``synchronous_standby_names`` or value of ``voters`` - field in the ``/sync`` key for *transition_type* values ``sync`` and ``quorum`` respectively. + field in the ``/sync`` key for :attr:`transition_type` values ``sync`` and ``quorum`` respectively. """ transition_type: str @@ -38,17 +38,22 @@ class QuorumError(Exception): class QuorumStateResolver(object): """Calculates a list of state transition tuples of the form `('sync'/'quorum'/'restart',leader,number,set_of_names)` - Synchronous replication state is set in two places. PostgreSQL configuration sets how many and which nodes are - needed for a commit to succeed, abbreviated as ``numsync`` and ``sync`` set here. DCS contains information about how - many and which nodes need to be interrogated to be sure to see an wal position containing latest confirmed commit, - abbreviated as ``quorum`` and ``voters`` set. Both pairs have the meaning "ANY n OF set". + Synchronous replication state is set in two places: - The number of nodes needed for commit to succeed, ``numsync``, is also called the replication factor. + * PostgreSQL configuration sets how many and which nodes are needed for a commit to succeed, abbreviated as + ``numsync`` and ``sync`` set here; + * DCS contains information about how many and which nodes need to be interrogated to be sure to see an wal position + containing latest confirmed commit, abbreviated as ``quorum`` and ``voters`` set. + + .. note:: + Both of above pairs have the meaning "ANY n OF set". + + The number of nodes needed for commit to succeed, ``numsync``, is also called the replication factor. To guarantee zero transaction loss on failover we need to keep the invariant that at all times any subset of nodes that can acknowledge a commit overlaps with any subset of nodes that can achieve quorum to promote a new leader. Given a desired replication factor and a set of nodes able to participate in sync replication there - is one optimal state satisfying this condition. Given the node set ``active``, the optimal state is: + is one optimal state satisfying this condition. Given the node set ``active``, the optimal state is:: sync = voters = active @@ -60,18 +65,25 @@ class QuorumStateResolver(object): other arbitrary state given arbitrary changes is node availability, configuration and interrupted transitions. To keep the invariant the rule to follow is that when increasing ``numsync`` or ``quorum``, we need to perform the - increasing operation first. When decreasing either, the decreasing operation needs to be performed later. + increasing operation first. When decreasing either, the decreasing operation needs to be performed later. In other + words: + + * If a user increases ``synchronous_node_count`` configuration, first we increase ``synchronous_standby_names`` + (``numsync``), then we decrease ``quorum`` field in the ``/sync`` key; + * If a user decreases ``synchronous_node_count`` configuration, first we increase ``quorum`` field in the ``/sync`` + key, then we decrease ``synchronous_standby_names`` (``numsync``). Order of adding or removing nodes from ``sync`` and ``voters`` depends on the state of ``synchronous_standby_names``. - When adding new nodes: + + When adding new nodes:: if ``sync`` (``synchronous_standby_names``) is empty: add new nodes first to ``sync`` and then to ``voters`` when ``numsync_confirmed`` > ``0``. else: add new nodes first to ``voters`` and then to ``sync``. - When removing nodes: + When removing nodes:: if ``sync`` (``synchronous_standby_names``) will become empty after removal: first remove nodes from ``voters`` and then from ``sync``. @@ -91,7 +103,7 @@ class QuorumStateResolver(object): :ivar active: set of node names that are replicating from the primary (according to ``pg_stat_replication``) and are eligible to be listed in ``synchronous_standby_names``. :ivar sync_wanted: desired number of synchronous nodes (``synchronous_node_count`` from the global configuration). - :ivar leader_wanted: the desired leader (could be different from the *leader* right after a failover). + :ivar leader_wanted: the desired leader (could be different from the :attr:`leader` right after a failover). """ def __init__(self, leader: str, quorum: int, voters: Collection[str], @@ -126,8 +138,10 @@ def __init__(self, leader: str, quorum: int, voters: Collection[str], self.leader_wanted = leader_wanted def check_invariants(self) -> None: - """Checks invatiant of ``synchronous_standby_names`` and ``/sync`` key in DCS. + """Checks invariant of ``synchronous_standby_names`` and ``/sync`` key in DCS. + .. seealso:: + Check :class:`QuorumStateResolver`'s docstring for more information. :raises: :exc:`QuorumError`: in case of broken state""" voters = CaseInsensitiveSet(self.voters | CaseInsensitiveSet([self.leader])) @@ -135,6 +149,7 @@ def check_invariants(self) -> None: # We need to verify that subset of nodes that can acknowledge a commit overlaps # with any subset of nodes that can achieve quorum to promote a new leader. + # ``+ 1`` is required because the leader is included in the set. if self.voters and not (len(voters | sync) <= self.quorum + self.numsync + 1): raise QuorumError("Quorum and sync not guaranteed to overlap: nodes %d >= quorum %d + sync %d" % (len(voters | sync), self.quorum, self.numsync)) @@ -146,7 +161,7 @@ def check_invariants(self) -> None: def quorum_update(self, quorum: int, voters: CaseInsensitiveSet, leader: Optional[str] = None, adjust_quorum: Optional[bool] = True) -> Iterator[Transition]: - """Updates quorum, voters and optionally leader fields. + """Updates :attr:`quorum`, :attr:`voters` and optionally :attr:`leader` fields. :param quorum: the new value for :attr:`quorum`, could be adjusted depending on values of :attr:`numsync_confirmed` and *adjust_quorum*. From 3a602f099e26fdba9fdec3d0983bac20edc5053a Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 24 Aug 2023 11:32:37 +0200 Subject: [PATCH 28/37] Address review feedback --- patroni/postgresql/sync.py | 4 ++-- patroni/quorum.py | 9 ++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index b503af43c..067583d76 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -159,8 +159,8 @@ class _SyncState(NamedTuple): :ivar sync_type: possible values: ``off``, ``priority``, ``quorum`` :ivar numsync: how many nodes are required to be synchronous (according to ``synchronous_standby_names``). Is ``0`` if ``synchronous_standby_names`` value is invalid or contains ``*``. - :ivar numsync_confirmed: how many nodes are known to be synchronous according to the ``pg_stat_replication`` - view. Only nodes that caught up with the :attr:`SyncHandler._primary_flush_lsn` are counted. + :ivar numsync_confirmed: how many nodes are known to be synchronous according to the ``pg_stat_replication`` view. + Only nodes that caught up with the :attr:`SyncHandler._primary_flush_lsn` are counted. :ivar sync: collection of synchronous node names. In case of quorum commit all nodes listed in ``synchronous_standby_names``, otherwise nodes that are confirmed to be synchronous according to the ``pg_stat_replication`` view. diff --git a/patroni/quorum.py b/patroni/quorum.py index 2d75f0c08..c510cfcad 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -1,8 +1,10 @@ +"""Implement state machine to manage ``synchronous_standby_names`` GUC and ``/sync`` key in DCS.""" import logging from typing import Collection, Iterator, NamedTuple, Optional from .collections import CaseInsensitiveSet +from .exceptions import PatroniException logger = logging.getLogger(__name__) @@ -31,12 +33,12 @@ class Transition(NamedTuple): names: CaseInsensitiveSet -class QuorumError(Exception): +class QuorumError(PatroniException): """Exception indicating that the quorum state is broken.""" -class QuorumStateResolver(object): - """Calculates a list of state transition tuples of the form `('sync'/'quorum'/'restart',leader,number,set_of_names)` +class QuorumStateResolver: + """Calculates a list of state transitions and yields them as :class:`Transition` named tuples. Synchronous replication state is set in two places: @@ -142,6 +144,7 @@ def check_invariants(self) -> None: .. seealso:: Check :class:`QuorumStateResolver`'s docstring for more information. + :raises: :exc:`QuorumError`: in case of broken state""" voters = CaseInsensitiveSet(self.voters | CaseInsensitiveSet([self.leader])) From 5f65b56045bf6de58de7ea97c431f109703b7620 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Mon, 11 Sep 2023 15:15:32 +0200 Subject: [PATCH 29/37] more f-strings --- patroni/quorum.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/patroni/quorum.py b/patroni/quorum.py index c510cfcad..5c85bd471 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -154,13 +154,15 @@ def check_invariants(self) -> None: # with any subset of nodes that can achieve quorum to promote a new leader. # ``+ 1`` is required because the leader is included in the set. if self.voters and not (len(voters | sync) <= self.quorum + self.numsync + 1): - raise QuorumError("Quorum and sync not guaranteed to overlap: nodes %d >= quorum %d + sync %d" % - (len(voters | sync), self.quorum, self.numsync)) + len_nodes = len(voters | sync) + raise QuorumError("Quorum and sync not guaranteed to overlap: " + f"nodes {len_nodes} >= quorum {self.quorum} + sync {self.sync} + 1") # unstable cases, we are changing synchronous_standby_names and /sync key # one after another, hence one set is allowed to be a subset of another if not (voters.issubset(sync) or sync.issubset(voters)): - raise QuorumError("Mismatched sets: quorum only=%s sync only=%s" % - (voters - sync, sync - voters)) + voters_only = voters - sync + sync_only = sync - voters + raise QuorumError(f"Mismatched sets: voter only={voters_only} sync only={sync_only}") def quorum_update(self, quorum: int, voters: CaseInsensitiveSet, leader: Optional[str] = None, adjust_quorum: Optional[bool] = True) -> Iterator[Transition]: From 61c3d7c80148a0245e138028c18d5e1774ade747 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Mon, 11 Sep 2023 15:54:04 +0200 Subject: [PATCH 30/37] Fix citus.rst --- docs/citus.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/citus.rst b/docs/citus.rst index cb3f9a716..db9bd1afb 100644 --- a/docs/citus.rst +++ b/docs/citus.rst @@ -34,7 +34,7 @@ There are only a few simple rules you need to follow: After that you just need to start Patroni and it will handle the rest: -0. Patroni will set ``bootstrap.dcs.synchronous_mode`` to :ref:`quorum ` if it is not explicitly set to any other value. 1. ``citus`` extension will be automatically added to ``shared_preload_libraries``. 2. If ``max_prepared_transactions`` isn't explicitly set in the global From d1dff7832640fbf5e67355a0c5c2155b36604eb7 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Tue, 12 Sep 2023 09:11:57 +0200 Subject: [PATCH 31/37] Rename methods in unit tests to match names of methods we are testing --- tests/test_ha.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/test_ha.py b/tests/test_ha.py index 89ed41fa7..cadb6eafd 100644 --- a/tests/test_ha.py +++ b/tests/test_ha.py @@ -1290,7 +1290,7 @@ def test_demote_immediate(self, follow): self.ha.demote('immediate') follow.assert_called_once_with(None) - def test_process__multisync_replication(self): + def test__process_multisync_replication(self): self.ha.has_lock = true mock_set_sync = self.p.sync_handler.set_synchronous_standby_names = Mock() self.p.name = 'leader' @@ -1685,7 +1685,7 @@ def test_process_sync_replication_prepromote(self): self.assertEqual(mock_set_sync.call_count, 1) self.assertEqual(mock_set_sync.call_args_list[0][0], ('ANY 3 (foo,other,postgresql0)',)) - def test_process_quorum_replication(self): + def test__process_quorum_replication(self): self.p._major_version = 150000 self.ha.has_lock = true mock_set_sync = self.p.config.set_synchronous_standby_names = Mock() From 73a5c9a245df4cc620570b4ba4b1d6f5dee96c53 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Thu, 14 Sep 2023 16:04:53 +0200 Subject: [PATCH 32/37] Apply suggestions from code review Co-authored-by: Israel --- patroni/quorum.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/patroni/quorum.py b/patroni/quorum.py index 5c85bd471..71e00f9df 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -189,12 +189,12 @@ def quorum_update(self, quorum: int, voters: CaseInsensitiveSet, leader: Optiona if leader is not None: # Change of leader was requested self.leader = leader elif self.numsync_confirmed == 0: - # If there are no nodes that known to caught up with the primary we want to reset quorum/votes in /sync key + # If there are no nodes that known to caught up with the primary we want to reset quorum/voters in /sync key quorum = 0 voters = CaseInsensitiveSet() elif adjust_quorum: # It could be that the number of nodes that are known to catch up with the primary is below desired numsync. - # We want to increase quorum to guaranty that the sync node will be found during the leader race. + # We want to increase quorum to guarantee that the sync node will be found during the leader race. quorum += max(self.numsync - self.numsync_confirmed, 0) if (self.leader, quorum, voters) == (old_leader, self.quorum, self.voters): @@ -233,9 +233,11 @@ def sync_update(self, numsync: int, sync: CaseInsensitiveSet) -> Iterator[Transi yield Transition('sync', self.leader, self.numsync, self.sync) def __iter__(self) -> Iterator[Transition]: - """Merge two transitions of the same type to a single one. + """Iterate over the transitions produced by :meth:`_generate_transitions`. .. note:: + Merge two transitions of the same type to a single one. + This is always safe because skipping the first transition is equivalent to no one observing the intermediate state. From 0f6e0696f8f5d5141850855074f661779c76d94b Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Fri, 15 Sep 2023 12:42:28 +0200 Subject: [PATCH 33/37] Address feedback --- patroni/quorum.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/patroni/quorum.py b/patroni/quorum.py index 5c85bd471..5d7cc75a2 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -256,7 +256,7 @@ def __handle_non_steady_cases(self) -> Iterator[Transition]: :yields: transitions as :class:`Transition` objects. """ if self.sync < self.voters: - logger.debug("Case 1: synchronous_standby_names subset of DCS state") + logger.debug("Case 1: synchronous_standby_names %s is a subset of DCS state %s", self.sync, self.voters) # Case 1: voters is superset of sync nodes. In the middle of changing voters (quorum). # Evict dead nodes from voters that are not being synced. remove_from_voters = self.voters - (self.sync | self.active) @@ -270,7 +270,7 @@ def __handle_non_steady_cases(self) -> Iterator[Transition]: if add_to_sync: yield from self.sync_update(self.numsync, CaseInsensitiveSet(self.sync | add_to_sync)) elif self.sync > self.voters: - logger.debug("Case 2: synchronous_standby_names superset of DCS state") + logger.debug("Case 2: synchronous_standby_names %s is a superset of DCS state %s", self.sync, self.voters) # Case 2: sync is superset of voters nodes. In the middle of changing replication factor (sync). # Add to voters nodes that are already synced and active add_to_voters = (self.sync - self.voters) & self.active @@ -290,11 +290,13 @@ def __handle_non_steady_cases(self) -> Iterator[Transition]: safety_margin = self.quorum + min(self.numsync, self.numsync_confirmed) - len(self.voters | self.sync) if safety_margin > 0: # In the middle of changing replication factor. if self.numsync > self.sync_wanted: - logger.debug('Case 3: replication factor is bigger than needed') - yield from self.sync_update(max(self.sync_wanted, len(self.voters) - self.quorum), self.sync) + numsync = max(self.sync_wanted, len(self.voters) - self.quorum) + logger.debug('Case 3: replication factor %d is bigger than needed %d', self.numsync, numsync) + yield from self.sync_update(numsync, self.sync) else: - logger.debug('Case 4: quorum is bigger than needed') - yield from self.quorum_update(len(self.sync) - self.numsync, self.voters) + quorum = len(self.sync) - self.numsync + logger.debug('Case 4: quorum %d is bigger than needed %d', self.quorum, quorum) + yield from self.quorum_update(quorum, self.voters) else: safety_margin = self.quorum + self.numsync - len(self.voters | self.sync) if self.numsync == self.sync_wanted and safety_margin > 0 and self.numsync > self.numsync_confirmed: From 7794f9c7c825bb22b8042c14c9b7655c2b56fb0d Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Tue, 24 Oct 2023 11:11:01 +0200 Subject: [PATCH 34/37] Apply suggestions from code review Co-authored-by: Israel --- patroni/config.py | 2 +- patroni/dcs/__init__.py | 15 +++++++++------ patroni/postgresql/sync.py | 7 ++++++- 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/patroni/config.py b/patroni/config.py index 88efa0570..99ce58f16 100644 --- a/patroni/config.py +++ b/patroni/config.py @@ -96,7 +96,7 @@ def is_paused(self) -> bool: @property def is_quorum_commit_mode(self) -> bool: - """:returns: `True` if quorum commit replication is requested""" + """:returns: ``True`` if quorum commit replication is requested""" return str(self.get('synchronous_mode')).lower() == 'quorum' @property diff --git a/patroni/dcs/__init__.py b/patroni/dcs/__init__.py index a50f3dce1..36e911217 100644 --- a/patroni/dcs/__init__.py +++ b/patroni/dcs/__init__.py @@ -615,8 +615,9 @@ class SyncState(NamedTuple): :ivar version: modification version of a synchronization key in a Configuration Store. :ivar leader: reference to member that was leader. :ivar sync_standby: synchronous standby list (comma delimited) which are last synchronized to leader. - :ivar quorum: if the node from sync_standby list is doing a leader race it should - see at least quorum other nodes from the sync_standby + leader list. + :ivar quorum: if the node from :attr:`~SyncState.sync_standby` list is doing a leader race it should + see at least :attr:`~SyncState.quorum` other nodes from the + :attr:`~SyncState.sync_standby` + :attr:`~SyncState.leader` list. """ version: Optional[_Version] @@ -1949,8 +1950,9 @@ def sync_state(leader: Optional[str], sync_standby: Optional[Collection[str]], :param leader: name of the leader node that manages ``/sync`` key. :param sync_standby: collection of currently known synchronous standby node names. - :param quorum: if the node from sync_standby list is doing a leader race it should - see at least quorum other nodes from the sync_standby + leader list + :param quorum: if the node from :attr:`~SyncState.sync_standby` list is doing a leader race it should + see at least :attr:`~SyncState.quorum` other nodes from the + :attr:`~SyncState.sync_standby` + :attr:`~SyncState.leader` list :returns: dictionary that later could be serialized to JSON or saved directly to DCS. """ @@ -1967,8 +1969,9 @@ def write_sync_state(self, leader: Optional[str], sync_standby: Optional[Collect :param leader: name of the leader node that manages ``/sync`` key. :param sync_standby: collection of currently known synchronous standby node names. :param version: for conditional update of the key/object. - :param quorum: if the node from sync_standby list is doing a leader race it should - see at least quorum other nodes from the sync_standby + leader list + :param quorum: if the node from :attr:`~SyncState.sync_standby` list is doing a leader race it should + see at least :attr:`~SyncState.quorum` other nodes from the + :attr:`~SyncState.sync_standby` + :attr:`~SyncState.leader` list :returns: the new :class:`SyncState` object or ``None``. """ diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index 067583d76..169f5f002 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -354,7 +354,9 @@ def current_state(self, cluster: Cluster) -> _SyncState: for replica in sorted(replica_list, key=lambda x: x.nofailover): if sync_node_maxlag <= 0 or replica_list.max_lsn - replica.lsn <= sync_node_maxlag: if self._postgresql.global_config.is_quorum_commit_mode: - # add nodes with nofailover tag only to get enough "active" nodes + # We do not add nodes with `nofailover` enabled because that reduces availability. + # We need to check LSN quorum only among nodes that are promotable because + # there is a chance that a non-promotable node is ahead of a promotable one. if not replica.nofailover or len(active) < sync_node_count: if replica.application_name in self._ready_replicas: numsync_confirmed += 1 @@ -379,6 +381,9 @@ def current_state(self, cluster: Cluster) -> _SyncState: def set_synchronous_standby_names(self, sync: Collection[str], num: Optional[int] = None) -> None: """Constructs and sets ``synchronous_standby_names`` GUC value. + + .. note:: + standbys in ``synchronous_standby_names`` will be sorted by name. :param sync: set of nodes to sync to :param num: specifies number of nodes to sync to. The *num* is set only in case if quorum commit is enabled From a9e1d67904fe68309805b5ab0136d26f9cd9f6a2 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Tue, 24 Oct 2023 11:43:26 +0200 Subject: [PATCH 35/37] Update sync.py --- patroni/postgresql/sync.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/patroni/postgresql/sync.py b/patroni/postgresql/sync.py index 169f5f002..0a2aceab0 100644 --- a/patroni/postgresql/sync.py +++ b/patroni/postgresql/sync.py @@ -381,7 +381,7 @@ def current_state(self, cluster: Cluster) -> _SyncState: def set_synchronous_standby_names(self, sync: Collection[str], num: Optional[int] = None) -> None: """Constructs and sets ``synchronous_standby_names`` GUC value. - + .. note:: standbys in ``synchronous_standby_names`` will be sorted by name. From ebdc197f088a3d186dfb664656b741174403b972 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Wed, 25 Oct 2023 11:23:39 +0200 Subject: [PATCH 36/37] Apply suggestions from code review Co-authored-by: Israel --- patroni/ha.py | 12 ++++++------ patroni/quorum.py | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/patroni/ha.py b/patroni/ha.py index b0f5f4768..41eadc10e 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -698,7 +698,7 @@ def is_synchronous_mode(self) -> bool: return self.global_config.is_synchronous_mode def is_quorum_commit_mode(self) -> bool: - """:returns: `True` if quorum commit replication is requested and "supported".""" + """``True`` if quorum commit replication is requested and "supported".""" return self.global_config.is_quorum_commit_mode and self.state_handler.supports_multiple_sync def is_failsafe_mode(self) -> bool: @@ -732,8 +732,8 @@ def disable_synchronous_replication(self) -> None: def _process_quorum_replication(self) -> None: """Process synchronous replication state when quorum commit is requested. - Synchronous standbys are registered in two places postgresql.conf and DCS. The order of updating them must - keep the invariant that `quorum + sync >= len(set(quorum pool)|set(sync pool))`. This is done using + Synchronous standbys are registered in two places: ``postgresql.conf`` and DCS. The order of updating them must + keep the invariant that ``quorum + sync >= len(set(quorum pool)|set(sync pool))``. This is done using :class:`QuorumStateResolver` that given a current state and set of desired synchronous nodes and replication level outputs changes to DCS and synchronous replication in correct order to reach the desired state. In case any of those steps causes an error we can just bail out and let next iteration rediscover the state @@ -876,7 +876,7 @@ def process_sync_replication_prepromote(self) -> bool: we can promote immediately and let normal quorum resolver process handle any membership changes later. Otherwise, we will just reset DCS state to ourselves and add replicas as they connect. - :returns: `True` if on success or `False` if failed to update /sync key in DCS. + :returns: ``True`` if on success or ``False`` if failed to update /sync key in DCS. """ if not self.is_synchronous_mode(): self.disable_synchronous_replication() @@ -1122,7 +1122,7 @@ def is_lagging(self, wal_position: int) -> bool: :param wal_position: Current wal position. - :returns `True` when node is lagging + :returns: ``True`` when node is lagging """ lag = (self.cluster.last_lsn or 0) - wal_position return lag > self.global_config.maximum_lag_on_failover @@ -1133,7 +1133,7 @@ def _is_healthiest_node(self, members: Collection[Member], check_replication_lag :param members: the list of nodes to check against :param check_replication_lag: whether to take the replication lag into account. If the lag exceeds configured threshold the node disqualifies itself. - :returns: `True` if the node is eligible to become the new leader. Since this method is executed + :returns: ``True`` if the node is eligible to become the new leader. Since this method is executed on multiple nodes independently it is possible that multiple nodes could count themselves as the healthiest because they received/replayed up to the same LSN, but this is totally fine. diff --git a/patroni/quorum.py b/patroni/quorum.py index 120c5e0d8..059c8acc7 100644 --- a/patroni/quorum.py +++ b/patroni/quorum.py @@ -253,7 +253,7 @@ def __iter__(self) -> Iterator[Transition]: break def __handle_non_steady_cases(self) -> Iterator[Transition]: - """Handle cases when set of transitions produces on previous run was interrupted. + """Handle cases when set of transitions produced on previous run was interrupted. :yields: transitions as :class:`Transition` objects. """ From 3b367d650ef71e251592586b30902720e8332fd3 Mon Sep 17 00:00:00 2001 From: Alexander Kukushkin Date: Wed, 29 Nov 2023 16:49:43 +0100 Subject: [PATCH 37/37] Revert unexpected change --- patroni/ha.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/patroni/ha.py b/patroni/ha.py index 0519ed06f..a90030f31 100644 --- a/patroni/ha.py +++ b/patroni/ha.py @@ -1939,7 +1939,7 @@ def _run_cycle(self) -> str: global_config.update(self.cluster) self.state_handler.reset_cluster_info_state(self.cluster, self.patroni) except Exception: - self.state_handler.reset_cluster_info_state(None, self.patroni.nofailover) + self.state_handler.reset_cluster_info_state(None) raise if self.is_paused():