Add auth token to websocket

When making a websocket request, browsers do not send the
"Authorization" header.  Therefore if a Zuul tenant is run in
a configuration where authz is required for read-only access,
the websocket-based log streaming will always fail.

To correct this, we will remove the http request authz check
from the console-stream endpoint, and add an optional token
parameter to the websocket message payload.  The JS web app
will be responsible for sending the auth token in the payload,
and the web server will validate it if it is required for the
tenant.  Thanks to Andrei Dmitriev for this suggestion.

Since we essentially have two different authz code paths in
zuul-web now, in order to share as much code as possible, the
authz sequence is refactored in such a way that the final authz
check can be deferred.  First we create an AuthContext at the
start of the request which stores tenant and header information,
then the actual validation is performed in a separate step where
the token can optionally be provided.

In the http code path, we create the AuthContext and validate
immediately, using the Authorization header, and we do all of that
in the cherrypy tool at the start of the request.

In the websocket code path, we create the AuthContext as the
websocket handler is being created by the cherrypy request handler,
then we perform validation after receiving a message on the
websocket.  We use the token supplied from the request.

Error handling is adjusted so in the http code path, exceptions
that return appropriate http errors are raised, but in the
websocket path, these are caught and translated into websocket
close calls.

A related issue is that we perform no validation that the
streaming build log being requested belongs to the tenant via
which the request is being sent.  This was unecessary before
read-only access was an option, but now that it is, we should
check that a streaming build request arrives via the correct
tenant URL.  This change adjusts that as well.

During testing, it was noted that the tenant configuration syntax
allows admin-rules and access-rules to use the scalar-or-list
pattern, however some parts of the code assumed only lists.  The
configloader is updated to use scalar-or-list for both of those
values.

Change-Id: Ifd4c21bb1fe962bf23acb5b4f10b3bbaba61e63a
Co-Authored-By: Andrei Dmitriev <andrei.dmitriev@nokia.com>
This commit is contained in:
James E. Blair 2023-10-23 16:06:25 -07:00
parent 17badd55a6
commit 18fb324f1e
12 changed files with 361 additions and 88 deletions

View File

@ -0,0 +1,29 @@
# NOTE(Shrews): Do not run any tasks that will need zuul_console to stream
# output because that will not work. Since we just need any output in our
# ansible log, the test coordination tasks should be sufficient.
- hosts: localhost
tasks:
- debug: var=waitpath
- debug:
msg: |
Multiline
Debug Test Token String
Message
# Logging of loops is special so we do a simple one iteration
# loop and check that we log things properly
- name: Override ansible_loop_var
set_fact:
_testing_fact: "{{ other_loop_var }}"
with_random_choice:
- "one"
- "two"
- "three"
loop_control:
loop_var: "other_loop_var"
# Do not finish until test creates the flag file
- wait_for:
state: present
path: "{{waitpath}}"

View File

@ -0,0 +1,30 @@
- pipeline:
name: check
manager: independent
trigger:
gerrit:
- event: patchset-created
success:
gerrit:
Verified: 1
failure:
gerrit:
Verified: -1
- job:
name: base
parent: null
nodeset:
nodes:
- name: test_node
label: test_label
- job:
name: python27
vars:
waitpath: '{{zuul._test.test_root}}/builds/{{zuul.build}}/test_wait'
run: playbooks/python27.yaml
nodeset:
nodes:
- name: controller
label: ubuntu-trusty

View File

@ -0,0 +1,5 @@
- project:
name: org/project
check:
jobs:
- python27

View File

@ -0,0 +1 @@
test

View File

@ -0,0 +1,25 @@
- authorization-rule:
name: user-rule
conditions:
- groups: users
- api-root:
access-rules: user-rule
- tenant:
name: tenant-one
access-rules: user-rule
source:
gerrit:
config-projects:
- common-config
untrusted-projects:
- org/project
- tenant:
name: tenant-two
access-rules: user-rule
source:
gerrit:
config-projects: []
untrusted-projects: []

View File

