Merge "Implement an OpenShift resource provider"

This commit is contained in:
Zuul 2019-01-17 15:39:06 +00:00 committed by Gerrit Code Review
commit e459ffa0fd
18 changed files with 1026 additions and 1 deletions

View File

@ -139,6 +139,21 @@
required-projects:
- openstack-infra/nodepool
- job:
description: |
Test that nodepool works with openshift.
name: nodepool-functional-openshift
pre-run: playbooks/nodepool-functional-openshift/pre.yaml
run: playbooks/nodepool-functional-openshift/run.yaml
nodeset:
nodes:
- name: cluster
label: centos-7
- name: launcher
label: fedora-28
required-projects:
- openstack-infra/nodepool
- project:
check:
jobs:
@ -154,6 +169,7 @@
- nodepool-functional-py35-src:
voting: false
- nodepool-functional-k8s
- nodepool-functional-openshift
- pbrx-build-container-images:
vars:
pbrx_prefix: zuul

View File

@ -13,3 +13,4 @@ musl-dev [compile test platform:apk]
python3-dev [compile test platform:dpkg]
python3-devel [compile test platform:rpm]
zookeeperd [platform:dpkg test]
zookeeper [platform:rpm test]

View File

@ -352,6 +352,13 @@ Options
kubernetes driver, see the separate section
:attr:`providers.[kubernetes]`
.. value:: openshift
For details on the extra options required and provided by the
openshift driver, see the separate section
:attr:`providers.[openshift]`
OpenStack Driver
----------------
@ -1134,3 +1141,132 @@ Selecting the kubernetes driver adds the following options to the
Only used by the
:value:`providers.[kubernetes].labels.type.pod` label type;
specifies the image name used by the pod.
Openshift Driver
----------------
Selecting the openshift driver adds the following options to the
:attr:`providers` section of the configuration.
.. attr-overview::
:prefix: providers.[openshift]
:maxdepth: 3
.. attr:: providers.[openshift]
:type: list
An Openshift provider's resources are partitioned into groups called `pool`
(see :attr:`providers.[openshift].pools` for details), and within a pool,
the node types which are to be made available are listed
(see :attr:`providers.[openshift].labels` for details).
.. note:: For documentation purposes the option names are prefixed
``providers.[openshift]`` to disambiguate from other
drivers, but ``[openshift]`` is not required in the
configuration (e.g. below
``providers.[openshift].pools`` refers to the ``pools``
key in the ``providers`` section when the ``openshift``
driver is selected).
Example:
.. code-block:: yaml
providers:
- name: cluster
driver: openshift
context: context-name
pools:
- name: main
labels:
- name: openshift-project
type: project
- name: openshift-pod
type: pod
image: docker.io/fedora:28
.. attr:: context
:required:
Name of the context configured in ``kube/config``.
Before using the driver, Nodepool services need a ``kube/config`` file
manually installed with self-provisioner (the service account needs to
be able to create project) context.
Make sure the context is present in ``oc config get-contexts`` command
output.
.. attr:: launch-retries
:default: 3
The number of times to retry launching a node before considering
the job failed.
.. attr:: max-projects
:default: infinite
:type: int
Maximum number of projects that can be used.
.. attr:: pools
:type: list
A pool defines a group of resources from an Openshift provider.
.. attr:: name
:required:
Project's name are prefixed with the pool's name.
.. attr:: labels
:type: list
Each entry in a pool`s `labels` section indicates that the
corresponding label is available for use in this pool.
Each entry is a dictionary with the following keys
.. attr:: name
:required:
Identifier for this label; references an entry in the
:attr:`labels` section.
.. attr:: type
The Openshift provider supports two types of labels:
.. value:: project
Project labels provide an empty project configured
with a service account that can creates pods, services,
configmaps, etc.
.. value:: pod
Pod labels provide a dedicated project with a single pod
created using the
:attr:`providers.[openshift].labels.image` parameter and it
is configured with a service account that can exec and get
the logs of the pod.
.. attr:: image
Only used by the
:value:`providers.[openshift].labels.type.pod` label type;
specifies the image name used by the pod.
.. attr:: cpu
:type: int
Only used by the
:value:`providers.[openshift].labels.type.pod` label type;
specifies the amount of cpu to request for the pod.
.. attr:: memory
:type: int
Only used by the
:value:`providers.[openshift].labels.type.pod` label type;
specifies the amount of memory in MB to request for the pod.

View File

@ -0,0 +1,37 @@
# 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.openshift.config import OpenshiftProviderConfig
from nodepool.driver.openshift.provider import OpenshiftProvider
from openshift import config
class OpenshiftDriver(Driver):
def __init__(self):
super().__init__()
def reset(self):
try:
config.load_kube_config(persist_config=True)
except FileNotFoundError:
pass
def getProviderConfig(self, provider):
return OpenshiftProviderConfig(self, provider)
def getProvider(self, provider_config, use_taskmanager):
return OpenshiftProvider(provider_config, use_taskmanager)

View File

@ -0,0 +1,128 @@
# 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 math
import voluptuous as v
from nodepool.driver import ConfigPool
from nodepool.driver import ConfigValue
from nodepool.driver import ProviderConfig
class OpenshiftLabel(ConfigValue):
def __eq__(self, other):
if isinstance(other, OpenshiftLabel):
return (other.name == self.name and
other.type == self.type and
other.image_pull == self.image_pull and
other.image == self.image and
other.cpu == self.cpu and
other.memory == self.memory)
return False
def __repr__(self):
return "<OpenshiftLabel %s>" % self.name
class OpenshiftPool(ConfigPool):
def __eq__(self, other):
if isinstance(other, OpenshiftPool):
return (super().__eq__(other) and
other.name == self.name and
other.labels == self.labels)
return False
def __repr__(self):
return "<OpenshiftPool %s>" % self.name
def load(self, pool_config, full_config):
super().load(pool_config)
self.name = pool_config['name']
self.labels = {}
for label in pool_config.get('labels', []):
pl = OpenshiftLabel()
pl.name = label['name']
pl.type = label['type']
pl.image = label.get('image')
pl.image_pull = label.get('image-pull', 'IfNotPresent')
pl.cpu = label.get('cpu')
pl.memory = label.get('memory')
pl.pool = self
self.labels[pl.name] = pl
full_config.labels[label['name']].pools.append(self)
class OpenshiftProviderConfig(ProviderConfig):
def __init__(self, driver, provider):
self.driver_object = driver
self.__pools = {}
super().__init__(provider)
def __eq__(self, other):
if isinstance(other, OpenshiftProviderConfig):
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']
self.max_projects = self.provider.get('max-projects', math.inf)
for pool in self.provider.get('pools', []):
pp = OpenshiftPool()
pp.load(pool, config)
pp.provider = self
self.pools[pp.name] = pp
def getSchema(self):
openshift_label = {
v.Required('name'): str,
v.Required('type'): str,
'image': str,
'image-pull': str,
'cpu': int,
'memory': int,
}
pool = {
v.Required('name'): str,
v.Required('labels'): [openshift_label],
}
schema = ProviderConfig.getCommonSchemaDict()
schema.update({
v.Required('pools'): [pool],
v.Required('context'): str,
'launch-retries': int,
'max-projects': int,
})
return v.Schema(schema)
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,138 @@
# 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 exceptions
from nodepool import zk
from nodepool.driver.utils import NodeLauncher
from nodepool.driver import NodeRequestHandler
class OpenShiftLauncher(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")
project = "%s-%s" % (self.handler.pool.name, self.node.id)
self.node.external_id = self.handler.manager.createProject(project)
self.zk.storeNode(self.node)
resource = self.handler.manager.prepareProject(project)
if self.label.type == "pod":
self.handler.manager.createPod(
project, self.label)
resource['pod'] = self.label.name
self.node.connection_type = "kubectl"
self.node.interface_ip = self.label.name
else:
self.node.connection_type = "project"
self.node.state = zk.READY
# NOTE: resource access token may be encrypted here
self.node.connection_port = resource
self.zk.storeNode(self.node)
self.log.info("Resource %s is ready", project)
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 as e:
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 'exceeded quota' in str(e).lower():
self.log.info("%s: quota exceeded", self.node.id)
raise exceptions.QuotaException("Quota exceeded")
if attempts == self._retries:
raise
attempts += 1
class OpenshiftNodeRequestHandler(NodeRequestHandler):
log = logging.getLogger("nodepool.driver.openshift."
"OpenshiftNodeRequestHandler")
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, zk.ABORTED)
for s in node_states):
return False
return True
def hasRemainingQuota(self, node_types):
if len(self.manager.listNodes()) + 1 > self.provider.max_projects:
return False
return True
def launch(self, node):
label = self.pool.labels[node.type[0]]
thd = OpenShiftLauncher(self, node, self.provider, label)
thd.start()
self._threads.append(thd)

View File

@ -0,0 +1,237 @@
# 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 client as os_client
from openshift import config
from nodepool import exceptions
from nodepool.driver import Provider
from nodepool.driver.openshift import handler
urllib3.disable_warnings()
class OpenshiftProvider(Provider):
log = logging.getLogger("nodepool.driver.openshift.OpenshiftProvider")
def __init__(self, provider, *args):
self.provider = provider
self.ready = False
try:
self.os_client, self.k8s_client = self._get_client(
provider.context)
except kce.ConfigException:
self.log.exception(
"Couldn't load context %s from config", provider.context)
self.os_client = None
self.k8s_client = None
self.project_names = set()
for pool in provider.pools.values():
self.project_names.add(pool.name)
def _get_client(self, context):
conf = config.new_client_from_config(context=context)
return (
os_client.OapiApi(conf),
k8s_client.CoreV1Api(conf))
def start(self, zk_conn):
self.log.debug("Starting")
if self.ready or not self.os_client or not self.k8s_client:
return
self.ready = True
def stop(self):
self.log.debug("Stopping")
def listNodes(self):
servers = []
class FakeServer:
def __init__(self, project, provider, valid_names):
self.id = project.metadata.name
self.name = project.metadata.name
self.metadata = {}
if [True for valid_name in valid_names
if project.metadata.name.startswith("%s-" % valid_name)]:
node_id = project.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 project, let's skip metadata
pass
def get(self, name, default=None):
return getattr(self, name, default)
if self.ready:
for project in self.os_client.list_project().items:
servers.append(FakeServer(
project, self.provider.name, self.project_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 project" % server_id)
try:
self.os_client.delete_project(server_id)
self.log.info("%s: project removed" % server_id)
except Exception:
# TODO: implement better exception handling
self.log.exception("Couldn't remove project %s" % server_id)
def waitForNodeCleanup(self, server_id):
for retry in range(300):
try:
self.os_client.read_project(server_id)
except Exception:
break
time.sleep(1)
def createProject(self, project):
self.log.debug("%s: creating project" % project)
# Create the project
proj_body = {
'apiVersion': 'v1',
'kind': 'ProjectRequest',
'metadata': {
'name': project,
}
}
self.os_client.create_project_request(proj_body)
return project
def prepareProject(self, project):
user = "zuul-worker"
# Create the service account
sa_body = {
'apiVersion': 'v1',
'kind': 'ServiceAccount',
'metadata': {'name': user}
}
self.k8s_client.create_namespaced_service_account(project, sa_body)
# Wait for the token to be created
for retry in range(30):
sa = self.k8s_client.read_namespaced_service_account(
user, project)
token = None
if sa.secrets:
for secret_obj in sa.secrets:
secret = self.k8s_client.read_namespaced_secret(
secret_obj.name, project)
token = secret.metadata.annotations.get(
'openshift.io/token-secret.value')
if token:
break
if token:
break
time.sleep(1)
if not token:
raise exceptions.LaunchNodepoolException(
"%s: couldn't find token for service account %s" %
(project, sa))
# Give service account admin access
role_body = {
'apiVersion': 'v1',
'kind': 'RoleBinding',
'metadata': {'name': 'admin-0'},
'roleRef': {'name': 'admin'},
'subjects': [{
'kind': 'ServiceAccount',
'name': user,
'namespace': project,
}],
'userNames': ['system:serviceaccount:%s:zuul-worker' % project]
}
try:
self.os_client.create_namespaced_role_binding(project, role_body)
except ValueError:
# https://github.com/ansible/ansible/issues/36939
pass
resource = {
'namespace': project,
'host': self.os_client.api_client.configuration.host,
'skiptls': not self.os_client.api_client.configuration.verify_ssl,
'token': token,
'user': user,
}
self.log.info("%s: project created" % project)
return resource
def createPod(self, project, label):
spec_body = {
'name': label.name,
'image': label.image,
'imagePullPolicy': label.image_pull,
'command': ["/bin/bash", "-c", "--"],
'args': ["while true; do sleep 30; done;"],
'workingDir': '/tmp',
}
if label.cpu or label.memory:
spec_body['resources'] = {}
for rtype in ('requests', 'limits'):
rbody = {}
if label.cpu:
rbody['cpu'] = int(label.cpu)
if label.memory:
rbody['memory'] = '%dMi' % int(label.memory)
spec_body['resources'][rtype] = rbody
pod_body = {
'apiVersion': 'v1',
'kind': 'Pod',
'metadata': {'name': label.name},
'spec': {
'containers': [spec_body],
},
'restartPolicy': 'Never',
}
self.k8s_client.create_namespaced_pod(project, pod_body)
for retry in range(300):
pod = self.k8s_client.read_namespaced_pod(label.name, project)
if pod.status.phase == "Running":
break
self.log.debug("%s: pod status is %s", project, pod.status.phase)
time.sleep(1)
if retry == 299:
raise exceptions.LaunchNodepoolException(
"%s: pod failed to initialize (%s)" % (
project, pod.status.phase))
def getRequestHandler(self, poolworker, request):
return handler.OpenshiftNodeRequestHandler(poolworker, request)

View File

@ -328,7 +328,7 @@ class DBTestCase(BaseTestCase):
self.log = logging.getLogger("tests")
self.setupZK()
def setup_config(self, filename, images_dir=None):
def setup_config(self, filename, images_dir=None, context_name=None):
if images_dir is None:
images_dir = fixtures.TempDir()
self.useFixture(images_dir)
@ -341,6 +341,7 @@ class DBTestCase(BaseTestCase):
config = conf_fd.read().decode('utf8')
data = config.format(images_dir=images_dir.path,
build_log_dir=build_log_dir.path,
context_name=context_name,
zookeeper_host=self.zookeeper_host,
zookeeper_port=self.zookeeper_port,
zookeeper_chroot=self.zookeeper_chroot)

View File

@ -21,6 +21,8 @@ labels:
- name: trusty-static
- name: kubernetes-namespace
- name: pod-fedora
- name: openshift-project
- name: openshift-pod
providers:
- name: cloud1
@ -116,6 +118,20 @@ providers:
type: pod
image: docker.io/fedora:28
- name: openshift
driver: openshift
context: "/hostname:8443/self-provisioner-service-account"
pools:
- name: main
labels:
- name: openshift-project
type: project
- name: openshift-pod
type: pod
image: docker.io/fedora:28
memory: 512
cpu: 2
diskimages:
- name: trusty
formats:

View File

@ -0,0 +1,23 @@
zookeeper-servers:
- host: {zookeeper_host}
port: {zookeeper_port}
chroot: {zookeeper_chroot}
labels:
- name: openshift-project
min-ready: 1
- name: openshift-pod
min-ready: 1
providers:
- name: openshift
driver: openshift
context: {context_name}
pools:
- name: main
labels:
- name: openshift-project
type: project
- name: openshift-pod
type: pod
image: docker.io/fedora:28

21
nodepool/tests/fixtures/openshift.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: openshift-project
providers:
- name: openshift
driver: openshift
context: admin-cluster.local
pools:
- name: main
labels:
- name: openshift-project
type: project
- name: pod-fedora
type: pod
image: docker.io/fedora:28

View File

@ -0,0 +1,50 @@
# 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 logging
import os
import yaml
from nodepool import tests
class TestOpenShift(tests.DBTestCase):
log = logging.getLogger("nodepool.TestOpenShift")
def setup_config(self, filename):
adjusted_filename = "functional/openshift/" + filename
# Openshift context name are not hardcoded,
# discover the name setup by oc login
kubecfg = yaml.safe_load(open(os.path.expanduser("~/.kube/config")))
try:
ctx_name = kubecfg['contexts'][0]['name']
except IndexError:
raise RuntimeError("Run oc login first")
self.log.debug("Using %s context name", ctx_name)
return super().setup_config(adjusted_filename, context_name=ctx_name)
def test_basic(self):
configfile = self.setup_config('basic.yaml')
pool = self.useNodepool(configfile, watermark_sleep=1)
pool.start()
nodes = self.waitForNodes("openshift-project", 1)
self.assertEqual(1, len(nodes))
self.assertEqual(nodes[0].connection_type, "project")
nodes = self.waitForNodes("openshift-pod", 1)
self.assertEqual(1, len(nodes))
self.assertEqual(nodes[0].connection_type, "kubectl")

View File

@ -0,0 +1,153 @@
# 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.openshift import provider
class FakeOpenshiftClient(object):
def __init__(self):
self.projects = []
class FakeApi:
class configuration:
host = "http://localhost:8080"
verify_ssl = False
self.api_client = FakeApi()
def list_project(self):
class FakeProjects:
items = self.projects
return FakeProjects
def create_project_request(self, proj_body):
class FakeProject:
class metadata:
name = proj_body['metadata']['name']
self.projects.append(FakeProject)
return FakeProject
def delete_project(self, name):
to_delete = None
for project in self.projects:
if project.metadata.name == name:
to_delete = project
break
if not to_delete:
raise RuntimeError("Unknown project %s" % name)
self.projects.remove(to_delete)
def create_namespaced_role_binding(self, ns, role_binding_body):
return
class FakeCoreClient(object):
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:
class metadata:
annotations = {'openshift.io/token-secret.value': '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 TestDriverOpenshift(tests.DBTestCase):
log = logging.getLogger("nodepool.TestDriverOpenshift")
def setUp(self):
super().setUp()
self.fake_os_client = FakeOpenshiftClient()
self.fake_k8s_client = FakeCoreClient()
def fake_get_client(*args):
return self.fake_os_client, self.fake_k8s_client
self.useFixture(fixtures.MockPatchObject(
provider.OpenshiftProvider, '_get_client',
fake_get_client
))
def test_openshift_machine(self):
configfile = self.setup_config('openshift.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_openshift_native(self):
configfile = self.setup_config('openshift.yaml')
pool = self.useNodepool(configfile, watermark_sleep=1)
pool.start()
req = zk.NodeRequest()
req.state = zk.REQUESTED
req.node_types.append('openshift-project')
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, 'project')
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,32 @@
- name: Configure a multi node environment
hosts: all
tasks:
- name: Set up multi-node firewall
include_role:
name: multi-node-firewall
- name: Set up multi-node firewall
include_role:
name: multi-node-hosts-file
- hosts: launcher
roles:
- role: bindep
tasks:
- name: Ensure nodepool services directories
file:
path: '{{ ansible_user_dir }}/{{ item }}'
state: directory
with_items:
- work/logs/nodepool
- work/etc
- work/images
- name: Ensure oc client is installed
package:
name: origin-clients
become: yes
- hosts: cluster
roles:
- install-openshift

View File

@ -0,0 +1,26 @@
- hosts: cluster
roles:
- deploy-openshift
- hosts: launcher
pre_tasks:
- name: Login to the openshift cluster as developer
command: >
oc login -u developer -p developer --insecure-skip-tls-verify=true
https://{{ hostvars['cluster']['ansible_hostname'] }}:8443
# Zookeeper service doesn't start by default on fedora
- name: Setup zoo.cfg
command: cp /etc/zookeeper/zoo_sample.cfg /etc/zookeeper/zoo.cfg
become: yes
ignore_errors: yes
- name: Start zookeeper
service:
name: zookeeper
state: started
become: yes
ignore_errors: yes
roles:
- role: tox
tox_envlist: functional_openshift

View File

@ -0,0 +1,5 @@
---
features:
- |
A new driver is available to support Openshift cluster as a resources provider
to enable project and pod request.

View File

@ -55,6 +55,11 @@ commands = {posargs}
commands = stestr --test-path ./nodepool/tests/functional/kubernetes run --no-subunit-trace {posargs}
stestr slowest
[testenv:functional_openshift]
basepython = python3
commands = stestr --test-path ./nodepool/tests/functional/openshift run --no-subunit-trace {posargs}
stestr slowest
[flake8]
# These are ignored intentionally in openstack-infra projects;
# please don't submit patches that solely correct them or enable them.