implement an initial development environment for receptor-based clusters

This commit is contained in:
Ryan Petrello 2021-03-16 13:30:56 -04:00 committed by Alan Rominger
parent 4a271d6897
commit 05cb876df5
No known key found for this signature in database
GPG Key ID: C2D7EAAA12B63559
22 changed files with 396 additions and 286 deletions

View File

@ -173,7 +173,20 @@ init:
. $(VENV_BASE)/awx/bin/activate; \
fi; \
$(MANAGEMENT_COMMAND) provision_instance --hostname=$(COMPOSE_HOST); \
$(MANAGEMENT_COMMAND) register_queue --queuename=controlplane --instance_percent=100;
$(MANAGEMENT_COMMAND) register_queue --queuename=controlplane --instance_percent=100;\
if [ "$(AWX_GROUP_QUEUES)" == "tower,thepentagon" ]; then \
$(MANAGEMENT_COMMAND) provision_instance --hostname=isolated; \
$(MANAGEMENT_COMMAND) register_queue --queuename='thepentagon' --hostnames=isolated --controller=tower; \
$(MANAGEMENT_COMMAND) generate_isolated_key > /awx_devel/awx/main/isolated/authorized_keys; \
fi;
if [ ! -f /etc/receptor/certs/awx.key ]; then \
rm -f /etc/receptor/certs/*; \
receptor --cert-init commonname="AWX Test CA" bits=2048 outcert=/etc/receptor/certs/ca.crt outkey=/etc/receptor/certs/ca.key; \
for node in $(RECEPTOR_MUTUAL_TLS); do \
receptor --cert-makereq bits=2048 commonname="$$node test cert" dnsname=$$node nodeid=$$node outreq=/etc/receptor/certs/$$node.csr outkey=/etc/receptor/certs/$$node.key; \
receptor --cert-signreq req=/etc/receptor/certs/$$node.csr cacert=/etc/receptor/certs/ca.crt cakey=/etc/receptor/certs/ca.key outcert=/etc/receptor/certs/$$node.crt verify=yes; \
done; \
fi; \
# Refresh development environment after pulling new code.
refresh: clean requirements_dev version_file develop migrate
@ -536,6 +549,9 @@ docker-refresh: docker-clean docker-compose
docker-compose-elk: docker-auth awx/projects docker-compose-sources
docker-compose -f tools/docker-compose/_sources/docker-compose.yml -f tools/elastic/docker-compose.logstash-link.yml -f tools/elastic/docker-compose.elastic-override.yml up --no-recreate
docker-compose-cluster: docker-auth awx/projects docker-compose-sources
docker-compose -f tools/docker-compose/_sources/docker-compose.yml -f tools/docker-receptor.yml up
docker-compose-cluster-elk: docker-auth awx/projects docker-compose-sources
docker-compose -f tools/docker-compose/_sources/docker-compose.yml -f tools/elastic/docker-compose.logstash-link-cluster.yml -f tools/elastic/docker-compose.elastic-override.yml up --no-recreate

View File

@ -4773,6 +4773,7 @@ class InstanceSerializer(BaseSerializer):
percent_capacity_remaining = serializers.SerializerMethodField()
jobs_running = serializers.IntegerField(help_text=_('Count of jobs in the running or waiting state that ' 'are targeted for this instance'), read_only=True)
jobs_total = serializers.IntegerField(help_text=_('Count of all jobs that target this instance'), read_only=True)
is_receptor = serializers.SerializerMethodField()
class Meta:
model = Instance
@ -4817,6 +4818,9 @@ class InstanceSerializer(BaseSerializer):
else:
return float("{0:.2f}".format(((float(obj.capacity) - float(obj.consumed_capacity)) / (float(obj.capacity))) * 100))
def get_is_receptor(self, obj):
return obj.is_receptor()
class InstanceGroupSerializer(BaseSerializer):

View File

@ -370,15 +370,6 @@ class InstanceDetail(RetrieveUpdateAPIView):
model = models.Instance
serializer_class = serializers.InstanceSerializer
def update(self, request, *args, **kwargs):
r = super(InstanceDetail, self).update(request, *args, **kwargs)
if status.is_success(r.status_code):
obj = self.get_object()
obj.refresh_capacity()
obj.save()
r.data = serializers.InstanceSerializer(obj, context=self.get_serializer_context()).to_representation(obj)
return r
class InstanceUnifiedJobsList(SubListAPIView):

View File

@ -48,7 +48,8 @@ class Command(BaseCommand):
if no_color:
color = ''
fmt = '\t' + color + '{0.hostname} capacity={0.capacity} version={1}'
if x.capacity:
fmt += ' heartbeat="{0.modified:%Y-%m-%d %H:%M:%S}"'
if x.last_isolated_check:
fmt += ' last_isolated_check="{0.last_isolated_check:%Y-%m-%d %H:%M:%S}"'
fmt += ' heartbeat="{0.modified:%Y-%m-%d %H:%M:%S}"'
print((fmt + '\033[0m').format(x, x.version or '?'))
print('')

View File

@ -125,33 +125,18 @@ class Instance(HasPolicyEditsMixin, BaseModel):
def jobs_total(self):
return UnifiedJob.objects.filter(execution_node=self.hostname).count()
@staticmethod
def choose_online_control_plane_node():
return random.choice(Instance.objects.filter(enabled=True).exclude(version__startswith='ansible-runner-').values_list('hostname', flat=True))
def is_lost(self, ref_time=None):
if ref_time is None:
ref_time = now()
grace_period = 120
return self.modified < ref_time - timedelta(seconds=grace_period)
def refresh_capacity(self):
cpu = get_cpu_capacity()
mem = get_mem_capacity()
if self.enabled:
self.capacity = get_system_task_capacity(self.capacity_adjustment)
else:
self.capacity = 0
try:
# if redis is down for some reason, that means we can't persist
# playbook event data; we should consider this a zero capacity event
redis.Redis.from_url(settings.BROKER_URL).ping()
except redis.ConnectionError:
self.capacity = 0
self.cpu = cpu[0]
self.memory = mem[0]
self.cpu_capacity = cpu[1]
self.mem_capacity = mem[1]
self.version = awx_application_version
self.save(update_fields=['capacity', 'version', 'modified', 'cpu', 'memory', 'cpu_capacity', 'mem_capacity'])
def is_receptor(self):
return self.version.startswith('ansible-runner-')
class InstanceGroup(HasPolicyEditsMixin, BaseModel, RelatedJobsMixin):

View File

@ -253,6 +253,12 @@ class TaskManager:
}
dependencies = [{'type': get_type_for_model(type(t)), 'id': t.id} for t in dependent_tasks]
try:
controller_node = Instance.choose_online_control_plane_node()
except IndexError:
logger.warning("No control plane nodes available to manage {}".format(task.log_format))
return
task.status = 'waiting'
(start_status, opts) = task.pre_start()
@ -289,9 +295,9 @@ class TaskManager:
logger.debug('Submitting containerized {} to queue {}.'.format(task.log_format, task.execution_node))
else:
task.instance_group = rampart_group
if instance is not None:
task.execution_node = instance.hostname
logger.debug('Submitting {} to <instance group, instance> <{},{}>.'.format(task.log_format, task.instance_group_id, task.execution_node))
task.execution_node = instance.hostname
task.controller_node = controller_node
logger.debug('Submitting job {} to queue {} controlled by {}.'.format(task.log_format, task.execution_node, controller_node))
with disable_activity_stream():
task.celery_task_id = str(uuid.uuid4())
task.save()

View File

@ -55,7 +55,11 @@ import ansible_runner
# Receptor
from receptorctl.socket_interface import ReceptorControl
# dateutil
from dateutil.parser import parse as parse_date
# AWX
from awx import MODE
from awx import __version__ as awx_application_version
from awx.main.constants import PRIVILEGE_ESCALATION_METHODS, STANDARD_INVENTORY_UPDATE_ENV, MINIMAL_EVENTS
from awx.main.access import access_registry
@ -98,6 +102,9 @@ from awx.main.utils.common import (
parse_yaml_or_json,
cleanup_new_process,
create_partition,
get_mem_capacity,
get_cpu_capacity,
get_system_task_capacity,
)
from awx.main.utils.execution_environments import get_default_pod_spec, CONTAINER_ROOT, to_container_path
from awx.main.utils.ansible import read_ansible_config
@ -114,6 +121,9 @@ from awx.main.analytics.subsystem_metrics import Metrics
from rest_framework.exceptions import PermissionDenied
RECEPTOR_SOCK = '/var/run/receptor/receptor.sock'
RECEPTOR_PENDING = 'ansible-runner-???'
__all__ = [
'RunJob',
@ -398,48 +408,74 @@ def cleanup_execution_environment_images():
if process.returncode != 0:
logger.debug(f"Failed to delete image {image_name}")
@task(queue=get_local_queuename)
def check_heartbeat(node):
AWXReceptorJob.check_heartbeat(node)
def discover_receptor_nodes():
ctl = ReceptorControl(RECEPTOR_SOCK)
connections = ctl.simple_command('status')['Advertisements']
nowtime = now()
for ad in connections:
hostname = ad['NodeID']
commands = ad['WorkCommands'] or []
if 'ansible-runner' not in commands:
if 'local' in commands:
# this node is strictly a control plane node, and does not
# provide ansible-runner as a work command
(changed, instance) = Instance.objects.register(hostname=hostname)
if changed:
logger.info("Registered tower control node '{}'".format(hostname))
instance.capacity = instance.cpu = instance.memory = instance.cpu_capacity = instance.mem_capacity = 0 # noqa
instance.version = get_awx_version()
instance.save(update_fields=['capacity', 'version', 'cpu', 'memory', 'cpu_capacity', 'mem_capacity'])
continue
(changed, instance) = Instance.objects.register(hostname=hostname)
was_lost = instance.is_lost(ref_time=nowtime)
if changed:
logger.info("Registered tower execution node '{}'".format(hostname))
instance.capacity = 0
instance.version = RECEPTOR_PENDING
instance.save(update_fields=['capacity', 'version', 'modified'])
check_heartbeat.apply_async([hostname])
else:
last_seen = parse_date(ad['Time'])
logger.debug("Updated tower control node '{}' last seen {}".format(hostname, last_seen))
instance.modified = last_seen
if instance.is_lost(ref_time=nowtime):
# if the instance hasn't advertised in awhile,
# don't save a new modified time
continue
instance.save(update_fields=['modified'])
if was_lost:
# if the instance *was* lost, but has appeared again,
# attempt to re-establish the initial capacity and version
# check
logger.warning('Attempting to rejoin the cluster as instance {}.'.format(hostname))
check_heartbeat.apply_async([hostname])
@task(queue=get_local_queuename)
def cluster_node_heartbeat():
logger.debug("Cluster node heartbeat task.")
nowtime = now()
instance_list = list(Instance.objects.all())
this_inst = None
lost_instances = []
(changed, instance) = Instance.objects.get_or_register()
if changed:
logger.info("Registered tower node '{}'".format(instance.hostname))
logger.info("Registered tower control node '{}'".format(instance.hostname))
discover_receptor_nodes()
for inst in list(instance_list):
if inst.hostname == settings.CLUSTER_HOST_ID:
this_inst = inst
instance_list.remove(inst)
elif inst.is_lost(ref_time=nowtime):
lost_instances.append(inst)
instance_list.remove(inst)
if this_inst:
startup_event = this_inst.is_lost(ref_time=nowtime)
this_inst.refresh_capacity()
if startup_event:
logger.warning('Rejoining the cluster as instance {}.'.format(this_inst.hostname))
return
else:
raise RuntimeError("Cluster Host Not Found: {}".format(settings.CLUSTER_HOST_ID))
# IFF any node has a greater version than we do, then we'll shutdown services
for other_inst in instance_list:
if other_inst.version == "":
continue
if Version(other_inst.version.split('-', 1)[0]) > Version(awx_application_version.split('-', 1)[0]) and not settings.DEBUG:
logger.error(
"Host {} reports version {}, but this node {} is at {}, shutting down".format(
other_inst.hostname, other_inst.version, this_inst.hostname, this_inst.version
)
)
# Shutdown signal will set the capacity to zero to ensure no Jobs get added to this instance.
# The heartbeat task will reset the capacity to the system capacity after upgrade.
stop_local_services(communicate=False)
raise RuntimeError("Shutting down.")
for other_inst in lost_instances:
try:
reaper.reap(other_inst)
@ -1720,6 +1756,7 @@ class RunJob(BaseTask):
]
)
params['process_isolation'] = False if MODE == 'development' else True
return params
def pre_run_hook(self, job, private_data_dir):
@ -2793,6 +2830,10 @@ class RunAdHocCommand(BaseTask):
d[r'Password:\s*?$'] = 'ssh_password'
return d
def build_execution_environment_params(self, instance, private_data_dir):
params = super(RunAdHocCommand, self).build_execution_environment_params(instance)
params['process_isolation'] = False if MODE == 'development' else True
return params
@task(queue=get_local_queuename)
class RunSystemJob(BaseTask):
@ -2927,18 +2968,120 @@ class AWXReceptorJob:
execution_environment_params = self.task.build_execution_environment_params(self.task.instance, runner_params['private_data_dir'])
self.runner_params['settings'].update(execution_environment_params)
def run(self):
def run(self, work_type=None):
# We establish a connection to the Receptor socket
receptor_ctl = get_receptor_ctl()
# TODO: Seems like we already have a method for doing this now?
# receptor_ctl = get_receptor_ctl()
receptor_ctl = ReceptorControl(RECEPTOR_SOCK)
try:
return self._run_internal(receptor_ctl)
return self._run_internal(receptor_ctl, work_type=work_type)
finally:
# Make sure to always release the work unit if we established it
if self.unit_id is not None and settings.RECEPTOR_RELEASE_WORK:
receptor_ctl.simple_command(f"work release {self.unit_id}")
def _run_internal(self, receptor_ctl):
@classmethod
def check_heartbeat(cls, node):
# make a private data dir and env dir
private_data_dir = tempfile.mkdtemp(prefix='awx_heartbeat_', dir=settings.AWX_ISOLATION_BASE_PATH)
env_path = os.path.join(private_data_dir, 'env')
os.makedirs(os.path.join(env_path), mode=0o700)
# write a cmdline file for adhoc
f = os.fdopen(os.open(os.path.join(env_path, 'cmdline'), os.O_RDWR | os.O_CREAT, stat.S_IREAD | stat.S_IWRITE), 'w')
f.write(ansible_runner.utils.args2cmdline('localhost'))
f.close()
# write a custom facts.d to report the runner version
facts_path = os.path.join(private_data_dir, 'facts.d')
os.makedirs(facts_path, mode=0o700)
with open(os.path.join(facts_path, 'ansible_runner.fact'), 'w') as f:
os.chmod(f.name, 0o700)
f.write("""#!/usr/bin/env sh\necho "{\\"version\\": \\"`ansible-runner --version`\\"}"\n""") # noqa
# write a local inventory
inventory_path = os.path.join(private_data_dir, 'inventory')
os.makedirs(inventory_path, mode=0o700)
fn = os.path.join(inventory_path, 'hosts')
with open(fn, 'w') as f:
os.chmod(fn, stat.S_IRUSR | stat.S_IXUSR | stat.S_IWUSR)
f.write('localhost ansible_connection=local')
runner_params = {
'ident': str(uuid4()),
'private_data_dir': private_data_dir,
'module': 'setup',
'module_args': f'fact_path={private_data_dir}/facts.d',
'inventory': inventory_path,
'only_transmit_kwargs': False,
'settings': {
"container_image": get_default_execution_environment().image,
"container_options": ['--user=root'],
"process_isolation": False if MODE == 'development' else True,
},
}
class _Instance(object):
pk = -1
job_env = {}
@property
def is_container_group_task(self):
return False
@property
def execution_node(self):
return node
class _BaseTask(object):
instance = _Instance()
cpus = 0
mem_mb = 0
version = RECEPTOR_PENDING
def build_execution_environment_params(self, instance, private_data_dir):
return {}
def event_handler(self, event_data):
if event_data.get('event') == 'runner_on_ok':
facts = event_data.get('event_data', {}).get('res', {}).get('ansible_facts', {})
if facts:
self.cpus = facts.get('ansible_processor_vcpus', 0)
self.mem_mb = facts.get('ansible_memtotal_mb', 0)
version = facts.get('ansible_local', {}).get('ansible_runner', {}).get('version', '') # noqa
if version:
self.version = f'ansible-runner-{version}'
def finished_callback(self, runner_obj):
pass
def cancel_callback(self):
pass
def status_handler(self, status_data, runner_config):
pass
def update_model(self, *args, **kw):
pass
task = _BaseTask()
receptor_job = cls(task, runner_params)
res = receptor_job.run(work_type='ansible-runner')
if res.status == 'successful':
cpu = get_cpu_capacity(task.cpus)
mem = get_cpu_capacity(task.mem_mb)
instance = Instance.objects.get(hostname=node)
instance.cpu = cpu[0]
instance.cpu_capacity = cpu[1]
instance.memory = mem[0]
instance.memory_capacity = mem[1]
instance.capacity = get_system_task_capacity(
instance.capacity_adjustment,
instance.cpu_capacity,
instance.mem_capacity,
)
instance.version = task.version
instance.save(update_fields=['capacity', 'version', 'cpu', 'memory', 'cpu_capacity', 'mem_capacity'])
def _run_internal(self, receptor_ctl, work_type=None):
# Create a socketpair. Where the left side will be used for writing our payload
# (private data dir, kwargs). The right side will be passed to Receptor for
# reading.
@ -2949,7 +3092,14 @@ class AWXReceptorJob:
# submit our work, passing
# in the right side of our socketpair for reading.
result = receptor_ctl.submit_work(worktype=self.work_type, payload=sockout.makefile('rb'), params=self.receptor_params)
_kw = {}
work_type = work_type or self.work_type
if work_type == 'ansible-runner':
_kw['node'] = self.task.instance.execution_node
logger.debug(f'receptorctl.submit_work(node={_kw["node"]})')
else:
logger.debug(f'receptorctl.submit_work({work_type})')
result = receptor_ctl.submit_work(worktype=work_type, payload=sockout.makefile('rb'), params=self.receptor_params, **_kw)
self.unit_id = result['unitid']
self.task.update_model(self.task.instance.pk, work_unit_id=result['unitid'])
@ -3005,7 +3155,7 @@ class AWXReceptorJob:
# write our payload to the left side of our socketpair.
@cleanup_new_process
def transmit(self, _socket):
if not settings.IS_K8S and self.work_type == 'local':
if not settings.IS_K8S and self.work_type == 'local' and 'only_transmit_kwargs' not in self.runner_params:
self.runner_params['only_transmit_kwargs'] = True
try:
@ -3052,6 +3202,8 @@ class AWXReceptorJob:
work_type = 'kubernetes-runtime-auth'
else:
work_type = 'kubernetes-incluster-auth'
elif isinstance(self.task.instance, (Job, AdHocCommand)):
work_type = 'ansible-runner'
else:
work_type = 'local'

View File

@ -1,11 +1,7 @@
import redis
import pytest
from unittest import mock
import json
from awx.main.models import Job, Instance, JobHostSummary, InventoryUpdate, InventorySource, Project, ProjectUpdate, SystemJob, AdHocCommand
from awx.main.tasks import cluster_node_heartbeat
from django.test.utils import override_settings
from awx.main.models import Job, JobHostSummary, InventoryUpdate, InventorySource, Project, ProjectUpdate, SystemJob, AdHocCommand
@pytest.mark.django_db
@ -19,36 +15,6 @@ def test_orphan_unified_job_creation(instance, inventory):
assert job2.launch_type == 'relaunch'
@pytest.mark.django_db
@mock.patch('awx.main.utils.common.get_cpu_capacity', lambda: (2, 8))
@mock.patch('awx.main.utils.common.get_mem_capacity', lambda: (8000, 62))
def test_job_capacity_and_with_inactive_node():
i = Instance.objects.create(hostname='test-1')
with mock.patch.object(redis.client.Redis, 'ping', lambda self: True):
i.refresh_capacity()
assert i.capacity == 62
i.enabled = False
i.save()
with override_settings(CLUSTER_HOST_ID=i.hostname):
cluster_node_heartbeat()
i = Instance.objects.get(id=i.id)
assert i.capacity == 0
@pytest.mark.django_db
@mock.patch('awx.main.utils.common.get_cpu_capacity', lambda: (2, 8))
@mock.patch('awx.main.utils.common.get_mem_capacity', lambda: (8000, 62))
def test_job_capacity_with_redis_disabled():
i = Instance.objects.create(hostname='test-1')
def _raise(self):
raise redis.ConnectionError()
with mock.patch.object(redis.client.Redis, 'ping', _raise):
i.refresh_capacity()
assert i.capacity == 0
@pytest.mark.django_db
def test_job_type_name():
job = Job.objects.create()

View File

@ -15,7 +15,6 @@ import urllib.parse
import threading
import contextlib
import tempfile
import psutil
from functools import reduce, wraps
from decimal import Decimal
@ -699,7 +698,7 @@ def parse_yaml_or_json(vars_str, silent_failure=True):
return vars_dict
def get_cpu_capacity():
def get_cpu_capacity(raw):
from django.conf import settings
settings_forkcpu = getattr(settings, 'SYSTEM_TASK_FORKS_CPU', None)
@ -713,18 +712,16 @@ def get_cpu_capacity():
elif settings_abscpu is not None:
return 0, int(settings_abscpu)
cpu = psutil.cpu_count()
if env_forkcpu:
forkcpu = int(env_forkcpu)
elif settings_forkcpu:
forkcpu = int(settings_forkcpu)
else:
forkcpu = 4
return (cpu, cpu * forkcpu)
return (raw, raw * forkcpu)
def get_mem_capacity():
def get_mem_capacity(raw_mb):
from django.conf import settings
settings_forkmem = getattr(settings, 'SYSTEM_TASK_FORKS_MEM', None)
@ -745,8 +742,7 @@ def get_mem_capacity():
else:
forkmem = 100
mem = psutil.virtual_memory().total
return (mem, max(1, ((mem // 1024 // 1024) - 2048) // forkmem))
return (raw_mb, max(1, ((raw_mb // 1024 // 1024) - 2048) // forkmem))
def get_system_task_capacity(scale=Decimal(1.0), cpu_capacity=None, mem_capacity=None):

View File

@ -32,7 +32,7 @@ def unwrap_broadcast_msg(payload: dict):
def get_broadcast_hosts():
Instance = apps.get_model('main', 'Instance')
instances = Instance.objects.exclude(hostname=Instance.objects.me().hostname).order_by('hostname').values('hostname', 'ip_address').distinct()
instances = Instance.objects.exclude(hostname=Instance.objects.me().hostname).exclude(version__startswith='ansible-runner-').order_by('hostname').values('hostname', 'ip_address').distinct()
return {i['hostname']: i['ip_address'] or i['hostname'] for i in instances}

View File

@ -125,6 +125,8 @@ if "pytest" in sys.modules:
}
}
CELERYBEAT_SCHEDULE['cluster_heartbeat']['schedule'] = timedelta(seconds=10)
CLUSTER_HOST_ID = socket.gethostname()
AWX_CALLBACK_PROFILE = True

View File

@ -65,6 +65,9 @@ function InstanceListItem({
const labelId = `check-action-${instance.id}`;
function usedCapacity(item) {
if (!item.is_receptor) {
return ``;
}
if (item.enabled) {
return (
<Progress

View File

@ -1,119 +0,0 @@
---
version: '2'
services:
haproxy:
build:
context: ./docker-compose
dockerfile: Dockerfile-haproxy
container_name: tools_haproxy_1
depends_on:
- "awx-1"
- "awx-2"
- "awx-3"
ports:
- "8013:8013"
- "8043:8043"
- "1936:1936"
awx-1:
user: ${CURRENT_UID}
container_name: tools_awx_1_1
privileged: true
image: ${DEV_DOCKER_TAG_BASE}/awx_devel:${TAG}
hostname: awx-1
environment:
CURRENT_UID:
SDB_HOST: 0.0.0.0
SDB_PORT: 5899
AWX_GROUP_QUEUES: alpha,tower
command: launch_awx.sh
working_dir: "/awx_devel"
volumes:
- "../:/awx_devel"
- "./redis/redis_socket_ha_1:/var/run/redis/"
- "./docker-compose/supervisor.conf:/etc/supervisord.conf"
- "./docker-compose-cluster/awx-1-receptor.conf:/etc/receptor/receptor.conf"
ports:
- "2222:2222"
- "5899-5999:5899-5999"
awx-2:
user: ${CURRENT_UID}
container_name: tools_awx_2_1
privileged: true
image: ${DEV_DOCKER_TAG_BASE}/awx_devel:${TAG}
hostname: awx-2
command: launch_awx.sh
working_dir: "/awx_devel"
environment:
CURRENT_UID:
SDB_HOST: 0.0.0.0
SDB_PORT: 7899
AWX_GROUP_QUEUES: bravo,tower
volumes:
- "../:/awx_devel"
- "./redis/redis_socket_ha_2:/var/run/redis/"
- "./docker-compose/supervisor.conf:/etc/supervisord.conf"
- "./docker-compose-cluster/awx-2-receptor.conf:/etc/receptor/receptor.conf"
ports:
- "2223:2222"
- "7899-7999:7899-7999"
awx-3:
user: ${CURRENT_UID}
container_name: tools_awx_3_1
privileged: true
image: ${DEV_DOCKER_TAG_BASE}/awx_devel:${TAG}
hostname: awx-3
command: launch_awx.sh
working_dir: "/awx_devel"
environment:
CURRENT_UID:
SDB_HOST: 0.0.0.0
SDB_PORT: 8899
AWX_GROUP_QUEUES: charlie,tower
volumes:
- "../:/awx_devel"
- "./redis/redis_socket_ha_3:/var/run/redis/"
- "./docker-compose/supervisor.conf:/etc/supervisord.conf"
- "./docker-compose-cluster/awx-3-receptor.conf:/etc/receptor/receptor.conf"
ports:
- "2224:2222"
- "8899-8999:8899-8999"
redis_1:
user: ${CURRENT_UID}
image: redis:latest
container_name: tools_redis_1_1
command: ["/usr/local/etc/redis/redis.conf"]
volumes:
- "./redis/redis.conf:/usr/local/etc/redis/redis.conf"
- "./redis/redis_socket_ha_1:/var/run/redis/"
redis_2:
user: ${CURRENT_UID}
image: redis:latest
container_name: tools_redis_2_1
command: ["/usr/local/etc/redis/redis.conf"]
volumes:
- "./redis/redis.conf:/usr/local/etc/redis/redis.conf"
- "./redis/redis_socket_ha_2:/var/run/redis/"
redis_3:
user: ${CURRENT_UID}
image: redis:latest
container_name: tools_redis_3_1
command: ["/usr/local/etc/redis/redis.conf"]
volumes:
- "./redis/redis.conf:/usr/local/etc/redis/redis.conf"
- "./redis/redis_socket_ha_3:/var/run/redis/"
postgres:
image: postgres:12
container_name: tools_postgres_1
environment:
POSTGRES_HOST_AUTH_METHOD: trust
volumes:
- "awx_db:/var/lib/postgresql/data"
volumes:
awx_db:

View File

View File

@ -1,23 +1,25 @@
---
- node:
id: receptor-1
- log-level: info
#- tls-client:
# name: mutual-tls-client
# cert: /etc/receptor/certs/receptor-1.crt
# key: /etc/receptor/certs/receptor-1.key
# rootcas: /etc/receptor/certs/ca.crt
- tcp-peer:
address: receptor-hop:5555
redial: true
#tls: mutual-tls-client
- work-command:
worktype: ansible-runner
command: ansible-runner
params: worker
allowruntimeparams: true
- control-service:
service: control
filename: /var/run/receptor/receptor.sock
- tcp-listener:
port: 2200
- tcp-peer:
address: awx-2:2200
- work-command:
worktype: worker
command: ansible-runner
params: worker
- work-kubernetes:
worktype: ocp
namespace: receptor
image: quay.io/shanemcd/ee
authmethod: kubeconfig

View File

@ -1,23 +1,25 @@
---
- node:
id: receptor-2
- log-level: info
#- tls-client:
# name: mutual-tls-client
# cert: /etc/receptor/certs/receptor-2.crt
# key: /etc/receptor/certs/receptor-2.key
# rootcas: /etc/receptor/certs/ca.crt
- tcp-peer:
address: receptor-hop:5555
redial: true
#tls: mutual-tls-client
- work-command:
worktype: ansible-runner
command: ansible-runner
params: worker
allowruntimeparams: true
- control-service:
service: control
filename: /var/run/receptor/receptor.sock
- tcp-listener:
port: 2200
- tcp-peer:
address: awx-3:2200
- work-command:
worktype: worker
command: ansible-runner
params: worker
- work-kubernetes:
worktype: ocp
namespace: receptor
image: quay.io/shanemcd/ee
authmethod: kubeconfig

View File

@ -1,23 +1,25 @@
---
- node:
id: receptor-3
- log-level: info
#- tls-client:
# name: mutual-tls-client
# cert: /etc/receptor/certs/receptor-3.crt
# key: /etc/receptor/certs/receptor-3.key
# rootcas: /etc/receptor/certs/ca.crt
- tcp-peer:
address: receptor-hop:5555
redial: true
#tls: mutual-tls-client
- work-command:
worktype: ansible-runner
command: ansible-runner
params: worker
allowruntimeparams: true
- control-service:
service: control
filename: /var/run/receptor/receptor.sock
- tcp-listener:
port: 2200
- tcp-peer:
address: awx-1:2200
- work-command:
worktype: worker
command: ansible-runner
params: worker
- work-kubernetes:
worktype: ocp
namespace: receptor
image: quay.io/shanemcd/ee
authmethod: kubeconfig

View File

@ -0,0 +1,26 @@
---
- node:
id: receptor-hop
- log-level: info
#- tls-server:
# name: mutual-tls-server
# cert: /etc/receptor/certs/receptor-hop.crt
# key: /etc/receptor/certs/receptor-hop.key
# clientcas: /etc/receptor/certs/ca.crt
- tls-client:
name: mutual-tls-client
cert: /etc/receptor/certs/receptor-hop.crt
key: /etc/receptor/certs/receptor-hop.key
rootcas: /etc/receptor/certs/ca.crt
- tcp-peer:
address: awx_1:4444
redial: true
#tls: mutual-tls-client
- tcp-listener:
port: 5555
#tls: mutual-tls-server

View File

@ -0,0 +1,2 @@
*
!.gitignore

View File

@ -36,6 +36,7 @@ services:
- "../../docker-compose/_sources/local_settings.py:/etc/tower/conf.d/local_settings.py"
- "../../docker-compose/_sources/SECRET_KEY:/etc/tower/SECRET_KEY"
- "../../docker-compose/receptor.conf:/etc/receptor/receptor.conf"
- "../../docker-compose-cluster/certs:/etc/receptor/certs"
- "/sys/fs/cgroup:/sys/fs/cgroup"
- "~/.kube/config:/var/lib/awx/.kube/config"
- "redis_socket_{{ container_postfix }}:/var/run/redis/:rw"

View File

@ -1,12 +1,20 @@
---
- node:
id: foo
id: awx_1
- log-level: debug
- log-level:
debug
- tcp-listener:
port: 2222
#- tls-server:
# name: mutual-tls
# cert: /etc/receptor/certs/awx.crt
# key: /etc/receptor/certs/awx.key
# requireclientcert: true
# clientcas: /etc/receptor/certs/ca.crt
- control-service:
service: control
filename: /var/run/receptor/receptor.sock
@ -30,3 +38,7 @@
allowruntimeauth: true
allowruntimepod: true
allowruntimeparams: true
- tcp-listener:
port: 4444
#tls: mutual-tls

60
tools/docker-receptor.yml Normal file
View File

@ -0,0 +1,60 @@
#
# a simple receptor network with a single control plane node and one hop
# to nodes where receptor + runner are installed
#
# <awx:4444>
# <receptor-hop:5555>
# ^-------------- <receptor-1>
# ^-------------- <receptor-2>
# ^-------------- <receptor-3>
#
---
version: '2'
services:
awx_1:
environment:
RECEPTOR_MUTUAL_TLS: awx receptor-hop receptor-1 receptor-2 receptor-3
volumes:
- "./docker-compose-cluster:/etc/receptor"
receptor-hop:
image: quay.io/ansible/awx-ee
user: root
container_name: tools_receptor_hop
hostname: receptor-hop
command: 'receptor --config /etc/receptor/awx-hop-receptor.conf'
links:
- awx_1
ports:
- "5555:5555"
volumes:
- "../../docker-compose-cluster:/etc/receptor"
receptor-1:
image: quay.io/ansible/awx-ee
user: root
container_name: tools_receptor_1
hostname: receptor-1
command: 'receptor --config /etc/receptor/awx-1-receptor.conf'
links:
- receptor-hop
volumes:
- "../../docker-compose-cluster:/etc/receptor"
receptor-2:
image: quay.io/ansible/awx-ee
user: root
container_name: tools_receptor_2
hostname: receptor-2
command: 'receptor --config /etc/receptor/awx-2-receptor.conf'
links:
- receptor-hop
volumes:
- "../../docker-compose-cluster:/etc/receptor"
receptor-3:
image: quay.io/ansible/awx-ee
user: root
container_name: tools_receptor_3
hostname: receptor-3
command: 'receptor --config /etc/receptor/awx-3-receptor.conf'
links:
- receptor-hop
volumes:
- "../../docker-compose-cluster:/etc/receptor"