Add support for Helm v3 and containerized armada

This adds support for Helm v3.

This replaces docker armada and tiller with armada Kubernetes pod
that has armada-api and tiller containers. The existing armada
commands i.e., in routine make_armada_request() get wrapped with
'kubectl exec -n armada armada-pod -c armada-api --'.
Redundant docker commands are removed.

Since the armada chart may run on either controller, the /manifests
and /overrides are no longer mounted since the underlying /opt/platform
is DRBD backed and only visible on the active controller. These files
are now copied from active controller into the armada-api container.

The /logs mount is no longer required. The 'tee' of output within
container was dropped. The output is captured and written directly to
controller host in the same as previous location /var/log/armada.
The monitor task scrapes these logs (as it did previously) for
specific success patterns.

Armada requests are now aborted by deleting the armada pod.
Kubernetes manages armada and automatically restarts the pod.

Change-Id: I7c55c9143d4fb794f52be57a6e68e84d6b917600
Story: 2007000
Task: 37774
Depends-On: https://review.opendev.org/719962
Signed-off-by: Jim Gauld <james.gauld@windriver.com>
This commit is contained in:
Jim Gauld 2020-04-07 15:02:40 -04:00
parent e3943e9a8b
commit f53c96f7df
6 changed files with 549 additions and 459 deletions

View File

@ -1,13 +0,0 @@
#
# Copyright (c) 2019 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
TILLER_CONTAINER_NAME = 'tiller'
TILLER_SELECTOR_NAME = 'tiller'
TILLER_IMAGE_NAME = '/kubernetes-helm/tiller'
TILLER_IMAGE_VERSION = "v2.13.1"
ARMADA_IMAGE_NAME = '/airshipit/armada'
ARMADA_IMAGE_VERSION = "8a1638098f88d92bf799ef4934abe569789b885e-ubuntu_bionic"

View File

@ -21,6 +21,7 @@ from kubernetes import config
from kubernetes import client
from kubernetes.client import Configuration
from kubernetes.client.rest import ApiException
from kubernetes.stream import stream
from six.moves import http_client as httplib
from oslo_log import log as logging
@ -652,3 +653,48 @@ class KubeOperator(object):
except Exception as e:
LOG.error("Kubernetes exception in "
"kube_get_pod %s/%s: %s" % (namespace, name, e))
def kube_get_pods_by_selector(self, namespace, label_selector,
field_selector):
c = self._get_kubernetesclient_core()
try:
api_response = c.list_namespaced_pod(namespace,
label_selector="%s" % label_selector,
field_selector="%s" % field_selector)
LOG.debug("Response: %s" % api_response)
return api_response.items
except ApiException as e:
LOG.error("Kubernetes exception in "
"kube_get_pods_by_selector %s/%s/%s: %s",
namespace, label_selector, field_selector, e)
raise
# NOTE: This is desired method to exec commands in a container.
# The minimal usage example indicates this can get separate streams for
# stdout and stderr. The code below produces a string of merged output,
# so we cannot deduce whether the provided exec_command is failing.
# This API can replace Popen/poll/kubectl exec calls if we peek at
# api_response. We require ability to poll, read and flush output from
# long running commands, wait for command completion, and timeout.
# See the following documentation:
# https://github.com/kubernetes-client/python/blob/master/examples/pod_exec.py
# https://github.com/kubernetes-client/python/blob/master/kubernetes/docs/CoreV1Api.md
def kube_exec_container_stream(self, name, namespace, exec_command, container=None):
c = self._get_kubernetesclient_core()
try:
api_response = stream(c.connect_get_namespaced_pod_exec,
name,
namespace,
container=container,
command=exec_command,
stderr=True, stdin=False,
stdout=True, tty=False)
return api_response
except ApiException as e:
LOG.error("Failed to exec Pod %s/%s: %s" % (namespace, name,
e.body))
raise
except Exception as e:
LOG.error("Kubernetes exception in "
"kube_exec_container %s/%s: %s" % (namespace, name, e))
raise

View File

