from ceph.deployment.drive_group import DriveGroupSpec
from ceph.deployment.service_spec import \
NFSServiceSpec, ServiceSpec, PlacementSpec, assert_valid_host, \
- HostPlacementSpec
+ HostPlacementSpec, IngressSpec
from ceph.utils import str_to_datetime, datetime_to_str, datetime_now
from cephadm.serve import CephadmServe
from cephadm.services.cephadmservice import CephadmDaemonDeploySpec
from .migrations import Migrations
from .services.cephadmservice import MonService, MgrService, MdsService, RgwService, \
RbdMirrorService, CrashService, CephadmService, CephfsMirrorService
+from .services.ingress import IngressService
from .services.container import CustomContainerService
from .services.iscsi import IscsiService
-from .services.ha_rgw import HA_RGWService
from .services.nfs import NFSService
from .services.osd import OSDRemovalQueue, OSDService, OSD, NotFoundError
from .services.monitoring import GrafanaService, AlertmanagerService, PrometheusService, \
OSDService, NFSService, MonService, MgrService, MdsService,
RgwService, RbdMirrorService, GrafanaService, AlertmanagerService,
PrometheusService, NodeExporterService, CrashService, IscsiService,
- HA_RGWService, CustomContainerService, CephadmExporter, CephfsMirrorService
+ IngressService, CustomContainerService, CephadmExporter, CephfsMirrorService
]
# https://github.com/python/mypy/issues/8993
if service_type == 'nfs':
spec = cast(NFSServiceSpec, spec)
sm[nm].rados_config_location = spec.rados_config_location()
- if spec.service_type == 'ha-rgw':
- # ha-rgw has 2 daemons running per host
+ if spec.service_type == 'ingress':
+ # ingress has 2 daemons running per host
sm[nm].size *= 2
# factor daemons into status
previews_for_specs.update({host: osd_reports})
return previews_for_specs
- def _calc_daemon_deps(self, daemon_type: str, daemon_id: str) -> List[str]:
- need = {
- 'prometheus': ['mgr', 'alertmanager', 'node-exporter'],
- 'grafana': ['prometheus'],
- 'alertmanager': ['mgr', 'alertmanager'],
- }
+ def _calc_daemon_deps(self,
+ spec: Optional[ServiceSpec],
+ daemon_type: str,
+ daemon_id: str) -> List[str]:
deps = []
- for dep_type in need.get(daemon_type, []):
- for dd in self.cache.get_daemons_by_service(dep_type):
- deps.append(dd.name())
+ if daemon_type == 'haproxy':
+ # because cephadm creates new daemon instances whenever
+ # port or ip changes, identifying daemons by name is
+ # sufficient to detect changes.
+ if not spec:
+ return []
+ ingress_spec = cast(IngressSpec, spec)
+ assert ingress_spec.backend_service
+ daemons = self.cache.get_daemons_by_service(ingress_spec.backend_service)
+ deps = [d.name() for d in daemons]
+ elif daemon_type == 'keepalived':
+ # because cephadm creates new daemon instances whenever
+ # port or ip changes, identifying daemons by name is
+ # sufficient to detect changes.
+ if not spec:
+ return []
+ daemons = self.cache.get_daemons_by_service(spec.service_name())
+ deps = [d.name() for d in daemons if d.daemon_type == 'haproxy']
+ else:
+ need = {
+ 'prometheus': ['mgr', 'alertmanager', 'node-exporter'],
+ 'grafana': ['prometheus'],
+ 'alertmanager': ['mgr', 'alertmanager'],
+ }
+ for dep_type in need.get(daemon_type, []):
+ for dd in self.cache.get_daemons_by_service(dep_type):
+ deps.append(dd.name())
return sorted(deps)
@forall_hosts
'mgr': PlacementSpec(count=2),
'mds': PlacementSpec(count=2),
'rgw': PlacementSpec(count=2),
- 'ha-rgw': PlacementSpec(count=2),
+ 'ingress': PlacementSpec(count=2),
'iscsi': PlacementSpec(count=1),
'rbd-mirror': PlacementSpec(count=2),
'cephfs-mirror': PlacementSpec(count=1),
return self._apply(spec)
@handle_orch_error
- def apply_ha_rgw(self, spec: ServiceSpec) -> str:
+ def apply_ingress(self, spec: ServiceSpec) -> str:
return self._apply(spec)
@handle_orch_error
from ceph.deployment import inventory
from ceph.deployment.drive_group import DriveGroupSpec
-from ceph.deployment.service_spec import ServiceSpec, HA_RGWSpec, CustomContainerSpec
+from ceph.deployment.service_spec import ServiceSpec, IngressSpec, CustomContainerSpec
from ceph.utils import str_to_datetime, datetime_now
import orchestrator
from orchestrator import OrchestratorError, set_exception_subject, OrchestratorEvent, \
DaemonDescriptionStatus, daemon_type_to_service, service_to_daemon_types
from cephadm.services.cephadmservice import CephadmDaemonDeploySpec
-from cephadm.schedule import HostAssignment, DaemonPlacement
+from cephadm.schedule import HostAssignment
from cephadm.utils import forall_hosts, cephadmNoImage, is_repo_digest, \
CephadmNoImage, CEPH_TYPES, ContainerInspectInfo
from mgr_module import MonCommandFailed
hosts=self.mgr._hosts_with_daemon_inventory(),
daemons=daemons,
networks=self.mgr.cache.networks,
- filter_new_host=matches_network if service_type == 'mon'
- else virtual_ip_allowed if service_type == 'ha-rgw' else None,
+ filter_new_host=(
+ matches_network if service_type == 'mon'
+ else virtual_ip_allowed if service_type == 'ingress'
+ else None
+ ),
allow_colo=svc.allow_colo(),
)
self.log.debug('Hosts that will receive new daemons: %s' % slots_to_add)
self.log.debug('Daemons that will be removed: %s' % daemons_to_remove)
- if service_type == 'ha-rgw':
- spec = self.update_ha_rgw_definitive_hosts(spec, all_slots, slots_to_add)
-
for slot in slots_to_add:
for daemon_type in service_to_daemon_types(service_type):
+ if daemon_type != 'keepalived':
+ slot_ports = slot.ports
+ slot_ip = slot.ip
+ else:
+ slot_ports = []
+ slot_ip = None
+
# first remove daemon on conflicting port?
- if slot.ports:
+ if slot_ports:
for d in daemons_to_remove:
if d.hostname != slot.hostname:
continue
- if not (set(d.ports or []) & set(slot.ports)):
+ if not (set(d.ports or []) & set(slot_ports)):
continue
- if d.ip and slot.ip and d.ip != slot.ip:
+ if d.ip and slot_ip and d.ip != slot_ip:
continue
self.log.info(
f'Removing {d.name()} before deploying to {slot} to avoid a port conflict'
daemon_spec = svc.make_daemon_spec(
slot.hostname, daemon_id, slot.network, spec, daemon_type=daemon_type,
- ports=slot.ports,
- ip=slot.ip,
+ ports=slot_ports,
+ ip=slot_ip,
)
self.log.debug('Placing %s.%s on host %s' % (
daemon_type, daemon_id, slot.hostname))
else:
dd.is_active = False
- deps = self.mgr._calc_daemon_deps(dd.daemon_type, dd.daemon_id)
+ deps = self.mgr._calc_daemon_deps(spec, dd.daemon_type, dd.daemon_id)
last_deps, last_config = self.mgr.cache.get_daemon_last_config_deps(
dd.hostname, dd.name())
if last_deps is None:
# FIXME: we assume the first digest here is the best
self.mgr.set_container_image(entity, image_info.repo_digests[0])
- # ha-rgw needs definitve host list to create keepalived config files
- # if definitive host list has changed, all ha-rgw daemons must get new
- # config, including those that are already on the correct host and not
- # going to be deployed
- def update_ha_rgw_definitive_hosts(
- self,
- spec: ServiceSpec,
- hosts: List[DaemonPlacement],
- add_hosts: List[DaemonPlacement]
- ) -> HA_RGWSpec:
- spec = cast(HA_RGWSpec, spec)
- hostnames = [p.hostname for p in hosts]
- add_hostnames = [p.hostname for p in add_hosts]
- if not (set(hostnames) == set(spec.definitive_host_list)):
- spec.definitive_host_list = hostnames
- ha_rgw_daemons = self.mgr.cache.get_daemons_by_service(spec.service_name())
- for daemon in ha_rgw_daemons:
- if daemon.hostname in hostnames and daemon.hostname not in add_hostnames:
- assert daemon.hostname is not None
- self.mgr.cache.schedule_daemon_action(
- daemon.hostname, daemon.name(), 'reconfig')
- return spec
-
def _create_daemon(self,
daemon_spec: CephadmDaemonDeploySpec,
reconfig: bool = False,
self._deploy_cephadm_binary(daemon_spec.host)
if daemon_spec.daemon_type == 'haproxy':
- haspec = cast(HA_RGWSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
+ haspec = cast(IngressSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
if haspec.haproxy_container_image:
image = haspec.haproxy_container_image
if daemon_spec.daemon_type == 'keepalived':
- haspec = cast(HA_RGWSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
+ haspec = cast(IngressSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
if haspec.keepalived_container_image:
image = haspec.keepalived_container_image
import errno
import json
-import re
import logging
+import re
from abc import ABCMeta, abstractmethod
from typing import TYPE_CHECKING, List, Callable, TypeVar, \
Optional, Dict, Any, Tuple, NewType, cast
"""
# despite this mapping entity names to daemons, self.TYPE within
# the CephService class refers to service types, not daemon types
- if self.TYPE in ['rgw', 'rbd-mirror', 'cephfs-mirror', 'nfs', "iscsi", 'ha-rgw']:
+ if self.TYPE in ['rgw', 'rbd-mirror', 'cephfs-mirror', 'nfs', "iscsi", 'ingress']:
return AuthEntity(f'client.{self.TYPE}.{daemon_id}')
elif self.TYPE == 'crash':
if host == "":
force: bool = False,
known: Optional[List[str]] = None # output argument
) -> HandleCommandResult:
- # if load balancer (ha-rgw) is present block if only 1 daemon up otherwise ok
+ # if load balancer (ingress) is present block if only 1 daemon up otherwise ok
# if no load balancer, warn if > 1 daemon, block if only 1 daemon
- def ha_rgw_present() -> bool:
- running_ha_rgw_daemons = [
- daemon for daemon in self.mgr.cache.get_daemons_by_type('ha-rgw') if daemon.status == 1]
+ def ingress_present() -> bool:
+ running_ingress_daemons = [
+ daemon for daemon in self.mgr.cache.get_daemons_by_type('ingress') if daemon.status == 1]
running_haproxy_daemons = [
- daemon for daemon in running_ha_rgw_daemons if daemon.daemon_type == 'haproxy']
+ daemon for daemon in running_ingress_daemons if daemon.daemon_type == 'haproxy']
running_keepalived_daemons = [
- daemon for daemon in running_ha_rgw_daemons if daemon.daemon_type == 'keepalived']
+ daemon for daemon in running_ingress_daemons if daemon.daemon_type == 'keepalived']
# check that there is at least one haproxy and keepalived daemon running
if running_haproxy_daemons and running_keepalived_daemons:
return True
# if reached here, there is > 1 rgw daemon.
# Say okay if load balancer present or force flag set
- if ha_rgw_present() or force:
+ if ingress_present() or force:
return HandleCommandResult(0, warn_message, '')
# if reached here, > 1 RGW daemon, no load balancer and no force flag.
+++ /dev/null
-import logging
-from typing import List, cast, Tuple, Dict, Any
-
-from ceph.deployment.service_spec import HA_RGWSpec
-
-from .cephadmservice import CephadmDaemonDeploySpec, CephService
-from ..utils import resolve_ip
-
-logger = logging.getLogger(__name__)
-
-
-class HA_RGWService(CephService):
- TYPE = 'ha-rgw'
-
- class rgw_server():
- def __init__(self, hostname: str, address: str, port: int):
- self.name = hostname
- self.ip = address
- self.port = port
-
- def prepare_create(self, daemon_spec: CephadmDaemonDeploySpec) -> CephadmDaemonDeploySpec:
- assert daemon_spec.daemon_type == 'haproxy' or daemon_spec.daemon_type == 'keepalived'
- if daemon_spec.daemon_type == 'haproxy':
- return self.haproxy_prepare_create(daemon_spec)
- else:
- return self.keepalived_prepare_create(daemon_spec)
-
- def generate_config(self, daemon_spec: CephadmDaemonDeploySpec) -> Tuple[Dict[str, Any], List[str]]:
- assert daemon_spec.daemon_type == 'haproxy' or daemon_spec.daemon_type == 'keepalived'
-
- if daemon_spec.daemon_type == 'haproxy':
- return self.haproxy_generate_config(daemon_spec)
- else:
- return self.keepalived_generate_config(daemon_spec)
-
- def haproxy_prepare_create(self, daemon_spec: CephadmDaemonDeploySpec) -> CephadmDaemonDeploySpec:
- assert daemon_spec.daemon_type == 'haproxy'
-
- daemon_id = daemon_spec.daemon_id
- host = daemon_spec.host
- spec = cast(HA_RGWSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
-
- logger.info('Create daemon %s on host %s with spec %s' % (
- daemon_id, host, spec))
-
- daemon_spec.final_config, daemon_spec.deps = self.haproxy_generate_config(daemon_spec)
-
- return daemon_spec
-
- def keepalived_prepare_create(self, daemon_spec: CephadmDaemonDeploySpec) -> CephadmDaemonDeploySpec:
- assert daemon_spec.daemon_type == 'keepalived'
-
- daemon_id = daemon_spec.daemon_id
- host = daemon_spec.host
- spec = cast(HA_RGWSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
-
- logger.info('Create daemon %s on host %s with spec %s' % (
- daemon_id, host, spec))
-
- daemon_spec.final_config, daemon_spec.deps = self.keepalived_generate_config(daemon_spec)
-
- return daemon_spec
-
- def haproxy_generate_config(self, daemon_spec: CephadmDaemonDeploySpec) -> Tuple[Dict[str, Any], List[str]]:
- spec = cast(HA_RGWSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
-
- rgw_daemons = self.mgr.cache.get_daemons_by_type('rgw')
- rgw_servers = []
- for daemon in rgw_daemons:
- assert daemon.hostname is not None
- rgw_servers.append(self.rgw_server(
- daemon.name(),
- resolve_ip(daemon.hostname),
- daemon.ports[0] if daemon.ports else 80
- ))
-
- # virtual ip address cannot have netmask attached when passed to haproxy config
- # since the port is added to the end and something like 123.123.123.10/24:8080 is invalid
- virtual_ip_address = spec.virtual_ip_address
- if "/" in str(spec.virtual_ip_address):
- just_ip = str(spec.virtual_ip_address).split('/')[0]
- virtual_ip_address = just_ip
-
- ha_context = {'spec': spec, 'rgw_servers': rgw_servers,
- 'virtual_ip_address': virtual_ip_address}
-
- haproxy_conf = self.mgr.template.render('services/haproxy/haproxy.cfg.j2', ha_context)
-
- config_files = {
- 'files': {
- "haproxy.cfg": haproxy_conf,
- }
- }
- if spec.ha_proxy_frontend_ssl_certificate:
- ssl_cert = spec.ha_proxy_frontend_ssl_certificate
- if isinstance(ssl_cert, list):
- ssl_cert = '\n'.join(ssl_cert)
- config_files['files']['haproxy.pem'] = ssl_cert
-
- return config_files, []
-
- def keepalived_generate_config(self, daemon_spec: CephadmDaemonDeploySpec) -> Tuple[Dict[str, Any], List[str]]:
- host = daemon_spec.host
-
- spec = cast(HA_RGWSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
-
- all_hosts = spec.definitive_host_list
-
- # set state. first host in placement is master all others backups
- state = 'BACKUP'
- if all_hosts[0] == host:
- state = 'MASTER'
-
- # remove host, daemon is being deployed on from all_hosts list for
- # other_ips in conf file and converter to ips
- all_hosts.remove(host)
- other_ips = [resolve_ip(h) for h in all_hosts]
-
- ka_context = {'spec': spec, 'state': state,
- 'other_ips': other_ips,
- 'host_ip': resolve_ip(host)}
-
- keepalived_conf = self.mgr.template.render(
- 'services/keepalived/keepalived.conf.j2', ka_context)
-
- config_file = {
- 'files': {
- "keepalived.conf": keepalived_conf,
- }
- }
-
- return config_file, []
--- /dev/null
+import ipaddress
+import logging
+import random
+import string
+from typing import List, Dict, Any, Tuple, cast
+
+from ceph.deployment.service_spec import IngressSpec
+from cephadm.utils import resolve_ip
+
+from cephadm.services.cephadmservice import CephadmDaemonDeploySpec, CephService
+
+logger = logging.getLogger(__name__)
+
+
+class IngressService(CephService):
+ TYPE = 'ingress'
+
+ def prepare_create(
+ self,
+ daemon_spec: CephadmDaemonDeploySpec,
+ ) -> CephadmDaemonDeploySpec:
+ if daemon_spec.daemon_type == 'haproxy':
+ return self.haproxy_prepare_create(daemon_spec)
+ if daemon_spec.daemon_type == 'keepalived':
+ return self.keepalived_prepare_create(daemon_spec)
+ assert False, "unexpected daemon type"
+
+ def generate_config(
+ self,
+ daemon_spec: CephadmDaemonDeploySpec
+ ) -> Tuple[Dict[str, Any], List[str]]:
+ if daemon_spec.daemon_type == 'haproxy':
+ return self.haproxy_generate_config(daemon_spec)
+ else:
+ return self.keepalived_generate_config(daemon_spec)
+ assert False, "unexpected daemon type"
+
+ def haproxy_prepare_create(
+ self,
+ daemon_spec: CephadmDaemonDeploySpec,
+ ) -> CephadmDaemonDeploySpec:
+ assert daemon_spec.daemon_type == 'haproxy'
+
+ daemon_id = daemon_spec.daemon_id
+ host = daemon_spec.host
+ spec = cast(IngressSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
+
+ logger.info('Create daemon haproxy.%s on host %s with spec %s' % (
+ daemon_id, host, spec))
+
+ daemon_spec.final_config, daemon_spec.deps = self.haproxy_generate_config(daemon_spec)
+
+ return daemon_spec
+
+ def haproxy_generate_config(
+ self,
+ daemon_spec: CephadmDaemonDeploySpec,
+ ) -> Tuple[Dict[str, Any], List[str]]:
+ spec = cast(IngressSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
+ assert spec.backend_service
+ daemons = self.mgr.cache.get_daemons_by_service(spec.backend_service)
+ deps = [d.name() for d in daemons]
+
+ # generate password?
+ pw_key = f'{spec.service_name()}/monitor_password'
+ password = self.mgr.get_store(pw_key)
+ if password is None:
+ if not spec.monitor_password:
+ password = ''.join(random.choice(string.ascii_lowercase) for _ in range(20))
+ self.mgr.set_store(pw_key, password)
+ else:
+ if spec.monitor_password:
+ self.mgr.set_store(pw_key, None)
+ if spec.monitor_password:
+ password = spec.monitor_password
+
+ haproxy_conf = self.mgr.template.render(
+ 'services/ingress/haproxy.cfg.j2',
+ {
+ 'spec': spec,
+ 'servers': [
+ {
+ 'name': d.name(),
+ 'ip': d.ip or resolve_ip(str(d.hostname)),
+ 'port': d.ports[0],
+ } for d in daemons if d.ports
+ ],
+ 'user': spec.monitor_user or 'admin',
+ 'password': password,
+ 'ip': daemon_spec.ip or '*',
+ 'frontend_port': daemon_spec.ports[0] if daemon_spec.ports else spec.frontend_port,
+ 'monitor_port': daemon_spec.ports[1] if daemon_spec.ports else spec.monitor_port,
+ }
+ )
+ config_files = {
+ 'files': {
+ "haproxy.cfg": haproxy_conf,
+ }
+ }
+ if spec.ssl_cert:
+ ssl_cert = spec.ssl_cert
+ if isinstance(ssl_cert, list):
+ ssl_cert = '\n'.join(ssl_cert)
+ config_files['files']['haproxy.pem'] = ssl_cert
+
+ return config_files, sorted(deps)
+
+ def keepalived_prepare_create(
+ self,
+ daemon_spec: CephadmDaemonDeploySpec,
+ ) -> CephadmDaemonDeploySpec:
+ assert daemon_spec.daemon_type == 'keepalived'
+
+ daemon_id = daemon_spec.daemon_id
+ host = daemon_spec.host
+ spec = cast(IngressSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
+
+ logger.info('Create daemon keepalived.%s on host %s with spec %s' % (
+ daemon_id, host, spec))
+
+ daemon_spec.final_config, daemon_spec.deps = self.keepalived_generate_config(daemon_spec)
+
+ return daemon_spec
+
+ def keepalived_generate_config(
+ self,
+ daemon_spec: CephadmDaemonDeploySpec,
+ ) -> Tuple[Dict[str, Any], List[str]]:
+ spec = cast(IngressSpec, self.mgr.spec_store[daemon_spec.service_name].spec)
+ assert spec.backend_service
+
+ # generate password?
+ pw_key = f'{spec.service_name()}/keepalived_password'
+ password = self.mgr.get_store(pw_key)
+ if password is None:
+ if not spec.keepalived_password:
+ password = ''.join(random.choice(string.ascii_lowercase) for _ in range(20))
+ self.mgr.set_store(pw_key, password)
+ else:
+ if spec.keepalived_password:
+ self.mgr.set_store(pw_key, None)
+ if spec.keepalived_password:
+ password = spec.keepalived_password
+
+ daemons = self.mgr.cache.get_daemons_by_service(spec.service_name())
+ deps = sorted([d.name() for d in daemons if d.daemon_type == 'haproxy'])
+
+ host = daemon_spec.host
+ hosts = sorted(list(set([str(d.hostname) for d in daemons])))
+
+ # interface
+ interface = 'eth0'
+ for subnet, ifaces in self.mgr.cache.networks.get(host, {}).items():
+ logger.info(f'subnet {subnet} ifaces {ifaces} virtual_ip {spec.virtual_ip}')
+ if ifaces and ipaddress.ip_address(spec.virtual_ip) in ipaddress.ip_network(subnet):
+ logger.info(f'{spec.virtual_ip} is in {subnet}')
+ interface = list(ifaces.keys())[0]
+ break
+
+ # script to monitor health
+ script = '/usr/bin/false'
+ for d in daemons:
+ if d.hostname == host:
+ if d.daemon_type == 'haproxy':
+ assert d.ports
+ port = d.ports[1] # monitoring port
+ script = f'/usr/bin/curl http://{d.ip or "localhost"}:{port}/health'
+ assert script
+
+ # set state. first host in placement is master all others backups
+ state = 'BACKUP'
+ if hosts[0] == host:
+ state = 'MASTER'
+
+ # remove host, daemon is being deployed on from hosts list for
+ # other_ips in conf file and converter to ips
+ hosts.remove(host)
+ other_ips = [resolve_ip(h) for h in hosts]
+
+ keepalived_conf = self.mgr.template.render(
+ 'services/ingress/keepalived.conf.j2',
+ {
+ 'spec': spec,
+ 'script': script,
+ 'password': password,
+ 'interface': interface,
+ 'state': state,
+ 'other_ips': other_ips,
+ 'host_ip': resolve_ip(host),
+ }
+ )
+
+ config_file = {
+ 'files': {
+ "keepalived.conf": keepalived_conf,
+ }
+ }
+
+ return config_file, deps
--- /dev/null
+# {{ cephadm_managed }}
+global
+ log 127.0.0.1 local2
+ chroot /var/lib/haproxy
+ pidfile /var/lib/haproxy/haproxy.pid
+ maxconn 8000
+ daemon
+ stats socket /var/lib/haproxy/stats
+{% if spec.ssl_cert %}
+ {% if spec.ssl_dh_param %}
+ tune.ssl.default-dh-param {{ spec.ssl_dh_param }}
+ {% endif %}
+ {% if spec.ssl_ciphers %}
+ ssl-default-bind-ciphers {{ spec.ssl_ciphers | join(':') }}
+ {% endif %}
+ {% if spec.ssl_options %}
+ ssl-default-bind-options {{ spec.ssl_options | join(' ') }}
+ {% endif %}
+{% endif %}
+
+defaults
+ mode http
+ log global
+ option httplog
+ option dontlognull
+ option http-server-close
+ option forwardfor except 127.0.0.0/8
+ option redispatch
+ retries 3
+ timeout http-request 1s
+ timeout queue 20s
+ timeout connect 5s
+ timeout client 1s
+ timeout server 1s
+ timeout http-keep-alive 5s
+ timeout check 5s
+ maxconn 8000
+
+frontend stats
+ bind {{ ip }}:{{ monitor_port }}
+ stats enable
+ stats uri /stats
+ stats refresh 10s
+ stats auth {{ user }}:{{ password }}
+ http-request use-service prometheus-exporter if { path /metrics }
+ monitor-uri /health
+
+frontend frontend
+{% if spec.ssl_cert %}
+ bind {{ ip }}:{{ frontend_port }} ssl crt /var/lib/haproxy/haproxy.pem
+{% else %}
+ bind {{ ip }}:{{ frontend_port }}
+{% endif %}
+ default_backend backend
+
+backend backend
+ option forwardfor
+ balance static-rr
+ option httpchk HEAD / HTTP/1.0
+ {% for server in servers %}
+ server {{ server.name }} {{ server.ip }}:{{ server.port }} check weight 100
+ {% endfor %}
--- /dev/null
+# {{ cephadm_managed }}
+vrrp_script check_backend {
+ script "{{ script }}"
+ weight -20
+ interval 2
+ rise 2
+ fall 2
+}
+
+vrrp_instance VI_0 {
+ state {{ state }}
+ priority 100
+ interface {{ interface }}
+ virtual_router_id 51
+ advert_int 1
+ authentication {
+ auth_type PASS
+ auth_pass {{ password }}
+ }
+ unicast_src_ip {{ host_ip }}
+ unicast_peer {
+ {% for ip in other_ips %}
+ {{ ip }}
+ {% endfor %}
+ }
+ virtual_ipaddress {
+ {{ spec.virtual_ip }} dev {{ interface }}
+ }
+ track_script {
+ check_backend
+ }
+}
import pytest
from ceph.deployment.hostspec import HostSpec
-from ceph.deployment.service_spec import ServiceSpec, PlacementSpec, ServiceSpecValidationError
+from ceph.deployment.service_spec import ServiceSpec, PlacementSpec, ServiceSpecValidationError, IngressSpec
from cephadm.module import HostAssignment
from cephadm.schedule import DaemonPlacement
'rgw:host1(10.0.0.1:82)', 'rgw:host2(10.0.0.2:82)'],
[]
),
+ NodeAssignmentTest4(
+ IngressSpec(
+ service_type='ingress',
+ service_id='rgw.foo',
+ frontend_port=443,
+ monitor_port=8888,
+ virtual_ip='10.0.0.20/8',
+ backend_service='rgw.foo',
+ placement=PlacementSpec(label='foo'),
+ networks=['10.0.0.0/8'],
+ ),
+ {
+ 'host1': {'10.0.0.0/8': {'eth0': ['10.0.0.1']}},
+ 'host2': {'10.0.0.0/8': {'eth1': ['10.0.0.2']}},
+ 'host3': {'192.168.0.0/16': {'eth2': ['192.168.0.1']}},
+ },
+ [],
+ ['haproxy:host1(10.0.0.1:443,8888)', 'haproxy:host2(10.0.0.2:443,8888)'],
+ ['haproxy:host1(10.0.0.1:443,8888)', 'haproxy:host2(10.0.0.2:443,8888)'],
+ []
+ ),
])
def test_node_assignment4(spec, networks, daemons,
expected, expected_add, expected_remove):
daemons=daemons,
allow_colo=True,
networks=networks,
+ primary_daemon_type='haproxy' if spec.service_type == 'ingress' else spec.service_type,
).place()
got = [str(p) for p in all_slots]
assert key not in config_json
-def test_HA_RGW_spec():
- yaml_str = """service_type: ha-rgw
-service_id: haproxy_for_rgw
+def test_ingress_spec():
+ yaml_str = """service_type: ingress
+service_id: rgw.foo
placement:
hosts:
- host1
- host2
- host3
spec:
- virtual_ip_interface: eth0
- virtual_ip_address: 192.168.20.1/24
+ virtual_ip: 192.168.20.1/24
+ backend_service: rgw.foo
frontend_port: 8080
- ha_proxy_port: 1967
- ha_proxy_stats_enabled: true
- ha_proxy_stats_user: admin
- ha_proxy_stats_password: admin
- ha_proxy_enable_prometheus_exporter: true
- ha_proxy_monitor_uri: /haproxy_health
- keepalived_password: admin
+ monitor_port: 8081
"""
yaml_file = yaml.safe_load(yaml_str)
spec = ServiceSpec.from_json(yaml_file)
- assert spec.service_type == "ha-rgw"
- assert spec.service_id == "haproxy_for_rgw"
- assert spec.virtual_ip_interface == "eth0"
- assert spec.virtual_ip_address == "192.168.20.1/24"
+ assert spec.service_type == "ingress"
+ assert spec.service_id == "rgw.foo"
+ assert spec.virtual_ip == "192.168.20.1/24"
assert spec.frontend_port == 8080
- assert spec.ha_proxy_port == 1967
- assert spec.ha_proxy_stats_enabled is True
- assert spec.ha_proxy_stats_user == "admin"
- assert spec.ha_proxy_stats_password == "admin"
- assert spec.ha_proxy_enable_prometheus_exporter is True
- assert spec.ha_proxy_monitor_uri == "/haproxy_health"
- assert spec.keepalived_password == "admin"
+ assert spec.monitor_port == 8081
from ceph.deployment import inventory
from ceph.deployment.service_spec import ServiceSpec, NFSServiceSpec, RGWSpec, \
- ServiceSpecValidationError, IscsiServiceSpec, HA_RGWSpec
+ ServiceSpecValidationError, IscsiServiceSpec, IngressSpec
from ceph.deployment.drive_group import DriveGroupSpec
from ceph.deployment.hostspec import HostSpec
from ceph.utils import datetime_to_str, str_to_datetime
'prometheus': self.apply_prometheus,
'rbd-mirror': self.apply_rbd_mirror,
'rgw': self.apply_rgw,
- 'ha-rgw': self.apply_ha_rgw,
+ 'ingress': self.apply_ingress,
'host': self.add_host,
'cephadm-exporter': self.apply_cephadm_exporter,
}
"""Update RGW cluster"""
raise NotImplementedError()
- def apply_ha_rgw(self, spec: HA_RGWSpec) -> OrchResult[str]:
- """Update ha-rgw daemons"""
+ def apply_ingress(self, spec: IngressSpec) -> OrchResult[str]:
+ """Update ingress daemons"""
raise NotImplementedError()
def apply_rbd_mirror(self, spec: ServiceSpec) -> OrchResult[str]:
'mds': 'mds',
'rgw': 'rgw',
'osd': 'osd',
- 'haproxy': 'ha-rgw',
- 'keepalived': 'ha-rgw',
+ 'haproxy': 'ingress',
+ 'keepalived': 'ingress',
'iscsi': 'iscsi',
'rbd-mirror': 'rbd-mirror',
'cephfs-mirror': 'cephfs-mirror',
'mds': ['mds'],
'rgw': ['rgw'],
'osd': ['osd'],
- 'ha-rgw': ['haproxy', 'keepalived'],
+ 'ingress': ['haproxy', 'keepalived'],
'iscsi': ['iscsi'],
'rbd-mirror': ['rbd-mirror'],
'cephfs-mirror': ['cephfs-mirror'],
# The type of service (osd, mon, mgr, etc.)
self.daemon_type = daemon_type
- assert daemon_type not in ['HA_RGW', 'ha-rgw']
-
# The orchestrator will have picked some names for daemons,
# typically either based on hostnames or on pod names.
# This is the <foo> in mds.<foo>, the ID that will appear
"""
KNOWN_SERVICE_TYPES = 'alertmanager crash grafana iscsi mds mgr mon nfs ' \
'node-exporter osd prometheus rbd-mirror rgw ' \
- 'container cephadm-exporter ha-rgw cephfs-mirror'.split()
- REQUIRES_SERVICE_ID = 'iscsi mds nfs osd rgw container ha-rgw '.split()
+ 'container cephadm-exporter ingress cephfs-mirror'.split()
+ REQUIRES_SERVICE_ID = 'iscsi mds nfs osd rgw container ingress '.split()
MANAGED_CONFIG_OPTIONS = [
'mds_join_fs',
]
'osd': DriveGroupSpec,
'iscsi': IscsiServiceSpec,
'alertmanager': AlertManagerSpec,
- 'ha-rgw': HA_RGWSpec,
+ 'ingress': IngressSpec,
'container': CustomContainerSpec,
}.get(service_type, cls)
if ret == ServiceSpec and not service_type:
yaml.add_representer(AlertManagerSpec, ServiceSpec.yaml_representer)
-class HA_RGWSpec(ServiceSpec):
+class IngressSpec(ServiceSpec):
def __init__(self,
- service_type: str = 'ha-rgw',
+ service_type: str = 'ingress',
service_id: Optional[str] = None,
config: Optional[Dict[str, str]] = None,
networks: Optional[List[str]] = None,
placement: Optional[PlacementSpec] = None,
- virtual_ip_interface: Optional[str] = None,
- virtual_ip_address: Optional[str] = None,
+ backend_service: Optional[str] = None,
frontend_port: Optional[int] = None,
- ha_proxy_port: Optional[int] = None,
- ha_proxy_stats_enabled: Optional[bool] = None,
- ha_proxy_stats_user: Optional[str] = None,
- ha_proxy_stats_password: Optional[str] = None,
- ha_proxy_enable_prometheus_exporter: Optional[bool] = None,
- ha_proxy_monitor_uri: Optional[str] = None,
+ ssl_cert: Optional[str] = None,
+ ssl_dh_param: Optional[str] = None,
+ ssl_ciphers: Optional[List[str]] = None,
+ ssl_options: Optional[List[str]] = None,
+ monitor_port: Optional[int] = None,
+ monitor_user: Optional[str] = None,
+ monitor_password: Optional[str] = None,
+ enable_stats: Optional[bool] = None,
keepalived_password: Optional[str] = None,
- ha_proxy_frontend_ssl_certificate: Optional[str] = None,
- ha_proxy_frontend_ssl_port: Optional[int] = None,
- ha_proxy_ssl_dh_param: Optional[str] = None,
- ha_proxy_ssl_ciphers: Optional[List[str]] = None,
- ha_proxy_ssl_options: Optional[List[str]] = None,
+ virtual_ip: Optional[str] = None,
haproxy_container_image: Optional[str] = None,
keepalived_container_image: Optional[str] = None,
- definitive_host_list: Optional[List[str]] = None
):
- assert service_type == 'ha-rgw'
- super(HA_RGWSpec, self).__init__('ha-rgw', service_id=service_id,
- placement=placement, config=config,
- networks=networks)
-
- self.virtual_ip_interface = virtual_ip_interface
- self.virtual_ip_address = virtual_ip_address
+ assert service_type == 'ingress'
+ super(IngressSpec, self).__init__(
+ 'ingress', service_id=service_id,
+ placement=placement, config=config,
+ networks=networks
+ )
+ self.backend_service = backend_service
self.frontend_port = frontend_port
- self.ha_proxy_port = ha_proxy_port
- self.ha_proxy_stats_enabled = ha_proxy_stats_enabled
- self.ha_proxy_stats_user = ha_proxy_stats_user
- self.ha_proxy_stats_password = ha_proxy_stats_password
- self.ha_proxy_enable_prometheus_exporter = ha_proxy_enable_prometheus_exporter
- self.ha_proxy_monitor_uri = ha_proxy_monitor_uri
+ self.ssl_cert = ssl_cert
+ self.ssl_dh_param = ssl_dh_param
+ self.ssl_ciphers = ssl_ciphers
+ self.ssl_options = ssl_options
+ self.monitor_port = monitor_port
+ self.monitor_user = monitor_user
+ self.monitor_password = monitor_password
self.keepalived_password = keepalived_password
- self.ha_proxy_frontend_ssl_certificate = ha_proxy_frontend_ssl_certificate
- self.ha_proxy_frontend_ssl_port = ha_proxy_frontend_ssl_port
- self.ha_proxy_ssl_dh_param = ha_proxy_ssl_dh_param
- self.ha_proxy_ssl_ciphers = ha_proxy_ssl_ciphers
- self.ha_proxy_ssl_options = ha_proxy_ssl_options
+ self.virtual_ip = virtual_ip
self.haproxy_container_image = haproxy_container_image
self.keepalived_container_image = keepalived_container_image
- # placeholder variable. Need definitive list of hosts this service will
- # be placed on in order to generate keepalived config. Will be populated
- # when applying spec
- self.definitive_host_list = [] # type: List[str]
+
+ def get_port_start(self) -> List[int]:
+ return [cast(int, self.frontend_port),
+ cast(int, self.monitor_port)]
def validate(self) -> None:
- super(HA_RGWSpec, self).validate()
+ super(IngressSpec, self).validate()
- if not self.virtual_ip_interface:
- raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: No Virtual IP Interface specified')
- if not self.virtual_ip_address:
+ if not self.backend_service:
raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: No Virtual IP Address specified')
- if not self.frontend_port and not self.ha_proxy_frontend_ssl_certificate:
+ 'Cannot add ingress: No backend_service specified')
+ if not self.frontend_port:
raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: No Frontend Port specified')
- if not self.ha_proxy_port:
+ 'Cannot add ingress: No frontend_port specified')
+ if not self.monitor_port:
raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: No HA Proxy Port specified')
- if not self.ha_proxy_stats_enabled:
+ 'Cannot add ingress: No monitor_port specified')
+ if not self.virtual_ip:
raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: Ha Proxy Stats Enabled option not set')
- if not self.ha_proxy_stats_user:
- raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: No HA Proxy Stats User specified')
- if not self.ha_proxy_stats_password:
- raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: No HA Proxy Stats Password specified')
- if not self.ha_proxy_enable_prometheus_exporter:
- raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: HA Proxy Enable Prometheus Exporter option not set')
- if not self.ha_proxy_monitor_uri:
- raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: No HA Proxy Monitor Uri specified')
- if not self.keepalived_password:
- raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: No Keepalived Password specified')
- if self.ha_proxy_frontend_ssl_certificate:
- if not self.ha_proxy_frontend_ssl_port:
- raise ServiceSpecValidationError(
- 'Cannot add ha-rgw: Specified Ha Proxy Frontend SSL ' +
- 'Certificate but no SSL Port')
+ 'Cannot add ingress: No virtual_ip provided')
class CustomContainerSpec(ServiceSpec):