Implement a Kubernetes driver

This changes implements a Kubernetes resource provider.
The driver supports namespace request and pod request to enable both
containers as machine and native containers workflow.

Depends-On: https://review.openstack.org/605823
Change-Id: I81b5dc5abe92b71cc98b0d71c8a2863cddff6027
This commit is contained in:
Tristan Cacqueray 2017-11-19 03:59:28 +00:00
parent 674d6d88b4
commit 4295ff6870
15 changed files with 1036 additions and 0 deletions

View File

@ -119,6 +119,17 @@
devstack_localrc:
NODEPOOL_PAUSE_DEBIAN_STRETCH_DIB: false
- job:
description: |
Test that nodepool works with kubernetes.
name: nodepool-functional-k8s
pre-run: playbooks/nodepool-functional-k8s/pre.yaml
run: playbooks/nodepool-functional-k8s/run.yaml
post-run: playbooks/nodepool-functional-k8s/post.yaml
nodeset: ubuntu-xenial
required-projects:
- openstack-infra/nodepool
- project:
check:
jobs:
@ -133,6 +144,7 @@
voting: false
- nodepool-functional-py35-src:
voting: false
- nodepool-functional-k8s
- pbrx-build-container-images:
vars:
pbrx_prefix: zuul

View File

@ -1004,3 +1004,92 @@ Selecting the static driver adds the following options to the
:default: 1
The number of jobs that can run in parallel on this node.
Kubernetes driver
^^^^^^^^^^^^^^^^^
A Kubernetes provider's resources are partitioned into groups called "pool"
(see :ref:`k8s_pools` for details), and within a pool, the node types which
are to be made available are listed (see :ref:`k8s_labels` for details).
Example::
providers:
- name: kubespray
driver: kubernetes
context: admin-cluster.local
pools:
- name: main
labels:
- name: kubernetes-namespace
type: namespace
- name: pod-fedora
type: pod
image: docker.io/fedora:28
**required**
``context``
Name of the context configured in ``kube/config``.
Before using the driver, Nodepool services need a ``kube/config`` file
manually installed with cluster admin context.
**optional**
``launch-retries``
The number of times to retry launching a node before considering the job
failed.
Default 3.
.. _k8s_pools:
Kubernetes pools
~~~~~~~~~~~~~~~~
A pool defines a group of resources from a Kubernetes provider. Each pool has a
maximum number of namespace which can be created (Not Implemented yet).
Example::
pools:
- name: main
labels: []
**required**
``name``
Namespace name are prefixed with the pool's name.
.. _k8s_labels:
Kubernetes labels
~~~~~~~~~~~~~~~~~
Each entry in a pool`s `labels` section indicates that the
corresponding label is available for use in this pool.
Example::
labels:
- name: kubernetes-namespace
type: namespace
- name: pod-fedora
type: pod
image: docker.io/fedora:28
Kubernetes provider support two types of labels:
Namespace labels provide an empty namespace configured with a service account
that can creates pods, services, configmaps, ...
Pod labels provide a dedicated namespace with a single pod created using the
``image`` parameter and it is configured with a service account that can
exec and get the logs of the pod.

View File

@ -0,0 +1,34 @@
# Copyright 2018 Red Hat
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
# implied.
#
# See the License for the specific language governing permissions and
# limitations under the License.
from nodepool.driver import Driver
from nodepool.driver.kubernetes.config import KubernetesProviderConfig
from nodepool.driver.kubernetes.provider import KubernetesProvider
from openshift import config
class KubernetesDriver(Driver):
def reset(self):
try:
config.load_kube_config(persist_config=True)
except FileNotFoundError:
pass
def getProviderConfig(self, provider):
return KubernetesProviderConfig(self, provider)
def getProvider(self, provider_config, use_taskmanager):
return KubernetesProvider(provider_config, use_taskmanager)

View File

@ -0,0 +1,114 @@
# Copyright 2018 Red Hat
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
# implied.
#
# See the License for the specific language governing permissions and
# limitations under the License.
import voluptuous as v
from nodepool.driver import ConfigPool
from nodepool.driver import ConfigValue
from nodepool.driver import ProviderConfig
class KubernetesLabel(ConfigValue):
def __eq__(self, other):
if isinstance(other, KubernetesLabel):
return (other.name == self.name and
other.type == self.type and
other.image_pull == self.image_pull and
other.image == self.image)
return False
def __repr__(self):
return "<KubternetesLabel %s>" % self.name
class KubernetesPool(ConfigPool):
def __eq__(self, other):
if isinstance(other, KubernetesPool):
return (super().__eq__(other) and
other.name == self.name and
other.labels == self.labels)
return False
def __repr__(self):
return "<KubernetesPool %s>" % self.name
class KubernetesProviderConfig(ProviderConfig):
def __init__(self, driver, provider):
self.driver_object = driver
self.__pools = {}
super().__init__(provider)
def __eq__(self, other):
if isinstance(other, KubernetesProviderConfig):
return (super().__eq__(other) and
other.context == self.context and
other.pools == self.pools)
return False
@property
def pools(self):
return self.__pools
@property
def manage_images(self):
return False
def load(self, config):
self.launch_retries = int(self.provider.get('launch-retries', 3))
self.context = self.provider['context']
for pool in self.provider.get('pools', []):
pp = KubernetesPool()
pp.name = pool['name']
pp.provider = self
self.pools[pp.name] = pp
pp.labels = {}
for label in pool.get('labels', []):
pl = KubernetesLabel()
pl.name = label['name']
pl.type = label['type']
pl.image = label.get('image')
pl.image_pull = label.get('image-pull', 'IfNotPresent')
pl.pool = pp
pp.labels[pl.name] = pl
config.labels[label['name']].pools.append(pp)
def getSchema(self):
k8s_label = {
v.Required('name'): str,
v.Required('type'): str,
'image': str,
'image-pull': str,
}
pool = {
v.Required('name'): str,
v.Required('labels'): [k8s_label],
}
provider = {
v.Required('pools'): [pool],
v.Required('context'): str,
'launch-retries': int,
}
return v.Schema(provider)
def getSupportedLabels(self, pool_name=None):
labels = set()
for pool in self.pools.values():
if not pool_name or (pool.name == pool_name):
labels.update(pool.labels.keys())
return labels

View File

@ -0,0 +1,126 @@
# Copyright 2018 Red Hat
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
from kazoo import exceptions as kze
from nodepool import zk
from nodepool.driver.utils import NodeLauncher
from nodepool.driver import NodeRequestHandler
class K8SLauncher(NodeLauncher):
def __init__(self, handler, node, provider_config, provider_label):
super().__init__(handler.zk, node, provider_config)
self.handler = handler
self.zk = handler.zk
self.label = provider_label
self._retries = provider_config.launch_retries
def _launchLabel(self):
self.log.debug("Creating resource")
if self.label.type == "namespace":
resource = self.handler.manager.createNamespace(
self.node, self.handler.pool.name)
else:
resource = self.handler.manager.createPod(
self.node, self.handler.pool.name, self.label)
self.node.state = zk.READY
# NOTE: resource access token may be encrypted here
self.node.connection_port = resource
if self.label.type == "namespace":
self.node.connection_type = "namespace"
else:
self.node.connection_type = "kubectl"
self.node.interface_ip = resource['pod']
self.zk.storeNode(self.node)
self.log.info("Resource %s is ready" % resource['name'])
def launch(self):
attempts = 1
while attempts <= self._retries:
try:
self._launchLabel()
break
except kze.SessionExpiredError:
# If we lost our ZooKeeper session, we've lost our node lock
# so there's no need to continue.
raise
except Exception:
if attempts <= self._retries:
self.log.exception(
"Launch attempt %d/%d failed for node %s:",
attempts, self._retries, self.node.id)
# If we created an instance, delete it.
if self.node.external_id:
self.handler.manager.cleanupNode(self.node.external_id)
self.handler.manager.waitForNodeCleanup(
self.node.external_id)
self.node.external_id = None
self.node.interface_ip = None
self.zk.storeNode(self.node)
if attempts == self._retries:
raise
attempts += 1
class KubernetesNodeRequestHandler(NodeRequestHandler):
log = logging.getLogger("nodepool.driver.kubernetes."
"KubernetesNodeRequestHandler")
def __init__(self, pw, request):
super().__init__(pw, request)
self._threads = []
@property
def alive_thread_count(self):
count = 0
for t in self._threads:
if t.isAlive():
count += 1
return count
def imagesAvailable(self):
return True
def launchesComplete(self):
'''
Check if all launch requests have completed.
When all of the Node objects have reached a final state (READY or
FAILED), we'll know all threads have finished the launch process.
'''
if not self._threads:
return True
# Give the NodeLaunch threads time to finish.
if self.alive_thread_count:
return False
node_states = [node.state for node in self.nodeset]
# NOTE: It very important that NodeLauncher always sets one of
# these states, no matter what.
if not all(s in (zk.READY, zk.FAILED) for s in node_states):
return False
return True
def launch(self, node):
label = self.pool.labels[node.type[0]]
thd = K8SLauncher(self, node, self.provider, label)
thd.start()
self._threads.append(thd)

View File

@ -0,0 +1,289 @@
# Copyright 2018 Red Hat
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
import urllib3
import time
from kubernetes.config import config_exception as kce
from kubernetes import client as k8s_client
from openshift import config
from nodepool import exceptions
from nodepool.driver import Provider
from nodepool.driver.kubernetes import handler
urllib3.disable_warnings()
class KubernetesProvider(Provider):
log = logging.getLogger("nodepool.driver.kubernetes.KubernetesProvider")
def __init__(self, provider, *args):
self.provider = provider
self.ready = False
try:
self.k8s_client, self.rbac_client = self._get_client(
provider.context)
except kce.ConfigException:
self.log.exception("Couldn't load client from config")
self.log.info("Get context list using this command: "
"python3 -c \"from openshift import config; "
"print('\\n'.join([i['name'] for i in "
"config.list_kube_config_contexts()[0]]))\"")
self.k8s_client = None
self.rbac_client = None
self.namespace_names = set()
for pool in provider.pools.values():
self.namespace_names.add(pool.name)
def _get_client(self, context):
conf = config.new_client_from_config(context=context)
return (
k8s_client.CoreV1Api(conf),
k8s_client.RbacAuthorizationV1beta1Api(conf))
def start(self, zk_conn):
self.log.debug("Starting")
if self.ready or not self.k8s_client or not self.rbac_client:
return
self.ready = True
def stop(self):
self.log.debug("Stopping")
self.ready = False
def listNodes(self):
servers = []
class FakeServer:
def __init__(self, namespace, provider, valid_names):
self.id = namespace.metadata.name
self.name = namespace.metadata.name
self.metadata = {}
if [True for valid_name in valid_names
if namespace.metadata.name.startswith("%s-" % valid_name)]:
node_id = namespace.metadata.name.split('-')[-1]
try:
# Make sure last component of name is an id
int(node_id)
self.metadata['nodepool_provider_name'] = provider
self.metadata['nodepool_node_id'] = node_id
except Exception:
# Probably not a managed namespace, let's skip metadata
pass
def get(self, name, default=None):
return getattr(self, name, default)
if self.ready:
for namespace in self.k8s_client.list_namespace().items:
servers.append(FakeServer(
namespace, self.provider.name, self.namespace_names))
return servers
def labelReady(self, name):
# Labels are always ready
return True
def join(self):
pass
def cleanupLeakedResources(self):
pass
def cleanupNode(self, server_id):
if not self.ready:
return
self.log.debug("%s: removing namespace" % server_id)
delete_body = {
"apiVersion": "v1",
"kind": "DeleteOptions",
"propagationPolicy": "Background"
}
try:
self.k8s_client.delete_namespace(server_id, delete_body)
self.log.info("%s: namespace removed" % server_id)
except Exception:
# TODO: implement better exception handling
self.log.exception("Couldn't remove namespace %s" % server_id)
def waitForNodeCleanup(self, server_id):
for retry in range(300):
try:
self.k8s_client.read_namespace(server_id)
except Exception:
break
time.sleep(1)
def createNamespace(self, node, pool, restricted_access=False):
name = node.id
namespace = "%s-%s" % (pool, name)
user = "zuul-worker"
self.log.debug("%s: creating namespace" % namespace)
# Create the namespace
ns_body = {
'apiVersion': 'v1',
'kind': 'Namespace',
'metadata': {
'name': namespace,
'nodepool_node_id': name
}
}
proj = self.k8s_client.create_namespace(ns_body)
node.external_id = namespace
# Create the service account
sa_body = {
'apiVersion': 'v1',
'kind': 'ServiceAccount',
'metadata': {'name': user}
}
self.k8s_client.create_namespaced_service_account(namespace, sa_body)
# Wait for the token to be created
for retry in range(30):
sa = self.k8s_client.read_namespaced_service_account(
user, namespace)
ca_crt = None
token = None
if sa.secrets:
for secret_obj in sa.secrets:
secret = self.k8s_client.read_namespaced_secret(
secret_obj.name, namespace)
ca_crt = secret.data.get('ca.crt')
token = secret.data.get('token')
if token and ca_crt:
break
if token and ca_crt:
break
time.sleep(1)
if not token or not ca_crt:
raise exceptions.LaunchNodepoolException(
"%s: couldn't find token for service account %s" %
(namespace, sa))
# Create service account role
all_verbs = ["create", "delete", "get", "list", "patch",
"update", "watch"]
if restricted_access:
role_name = "zuul-restricted"
role_body = {
'kind': 'Role',
'apiVersion': 'rbac.authorization.k8s.io/v1beta1',
'metadata': {
'name': role_name,
},
'rules': [{
'apiGroups': [""],
'resources': ["pods"],
'verbs': ["get", "list"],
}, {
'apiGroups': [""],
'resources': ["pods/exec"],
'verbs': all_verbs
}, {
'apiGroups': [""],
'resources': ["pods/logs"],
'verbs': all_verbs
}]
}
else:
role_name = "zuul"
role_body = {
'kind': 'Role',
'apiVersion': 'rbac.authorization.k8s.io/v1beta1',
'metadata': {
'name': role_name,
},
'rules': [{
'apiGroups': [""],
'resources': ["pods", "pods/exec", "pods/log",
"services", "endpoints", "crontabs", "jobs",
"deployments", "replicasets",
"configmaps", "secrets"],
'verbs': all_verbs,
}]
}
self.rbac_client.create_namespaced_role(namespace, role_body)
# Give service account admin access
role_binding_body = {
'apiVersion': 'rbac.authorization.k8s.io/v1beta1',
'kind': 'RoleBinding',
'metadata': {'name': 'zuul-role'},
'roleRef': {
'apiGroup': 'rbac.authorization.k8s.io',
'kind': 'Role',
'name': role_name,
},
'subjects': [{
'kind': 'ServiceAccount',
'name': user,
'namespace': namespace,
}],
'userNames': ['system:serviceaccount:%s:zuul-worker' % namespace]
}
self.rbac_client.create_namespaced_role_binding(
namespace, role_binding_body)
resource = {
'name': proj.metadata.name,
'namespace': namespace,
'host': self.k8s_client.api_client.configuration.host,
'skiptls': not self.k8s_client.api_client.configuration.verify_ssl,
'token': token,
'ca_crt': ca_crt,
'user': user,
}
self.log.info("%s: namespace created" % namespace)
return resource
def createPod(self, node, pool, label):
resource = self.createNamespace(node, pool, restricted_access=True)
namespace = resource['namespace']
pod_body = {
'apiVersion': 'v1',
'kind': 'Pod',
'metadata': {'name': label.name},
'spec': {
'containers': [{
'name': label.name,
'image': label.image,
'imagePullPolicy': label.image_pull,
'command': ["/bin/bash", "-c", "--"],
'args': ["while true; do sleep 30; done;"],
'workingDir': '/tmp'
}]
},
'restartPolicy': 'Never',
}
self.k8s_client.create_namespaced_pod(namespace, pod_body)
for retry in range(300):
pod = self.k8s_client.read_namespaced_pod(label.name, namespace)
if pod.status.phase == "Running":
break
self.log.debug("%s: pod status is %s", namespace, pod.status.phase)
time.sleep(1)
if retry == 299:
raise exceptions.LaunchNodepoolException(
"%s: pod failed to initialize (%s)" % (
namespace, pod.status.phase))
resource["pod"] = label.name
return resource
def getRequestHandler(self, poolworker, request):
return handler.KubernetesNodeRequestHandler(poolworker, request)

View File

@ -19,6 +19,8 @@ labels:
- name: trusty-external
min-ready: 1
- name: trusty-static
- name: kubernetes-namespace
- name: pod-fedora
providers:
- name: cloud1
@ -99,6 +101,18 @@ providers:
username: zuul
max-parallel-jobs: 1
- name: kubespray
driver: kubernetes
context: admin-cluster.local
pools:
- name: main
labels:
- name: kubernetes-namespace
type: namespace
- name: pod-fedora
type: pod
image: docker.io/fedora:28
diskimages:
- name: trusty
formats:

21
nodepool/tests/fixtures/kubernetes.yaml vendored Normal file
View File

@ -0,0 +1,21 @@
zookeeper-servers:
- host: {zookeeper_host}
port: {zookeeper_port}
chroot: {zookeeper_chroot}
labels:
- name: pod-fedora
- name: kubernetes-namespace
providers:
- name: kubespray
driver: kubernetes
context: admin-cluster.local
pools:
- name: main
labels:
- name: kubernetes-namespace
type: namespace
- name: pod-fedora
type: pod
image: docker.io/fedora:28

View File

@ -0,0 +1,155 @@
# Copyright (C) 2018 Red Hat
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
# implied.
# See the License for the specific language governing permissions and
# limitations under the License.
import fixtures
import logging
from nodepool import tests
from nodepool import zk
from nodepool.driver.kubernetes import provider
class FakeCoreClient(object):
def __init__(self):
self.namespaces = []
class FakeApi:
class configuration:
host = "http://localhost:8080"
verify_ssl = False
self.api_client = FakeApi()
def list_namespace(self):
class FakeNamespaces:
items = self.namespaces
return FakeNamespaces
def create_namespace(self, ns_body):
class FakeNamespace:
class metadata:
name = ns_body['metadata']['name']
self.namespaces.append(FakeNamespace)
return FakeNamespace
def delete_namespace(self, name, delete_body):
to_delete = None
for namespace in self.namespaces:
if namespace.metadata.name == name:
to_delete = namespace
break
if not to_delete:
raise RuntimeError("Unknown namespace %s" % name)
self.namespaces.remove(to_delete)
def create_namespaced_service_account(self, ns, sa_body):
return
def read_namespaced_service_account(self, user, ns):
class FakeSA:
class secret:
name = "fake"
FakeSA.secrets = [FakeSA.secret]
return FakeSA
def read_namespaced_secret(self, name, ns):
class FakeSecret:
data = {'ca.crt': 'fake-ca', 'token': 'fake-token'}
return FakeSecret
def create_namespaced_pod(self, ns, pod_body):
return
def read_namespaced_pod(self, name, ns):
class FakePod:
class status:
phase = "Running"
return FakePod
class FakeRbacClient(object):
def create_namespaced_role(self, ns, role_body):
return
def create_namespaced_role_binding(self, ns, role_binding_body):
return
class TestDriverKubernetes(tests.DBTestCase):
log = logging.getLogger("nodepool.TestDriverKubernetes")
def setUp(self):
super().setUp()
self.fake_k8s_client = FakeCoreClient()
self.fake_rbac_client = FakeRbacClient()
def fake_get_client(*args):
return self.fake_k8s_client, self.fake_rbac_client
self.useFixture(fixtures.MockPatchObject(
provider.KubernetesProvider, '_get_client',
fake_get_client
))
def test_kubernetes_machine(self):
configfile = self.setup_config('kubernetes.yaml')
pool = self.useNodepool(configfile, watermark_sleep=1)
pool.start()
req = zk.NodeRequest()
req.state = zk.REQUESTED
req.node_types.append('pod-fedora')
self.zk.storeNodeRequest(req)
self.log.debug("Waiting for request %s", req.id)
req = self.waitForNodeRequest(req)
self.assertEqual(req.state, zk.FULFILLED)
self.assertNotEqual(req.nodes, [])
node = self.zk.getNode(req.nodes[0])
self.assertEqual(node.allocated_to, req.id)
self.assertEqual(node.state, zk.READY)
self.assertIsNotNone(node.launcher)
self.assertEqual(node.connection_type, 'kubectl')
self.assertEqual(node.connection_port.get('token'), 'fake-token')
node.state = zk.DELETING
self.zk.storeNode(node)
self.waitForNodeDeletion(node)
def test_kubernetes_native(self):
configfile = self.setup_config('kubernetes.yaml')
pool = self.useNodepool(configfile, watermark_sleep=1)
pool.start()
req = zk.NodeRequest()
req.state = zk.REQUESTED
req.node_types.append('kubernetes-namespace')
self.zk.storeNodeRequest(req)
self.log.debug("Waiting for request %s", req.id)
req = self.waitForNodeRequest(req)
self.assertEqual(req.state, zk.FULFILLED)
self.assertNotEqual(req.nodes, [])
node = self.zk.getNode(req.nodes[0])
self.assertEqual(node.allocated_to, req.id)
self.assertEqual(node.state, zk.READY)
self.assertIsNotNone(node.launcher)
self.assertEqual(node.connection_type, 'namespace')
self.assertEqual(node.connection_port.get('token'), 'fake-token')
node.state = zk.DELETING
self.zk.storeNode(node)
self.waitForNodeDeletion(node)

View File

@ -0,0 +1,15 @@
- hosts: all
vars:
nodepool_log_dir: '{{ ansible_user_dir }}/work/logs/nodepool'
nodepool_etc_dir: '{{ ansible_user_dir }}/work/etc'
tasks:
- name: 'Copy files from {{ nodepool_log_dir }}'
synchronize:
src: '{{ nodepool_log_dir }}'
dest: '{{ zuul.executor.log_root }}/{{ inventory_hostname }}'
mode: pull
- name: 'Copy files from {{ nodepool_etc_dir }}'
synchronize:
src: '{{ nodepool_etc_dir }}'
dest: '{{ zuul.executor.log_root }}/{{ inventory_hostname }}'
mode: pull

View File

@ -0,0 +1,14 @@
- hosts: all
roles:
- role: bindep
bindep_profile: dev
tasks:
- name: Ensure nodepool services directories
file:
path: '{{ ansible_user_dir }}/work/{{ item }}'
state: directory
with_items:
- logs/nodepool
- etc
- images

View File

@ -0,0 +1,5 @@
- hosts: all
roles:
- install-nodepool
- install-kubernetes
- nodepool-k8s-functional

View File

@ -12,3 +12,4 @@ voluptuous
kazoo
Paste
WebOb>=1.8.1
openshift

View File

@ -0,0 +1,88 @@
- name: Install zookeeper
package:
name: zookeeperd
become: yes
- name: Start zookeeper
service:
name: zookeeper
state: started
become: yes
- name: Install nodepool
command: pip3 install .
args:
chdir: "{{ zuul.projects['git.openstack.org/openstack-infra/nodepool'].src_dir }}"
become: yes
- name: Setup logging.conf
copy:
content: |
[loggers]
keys=root,nodepool,requests,openstack
[handlers]
keys=console,normal
[formatters]
keys=simple
[logger_root]
level=WARNING
handlers=console
[logger_requests]
level=WARNING
handlers=normal
qualname=requests
[logger_openstack]
level=WARNING
handlers=normal
qualname=openstack
[logger_gear]
level=DEBUG
handlers=normal
qualname=gear
[logger_nodepool]
level=DEBUG
handlers=normal
qualname=nodepool
[handler_console]
level=WARNING
class=StreamHandler
formatter=simple
args=(sys.stdout,)
[handler_normal]
level=DEBUG
class=FileHandler
formatter=simple
args=('{{ ansible_user_dir }}/work/logs/nodepool/launcher.log',)
[formatter_simple]
format=%(asctime)s %(levelname)s %(name)s: %(message)s
datefmt=
dest: "{{ ansible_user_dir }}/work/etc/logging.conf"
- name: Setup nodepool.yaml
copy:
content: |
zookeeper-servers:
- host: localhost
images-dir: "{{ ansible_user_dir }}/work/images/"
build-log-dir: "{{ ansible_user_dir }}/work/logs/nodepool/"
dest: "{{ ansible_user_dir }}/work/etc/nodepool.yaml"
- name: Setup secure.conf
copy:
content: ""
dest: "{{ ansible_user_dir }}/work/etc/secure.conf"
- name: Start the service
command: nodepool-launcher -c etc/nodepool.yaml -s etc/secure.conf -l etc/logging.conf -p launcher.pid
args:
chdir: "{{ ansible_user_dir }}/work/"

View File

@ -0,0 +1,59 @@
- name: debug context names
command: kubectl config get-contexts
- name: Update nodepool.yaml
copy:
content: |
zookeeper-servers:
- host: localhost
images-dir: "{{ ansible_user_dir }}/work/images/"
build-log-dir: "{{ ansible_user_dir }}/work/logs/nodepool/"
labels:
- name: kubernetes-namespace
min-ready: 1
- name: pod-fedora
min-ready: 1
providers:
- name: minikube
driver: kubernetes
context: minikube
pools:
- name: main
labels:
- name: kubernetes-namespace
type: namespace
- name: pod-fedora
type: pod
image: docker.io/fedora:28
dest: "{{ ansible_user_dir }}/work/etc/nodepool.yaml"
- name: Set nodepool_command facts
set_fact:
nodepool_command: nodepool -c "{{ ansible_user_dir }}/work/etc/nodepool.yaml"
- name: Wait for nodes
command: "{{ nodepool_command }} list"
register: nodepool_list
until: nodepool_list.stdout
retries: 120
delay: 2
- name: Show nodes
command: "{{ nodepool_command }} list --detail"
- name: Wait for fedora pod to be running
shell: "{{ nodepool_command }} list | grep 'pod-fedora.*running'"
register: nodepool_list
until: nodepool_list.stdout
retries: 120
delay: 2
ignore_errors: yes
- name: Show nodes
command: "{{ nodepool_command }} list --detail"
- name: Show namespace
command: kubectl get namespaces
- name: Show pods
command: kubectl get --all-namespaces=true pods