@ -42,7 +42,6 @@ from sysinv.api.controllers.v1 import kube_app
from sysinv.common import constants
from sysinv.common import exception
from sysinv.common import kubernetes
from sysinv.common import image_versions
from sysinv.common.retrying import retry
from sysinv.common import utils as cutils
from sysinv.common.storage_backend_conf import K8RbdProvisioner
@ -58,11 +57,13 @@ LOG = logging.getLogger(__name__)
# Constants
APPLY_SEARCH_PATTERN = 'Processing Chart,'
ARMADA_CONTAINER_NAME = 'armada_service'
ARMADA_NAMESPACE = 'armada'
ARMADA_APPLICATION = 'armada'
ARMADA_CONTAINER_NAME = 'armada-api'
ARMADA_MANIFEST_APPLY_SUCCESS_MSG = 'Done applying manifest'
ARMADA_RELEASE_ROLLBACK_FAILURE_MSG = 'Error while rolling back tiller release'
CONTAINER_ABNORMAL_EXIT_CODE = 137
DELETE_SEARCH_PATTERN = 'Deleting release'
DELETE_SEARCH_PATTERN = 'Deleting release|no release to delete'
ROLLBACK_SEARCH_PATTERN = 'Helm rollback of release'
INSTALLATION_TIMEOUT = 3600
MAX_DOWNLOAD_THREAD = 5
@ -76,9 +77,10 @@ CHARTS_PENDING_INSTALL_ITERATIONS = 60
ARMADA_LOG_MAX = 10
ARMADA_HOST_LOG_LOCATION = '/var/log/armada'
ARMADA_CONTAINER_LOG_LOCATION = '/logs'
ARMADA_CONTAINER_TMP = '/tmp'
ARMADA_LOCK_GROUP = 'armada.process'
ARMADA_LOCK_VERSION = 'v1'
ARMADA_LOCK_NAMESPACE = 'kube-system'
ARMADA_LOCK_NAMESPACE = 'armada'
ARMADA_LOCK_PLURAL = 'locks'
ARMADA_LOCK_NAME = 'lock'
@ -151,6 +153,7 @@ class AppOperator(object):
self._image = AppImageParser()
self._kube_pod = kube_pod.K8sPodOperator(self._kube)
self._lock = threading.Lock()
self._armada = ArmadaHelper(self._kube)
if not os.path.isfile(constants.ANSIBLE_BOOTSTRAP_FLAG):
self._clear_stuck_applications()
@ -786,7 +789,7 @@ class AppOperator(object):
continue
try:
output = subprocess.check_output(['helm', 'lint', r])
if "no failures" in output:
if "linted, 0 chart(s) failed" in output:
LOG.info("Helm chart %s validated" % os.path.basename(r))
else:
LOG.error("Validation failed for helm chart %s" %
@ -1381,13 +1384,15 @@ class AppOperator(object):
overrides_str = ""
if helm_files:
overrides_str += " ".join([
' --values /overrides/{0}/{1}/{2}'.format(
' --values {0}/overrides/{1}/{2}/{3}'.format(
ARMADA_CONTAINER_TMP,
app_name, app_version, os.path.basename(i))
for i in helm_files
])
if armada_files:
overrides_str += " ".join([
' --values /manifests/{0}/{1}/{2}'.format(
' --values {0}/manifests/{1}/{2}/{3}'.format(
ARMADA_CONTAINER_TMP,
app_name, app_version, os.path.basename(i))
for i in armada_files
])
@ -1410,11 +1415,13 @@ class AppOperator(object):
"""
try:
deployed_releases = helm_utils.retrieve_helm_releases()
LOG.debug('deployed_releases = %s', deployed_releases)
app = self._dbapi.kube_app_get(app_name)
app_releases = self._dbapi.kube_app_chart_release_get_all(app.id)
for r in app_releases:
LOG.debug('app.id=%r, release=%r, version=%r, namespace=%r',
app.id, r.release, r.version, r.namespace)
if (r.release in deployed_releases and
r.namespace in deployed_releases[r.release] and
r.version != deployed_releases[r.release][r.namespace]):
@ -1535,14 +1542,18 @@ class AppOperator(object):
print_chart = '{print $10}'
else:
print_chart = '{print $NF}'
p1 = subprocess.Popen(['docker', 'exec', ARMADA_CONTAINER_NAME,
'grep', pattern, logfile],
p1 = subprocess.Popen(['grep', pattern, logfile],
stdout=subprocess.PIPE)
p2 = subprocess.Popen(['awk', print_chart], stdin=p1.stdout,
stdout=subprocess.PIPE, stderr=subprocess.PIPE)
p1.stdout.close()
result, err = p2.communicate()
if result:
# Scrape information from command output, example 'validate' log:
# 2020-03-26 09:47:58.594 1105 INFO armada.cli [-] Successfully validated:\
# ('/tmp/manifests/oidc-auth-apps/1.0-0/oidc-auth-apps-manifest.yaml',)
# Strip out ANSI color code that might be in the text stream
r = re.compile("\x1b\[[0-9;]*m")
result = r.sub('', result).replace(',', '')
@ -1587,7 +1598,7 @@ class AppOperator(object):
(len(app.charts) - adjust) * 100)
else:
percent = round(float(num) / len(app.charts) * 100)
progress_str = 'processing chart: ' + last +\
progress_str = 'processing chart: ' + str(last) +\
', overall completion: ' + str(percent) + '%'
if app.progress != progress_str:
LOG.info("%s" % progress_str)
@ -1623,7 +1634,7 @@ class AppOperator(object):
mqueue = queue.Queue()
rc = True
logname = time.strftime(app.name + '-' + request + '_%Y-%m-%d-%H-%M-%S.log')
logfile = ARMADA_CONTAINER_LOG_LOCATION + '/' + logname
logfile = ARMADA_HOST_LOG_LOCATION + '/' + logname
if request == constants.APP_APPLY_OP:
pattern = APPLY_SEARCH_PATTERN
@ -1634,7 +1645,7 @@ class AppOperator(object):
monitor = greenthread.spawn_after(1, _check_progress, mqueue, app,
pattern, logfile)
rc = self._docker.make_armada_request(request, app.armada_service_mfile,
rc = self._armada.make_armada_request(request, app.armada_service_mfile,
overrides_str, app.releases, logfile)
_cleanup_armada_log(ARMADA_HOST_LOG_LOCATION, app.name, request)
@ -1957,7 +1968,7 @@ class AppOperator(object):
app.sync_armada_mfile_dir, constants.APP_METADATA_FILE)
shutil.copy(inst_metadata_file, sync_metadata_file)
if not self._docker.make_armada_request(
if not self._armada.make_armada_request(
'validate', manifest_file=app.armada_service_mfile):
raise exception.KubeAppUploadFailure(
name=app.name,
@ -2495,8 +2506,12 @@ class AppOperator(object):
# Stop the Armada request in case it has reached this far and
# remove locks.
# TODO(jgauld): Need to correct lock mechanism, something is no
# longer working for application aborts. The lock lingers around,
# and only automatically get cleaned up after a long period.
# Subsequent reapply fails since it we cannot get lock.
with self._lock:
self._docker.stop_armada_request()
self._armada.stop_armada_request()
self._clear_armada_locks()
else:
# Either the previous operation has completed or already failed
@ -2674,7 +2689,6 @@ class DockerHelper(object):
def __init__(self, dbapi):
self._dbapi = dbapi
self._lock = threading.Lock()
self.registries_info = \
copy.deepcopy(constants.DEFAULT_REGISTRIES_INFO)
@ -2798,226 +2812,6 @@ class DockerHelper(object):
# must be unauthenticated in this case.)
return pub_img_tag, None
def _start_armada_service(self, client):
try:
container = client.containers.get(ARMADA_CONTAINER_NAME)
if container.status != 'running':
LOG.info("Restarting Armada service...")
container.restart()
return container
except Exception:
LOG.info("Starting Armada service...")
try:
# Create the armada log folder if it does not exists
if not os.path.exists(ARMADA_HOST_LOG_LOCATION):
os.mkdir(ARMADA_HOST_LOG_LOCATION)
os.chmod(ARMADA_HOST_LOG_LOCATION, 0o755)
os.chown(ARMADA_HOST_LOG_LOCATION, 1000, grp.getgrnam("sys_protected").gr_gid)
# First make kubernetes config accessible to Armada. This
# is a work around the permission issue in Armada container.
kube_config = os.path.join(constants.APP_SYNCED_ARMADA_DATA_PATH,
'admin.conf')
shutil.copy(kubernetes.KUBERNETES_ADMIN_CONF, kube_config)
os.chown(kube_config, 1000, grp.getgrnam("sys_protected").gr_gid)
overrides_dir = common.HELM_OVERRIDES_PATH
manifests_dir = constants.APP_SYNCED_ARMADA_DATA_PATH
logs_dir = ARMADA_HOST_LOG_LOCATION
LOG.info("kube_config=%s, manifests_dir=%s, "
"overrides_dir=%s, logs_dir=%s." %
(kube_config, manifests_dir, overrides_dir, logs_dir))
binds = {
kube_config: {'bind': '/armada/.kube/config', 'mode': 'ro'},
manifests_dir: {'bind': '/manifests', 'mode': 'ro'},
overrides_dir: {'bind': '/overrides', 'mode': 'ro'},
logs_dir: {'bind': ARMADA_CONTAINER_LOG_LOCATION, 'mode': 'rw'}}
# Armada img was pushed to local registry (registry.local:9001/quay.io/)
# at bootstrap time
armada_image_tag = constants.DOCKER_REGISTRY_SERVER + '/' + \
constants.DEFAULT_DOCKER_QUAY_REGISTRY + \
image_versions.ARMADA_IMAGE_NAME + ":" + \
image_versions.ARMADA_IMAGE_VERSION
armada_image = client.images.list(armada_image_tag)
# Pull Armada image if it's not available
if not armada_image:
LOG.info("Downloading Armada image %s ..." % armada_image_tag)
local_registry_auth = cutils.get_local_docker_registry_auth()
client.images.pull(armada_image_tag,
auth_config=local_registry_auth)
LOG.info("Armada image %s downloaded!" % armada_image_tag)
container = client.containers.run(
armada_image_tag,
name=ARMADA_CONTAINER_NAME,
detach=True,
volumes=binds,
restart_policy={'Name': 'always'},
network_mode='host',
command=None)
LOG.info("Armada service started!")
return container
except IOError as ie:
if not kubernetes.is_k8s_configured():
LOG.error("Unable to start Armada service: %s" % ie)
except OSError as oe:
LOG.error("Unable to make kubernetes config accessible to "
"armada: %s" % oe)
except Exception as e:
# Possible docker exceptions are: RuntimeError, ContainerError,
# ImageNotFound and APIError
LOG.error("Docker error while launching Armada container: %s", e)
os.unlink(kube_config)
return None
def make_armada_request(self, request, manifest_file='', overrides_str='',
app_releases=None, logfile=None):
if logfile is None:
logfile = time.strftime(request + '_%Y-%m-%d-%H-%M-%S.log')
if app_releases is None:
app_releases = []
rc = True
# Instruct Armada to use the tiller service since it does not properly
# process IPv6 endpoints, therefore use a resolvable hostname
tiller_host = " --tiller-host tiller-deploy.kube-system.svc.cluster.local"
try:
client = docker.from_env(timeout=INSTALLATION_TIMEOUT)
# It causes problem if multiple threads attempt to start the
# same container, so add lock to ensure only one thread can
# start the Armada container at a time
with self._lock:
armada_svc = self._start_armada_service(client)
if armada_svc:
if request == 'validate':
cmd = 'armada validate ' + manifest_file
(exit_code, exec_logs) = armada_svc.exec_run(cmd)
if exit_code == 0:
LOG.info("Manifest file %s was successfully validated." %
manifest_file)
else:
rc = False
if exit_code == CONTAINER_ABNORMAL_EXIT_CODE:
LOG.error("Failed to validate application manifest %s. "
"Armada service has exited abnormally." %
manifest_file)
else:
LOG.error("Failed to validate application manifest "
"%s: %s." % (manifest_file, exec_logs))
elif request == constants.APP_APPLY_OP:
cmd = ("/bin/bash -c 'set -o pipefail; armada apply "
"--enable-chart-cleanup --debug {m} {o} {t} | "
"tee {lf}'".format(m=manifest_file, o=overrides_str,
t=tiller_host, lf=logfile))
LOG.info("Armada apply command = %s" % cmd)
(exit_code, exec_logs) = armada_svc.exec_run(cmd)
if exit_code == 0:
LOG.info("Application manifest %s was successfully "
"applied/re-applied." % manifest_file)
else:
rc = False
if exit_code == CONTAINER_ABNORMAL_EXIT_CODE:
LOG.error("Failed to apply application manifest %s. "
"Armada service has exited abnormally." %
manifest_file)
else:
LOG.error("Failed to apply application manifest %s. See "
"/var/log/armada/%s for details." %
(manifest_file, os.path.basename(logfile)))
elif request == constants.APP_ROLLBACK_OP:
cmd_rm = "rm " + logfile
armada_svc.exec_run(cmd_rm)
for app_release in app_releases:
release = app_release.get('release')
version = app_release.get('version')
sequenced = app_release.get('sequenced')
if sequenced:
cmd = "/bin/bash -c 'set -o pipefail; armada rollback " +\
"--debug --wait --timeout 1800 --release " +\
release + " --version " + str(version) + tiller_host +\
" | tee -a " + logfile + "'"
else:
cmd = "/bin/bash -c 'set -o pipefail; armada rollback " +\
"--debug --release " + release + " --version " +\
str(version) + tiller_host + " | tee -a " + logfile + "'"
(exit_code, exec_logs) = armada_svc.exec_run(cmd)
if exit_code != 0:
rc = False
if exit_code == CONTAINER_ABNORMAL_EXIT_CODE:
LOG.error("Failed to rollback release (%s). "
"Armada service has exited abnormally."
% release)
else:
LOG.error("Failed to rollback release %s. See "
"/var/log/armada/%s for details." %
(release, os.path.basename(logfile)))
break
if rc:
LOG.info("Application releases %s were successfully "
"rolled back." % app_releases)
elif request == constants.APP_DELETE_OP:
# Since armada delete doesn't support --values overrides
# files, use the delete manifest generated from the
# ArmadaManifestOperator during overrides generation. It
# will contain an accurate view of what was applied
manifest_delete_file = "%s-del%s" % os.path.splitext(manifest_file)
cmd = "/bin/bash -c 'set -o pipefail; armada delete --debug " +\
"--manifest " + manifest_delete_file + tiller_host + " | tee " +\
logfile + "'"
LOG.info("Armada delete command = %s" % cmd)
(exit_code, exec_logs) = armada_svc.exec_run(cmd)
if exit_code == 0:
LOG.info("Application charts were successfully "
"deleted with manifest %s." % manifest_delete_file)
else:
rc = False
if exit_code == CONTAINER_ABNORMAL_EXIT_CODE:
LOG.error("Failed to delete application manifest %s. "
"Armada service has exited abnormally." %
manifest_file)
else:
LOG.error("Failed to delete application manifest %s. See "
"/var/log/armada/%s for details." %
(manifest_file, os.path.basename(logfile)))
else:
rc = False
LOG.error("Unsupported armada request: %s." % request)
else:
# Armada sevice failed to start/restart
rc = False
except Exception as e:
# Failed to get a docker client
rc = False
LOG.error("Armada request %s for manifest %s failed: %s " %
(request, manifest_file, e))
return rc
def stop_armada_request(self):
"""A simple way to cancel an on-going manifest apply/rollback/delete
request. This logic will be revisited in the future.
"""
try:
client = docker.from_env(timeout=INSTALLATION_TIMEOUT)
container = client.containers.get(ARMADA_CONTAINER_NAME)
if container.status == 'running':
LOG.info("Stopping Armada service...")
container.stop()
except Exception as e:
# Failed to get a docker client
LOG.error("Failed to stop Armada service : %s " % e)
def download_an_image(self, app_name, img_tag):
rc = True
@ -3093,6 +2887,379 @@ class DockerHelper(object):
return img_tag, rc
class ArmadaHelper(object):
""" Armada class to encapsulate Armada related operations """
def __init__(self, kube):
self._kube = kube
self._lock = threading.Lock()
self.overrides_dir = common.HELM_OVERRIDES_PATH
self.manifests_dir = constants.APP_SYNCED_ARMADA_DATA_PATH
self.logs_dir = ARMADA_HOST_LOG_LOCATION
# Generate kubectl wrapped bash command that can run in
# a specific container of a namespaced pod.
def wrap_kubectl_bash(self, name, namespace, exec_command,
container=None):
kcmd = ['kubectl', '--kubeconfig', kubernetes.KUBERNETES_ADMIN_CONF,
'exec', '-n', namespace, name]
if container is not None:
kcmd.extend(['--container', container])
kcmd.extend(['--', '/bin/bash', '-c', exec_command])
return kcmd
# Wrapper for kubectl exec to run bash commands in a specific container
# of a namespaced pod.
# Returns command stdout and stderr, and stderr if kubectl command fails.
# This should be replaced with the core kubernetes client API
# connect_get_namespaced_pod_exec when that can be made to work properly
# with error handling, separate stdout, stderr, timeout, poll and flush
# of output streams, and wait for command completion.
def kube_exec_container_bash(self, name, namespace, exec_command,
container=None):
kcmd = self.wrap_kubectl_bash(name, namespace, exec_command,
container=container)
stdout, stderr = cutils.trycmd(*kcmd, discard_warnings=True,
run_as_root=False)
return stdout, stderr
# Wrapper for kubectl cp to a container. One of 'src' and 'dest' must
# be a remote file specification.
# Returns command stdout and stderr, and stderr if kubectl command fails.
# Limitation: kubectl cp command does not return an error when
# the source file does not exist.
# https://github.com/kubernetes/kubernetes/issues/78879
def kube_cp_container(self, namespace, src, dest, container=None):
kcmd = ['kubectl', '--kubeconfig', kubernetes.KUBERNETES_ADMIN_CONF,
'cp', '-n', namespace, src, dest]
if container is not None:
kcmd.extend(['--container', container])
stdout, stderr = cutils.trycmd(*kcmd, discard_warnings=True,
run_as_root=False)
return stdout, stderr
def copy_manifests_and_overrides_to_armada(self, armada_pod, mfile):
# NOTE: The armada pod may run on either controller.
# We do not want to mount host directories since DRBD
# /opt/platform is only visible on active controller.
# As a workaround, we can copy the required files into
# the armada container.
# Derive manifests and overrides directories for both
# source source and destination paths. We use well-known
# directories and a filename given the following format.
# /manifests/oidc-auth-apps/1.0-0/oidc-auth-apps-manifest-del.yaml
manifests_dest = '{}/{}'.format(ARMADA_CONTAINER_TMP, 'manifests')
overrides_dest = '{}/{}'.format(ARMADA_CONTAINER_TMP, 'overrides')
app_name = mfile.split('/', 3)[2]
# Create manifests and overrides directories in container
cmd = 'mkdir -v -p {}; mkdir -v -p {}'.\
format(manifests_dest, overrides_dest)
stdout, stderr = self.kube_exec_container_bash(
armada_pod, ARMADA_NAMESPACE, cmd, container=ARMADA_CONTAINER_NAME)
if stderr:
LOG.error("Failed to create manifests and overrides, error: %s",
stderr)
return False
# Copy manifests and overrides directories to container
# NOTE: kubectl cp command does not return an error when
# the source file does not exist.
# https://github.com/kubernetes/kubernetes/issues/78879
src_dest_dirs = \
[('{}/{}'.format(self.manifests_dir, app_name),
'{}:{}'.format(armada_pod, manifests_dest)),
('{}/{}'.format(self.overrides_dir, app_name),
'{}:{}'.format(armada_pod, overrides_dest))]
for src_dir, dest_dir in src_dest_dirs:
LOG.info("Copy %s to %s ." % (src_dir, dest_dir))
stdout, stderr = self.kube_cp_container(
ARMADA_NAMESPACE, src_dir, dest_dir,
container=ARMADA_CONTAINER_NAME)
if stderr:
LOG.error("Failed to copy %s to %s, error: %s",
src_dir, dest_dir, stderr)
return False
return True
def _start_armada_service(self):
"""Armada pod is managed by Kubernetes / Helm.
This routine checks and waits for armada to be providing service.
"""
self.overrides_dir = common.HELM_OVERRIDES_PATH
self.manifests_dir = constants.APP_SYNCED_ARMADA_DATA_PATH
try:
# Create the armada log folder if it does not exists
if not os.path.exists(ARMADA_HOST_LOG_LOCATION):
os.mkdir(ARMADA_HOST_LOG_LOCATION)
os.chmod(ARMADA_HOST_LOG_LOCATION, 0o755)
os.chown(ARMADA_HOST_LOG_LOCATION, 1000,
grp.getgrnam("sys_protected").gr_gid)
if not os.path.exists(common.HELM_OVERRIDES_PATH):
os.makedirs(common.HELM_OVERRIDES_PATH, 0o755)
except OSError as oe:
LOG.error("Unable to create armada log folder : %s" % oe)
return False
# Wait for armada to be ready for cmd execution.
# NOTE: make_armada_requests() also has retry mechanism
timeout = 30
while True:
try:
pods = self._kube.kube_get_pods_by_selector(
ARMADA_NAMESPACE,
"application=%s" % ARMADA_APPLICATION, "")
if not pods:
raise RuntimeError('armada pod not found')
pod = pods[0]
if pod and pod.status.phase != 'Running':
# Delete the pod, it should restart if it can
if not self._kube.kube_delete_pod(pod.metadata.name,
ARMADA_NAMESPACE, grace_periods_seconds=0):
LOG.warning("Pod %s/%s deletion unsuccessful...",
ARMADA_NAMESPACE, pod.metadata.name)
if pod and pod.status.phase == 'Running':
# Test that we can copy files into armada-api container
src = '/etc/build.info'
dest_dir = '{}:{}'.format(pod.metadata.name, '/tmp')
stdout, stderr = self.kube_cp_container(
ARMADA_NAMESPACE, src, dest_dir,
container=ARMADA_CONTAINER_NAME)
if stderr:
LOG.error("Failed to copy %s to %s, error: %s",
src, dest_dir, stderr)
raise RuntimeError('armada pod not ready')
else:
return True
return True
except Exception as e:
LOG.info("Could not get Armada service : %s " % e)
if timeout <= 0:
break
time.sleep(5)
timeout -= 5
LOG.error("Failed to get Armada service after 30 seconds.")
return False
def stop_armada_request(self):
"""A simple way to cancel an on-going manifest apply/rollback/delete
request. This logic will be revisited in the future.
"""
try:
pods = self._kube.kube_get_pods_by_selector(
ARMADA_NAMESPACE, "application=%s" % ARMADA_APPLICATION, "")
if not pods:
raise RuntimeError('armada pod not found')
for pod in pods:
if pod.status.phase == 'Running':
# Delete the pod, it should restart if it can
LOG.info("Stopping Armada service %s.", pod.metadata.name)
if not self._kube.kube_delete_pod(pod.metadata.name,
ARMADA_NAMESPACE,
grace_periods_seconds=0):
LOG.warning("Pod %s/%s deletion unsuccessful.",
ARMADA_NAMESPACE, pod.metadata.name)
except Exception as e:
LOG.error("Failed to stop Armada service : %s " % e)
def make_armada_request(self, request, manifest_file='', overrides_str='',
app_releases=None, logfile=None):
if logfile is None:
# Infer app name from the manifest file
# e.g., /tmp/manifests/oidc-auth-apps/1.0-0/oidc-auth-apps-manifest.yaml
app_name = manifest_file.split('/', 3)[2]
logname = time.strftime(app_name + '-' + request + '_%Y-%m-%d-%H-%M-%S.log')
logfile = ARMADA_HOST_LOG_LOCATION + '/' + logname
if app_releases is None:
app_releases = []
rc = True
# Configure additional armada options (e.g., such as --tiller-host),
# currently none are required.
tiller_host = " "
LOG.debug('make_armada_request: request=%s, '
'manifest_file=%s, overrides_str=%s, '
'app_releases=%r, logfile=%r',
request, manifest_file, overrides_str,
app_releases, logfile)
try:
# Ensure armada service is ready.
with self._lock:
ret = self._start_armada_service()
if ret:
# The armada pod name may change, get it each time
pods = self._kube.kube_get_pods_by_selector(
ARMADA_NAMESPACE, "application=%s" % ARMADA_APPLICATION,
"status.phase=Running")
if not pods:
raise RuntimeError('armada pod not found')
armada_pod = pods[0].metadata.name
if not self.copy_manifests_and_overrides_to_armada(armada_pod, manifest_file):
raise RuntimeError('could not access armada pod')
if request == 'validate':
cmd = ''.join(['armada validate ',
ARMADA_CONTAINER_TMP,
manifest_file])
LOG.info("Armada %s command: '%s'", request, cmd)
kcmd = self.wrap_kubectl_bash(
armada_pod, ARMADA_NAMESPACE, cmd,
container=ARMADA_CONTAINER_NAME)
p = subprocess.Popen(kcmd,
universal_newlines=True,
stdout=subprocess.PIPE,
stderr=subprocess.STDOUT)
with p.stdout, open(logfile, 'wb') as log:
while p.poll() is None:
line = p.stdout.readline()
if line != b"":
log.write(line)
log.flush()
if p.returncode != 0:
rc = False
LOG.error("Failed to validate application manifest %s "
"with exit code %s. See %s for details." %
(manifest_file, p.returncode, logfile))
else:
LOG.info("Manifest file %s was successfully validated." %
manifest_file)
elif request == constants.APP_APPLY_OP:
cmd = ''.join(['armada apply --debug ',
'--enable-chart-cleanup ',
ARMADA_CONTAINER_TMP,
manifest_file,
overrides_str,
tiller_host])
LOG.info("Armada %s command: '%s'", request, cmd)
kcmd = self.wrap_kubectl_bash(
armada_pod, ARMADA_NAMESPACE, cmd,
container=ARMADA_CONTAINER_NAME)
p = subprocess.Popen(kcmd,
universal_newlines=True,
stdout=subprocess.PIPE,
stderr=subprocess.STDOUT)
with p.stdout, open(logfile, 'wb') as log:
while p.poll() is None:
line = p.stdout.readline()
if line != b"":
LOG.debug('%s: %s', request, line.encode('utf-8'))
log.write(line)
log.flush()
if p.returncode != 0:
rc = False
LOG.error("Failed to apply application manifest %s "
"with exit code %s. See %s for details." %
(manifest_file, p.returncode, logfile))
else:
LOG.info("Application manifest %s was successfully "
"applied/re-applied." % manifest_file)
elif request == constants.APP_ROLLBACK_OP:
for app_release in app_releases:
release = app_release.get('release')
version = app_release.get('version')
sequenced = app_release.get('sequenced')
if sequenced:
cmd = ''.join(['armada rollback --debug ',
'--wait --timeout 1800 ',
'--release ' + release + ' ',
'--version ' + str(version),
tiller_host])
else:
cmd = ''.join(['armada rollback --debug ',
'--release ' + release + ' ',
'--version ' + str(version),
tiller_host])
LOG.info("Armada %s command: '%s'", request, cmd)
kcmd = self.wrap_kubectl_bash(
armada_pod, ARMADA_NAMESPACE, cmd,
container=ARMADA_CONTAINER_NAME)
p = subprocess.Popen(kcmd,
universal_newlines=True,
stdout=subprocess.PIPE,
stderr=subprocess.STDOUT)
with p.stdout, open(logfile, 'wb') as log:
while p.poll() is None:
line = p.stdout.readline()
if line != b"":
log.write(line)
log.flush()
if p.returncode != 0:
rc = False
LOG.error("Failed to rollback release %s "
"with exit code %s. See %s for details." %
(release, p.returncode, logfile))
break
if rc:
LOG.info("Application releases %s were successfully "
"rolled back." % app_releases)
elif request == constants.APP_DELETE_OP:
# Since armada delete doesn't support --values overrides
# files, use the delete manifest generated from the
# ArmadaManifestOperator during overrides generation. It
# will contain an accurate view of what was applied
manifest_delete_file = "%s-del%s" % os.path.splitext(manifest_file)
cmd = ''.join(['armada delete --debug ',
'--manifest ',
ARMADA_CONTAINER_TMP,
manifest_delete_file,
tiller_host])
LOG.info("Armada %s command: '%s'", request, cmd)
kcmd = self.wrap_kubectl_bash(
armada_pod, ARMADA_NAMESPACE, cmd,
container=ARMADA_CONTAINER_NAME)
p = subprocess.Popen(kcmd,
universal_newlines=True,
stdout=subprocess.PIPE,
stderr=subprocess.STDOUT)
with p.stdout, open(logfile, 'wb') as log:
while p.poll() is None:
line = p.stdout.readline()
if line != b"":
log.write(line)
log.flush()
if p.returncode != 0:
rc = False
LOG.error("Failed to delete application manifest %s "
"with exit code %s. See %s for details." %
(manifest_file, p.returncode, logfile))
else:
LOG.info("Application charts were successfully "
"deleted with manifest %s." % manifest_delete_file)
else:
rc = False
LOG.error("Unsupported armada request: %s." % request)
else:
# Armada sevice failed to start/restart
rc = False
LOG.error("Armada service failed to start/restart")
except Exception as e:
rc = False
LOG.error("Armada request %s for manifest %s failed: %s " %
(request, manifest_file, e))
return rc
class AppImageParser(object):
"""Utility class to help find images for an application"""

View File

@ -69,7 +69,6 @@ from oslo_utils import excutils
from oslo_utils import timeutils
from oslo_utils import uuidutils
from platform_util.license import license
from ruamel import yaml
from sqlalchemy.orm import exc
from six.moves import http_client as httplib
from sysinv._i18n import _
@ -84,7 +83,6 @@ from sysinv.common import constants
from sysinv.common import ceph as cceph
from sysinv.common import device as dconstants
from sysinv.common import exception
from sysinv.common import image_versions
from sysinv.common import fm
from sysinv.common import fernet
from sysinv.common import health
@ -111,7 +109,6 @@ from sysinv.puppet import common as puppet_common
from sysinv.puppet import puppet
from sysinv.helm import common as helm_common
from sysinv.helm import helm
from sysinv.helm import utils as helm_utils
MANAGER_TOPIC = 'sysinv.conductor_manager'
@ -5387,6 +5384,19 @@ class ConductorManager(service.PeriodicService):
self._kube_pod.delete_failed_pods_by_reason(
reason='NodeAffinity')
# Ensure that armada pod is running.
pods = self._kube.kube_get_pods_by_selector("armada",
"application=armada",
"status.phase=Running")
if not pods:
LOG.warning("armada pod not present")
return
# Disable application apply during upgrades
if self.is_upgrade_in_progress():
LOG.info("Upgrade in progress - disable audit")
return
# Check the application state and take the approprate action
for app_name in constants.HELM_APPS_PLATFORM_MANAGED:
@ -5449,167 +5459,10 @@ class ConductorManager(service.PeriodicService):
return
self.reapply_app(context, app_name)
def _patch_tiller_deployment(self):
""" Ensure tiller is patched with restart logic."""
LOG.info("Attempt to patch tiller deployment")
try:
# We have a race condition that may cause the tiller pod to not have
# its environment set up correctly. This will patch the tiller
# deployment to ensure that tiller can recover if that occurs. The
# deployment is patched during the initial ansible run. This will
# re-patch the deployment in the case when tiller has been removed
# and reinstalled in the cluster after the system has been
# installed. If tiller is already patched then the patch execution
# is successful causing no change to the deployment. Specify the
# update strategy to allow tiller deployment patching in a simplex
# controller configuration.
patch = {
'spec': {
'strategy': {
'type': 'RollingUpdate',
'rollingUpdate': {
'maxUnavailable': 1,
'maxSurge': 1,
}
},
'template': {
'spec': {
'containers': [{
'name': 'tiller',
'command': [
'/bin/sh',
'-cex',
'#!/bin/sh\n'
'env | grep -q -e ^TILLER_DEPLOY || exit\n'
'env | grep -q -e ^KUBE_DNS || exit\n'
'env | grep -q -e ^KUBERNETES_PORT || exit\n'
'env | grep -q -e ^KUBERNETES_SERVICE || exit\n'
'/tiller\n'
]
}]
}
}
}
}
cmd = ['kubectl',
'--kubeconfig={}'.format(kubernetes.KUBERNETES_ADMIN_CONF),
'patch', 'deployment', '-n', 'kube-system', 'tiller-deploy',
'-p', yaml.dump(patch)]
stdout, stderr = cutils.execute(*cmd, run_as_root=False)
except exception.ProcessExecutionError as e:
raise exception.SysinvException(
_("Error patching the tiller deployment, "
"Details: %s") % str(e))
LOG.info("Tiller deployment has been patched")
def _upgrade_downgrade_kube_components(self):
self._upgrade_downgrade_static_images()
self._upgrade_downgrade_tiller()
self._upgrade_downgrade_kube_networking()
@retry(retry_on_result=lambda x: x is False,
wait_fixed=(CONF.conductor.kube_upgrade_downgrade_retry_interval * 1000))
def _upgrade_downgrade_tiller(self):
"""Check if tiller needs to be upgraded or downgraded"""
LOG.info("_upgrade_downgrade_tiller")
FIVE_MIN_IN_SECS = 300
in_progress_statuses = [constants.APP_APPLY_IN_PROGRESS,
constants.APP_UPLOAD_IN_PROGRESS,
constants.APP_REMOVE_IN_PROGRESS,
constants.APP_UPDATE_IN_PROGRESS,
constants.APP_RECOVER_IN_PROGRESS]
# Check if we are in the middle of an application apply. If so wait
# 5 minutes and retry.
while True:
try:
in_progress = False
for app in self.dbapi.kube_app_get_all():
if app.status in in_progress_statuses:
LOG.info("_upgrade_downgrade_tiller kubernetes application "
"'%s' in progress, status is '%s'" %
(app.name, app.status))
in_progress = True
break
if in_progress:
greenthread.sleep(FIVE_MIN_IN_SECS)
continue
except Exception as e:
LOG.error("{}. Failed to get kubernetes application list.".format(e))
break
# Upgrade or downgrade the tiller image
try:
running_image = self._kube.kube_get_image_by_selector(
image_versions.TILLER_SELECTOR_NAME,
helm_common.HELM_NS_KUBE_SYSTEM,
image_versions.TILLER_CONTAINER_NAME)
if running_image is None:
LOG.warning("Failed to get tiller image")
return False
LOG.info("Running tiller image: %s" % running_image)
LOG.info("Requested tiller version: %s" % image_versions.TILLER_IMAGE_VERSION)
# Grab the version from the image name. Version is preceded
# by a ":" e.g.
# registry.local:9001/gcr.io/kubernetes-helm/tiller:v2.13.0
running_image_name, running_version = running_image.rsplit(":", 1)
if not running_version:
LOG.warning("Failed to get version from tiller image")
return False
# Verify the tiller version running
if running_version != image_versions.TILLER_IMAGE_VERSION:
LOG.info("Running version of tiller does not match patching version of %s. "
"Upgrade in progress."
% image_versions.TILLER_IMAGE_VERSION)
download_image = running_image_name + ":" + image_versions.TILLER_IMAGE_VERSION
self._docker._retrieve_specified_registries()
# download the image
try:
img_tag, ret = self._docker.download_an_image("helm",
download_image)
if not ret:
raise Exception
except Exception as e:
LOG.warning("Failed to download image '%s'. %s" % (download_image, e))
return False
# reset the cached registries
self._docker._reset_registries_info()
# Update the new image
try:
helm_utils.helm_upgrade_tiller(download_image)
except Exception as e:
LOG.warning("Failed to update the new image: %s" % e)
return False
except Exception as e:
LOG.error("{}. Failed to upgrade/downgrade tiller.".format(e))
return False
# Patch tiller to allow restarts if the environment is incomplete
#
# NOTE: This patch along with this upgrade functionality can be removed
# once StarlingX moves to Helm v3
try:
self._patch_tiller_deployment()
except Exception as e:
LOG.error("{}. Failed to patch tiller deployment.".format(e))
return False
return True
@retry(retry_on_result=lambda x: x is False,
wait_fixed=(CONF.conductor.kube_upgrade_downgrade_retry_interval * 1000))
def _upgrade_downgrade_kube_networking(self):
@ -6823,9 +6676,25 @@ class ConductorManager(service.PeriodicService):
" report_config_status! iconfig: %(iconfig)s" %
{'iconfig': iconfig, 'cfg': reported_cfg})
# Skip application reapply during upgrades
if self.is_upgrade_in_progress():
LOG.info("Upgrade in progress - skip pending app reapply")
return
if success:
self.check_pending_app_reapply(context)
def is_upgrade_in_progress(self):
""" Check if there is an upgrade in progress.
"""
try:
self.dbapi.software_upgrade_get_one()
except exception.NotFound:
# No upgrade in progress
return False
else:
return True
def report_partition_mgmt_success(self, host_uuid, idisk_uuid,
partition_uuid):
""" Disk partition management success callback for Sysinv Agent

View File

@ -19,6 +19,7 @@ from six import iteritems
from stevedore import extension
from oslo_log import log as logging
from sysinv.common import constants
from sysinv.common import exception
from sysinv.common import kubernetes
from sysinv.common import utils
@ -423,7 +424,15 @@ class HelmOperator(object):
if chart_tarfile is None:
# TODO: Clean up the assumption
chart_tarfile = chart_name + '-0.1.0'
return 'http://controller:{}/helm_charts/{}/{}.tgz'.format(
# Set the location based on ip address since
# http://controller does not resolve in armada container.
sys_controller_network = self.dbapi.network_get_by_type(constants.NETWORK_TYPE_CLUSTER_HOST)
sys_controller_network_addr_pool = self.dbapi.address_pool_get(sys_controller_network.pool_uuid)
sc_float_ip = sys_controller_network_addr_pool.floating_address
if utils.is_valid_ipv6(sc_float_ip):
sc_float_ip = '[' + sc_float_ip + ']'
return 'http://{}:{}/helm_charts/{}/{}.tgz'.format(
sc_float_ip,
utils.get_http_port(self.dbapi), repo_name, chart_tarfile)
def _add_armada_override_header(self, chart_name, chart_metadata_name, repo_name,
@ -503,7 +512,7 @@ class HelmOperator(object):
# specified by system or user, values from files and values passed in
# via --set . We need to ensure that we call helm using the same
# mechanisms to ensure the same behaviour.
cmd = ['helm', 'install', '--dry-run', '--debug']
cmd = ['helm', 'install', '--dry-run', '--debug', '--generate-name']
# Process the newly-passed-in override values
tmpfiles = []

View File

@ -9,6 +9,7 @@
"""Helm utilities and helper functions."""
import os
from eventlet.green import subprocess
import ruamel.yaml as yaml
from oslo_log import log as logging
@ -43,15 +44,56 @@ def retrieve_helm_releases():
by querying helm tiller
:return: a dict of deployed helm releases
"""
deployed_releases = {}
# Helm v3 releases
helm_list = subprocess.Popen(
['helm', '--kubeconfig', kubernetes.KUBERNETES_ADMIN_CONF,
'list', '--output', 'yaml'],
'list', '--all-namespaces', '--output', 'yaml'],
stdout=subprocess.PIPE, stderr=subprocess.PIPE)
timer = threading.Timer(20, helm_list.kill)
try:
releases = {}
deployed_releases = {}
timer.start()
out, err = helm_list.communicate()
if out and not err:
releases = yaml.safe_load(out)
elif err and not out:
raise exception.HelmTillerFailure(
reason="Failed to retrieve releases: %s" % err)
elif not err and not out:
err_msg = "Failed to retrieve releases. " \
"Helm tiller response timeout."
raise exception.HelmTillerFailure(reason=err_msg)
for r in releases:
r_name = r.get('name')
r_version = r.get('revision')
r_namespace = r.get('namespace')
deployed_releases.setdefault(r_name, {}).update(
{r_namespace: r_version})
except Exception as e:
raise exception.HelmTillerFailure(
reason="Failed to retrieve releases: %s" % e)
finally:
timer.cancel()
# Helm v2 releases
env = os.environ.copy()
env['PATH'] = '/usr/local/sbin:' + env['PATH']
env['KUBECONFIG'] = kubernetes.KUBERNETES_ADMIN_CONF
helm_list = subprocess.Popen(
['helmv2-cli', '--',
'helm',
'list', '--output', 'yaml'],
env=env, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
timer = threading.Timer(20, helm_list.kill)
try:
releases = {}
timer.start()
out, err = helm_list.communicate()
@ -78,6 +120,7 @@ def retrieve_helm_releases():
reason="Failed to retrieve releases: %s" % e)
finally:
timer.cancel()
return deployed_releases
@ -93,10 +136,16 @@ def delete_helm_release(release):
:param release: the name of the helm release
"""
# NOTE: This mechanism deletes armada/tiller managed releases.
# This could be adapted to also delete helm v3 releases using
# 'helm uninstall'.
env = os.environ.copy()
env['PATH'] = '/usr/local/sbin:' + env['PATH']
env['KUBECONFIG'] = kubernetes.KUBERNETES_ADMIN_CONF
helm_cmd = subprocess.Popen(
['helm', '--kubeconfig', kubernetes.KUBERNETES_ADMIN_CONF,
'delete', release],
stdout=subprocess.PIPE, stderr=subprocess.PIPE)
['helmv2-cli', '--',
'helm', 'delete', release],
env=env, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
timer = threading.Timer(20, helm_cmd.kill)
try:
@ -122,51 +171,14 @@ def delete_helm_release(release):
def get_openstack_pending_install_charts():
env = os.environ.copy()
env['PATH'] = '/usr/local/sbin:' + env['PATH']
env['KUBECONFIG'] = kubernetes.KUBERNETES_ADMIN_CONF
try:
return subprocess.check_output(
['helm', '--kubeconfig', kubernetes.KUBERNETES_ADMIN_CONF,
'list', '--namespace', 'openstack', '--pending'])
['helmv2-cli', '--',
'helm', 'list', '--namespace', 'openstack', '--pending'],
env=env)
except Exception as e:
raise exception.HelmTillerFailure(
reason="Failed to obtain pending charts list: %s" % e)
def helm_upgrade_tiller(image):
LOG.info("Attempt to update image to %s" % image)
try:
# Adding temporary workaround using helm init command with
# sed command until helm and tiller provide a fix for
# https://github.com/helm/helm/issues/6374
workaround_part1 = '--skip-refresh ' \
'--service-account tiller ' \
'--node-selectors "node-role.kubernetes.io/master"="" ' \
'--override spec.template.spec.hostNetwork=true ' \
'--override spec.selector.matchLabels.app=helm ' \
'--override spec.selector.matchLabels.name=tiller ' \
'--output yaml'
workaround_part2 = \
'| sed "s@apiVersion: extensions/v1beta1@apiVersion: apps/v1@" ' \
'| kubectl --kubeconfig {} replace --force -f -'.format(
kubernetes.KUBERNETES_ADMIN_CONF)
cmd = '{} {} {} {} {} {}'.format(
'helm init --upgrade --kubeconfig',
kubernetes.KUBERNETES_ADMIN_CONF,
'--tiller-image',
image,
workaround_part1,
workaround_part2)
LOG.info("Execute command: %s" % cmd)
proc = subprocess.Popen(cmd, stdout=subprocess.PIPE, shell=True)
out, err = proc.communicate()
if err:
raise exception.HelmTillerFailure(
reason="Failed to upgrade/downgrade image: %s" % err)
LOG.info("Image was updated to %s" % image)
except Exception as e:
raise exception.HelmTillerFailure(
reason="Failed to upgrade/downgrade image: %s" % e)