[Postgresql] Create replica

Change-Id: Ia00032074dc44a6fbfc1e2d5ab16d1734a1a732c
This commit is contained in:
Lingxian Kong 2020-09-08 23:03:34 +12:00
parent 8761f327fe
commit 5482c54645
22 changed files with 467 additions and 168 deletions

View File

@ -10,6 +10,7 @@ RUN export DEBIAN_FRONTEND="noninteractive" \
RUN apt-get update \
&& apt-get install $APTOPTS gnupg2 lsb-release apt-utils apt-transport-https ca-certificates software-properties-common curl \
&& apt-get -o Dpkg::Options::="--force-confmiss" install --reinstall netbase \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/*
@ -20,6 +21,7 @@ RUN ./install.sh $DATASTORE ${PERCONA_XTRABACKUP_VERSION}
RUN apt-get update \
&& apt-get install $APTOPTS build-essential python3-setuptools python3-all python3-all-dev python3-pip libffi-dev libssl-dev libxml2-dev libxslt1-dev libyaml-dev \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* \
&& pip3 --no-cache-dir install -U -r requirements.txt \
&& curl -sSL https://github.com/Yelp/dumb-init/releases/download/v1.2.2/dumb-init_1.2.2_amd64 -o /usr/local/bin/dumb-init \
&& chmod +x /usr/local/bin/dumb-init

View File

@ -42,7 +42,7 @@ class InnoBackupEx(mysql_base.MySQLBaseRunner):
cmd = ('innobackupex'
' --stream=xbstream'
' --parallel=2 ' +
self.user_and_pass + ' %s' % self.default_data_dir +
self.user_and_pass + ' %s' % self.datadir +
' 2>' + self.backup_log
)
return cmd + self.zip_cmd + self.encrypt_cmd
@ -111,7 +111,7 @@ class InnoBackupExIncremental(InnoBackupEx):
' --stream=xbstream'
' --incremental'
' --incremental-lsn=%(lsn)s ' +
self.user_and_pass + ' %s' % self.default_data_dir +
self.user_and_pass + ' %s' % self.datadir +
' 2>' + self.backup_log)
return cmd + self.zip_cmd + self.encrypt_cmd

View File

@ -53,7 +53,8 @@ class MySQLBaseRunner(base.BaseRunner):
last_line = output.splitlines()[-1].strip()
if not re.search('completed OK!', last_line):
LOG.error("Backup did not complete successfully.")
LOG.error(f"Backup did not complete successfully, last line:\n"
f"{last_line}")
return False
return True

View File

@ -9,23 +9,25 @@ case "$1" in
curl -sSL https://repo.percona.com/apt/percona-release_latest.$(lsb_release -sc)_all.deb -o percona-release.deb
dpkg -i percona-release.deb
percona-release enable-only tools release
apt-get update
apt-get install $APTOPTS percona-xtrabackup-$2
apt-get clean
rm -f percona-release.deb
;;
"mariadb")
apt-key adv --fetch-keys 'https://mariadb.org/mariadb_release_signing_key.asc'
add-apt-repository "deb [arch=amd64] http://mirror2.hs-esslingen.de/mariadb/repo/10.4/ubuntu $(lsb_release -cs) main"
apt-get install $APTOPTS mariadb-backup
apt-get clean
;;
"postgresql")
apt-key adv --fetch-keys 'https://www.postgresql.org/media/keys/ACCC4CF8.asc'
add-apt-repository "deb [arch=amd64] http://apt.postgresql.org/pub/repos/apt/ $(lsb_release -cs)-pgdg main"
apt-get install $APTOPTS postgresql-client-12
apt-get clean
;;
*)
echo "datastore $1 not supported"
exit 1
;;
esac
apt-get clean
rm -rf /var/lib/apt/lists/*

View File

@ -80,19 +80,19 @@ class Commands(object):
def db_load_datastore_config_parameters(self,
datastore,
datastore_version,
datastore_version_name,
config_file_location):
print("Loading config parameters for datastore (%s) version (%s)"
% (datastore, datastore_version))
% (datastore, datastore_version_name))
config_models.load_datastore_configuration_parameters(
datastore, datastore_version, config_file_location)
datastore, datastore_version_name, config_file_location)
def db_remove_datastore_config_parameters(self, datastore,
datastore_version):
datastore_version_name):
print("Removing config parameters for datastore (%s) version (%s)"
% (datastore, datastore_version))
% (datastore, datastore_version_name))
config_models.remove_datastore_configuration_parameters(
datastore, datastore_version)
datastore, datastore_version_name)
def datastore_version_flavor_add(self, datastore_name,
datastore_version_name, flavor_ids):
@ -230,7 +230,7 @@ def main():
'datastore',
help='Name of the datastore.')
parser.add_argument(
'datastore_version',
'datastore_version_name',
help='Name of the datastore version.')
parser.add_argument(
'config_file_location',
@ -245,7 +245,7 @@ def main():
'datastore',
help='Name of the datastore.')
parser.add_argument(
'datastore_version',
'datastore_version_name',
help='Name of the datastore version.')
parser = subparser.add_parser(

View File

@ -249,7 +249,10 @@ common_opts = [
'becomes required in the instance create request.'),
cfg.StrOpt('datastore_manager', default=None,
help='Manager class in the Guest Agent, set up by the '
'Taskmanager on instance provision.'),
'Taskmanager on instance provision.'),
cfg.StrOpt('datastore_version', default=None,
help='The guest datastore version that is set by the '
'Taskmanager during instance provision.'),
cfg.StrOpt('block_device_mapping', default='vdb',
help='Block device to map onto the created instance.'),
cfg.IntOpt('server_delete_time_out', default=60,
@ -1076,13 +1079,16 @@ postgresql_opts = [
help='The TCP port the server listens on.'),
cfg.StrOpt('backup_strategy', default='pg_basebackup',
help='Default strategy to perform backups.'),
cfg.StrOpt('replication_strategy',
default='PostgresqlReplicationStreaming',
help='Default strategy for replication.'),
cfg.StrOpt('replication_namespace',
default='trove.guestagent.strategies.replication.experimental.'
'postgresql_impl',
help='Namespace to load replication strategies from.'),
cfg.StrOpt(
'replication_strategy',
default='PostgresqlReplicationStreaming',
help='Default strategy for replication.'
),
cfg.StrOpt(
'replication_namespace',
default='trove.guestagent.strategies.replication.postgresql',
help='Namespace to load replication strategies from.'
),
cfg.StrOpt('mount_point', default='/var/lib/postgresql',
help="Filesystem path for mounting "
"volumes if volume support is enabled."),

View File

@ -542,13 +542,14 @@ class API(object):
return self._call("get_replica_context",
self.agent_high_timeout, version=version)
def attach_replica(self, replica_info, slave_config):
def attach_replica(self, replica_info, slave_config, restart=False):
LOG.debug("Attaching replica %s.", replica_info)
version = self.API_BASE_VERSION
self._call("attach_replica",
self.agent_high_timeout, version=version,
replica_info=replica_info, slave_config=slave_config)
replica_info=replica_info, slave_config=slave_config,
restart=restart)
def make_read_only(self, read_only):
LOG.debug("Executing make_read_only(%s)", read_only)

View File

@ -115,6 +115,19 @@ class Manager(periodic_task.PeriodicTasks):
"""
return None
@property
def replication(self):
"""If the datastore supports replication, return an instance of
the strategy.
"""
try:
return repl_strategy.get_instance(self.manager)
except Exception as ex:
LOG.warning("Cannot get replication instance for '%(manager)s': "
"%(msg)s", {'manager': self.manager, 'msg': str(ex)})
return None
@property
def replication_strategy(self):
"""If the datastore supports replication, return the strategy."""
@ -825,41 +838,63 @@ class Manager(periodic_task.PeriodicTasks):
################
# Replication related
################
def backup_required_for_replication(self, context):
return self.replication.backup_required_for_replication()
def get_replication_snapshot(self, context, snapshot_info,
replica_source_config=None):
LOG.debug("Getting replication snapshot.")
raise exception.DatastoreOperationNotSupported(
operation='get_replication_snapshot', datastore=self.manager)
LOG.info("Getting replication snapshot, snapshot_info: %s",
snapshot_info)
def attach_replication_slave(self, context, snapshot, slave_config):
LOG.debug("Attaching replication slave.")
self.replication.enable_as_master(self.app, replica_source_config)
LOG.info('Enabled as replication master')
snapshot_id, log_position = self.replication.snapshot_for_replication(
context, self.app, self.adm, None, snapshot_info)
volume_stats = self.get_filesystem_stats(context, None)
replication_snapshot = {
'dataset': {
'datastore_manager': self.manager,
'dataset_size': volume_stats.get('used', 0.0),
'volume_size': volume_stats.get('total', 0.0),
'snapshot_id': snapshot_id
},
'replication_strategy': self.replication_strategy,
'master': self.replication.get_master_ref(self.app, snapshot_info),
'log_position': log_position
}
return replication_snapshot
def attach_replica(self, context, snapshot, slave_config, restart=False):
raise exception.DatastoreOperationNotSupported(
operation='attach_replication_slave', datastore=self.manager)
def detach_replica(self, context, for_failover=False):
LOG.debug("Detaching replica.")
raise exception.DatastoreOperationNotSupported(
operation='detach_replica', datastore=self.manager)
"""Running on replica, detach from the primary."""
LOG.info("Detaching replica.")
replica_info = self.replication.detach_slave(self.app, for_failover)
return replica_info
def get_replica_context(self, context):
LOG.debug("Getting replica context.")
raise exception.DatastoreOperationNotSupported(
operation='get_replica_context', datastore=self.manager)
"""Running on primary."""
LOG.info("Getting replica context.")
replica_info = self.replication.get_replica_context(self.app, self.adm)
return replica_info
def make_read_only(self, context, read_only):
LOG.debug("Making datastore read-only.")
raise exception.DatastoreOperationNotSupported(
operation='make_read_only', datastore=self.manager)
def enable_as_master(self, context, replica_source_config):
LOG.debug("Enabling as master.")
raise exception.DatastoreOperationNotSupported(
operation='enable_as_master', datastore=self.manager)
LOG.info("Enable as master")
self.replication.enable_as_master(self.app, replica_source_config)
def demote_replication_master(self, context):
LOG.debug("Demoting replication master.")
raise exception.DatastoreOperationNotSupported(
operation='demote_replication_master', datastore=self.manager)
LOG.info("Demoting replication master.")
self.replication.demote_master(self.app)
def get_txn_count(self, context):
LOG.debug("Getting transaction count.")
@ -867,11 +902,9 @@ class Manager(periodic_task.PeriodicTasks):
operation='get_txn_count', datastore=self.manager)
def get_latest_txn_id(self, context):
LOG.debug("Getting latest transaction id.")
raise exception.DatastoreOperationNotSupported(
operation='get_latest_txn_id', datastore=self.manager)
def wait_for_txn(self, context, txn):
LOG.debug("Waiting for transaction.")
raise exception.DatastoreOperationNotSupported(
operation='wait_for_txn', datastore=self.manager)

View File

@ -27,7 +27,6 @@ from trove.common.notification import EndNotification
from trove.guestagent import guest_log
from trove.guestagent.common import operating_system
from trove.guestagent.datastore import manager
from trove.guestagent.strategies import replication as repl_strategy
from trove.guestagent.utils import docker as docker_util
from trove.guestagent.utils import mysql as mysql_util
from trove.instance import service_status
@ -50,19 +49,6 @@ class MySqlManager(manager.Manager):
def configuration_manager(self):
return self.app.configuration_manager
@property
def replication(self):
"""If the datastore supports replication, return an instance of
the strategy.
"""
try:
return repl_strategy.get_instance(self.manager)
except Exception as ex:
LOG.warning("Cannot get replication instance for '%(manager)s': "
"%(msg)s", {'manager': self.manager, 'msg': str(ex)})
return None
def get_service_status(self):
try:
with mysql_util.SqlClient(self.app.get_engine()) as client:
@ -133,7 +119,8 @@ class MySqlManager(manager.Manager):
LOG.info(f"Creating backup {backup_info['id']}")
with EndNotification(context):
volumes_mapping = {
'/var/lib/mysql': {'bind': '/var/lib/mysql', 'mode': 'rw'}
'/var/lib/mysql': {'bind': '/var/lib/mysql', 'mode': 'rw'},
'/tmp': {'bind': '/tmp', 'mode': 'rw'}
}
self.app.create_backup(context, backup_info,
volumes_mapping=volumes_mapping,
@ -273,7 +260,7 @@ class MySqlManager(manager.Manager):
'slave_volume_size': volume_stats.get('total', 0.0)
}))
def attach_replica(self, context, replica_info, slave_config):
def attach_replica(self, context, replica_info, slave_config, **kwargs):
LOG.info("Attaching replica, replica_info: %s", replica_info)
try:
if 'replication_strategy' in replica_info:
@ -286,45 +273,6 @@ class MySqlManager(manager.Manager):
self.status.set_status(service_status.ServiceStatuses.FAILED)
raise
def detach_replica(self, context, for_failover=False):
LOG.info("Detaching replica.")
replica_info = self.replication.detach_slave(self.app, for_failover)
return replica_info
def backup_required_for_replication(self, context):
return self.replication.backup_required_for_replication()
def get_replication_snapshot(self, context, snapshot_info,
replica_source_config=None):
LOG.info("Getting replication snapshot, snapshot_info: %s",
snapshot_info)
self.replication.enable_as_master(self.app, replica_source_config)
LOG.info('Enabled as replication master')
snapshot_id, log_position = self.replication.snapshot_for_replication(
context, self.app, self.adm, None, snapshot_info)
volume_stats = self.get_filesystem_stats(context, None)
replication_snapshot = {
'dataset': {
'datastore_manager': self.manager,
'dataset_size': volume_stats.get('used', 0.0),
'volume_size': volume_stats.get('total', 0.0),
'snapshot_id': snapshot_id
},
'replication_strategy': self.replication_strategy,
'master': self.replication.get_master_ref(self.app, snapshot_info),
'log_position': log_position
}
return replication_snapshot
def enable_as_master(self, context, replica_source_config):
LOG.info("Enable as master")
self.replication.enable_as_master(self.app, replica_source_config)
def make_read_only(self, context, read_only):
LOG.info("Executing make_read_only(%s)", read_only)
self.app.make_read_only(read_only)
@ -341,15 +289,6 @@ class MySqlManager(manager.Manager):
LOG.info("Calling wait_for_txn.")
self.app.wait_for_txn(txn)
def get_replica_context(self, context):
LOG.info("Getting replica context.")
replica_info = self.replication.get_replica_context(self.app, self.adm)
return replica_info
def demote_replication_master(self, context):
LOG.info("Demoting replication master.")
self.replication.demote_master(self.app)
def upgrade(self, context, upgrade_info):
"""Upgrade the database."""
LOG.info('Starting to upgrade database, upgrade_info: %s',

View File

@ -16,6 +16,8 @@ import os
from oslo_log import log as logging
from trove.common import cfg
from trove.common import exception
from trove.common import utils
from trove.common.notification import EndNotification
from trove.guestagent import guest_log
from trove.guestagent.common import operating_system
@ -56,25 +58,29 @@ class PostgresManager(manager.Manager):
self.app.set_data_dir(self.app.datadir)
self.app.update_overrides(overrides)
# Prepare pg_hba.conf
self.app.apply_access_rules()
self.configuration_manager.apply_system_override(
{'hba_file': service.HBA_CONFIG_FILE})
# Restore data from backup and reset root password
if backup_info:
self.perform_restore(context, self.app.datadir, backup_info)
if not snapshot:
signal_file = f"{self.app.datadir}/recovery.signal"
operating_system.execute_shell_cmd(
f"touch {signal_file}", [], shell=True, as_root=True)
operating_system.chown(signal_file, CONF.database_service_uid,
CONF.database_service_uid, force=True,
as_root=True)
signal_file = f"{self.app.datadir}/recovery.signal"
operating_system.execute_shell_cmd(
f"touch {signal_file}", [], shell=True, as_root=True)
operating_system.chown(signal_file, CONF.database_service_uid,
CONF.database_service_uid, force=True,
as_root=True)
if snapshot:
# This instance is a replica
self.attach_replica(context, snapshot, snapshot['config'])
# config_file can only be set on the postgres command line
command = f"postgres -c config_file={service.CONFIG_FILE}"
self.app.start_db(ds_version=ds_version, command=command)
self.app.secure()
# if snapshot:
# # This instance is a replication slave
# self.attach_replica(context, snapshot, snapshot['config'])
def apply_overrides(self, context, overrides):
pass
@ -134,3 +140,46 @@ class PostgresManager(manager.Manager):
volumes_mapping=volumes_mapping,
need_dbuser=False,
extra_params=extra_params)
def attach_replica(self, context, replica_info, slave_config,
restart=False):
"""Set up the standby server."""
self.replication.enable_as_slave(self.app, replica_info, None)
# For the previous primary, don't start db service in order to run
# pg_rewind command next.
if restart:
self.app.restart()
def make_read_only(self, context, read_only):
"""There seems to be no way to flag this at the database level in
PostgreSQL at the moment -- see discussion here:
http://www.postgresql.org/message-id/flat/CA+TgmobWQJ-GCa_tWUc4=80A
1RJ2_+Rq3w_MqaVguk_q018dqw@mail.gmail.com#CA+TgmobWQJ-GCa_tWUc4=80A1RJ
2_+Rq3w_MqaVguk_q018dqw@mail.gmail.com
"""
pass
def get_latest_txn_id(self, context):
if self.app.is_replica():
lsn = self.app.get_last_wal_replay_lsn()
else:
lsn = self.app.get_current_wal_lsn()
LOG.info("Last wal location found: %s", lsn)
return lsn
def wait_for_txn(self, context, txn):
if not self.app.is_replica():
raise exception.TroveError("Attempting to wait for a txn on a "
"non-replica server")
def _wait_for_txn():
lsn = self.app.get_last_wal_replay_lsn()
LOG.info("Last wal location found: %s", lsn)
return lsn >= txn
try:
utils.poll_until(_wait_for_txn, time_out=60)
except exception.PollTimeOut:
raise exception.TroveError(
f"Timeout occurred waiting for wal offset to change to {txn}")

View File

@ -32,7 +32,6 @@ from trove.instance import service_status
LOG = logging.getLogger(__name__)
CONF = cfg.CONF
ADMIN_USER_NAME = "os_admin"
SUPER_USER_NAME = "postgres"
CONFIG_FILE = "/etc/postgresql/postgresql.conf"
CNF_EXT = 'conf'
@ -95,6 +94,7 @@ class PgSqlApp(service.BaseDbApp):
# https://github.com/docker-library/docs/blob/master/postgres/README.md#pgdata
mount_point = cfg.get_configuration_property('mount_point')
self.datadir = f"{mount_point}/data/pgdata"
self.adm = PgSqlAdmin(SUPER_USER_NAME)
@classmethod
def get_data_dir(cls):
@ -109,25 +109,6 @@ class PgSqlApp(service.BaseDbApp):
cmd = f"pg_ctl reload -D {self.datadir}"
docker_util.run_command(self.docker_client, cmd)
def secure(self):
LOG.info("Securing PostgreSQL now.")
admin_password = utils.generate_random_password()
os_admin = models.PostgreSQLUser(ADMIN_USER_NAME, admin_password)
# Drop os_admin user if exists, this is needed for restore.
PgSqlAdmin(SUPER_USER_NAME).delete_user({'_name': ADMIN_USER_NAME})
PgSqlAdmin(SUPER_USER_NAME).create_admin_user(os_admin,
encrypt_password=True)
self.save_password(ADMIN_USER_NAME, admin_password)
self.apply_access_rules()
self.configuration_manager.apply_system_override(
{'hba_file': HBA_CONFIG_FILE})
self.restart()
LOG.info("PostgreSQL secure complete.")
def apply_access_rules(self):
"""PostgreSQL Client authentication settings
@ -137,17 +118,15 @@ class PgSqlApp(service.BaseDbApp):
"""
LOG.debug("Applying client authentication access rules.")
local_admins = ','.join([SUPER_USER_NAME, ADMIN_USER_NAME])
remote_admins = SUPER_USER_NAME
access_rules = OrderedDict(
[('local', [['all', local_admins, None, 'trust'],
['replication', local_admins, None, 'trust'],
[('local', [['all', SUPER_USER_NAME, None, 'trust'],
['replication', SUPER_USER_NAME, None, 'trust'],
['all', 'all', None, 'md5']]),
('host', [['all', local_admins, '127.0.0.1/32', 'trust'],
['all', local_admins, '::1/128', 'trust'],
['all', local_admins, 'localhost', 'trust'],
['all', remote_admins, '0.0.0.0/0', 'reject'],
['all', remote_admins, '::/0', 'reject'],
('host', [['all', SUPER_USER_NAME, '127.0.0.1/32', 'trust'],
['all', SUPER_USER_NAME, '::1/128', 'trust'],
['all', SUPER_USER_NAME, 'localhost', 'trust'],
['all', SUPER_USER_NAME, '0.0.0.0/0', 'reject'],
['all', SUPER_USER_NAME, '::/0', 'reject'],
['all', 'all', '0.0.0.0/0', 'md5'],
['all', 'all', '::/0', 'md5']])
])
@ -307,6 +286,57 @@ class PgSqlApp(service.BaseDbApp):
CONF.database_service_uid, force=True,
as_root=True)
def is_replica(self):
"""Wrapper for pg_is_in_recovery() for detecting a server in
standby mode
"""
r = self.adm.query("SELECT pg_is_in_recovery()")
return r[0][0]
def get_current_wal_lsn(self):
"""Wrapper for pg_current_wal_lsn()
Cannot be used against a running replica
"""
r = self.adm.query("SELECT pg_current_wal_lsn()")
return r[0][0]
def get_last_wal_replay_lsn(self):
"""Wrapper for pg_last_wal_replay_lsn()
For use on replica servers
"""
r = self.adm.query("SELECT pg_last_wal_replay_lsn()")
return r[0][0]
def pg_rewind(self, conn_info):
docker_image = CONF.get(CONF.datastore_manager).docker_image
image = f'{docker_image}:{CONF.datastore_version}'
user = "%s:%s" % (CONF.database_service_uid, CONF.database_service_uid)
volumes = {
"/var/run/postgresql": {"bind": "/var/run/postgresql",
"mode": "rw"},
"/var/lib/postgresql": {"bind": "/var/lib/postgresql",
"mode": "rw"},
"/var/lib/postgresql/data": {"bind": "/var/lib/postgresql/data",
"mode": "rw"},
}
command = (f"pg_rewind --target-pgdata={self.datadir} "
f"--source-server='{conn_info}'")
docker_util.remove_container(self.docker_client, name='pg_rewind')
LOG.info('Running pg_rewind in container')
output, ret = docker_util.run_container(
self.docker_client, image, 'pg_rewind',
volumes=volumes, command=command, user=user)
result = output[-1]
LOG.debug(f"Finished running pg_rewind, last output: {result}")
if not ret:
msg = f'Failed to run pg_rewind in container, error: {result}'
LOG.error(msg)
raise Exception(msg)
class PgSqlAdmin(object):
# Default set of options of an administrative account.

View File

@ -440,7 +440,7 @@ class BaseDbApp(object):
swift_container = (backup_info.get('swift_container') or
CONF.backup_swift_container)
swift_params = (f'--swift-extra-metadata={swift_metadata} '
f'--swift-container {swift_container}')
f'--swift-container={swift_container}')
command = (
f'/usr/bin/python3 main.py --backup --backup-id={backup_id} '
@ -449,7 +449,7 @@ class BaseDbApp(object):
f'{db_userinfo} '
f'{swift_params} '
f'{incremental} '
f'{extra_params} '
f'{extra_params}'
)
# Update backup status in db
@ -489,11 +489,13 @@ class BaseDbApp(object):
'state': BackupState.COMPLETED,
})
else:
LOG.error(f'Cannot parse backup output: {result}')
msg = f'Cannot parse backup output: {result}'
LOG.error(msg)
backup_state.update({
'success': False,
'state': BackupState.FAILED,
})
raise Exception(msg)
except Exception as err:
LOG.error("Failed to create backup %s", backup_id)
backup_state.update({

View File

@ -41,7 +41,7 @@ def get_instance(manager):
replication_strategy, __replication_namespace)
__replication_instance = replication_strategy_cls()
__replication_manager = manager
LOG.debug('Got replication instance from: %(namespace)s.%(strategy)s',
LOG.debug('Replication instance from: %(namespace)s.%(strategy)s',
{'namespace': __replication_namespace,
'strategy': __replication_strategy})
return __replication_instance

View File

@ -79,7 +79,13 @@ class MysqlReplicationBase(base.Replication):
def snapshot_for_replication(self, context, service, adm, location,
snapshot_info):
LOG.info("Creating backup for replication")
service.create_backup(context, snapshot_info)
volumes_mapping = {
'/var/lib/mysql': {'bind': '/var/lib/mysql', 'mode': 'rw'},
'/tmp': {'bind': '/tmp', 'mode': 'rw'}
}
service.create_backup(context, snapshot_info,
volumes_mapping=volumes_mapping)
LOG.info('Creating replication user')
replication_user = self._create_replication_user(service, adm)

View File

@ -0,0 +1,220 @@
# Copyright 2020 Catalyst Cloud
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
import os
from oslo_log import log as logging
from oslo_utils import netutils
from trove.common import cfg
from trove.common import exception
from trove.common import utils
from trove.common.db.postgresql import models
from trove.guestagent.common import operating_system
from trove.guestagent.common.operating_system import FileMode
from trove.guestagent.datastore.postgres import service as pg_service
from trove.guestagent.strategies.replication import base
LOG = logging.getLogger(__name__)
CONF = cfg.CONF
REPL_USER = 'replicator'
class PostgresqlReplicationStreaming(base.Replication):
def _create_replication_user(self, service, adm_mgr, pwfile):
"""Create the replication user and password file.
Unfortunately, to be able to run pg_rewind, we need SUPERUSER, not just
REPLICATION privilege
"""
pw = utils.generate_random_password()
operating_system.write_file(pwfile, pw, as_root=True)
operating_system.chown(pwfile, user=CONF.database_service_uid,
group=CONF.database_service_uid, as_root=True)
operating_system.chmod(pwfile, FileMode.SET_USR_RWX(),
as_root=True)
LOG.debug(f"File {pwfile} created")
LOG.debug(f"Creating replication user {REPL_USER}")
repl_user = models.PostgreSQLUser(name=REPL_USER, password=pw)
adm_mgr.create_user(repl_user, None,
*('REPLICATION', 'SUPERUSER', 'LOGIN'))
return pw
def _get_or_create_replication_user(self, service):
"""There are three scenarios we need to deal with here:
- This is a fresh master, with no replicator user created.
Generate a new u/p
- We are attaching a new slave and need to give it the login creds
Send the creds we have stored in PGDATA/.replpass
- This is a failed-over-to slave, who will have the replicator user
but not the credentials file. Recreate the repl user in this case
"""
LOG.debug("Checking for replication user")
pwfile = os.path.join(service.datadir, ".replpass")
adm_mgr = service.adm
if adm_mgr.user_exists(REPL_USER):
if operating_system.exists(pwfile, as_root=True):
LOG.debug("Found existing .replpass")
pw = operating_system.read_file(pwfile, as_root=True)
else:
LOG.debug("Found user but not .replpass, recreate")
adm_mgr.delete_user(models.PostgreSQLUser(REPL_USER))
pw = self._create_replication_user(service, adm_mgr, pwfile)
else:
LOG.debug("Found no replicator user, create one")
pw = self._create_replication_user(service, adm_mgr, pwfile)
repl_user_info = {
'name': REPL_USER,
'password': pw
}
return repl_user_info
def enable_as_master(self, service, master_config):
"""Primary postgredql settings.
For a server to be a master in postgres, we need to enable
the replication user in pg_hba.conf
"""
self._get_or_create_replication_user(service)
hba_entry = f"host replication {REPL_USER} 0.0.0.0/0 md5\n"
tmp_hba = '/tmp/pg_hba'
operating_system.copy(pg_service.HBA_CONFIG_FILE, tmp_hba,
force=True, as_root=True)
operating_system.chmod(tmp_hba, FileMode.SET_ALL_RWX(),
as_root=True)
with open(tmp_hba, 'a+') as hba_file:
hba_file.write(hba_entry)
operating_system.copy(tmp_hba, pg_service.HBA_CONFIG_FILE,
force=True, as_root=True)
operating_system.chown(pg_service.HBA_CONFIG_FILE,
user=CONF.database_service_uid,
group=CONF.database_service_uid, as_root=True)
operating_system.chmod(pg_service.HBA_CONFIG_FILE,
FileMode.SET_USR_RWX(),
as_root=True)
operating_system.remove(tmp_hba, as_root=True)
LOG.debug(f"{pg_service.HBA_CONFIG_FILE} changed")
service.restart()
def snapshot_for_replication(self, context, service, adm, location,
snapshot_info):
LOG.info("Creating backup for replication")
volumes_mapping = {
'/var/lib/postgresql/data': {
'bind': '/var/lib/postgresql/data', 'mode': 'rw'
},
"/var/run/postgresql": {"bind": "/var/run/postgresql",
"mode": "ro"},
}
extra_params = f"--pg-wal-archive-dir {pg_service.WAL_ARCHIVE_DIR}"
service.create_backup(context, snapshot_info,
volumes_mapping=volumes_mapping,
need_dbuser=False,
extra_params=extra_params)
LOG.info('Getting or creating replication user')
replication_user = self._get_or_create_replication_user(service)
log_position = {
'replication_user': replication_user
}
return snapshot_info['id'], log_position
def get_master_ref(self, service, snapshot_info):
master_ref = {
'host': netutils.get_my_ipv4(),
'port': cfg.get_configuration_property('postgresql_port')
}
return master_ref
def enable_as_slave(self, service, snapshot, slave_config):
"""Set up the replica server."""
signal_file = f"{service.datadir}/standby.signal"
operating_system.execute_shell_cmd(
f"touch {signal_file}", [], shell=True, as_root=True)
operating_system.chown(signal_file, CONF.database_service_uid,
CONF.database_service_uid, force=True,
as_root=True)
LOG.debug("Standby signal file created")
user = snapshot['log_position']['replication_user']
conninfo = (f"host={snapshot['master']['host']} "
f"port={snapshot['master']['port']} "
f"dbname=postgres "
f"user={user['name']} password={user['password']}")
service.configuration_manager.apply_system_override(
{'primary_conninfo': conninfo})
LOG.debug("primary_conninfo is set in the config file.")
def detach_slave(self, service, for_failover):
"""Promote replica and wait for its running.
Running on replica, detach from the primary.
"""
service.adm.query("select pg_promote()")
def _wait_for_failover():
"""Wait until slave has switched out of recovery mode"""
return not service.is_replica()
try:
utils.poll_until(_wait_for_failover, time_out=60)
except exception.PollTimeOut:
raise exception.TroveError(
"Timeout occurred waiting for replica to exit standby mode")
def get_replica_context(self, service, adm):
"""Running on primary."""
repl_user_info = self._get_or_create_replication_user(service)
return {
'master': self.get_master_ref(None, None),
'log_position': {'replication_user': repl_user_info}
}
def cleanup_source_on_replica_detach(self, admin_service, replica_info):
pass
def _pg_rewind(self, service):
conn_info = service.configuration_manager.get_value('primary_conninfo')
service.pg_rewind(conn_info)
signal_file = f"{service.datadir}/standby.signal"
operating_system.execute_shell_cmd(
f"touch {signal_file}", [], shell=True, as_root=True)
operating_system.chown(signal_file, CONF.database_service_uid,
CONF.database_service_uid, force=True,
as_root=True)
LOG.debug("Standby signal file created")
def demote_master(self, service):
"""Running on the old primary.
In order to demote a master we need to shutdown the server and call
pg_rewind against the new master to enable a proper timeline
switch.
"""
service.stop_db()
self._pg_rewind(service)
service.restart()

View File

@ -83,7 +83,7 @@ def _decode_output(output):
def run_container(client, image, name, network_mode="host", volumes={},
command=""):
command="", user=""):
"""Run command in a container and return the string output list.
:returns output: The log output.
@ -103,6 +103,7 @@ def run_container(client, image, name, network_mode="host", volumes={},
volumes=volumes,
remove=False,
command=command,
user=user,
)
except docker.errors.ContainerError as err:
output = err.container.logs()

View File

@ -929,7 +929,7 @@ class BaseInstance(SimpleInstance):
self._server_group_loaded = True
return self._server_group
def get_injected_files(self, datastore_manager):
def get_injected_files(self, datastore_manager, datastore_version):
injected_config_location = CONF.get('injected_config_location')
guest_info = CONF.get('guest_info')
@ -946,8 +946,10 @@ class BaseInstance(SimpleInstance):
"[DEFAULT]\n"
"guest_id=%s\n"
"datastore_manager=%s\n"
"datastore_version=%s\n"
"tenant_id=%s\n"
% (self.id, datastore_manager, self.tenant_id)
% (self.id, datastore_manager, datastore_version,
self.tenant_id)
)
}

View File

@ -136,7 +136,7 @@ class Manager(periodic_task.PeriodicTasks):
try:
if replica.id != master_candidate.id:
replica.detach_replica(old_master, for_failover=True)
replica.attach_replica(master_candidate)
replica.attach_replica(master_candidate, restart=True)
except exception.TroveError as ex:
log_fmt = ("Unable to migrate replica %(slave)s from "
"old replica source %(old_master)s to "
@ -156,7 +156,7 @@ class Manager(periodic_task.PeriodicTasks):
# dealing with the old master after all the other replicas
# has been migrated.
old_master.attach_replica(master_candidate)
old_master.attach_replica(master_candidate, restart=False)
try:
old_master.demote_replication_master()
except Exception as ex:

View File

@ -565,7 +565,7 @@ class FreshInstanceTasks(FreshInstance, NotifyMixin, ConfigurationMixin):
networks = self._prepare_networks_for_instance(
datastore_manager, nics, access=access
)
files = self.get_injected_files(datastore_manager)
files = self.get_injected_files(datastore_manager, ds_version)
cinder_volume_type = volume_type or CONF.cinder_volume_type
volume_info = self._create_server_volume(
flavor['id'], image_id,
@ -1165,13 +1165,14 @@ class BuiltInstanceTasks(BuiltInstance, NotifyMixin, ConfigurationMixin):
if not for_failover:
self.reset_task_status()
def attach_replica(self, master):
def attach_replica(self, master, restart=False):
LOG.info("Attaching replica %s to master %s", self.id, master.id)
try:
replica_info = master.guest.get_replica_context()
flavor = self.nova_client.flavors.get(self.flavor_id)
slave_config = self._render_replica_config(flavor).config_contents
self.guest.attach_replica(replica_info, slave_config)
self.guest.attach_replica(replica_info, slave_config,
restart=restart)
self.update_db(slave_of_id=master.id)
self.slave_list = None
except (GuestError, GuestTimeout):
@ -2047,7 +2048,9 @@ class RebuildAction(ResizeActionBase):
self.wait_status = ['ACTIVE']
def _initiate_nova_action(self):
files = self.instance.get_injected_files(self.instance.datastore.name)
files = self.instance.get_injected_files(
self.instance.datastore.name,
self.instance.datastore_version.name)
LOG.debug(f"Rebuilding Nova server {self.instance.server.id}")
# Before Nova version 2.57, userdata is not supported when doing

View File

@ -214,8 +214,9 @@ wal_level = replica # minimal, replica, or logical
# open_sync
#full_page_writes = on # recover from partial page writes
#wal_compression = off # enable compression of full-page writes
#wal_log_hints = off # also do full page writes of non-critical updates
wal_log_hints = on # also do full page writes of non-critical updates
# (change requires restart)
# (Trove default)
#wal_init_zero = on # zero-fill new WAL files
#wal_recycle = on # recycle WAL files
#wal_buffers = -1 # min 32kB, -1 sets based on shared_buffers

View File

@ -94,12 +94,13 @@ class TestManager(trove_testtools.TestCase):
self.mock_slave1.detach_replica.assert_called_with(
self.mock_old_master, for_failover=True)
self.mock_old_master.attach_replica.assert_called_with(
self.mock_slave1)
self.mock_slave1, restart=False)
self.mock_slave1.make_read_only.assert_called_with(False)
self.mock_slave2.detach_replica.assert_called_with(
self.mock_old_master, for_failover=True)
self.mock_slave2.attach_replica.assert_called_with(self.mock_slave1)
self.mock_slave2.attach_replica.assert_called_with(self.mock_slave1,
restart=True)
self.mock_old_master.demote_replication_master.assert_any_call()

View File

@ -249,7 +249,7 @@ class FreshInstanceTasksTest(BaseFreshInstanceTasksTest):
cfg.CONF.set_override('injected_config_location', '/etc/trove/conf.d')
# execute
files = self.freshinstancetasks.get_injected_files("test")
files = self.freshinstancetasks.get_injected_files("test", 'test')
# verify
self.assertTrue(
'/etc/trove/conf.d/guest_info.conf' in files)
@ -266,7 +266,7 @@ class FreshInstanceTasksTest(BaseFreshInstanceTasksTest):
cfg.CONF.set_override('injected_config_location', '/etc')
# execute
files = self.freshinstancetasks.get_injected_files("test")
files = self.freshinstancetasks.get_injected_files("test", 'test')
# verify
self.assertTrue(
'/etc/guest_info' in files)
@ -920,7 +920,7 @@ class BuiltInstanceTasksTest(trove_testtools.TestCase):
return_value=replica_config):
self.instance_task.attach_replica(master)
self.instance_task._guest.attach_replica.assert_called_with(
replica_context, config_content)
replica_context, config_content, restart=False)
mock_update_db.assert_called_with(slave_of_id=master.id)
@patch('trove.taskmanager.models.LOG')