@ -1,4 +1,5 @@
# Copyright 2017 Red Hat, Inc.
# Copyright 2023 Acme Gating, LLC
#
# 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
@ -32,16 +33,19 @@ from zuul.lib.statsd import normalize_statsd_name
import tests.base
from tests.base import iterate_timeout, ZuulWebFixture, FIXTURE_DIR
import jwt
from ws4py.client import WebSocketBaseClient
class WSClient(WebSocketBaseClient):
def __init__(self, port, build_uuid):
def __init__(self, port, build_uuid, tenant='tenant-one', token=None):
self.port = port
self.build_uuid = build_uuid
self.token = token
self.results = ''
self.close_results = None
self.event = threading.Event()
uri = 'ws://[::1]:%s/api/tenant/tenant-one/console-stream' % port
uri = f'ws://[::1]:{port}/api/tenant/{tenant}/console-stream'
super(WSClient, self).__init__(uri)
self.thread = threading.Thread(target=self.run)
@ -51,9 +55,15 @@ class WSClient(WebSocketBaseClient):
if message.is_text:
self.results += message.data.decode('utf-8')
def closed(self, code, reason=None):
self.close_results = (code, reason)
super().closed(code, reason)
def run(self):
self.connect()
req = {'uuid': self.build_uuid, 'logfile': None}
if self.token:
req['token'] = self.token
self.send(json.dumps(req))
self.event.set()
super(WSClient, self).run()
@ -225,6 +235,11 @@ class TestStreamingBase(tests.base.AnsibleZuulTestCase):
gateway_port = gateway.server.socket.getsockname()[1]
return gateway, (self.host, gateway_port)
def runWSClient(self, *args, **kw):
client = WSClient(*args, **kw)
client.event.wait()
return client
class TestStreaming(TestStreamingBase):
@ -304,11 +319,6 @@ class TestStreaming(TestStreamingBase):
m = re.search(pattern, self.streaming_data[None])
self.assertNotEqual(m, None)
def runWSClient(self, port, build_uuid):
client = WSClient(port, build_uuid)
client.event.wait()
return client
def test_decode_boundaries(self):
'''
Test multi-byte characters crossing read buffer boundaries.
@ -630,6 +640,104 @@ class TestStreaming(TestStreamingBase):
self.assertEqual(file_contents, self.streaming_data[None])
class TestAuthWebsocketStreaming(TestStreamingBase):
config_file = 'zuul-admin-web.conf'
tenant_config_file = 'config/auth-streamer/main.yaml'
def test_auth_websocket_streaming(self):
# Start the web server
web = self.useFixture(
ZuulWebFixture(self.changes, self.config,
self.additional_event_queues, self.upstream_root,
self.poller_events,
self.git_url_with_auth, self.addCleanup,
self.test_root))
# Start the finger streamer daemon
streamer = zuul.lib.log_streamer.LogStreamer(
self.host, 0, self.executor_server.jobdir_root)
self.addCleanup(streamer.stop)
# Need to set the streaming port before submitting the job
finger_port = streamer.server.socket.getsockname()[1]
self.executor_server.log_streaming_port = finger_port
A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')
self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
# We don't have any real synchronization for the ansible jobs, so
# just wait until we get our running build.
for x in iterate_timeout(30, "build"):
if len(self.builds):
break
build = self.builds[0]
self.assertEqual(build.name, 'python27')
build_dir = os.path.join(self.executor_server.jobdir_root, build.uuid)
for x in iterate_timeout(30, "build dir"):
if os.path.exists(build_dir):
break
# Need to wait to make sure that jobdir gets set
for x in iterate_timeout(30, "jobdir"):
if build.jobdir is not None:
break
build = self.builds[0]
# Wait for the job to begin running and create the ansible log file.
# The job waits to complete until the flag file exists, so we can
# safely access the log here. We only open it (to force a file handle
# to be kept open for it after the job finishes) but wait to read the
# contents until the job is done.
ansible_log = os.path.join(build.jobdir.log_root, 'job-output.txt')
for x in iterate_timeout(30, "ansible log"):
if os.path.exists(ansible_log):
break
logfile = open(ansible_log, 'r')
self.addCleanup(logfile.close)
# Attempt to stream the log without an authz token
client1 = self.runWSClient(web.port, build.uuid)
client1.thread.join()
self.assertEqual(
(4000, b'Missing "Authorization" header'),
client1.close_results)
# Attempt to bypass authz by using a valid token with a
# different tenant
authz = {'iss': 'zuul_operator',
'aud': 'zuul.example.com',
'sub': 'testuser',
'groups': ['users'],
'exp': int(time.time()) + 3600}
token = jwt.encode(authz, key='NoDanaOnlyZuul',
algorithm='HS256')
client2 = self.runWSClient(web.port, build.uuid, tenant='tenant-two',
token=token)
client2.thread.join()
self.assertEqual(
(4011, b'Build not found'),
client2.close_results)
# Finally, use a valid token
client3 = self.runWSClient(web.port, build.uuid, token=token)
# Allow the job to complete
flag_file = os.path.join(build_dir, 'test_wait')
open(flag_file, 'w').close()
# Wait for the websocket client to complete, which it should when
# it's received the full log.
client3.thread.join()
self.waitUntilSettled()
file_contents = logfile.read()
self.log.debug("\n\nFile contents: %s\n\n", file_contents)
self.log.debug("\n\nStreamed: %s\n\n", client3.results)
self.assertEqual(file_contents, client3.results)
class CountingFingerRequestHandler(zuul.lib.fingergw.RequestHandler):
def _fingerClient(self, server, port, build_uuid, use_ssl):

View File

@ -3656,7 +3656,8 @@ class TestWebApiAccessRules(BaseTestWeb):
'/api/tenant/{tenant}/nodes',
'/api/tenant/{tenant}/key/{project}.pub',
'/api/tenant/{tenant}/project-ssh-key/{project}.pub',
'/api/tenant/{tenant}/console-stream',
# console-stream is tested by test_auth_websocket_streaming
# '/api/tenant/{tenant}/console-stream',
'/api/tenant/{tenant}/badge',
'/api/tenant/{tenant}/builds',
'/api/tenant/{tenant}/build/{uuid}',

View File

@ -77,6 +77,10 @@ function getStreamUrl(apiPrefix) {
return streamUrl
}
function getAuthToken() {
return authToken
}
function makeRequest(url, method, data) {
if (method === undefined) {
method = 'get'
@ -325,6 +329,7 @@ export {
apiUrl,
getHomepageUrl,
getStreamUrl,
getAuthToken,
fetchChangeStatus,
fetchConfigErrors,
fetchStatus,

View File

@ -25,7 +25,7 @@ import { FitAddon} from 'xterm-addon-fit'
import { WebLinksAddon } from 'xterm-addon-web-links'
import { SearchAddon } from 'xterm-addon-search'
import { getStreamUrl } from '../api'
import { getStreamUrl, getAuthToken } from '../api'
class StreamPage extends React.Component {
static propTypes = {
@ -72,6 +72,10 @@ class StreamPage extends React.Component {
if (logfile) {
params.logfile = logfile
}
const authToken = getAuthToken()
if (authToken) {
params.token = authToken
}
document.title = 'Zuul Stream | ' + params.uuid.slice(0, 7)
const term = new Terminal()

View File

@ -1983,9 +1983,9 @@ class TenantParser(object):
tenant.exclude_unprotected_branches = \
conf['exclude-unprotected-branches']
if conf.get('admin-rules') is not None:
tenant.admin_rules = conf['admin-rules']
tenant.admin_rules = as_list(conf['admin-rules'])
if conf.get('access-rules') is not None:
tenant.access_rules = conf['access-rules']
tenant.access_rules = as_list(conf['access-rules'])
if conf.get('authentication-realm') is not None:
tenant.default_auth_realm = conf['authentication-realm']
if conf.get('semaphores') is not None:

View File

@ -176,7 +176,8 @@ class CustomThreadingTCPServer(socketserver.ThreadingTCPServer):
return sock, addr
def getJobLogStreamAddress(executor_api, uuid, source_zone):
def getJobLogStreamAddress(executor_api, uuid, source_zone,
tenant_name=None):
"""
Looks up the log stream address for the given build UUID.
@ -196,6 +197,11 @@ def getJobLogStreamAddress(executor_api, uuid, source_zone):
if build_request is None:
raise StreamingError("Build not found")
if tenant_name is not None and build_request.tenant_name != tenant_name:
# Intentionally the same error as above to avoid leaking
# out-of-tenant build information.
raise StreamingError("Build not found")
worker_info = build_request.worker_info
if not worker_info:
raise StreamingError("Build did not start yet")

View File

@ -1,5 +1,5 @@
# Copyright (c) 2017 Red Hat
# Copyright 2021-2022 Acme Gating, LLC
# Copyright 2021-2023 Acme Gating, LLC
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@ -109,13 +109,15 @@ def get_request_logger(logger=None):
class APIError(cherrypy.HTTPError):
def __init__(self, code, json_doc=None):
def __init__(self, code, json_doc=None, headers=None):
self._headers = headers or {}
self._json_doc = json_doc
super().__init__(code)
def set_response(self):
super().set_response()
resp = cherrypy.response
resp.headers.update(self._headers)
if self._json_doc:
ret = json.dumps(self._json_doc).encode('utf8')
resp.body = ret
@ -183,35 +185,108 @@ class AuthInfo:
self.admin = admin
def _check_auth(require_admin=False, require_auth=False, tenant=None):
if require_admin:
require_auth = True
request = cherrypy.serving.request
zuulweb = request.app.root
class AuthContext:
"""This stores common information about the authorization context for
a resource so that the request for the resource can be authorized
either in a Cherrpy tool or a Websocket handler.
if tenant:
if not require_auth and tenant.access_rules:
# This tenant requires auth for read-only access
:param Tenant tenant: The Zuul tenant object; supply if it is relevant
for authorization to the protected resource
:param bool require_admin: Whether admin access is required for
this resource
:param bool require_auth: Whether authenticated access is required for
this resource
"""
def __init__(self, tenant=None, require_admin=None, require_auth=None):
request = cherrypy.serving.request
zuulweb = request.app.root.zuulweb
if require_admin:
require_auth = True
else:
if not require_auth and zuulweb.zuulweb.abide.api_root.access_rules:
# The API root requires auth for read-only access
require_auth = True
# Always set the auth variable
request.params['auth'] = None
basic_error = zuulweb._basic_auth_header_check(required=require_auth)
if basic_error is not None:
return
claims, token_error = zuulweb._auth_token_check(required=require_auth)
if token_error is not None:
return
access, admin = zuulweb._isAuthorized(tenant, claims)
if (require_auth and not access) or (require_admin and not admin):
raise APIError(403)
if tenant:
if not require_auth and tenant.access_rules:
# This tenant requires auth for read-only access
require_auth = True
else:
if not require_auth and zuulweb.abide.api_root.access_rules:
# The API root requires auth for read-only access
require_auth = True
self.require_admin = require_admin
self.require_auth = require_auth
self.headers = request.headers
self.tenant = tenant
self.zuulweb = zuulweb
request.params['auth'] = AuthInfo(claims['__zuul_uid_claim'],
admin)
def validate(self, token=None):
"""Validate access to the resource
:param str token: The bearer token; if not supplied, it will be
retreieved from the request header
Raises an exception if authorization failed and was required.
:returns: an AuthInfo instance if authorization succeeded;
None if it failed but was not required.
"""
if token is None:
try:
token = self._getTokenFromHeader()
except APIError:
if self.require_auth:
raise
return None
try:
claims = self._getClaims(token)
except APIError:
if self.require_auth:
raise
return None
access, admin = self.zuulweb.api._isAuthorized(self.tenant, claims)
if ((self.require_auth and not access) or
(self.require_admin and not admin)):
raise APIError(403)
return AuthInfo(claims['__zuul_uid_claim'], admin)
def _getTokenFromHeader(self):
"""Make sure protected endpoints have a Authorization header with the
bearer token."""
token_header = self.headers.get('Authorization', None)
# Add basic checks here
if token_header is None:
e = 'Missing "Authorization" header'
e_desc = e
elif not token_header.lower().startswith('bearer '):
e = 'Invalid Authorization header format'
e_desc = '"Authorization" header must start with "Bearer"'
else:
token = token_header[len('Bearer '):]
return token
error_header = '''Bearer realm="%s"
error="%s"
error_description="%s"''' % (self.zuulweb.authenticators.default_realm,
e,
e_desc)
error_data = {'description': e_desc,
'error': e,
'realm': self.zuulweb.authenticators.default_realm}
raise APIError(401, json_doc=error_data, headers={
"WWW-Authenticate": error_header
})
def _getClaims(self, token):
try:
claims = self.zuulweb.authenticators.authenticate(token)
except exceptions.AuthTokenException as e:
error_data = {'description': str(e.error_description),
'error': str(e.error),
'realm': str(e.realm)}
raise APIError(e.HTTPError, json_doc=error_data,
headers=e.getAdditionalHeaders())
return claims
def check_root_auth(**kw):
@ -221,7 +296,8 @@ def check_root_auth(**kw):
if request.handler is None:
# handle_options has already aborted the request.
return
return _check_auth(**kw)
auth_context = AuthContext(**kw)
request.params['auth'] = auth_context.validate()
def check_tenant_auth(**kw):
@ -237,7 +313,8 @@ def check_tenant_auth(**kw):
# Always set the tenant variable
tenant = zuulweb._getTenantOrRaise(tenant_name)
request.params['tenant'] = tenant
return _check_auth(**kw, tenant=tenant)
auth_context = AuthContext(tenant=tenant, **kw)
request.params['auth'] = auth_context.validate()
cherrypy.tools.check_root_auth = cherrypy.Tool('on_start_resource',
@ -324,6 +401,16 @@ class LogStreamHandler(WebSocket):
super(LogStreamHandler, self).__init__(*args, **kw)
self.streamer = None
# Because we lose our request context by the time we get the
# authorization token over the websocket protocol, we create
# an AuthContext here, and then perform delayed validation on
# messages we recieve.
request = cherrypy.serving.request
self.zuulweb = request.app.root.zuulweb
self.tenant_name = request.params.get('tenant_name')
tenant = self.zuulweb.api._getTenantOrRaise(self.tenant_name)
self.auth_context = AuthContext(tenant=tenant)
def received_message(self, message):
if message.is_text:
req = json.loads(message.data.decode('utf-8'))
@ -331,6 +418,17 @@ class LogStreamHandler(WebSocket):
if self.streamer:
self.log.debug("Ignoring request due to existing streamer")
return
token = req.get('token')
try:
self.auth_context.validate(token=token)
except APIError as e:
if e._json_doc and e._json_doc.get('error'):
msg = e._json_doc.get('error').encode('utf8')[:123]
else:
msg = b'Authorization error'
return self.logClose(4000, msg)
try:
self._streamLog(req)
except Exception:
@ -369,7 +467,8 @@ class LogStreamHandler(WebSocket):
try:
port_location = streamer_utils.getJobLogStreamAddress(
self.zuulweb.executor_api,
request['uuid'], source_zone=self.zuulweb.zone)
request['uuid'], source_zone=self.zuulweb.zone,
tenant_name=self.tenant_name)
except exceptions.StreamingError as e:
return self.logClose(4011, str(e))
@ -475,48 +574,6 @@ class ZuulWebAPI(object):
def log(self):
return get_request_logger()
def _basic_auth_header_check(self, required=True):
"""make sure protected endpoints have a Authorization header with the
bearer token."""
token = cherrypy.request.headers.get('Authorization', None)
# Add basic checks here
if token is None:
e = 'Missing "Authorization" header'
e_desc = e
elif not token.lower().startswith('bearer '):
e = 'Invalid Authorization header format'
e_desc = '"Authorization" header must start with "Bearer"'
else:
return None
error_header = '''Bearer realm="%s"
error="%s"
error_description="%s"''' % (self.zuulweb.authenticators.default_realm,
e,
e_desc)
error_data = {'description': e_desc,
'error': e,
'realm': self.zuulweb.authenticators.default_realm}
if required:
cherrypy.response.headers["WWW-Authenticate"] = error_header
raise APIError(401, error_data)
return error_data
def _auth_token_check(self, required=True):
rawToken = \
cherrypy.request.headers['Authorization'][len('Bearer '):]
try:
claims = self.zuulweb.authenticators.authenticate(rawToken)
except exceptions.AuthTokenException as e:
if required:
for header, contents in e.getAdditionalHeaders().items():
cherrypy.response.headers[header] = contents
raise APIError(e.HTTPError)
return ({},
{'description': e.error_description,
'error': e.error,
'realm': e.realm})
return (claims, None)
@cherrypy.expose
@cherrypy.tools.json_in()
@cherrypy.tools.json_out(content_type='application/json; charset=utf-8')
@ -1642,15 +1699,17 @@ class ZuulWebAPI(object):
@cherrypy.tools.handle_options()
# We don't check auth here since we would never fall through to it
def console_stream_options(self, tenant_name):
cherrypy.request.ws_handler.zuulweb = self.zuulweb
pass
@cherrypy.expose
@cherrypy.tools.save_params()
@cherrypy.tools.websocket(handler_cls=LogStreamHandler)
# Options handling in _options method
@cherrypy.tools.check_tenant_auth()
def console_stream_get(self, tenant_name, tenant, auth):
cherrypy.request.ws_handler.zuulweb = self.zuulweb
# The Authorization header is not included when upgrading to
# websocket, so the websocket handler itself will validate auth
# using the websocket protocol.
def console_stream_get(self, tenant_name):
pass
@cherrypy.expose
@cherrypy.tools.save_params()