Move multi-threading code to a library.

This patch extracts the multi-threading code from bin/swift into
swiftclient/multithreading and adds tests.  In particular, this new way
of doing it (with context managers) will prevent non-daemonic threads
from wedging the process when unexpected exceptions happen.

I enabled reporting of which lines, specifically, are not covered by
unit tests (added -m option to "coverage report" in .unittests).

This patch includes a drive-by fix for uploading a segmented file with
--use-slo when that object already exists.  A key of "name" was used
instead of "path", raising KeyError.

There's also another drive-by fix for uploading segmented objects with
--use-slo.  Commit 874e0e4427 regressed
this by removing the capturing of thread-worker results in
QueueFunctionThread.run().  This patch restores that functionality and
the feature (uploading SLO objects).

Change-Id: I0b4f677e4a734e83d1a25088d9a74f7d46384e53
This commit is contained in:
Darrell Bishop 2013-06-26 22:47:49 -07:00
parent 5d9c6f845c
commit 9198e95468
9 changed files with 927 additions and 462 deletions

View File

@ -3,6 +3,6 @@ set -e
python setup.py testr --coverage
RET=$?
coverage report
coverage report -m
rm -f .coverage
exit $RET

656
bin/swift

File diff suppressed because it is too large Load Diff

View File

@ -15,8 +15,6 @@
import sys
import os
import swiftclient
# If extensions (or modules to document with autodoc) are in another directory,
# add these directories to sys.path here. If the directory is relative to the
@ -36,6 +34,9 @@ sys.path.insert(0, ROOT)
extensions = ['sphinx.ext.autodoc', 'sphinx.ext.doctest', 'sphinx.ext.todo',
'sphinx.ext.coverage']
autoclass_content = 'both'
autodoc_default_flags = ['members', 'undoc-members', 'show-inheritance']
# Add any paths that contain templates here, relative to this directory.
templates_path = ['_templates']
@ -50,7 +51,7 @@ master_doc = 'index'
# General information about the project.
project = u'Swiftclient'
copyright = u'2012 OpenStack, LLC.'
copyright = u'2013 OpenStack, LLC.'
# The version info for the project you're documenting, acts as replacement for
# |version| and |release|, also used in various other places throughout the

View File

@ -4,14 +4,18 @@ swiftclient
==============
.. automodule:: swiftclient
:members:
:undoc-members:
:show-inheritance:
swiftclient.client
==================
.. automodule:: swiftclient.client
:members:
:undoc-members:
:show-inheritance:
swiftclient.exceptions
======================
.. automodule:: swiftclient.exceptions
swiftclient.multithreading
==========================
.. automodule:: swiftclient.multithreading

View File

@ -28,6 +28,8 @@ from urlparse import urlparse, urlunparse
from httplib import HTTPException, HTTPConnection, HTTPSConnection
from time import sleep
from swiftclient.exceptions import ClientException, InvalidHeadersException
try:
from swiftclient.https_connection import HTTPSConnectionNoSSLComp
except ImportError:
@ -102,64 +104,6 @@ except ImportError:
from json import loads as json_loads
class InvalidHeadersException(Exception):
pass
class ClientException(Exception):
def __init__(self, msg, http_scheme='', http_host='', http_port='',
http_path='', http_query='', http_status=0, http_reason='',
http_device='', http_response_content=''):
Exception.__init__(self, msg)
self.msg = msg
self.http_scheme = http_scheme
self.http_host = http_host
self.http_port = http_port
self.http_path = http_path
self.http_query = http_query
self.http_status = http_status
self.http_reason = http_reason
self.http_device = http_device
self.http_response_content = http_response_content
def __str__(self):
a = self.msg
b = ''
if self.http_scheme:
b += '%s://' % self.http_scheme
if self.http_host:
b += self.http_host
if self.http_port:
b += ':%s' % self.http_port
if self.http_path:
b += self.http_path
if self.http_query:
b += '?%s' % self.http_query
if self.http_status:
if b:
b = '%s %s' % (b, self.http_status)
else:
b = str(self.http_status)
if self.http_reason:
if b:
b = '%s %s' % (b, self.http_reason)
else:
b = '- %s' % self.http_reason
if self.http_device:
if b:
b = '%s: device %s' % (b, self.http_device)
else:
b = 'device %s' % self.http_device
if self.http_response_content:
if len(self.http_response_content) <= 60:
b += ' %s' % self.http_response_content
else:
b += ' [first 60 chars of response] %s' \
% self.http_response_content[:60]
return b and '%s: %s' % (a, b) or a
def http_connection(url, proxy=None, ssl_compression=True):
"""
Make an HTTPConnection or HTTPSConnection

72
swiftclient/exceptions.py Normal file
View File

@ -0,0 +1,72 @@
# Copyright (c) 2010-2013 OpenStack, 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 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.
class ClientException(Exception):
def __init__(self, msg, http_scheme='', http_host='', http_port='',
http_path='', http_query='', http_status=0, http_reason='',
http_device='', http_response_content=''):
Exception.__init__(self, msg)
self.msg = msg
self.http_scheme = http_scheme
self.http_host = http_host
self.http_port = http_port
self.http_path = http_path
self.http_query = http_query
self.http_status = http_status
self.http_reason = http_reason
self.http_device = http_device
self.http_response_content = http_response_content
def __str__(self):
a = self.msg
b = ''
if self.http_scheme:
b += '%s://' % self.http_scheme
if self.http_host:
b += self.http_host
if self.http_port:
b += ':%s' % self.http_port
if self.http_path:
b += self.http_path
if self.http_query:
b += '?%s' % self.http_query
if self.http_status:
if b:
b = '%s %s' % (b, self.http_status)
else:
b = str(self.http_status)
if self.http_reason:
if b:
b = '%s %s' % (b, self.http_reason)
else:
b = '- %s' % self.http_reason
if self.http_device:
if b:
b = '%s: device %s' % (b, self.http_device)
else:
b = 'device %s' % self.http_device
if self.http_response_content:
if len(self.http_response_content) <= 60:
b += ' %s' % self.http_response_content
else:
b += ' [first 60 chars of response] %s' \
% self.http_response_content[:60]
return b and '%s: %s' % (a, b) or a
class InvalidHeadersException(Exception):
pass

View File

@ -0,0 +1,241 @@
# Copyright (c) 2010-2012 OpenStack, 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 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 sys
from time import sleep
from Queue import Queue
from threading import Thread
from traceback import format_exception
from swiftclient.exceptions import ClientException
class StopWorkerThreadSignal(object):
pass
class QueueFunctionThread(Thread):
"""
Calls `func`` for each item in ``queue``; ``func`` is called with a
de-queued item as the first arg followed by ``*args`` and ``**kwargs``.
Any exceptions raised by ``func`` are stored in :attr:`self.exc_infos`.
If the optional kwarg ``store_results`` is specified, it must be a list and
each result of invoking ``func`` will be appended to that list.
Putting a :class:`StopWorkerThreadSignal` instance into queue will cause
this thread to exit.
"""
def __init__(self, queue, func, *args, **kwargs):
"""
:param queue: A :class:`Queue` object from which work jobs will be
pulled.
:param func: A callable which will be invoked with a dequeued item
followed by ``*args`` and ``**kwargs``.
:param \*args: Optional positional arguments for ``func``.
:param \*\*kwargs: Optional kwargs for func. If the kwarg
``store_results`` is specified, its value must be a
list, and every result from invoking ``func`` will
be appended to the supplied list. The kwarg
``store_results`` will not be passed into ``func``.
"""
Thread.__init__(self)
self.queue = queue
self.func = func
self.args = args
self.kwargs = kwargs
self.exc_infos = []
self.store_results = kwargs.pop('store_results', None)
def run(self):
while True:
item = self.queue.get()
if isinstance(item, StopWorkerThreadSignal):
break
try:
result = self.func(item, *self.args, **self.kwargs)
if self.store_results is not None:
self.store_results.append(result)
except Exception:
self.exc_infos.append(sys.exc_info())
class QueueFunctionManager(object):
"""
A context manager to handle the life-cycle of a single :class:`Queue`
and a list of associated :class:`QueueFunctionThread` instances.
This class is not usually instantiated directly. Instead, call the
:meth:`MultiThreadingManager.queue_manager` object method,
which will return an instance of this class.
When entering the context, ``thread_count`` :class:`QueueFunctionThread`
instances are created and started. The input queue is returned. Inside
the context, any work item put into the queue will get worked on by one of
the :class:`QueueFunctionThread` instances.
When the context is exited, all threads are sent a
:class:`StopWorkerThreadSignal` instance and then all threads are waited
upon. Finally, any exceptions from any of the threads are reported on via
the supplied ``thread_manager``'s :meth:`error` method. If an
``error_counter`` list was supplied on instantiation, its first element is
incremented once for every exception which occurred.
"""
def __init__(self, func, thread_count, thread_manager, thread_args=None,
thread_kwargs=None, error_counter=None,
connection_maker=None):
"""
:param func: The worker function which will be passed into each
:class:`QueueFunctionThread`'s constructor.
:param thread_count: The number of worker threads to run.
:param thread_manager: An instance of :class:`MultiThreadingManager`.
:param thread_args: Optional positional arguments to be passed into
each invocation of ``func`` after the de-queued
work item.
:param thread_kwargs: Optional keyword arguments to be passed into each
invocation of ``func``. If a list is supplied as
the ``store_results`` keyword argument, it will
be filled with every result of invoking ``func``
in all threads.
:param error_counter: Optional list containing one integer. If
supplied, the list's first element will be
incremented once for each exception in any
thread. This happens only when exiting the
context.
:param connection_maker: Optional callable. If supplied, this callable
will be invoked once per created thread, and
the result will be passed into func after the
de-queued work item but before ``thread_args``
and ``thread_kwargs``. This is used to ensure
each thread has its own connection to Swift.
"""
self.func = func
self.thread_count = thread_count
self.thread_manager = thread_manager
self.error_counter = error_counter
self.connection_maker = connection_maker
self.queue = Queue(10000)
self.thread_list = []
self.thread_args = thread_args if thread_args else ()
self.thread_kwargs = thread_kwargs if thread_kwargs else {}
def __enter__(self):
for _junk in xrange(self.thread_count):
if self.connection_maker:
thread_args = (self.connection_maker(),) + self.thread_args
else:
thread_args = self.thread_args
qf_thread = QueueFunctionThread(self.queue, self.func,
*thread_args, **self.thread_kwargs)
qf_thread.start()
self.thread_list.append(qf_thread)
return self.queue
def __exit__(self, exc_type, exc_value, traceback):
for thread in [t for t in self.thread_list if t.isAlive()]:
self.queue.put(StopWorkerThreadSignal())
while any(map(QueueFunctionThread.is_alive, self.thread_list)):
sleep(0.05)
for thread in self.thread_list:
for info in thread.exc_infos:
if self.error_counter:
self.error_counter[0] += 1
if isinstance(info[1], ClientException):
self.thread_manager.error(str(info[1]))
else:
self.thread_manager.error(''.join(format_exception(*info)))
class MultiThreadingManager(object):
"""
One object to manage context for multi-threading. This should make
bin/swift less error-prone and allow us to test this code.
This object is a context manager and returns itself into the context. When
entering the context, two printing threads are created (see below) and they
are waited on and cleaned up when exiting the context.
A convenience method, :meth:`queue_manager`, is provided to create a
:class:`QueueFunctionManager` context manager (a thread-pool with an
associated input queue for work items).
Also, thread-safe printing to two streams is provided. The
:meth:`print_msg` method will print to the supplied ``print_stream``
(defaults to ``sys.stdout``) and the :meth:`error` method will print to the
supplied ``error_stream`` (defaults to ``sys.stderr``). Both of these
printing methods will format the given string with any supplied ``*args``
(a la printf) and encode the result to utf8 if necessary.
The attribute :attr:`self.error_count` is incremented once per error
message printed, so an application can tell if any worker threads
encountered exceptions or otherwise called :meth:`error` on this instance.
The swift command-line tool uses this to exit non-zero if any error strings
were printed.
"""
def __init__(self, print_stream=sys.stdout, error_stream=sys.stderr):
"""
:param print_stream: The stream to which :meth:`print_msg` sends
formatted messages, encoded to utf8 if necessary.
:param error_stream: The stream to which :meth:`error` sends formatted
messages, encoded to utf8 if necessary.
"""
self.print_stream = print_stream
self.printer = QueueFunctionManager(self._print, 1, self)
self.error_stream = error_stream
self.error_printer = QueueFunctionManager(self._print_error, 1, self)
self.error_count = 0
def __enter__(self):
self.printer.__enter__()
self.error_printer.__enter__()
return self
def __exit__(self, exc_type, exc_value, traceback):
self.error_printer.__exit__(exc_type, exc_value, traceback)
self.printer.__exit__(exc_type, exc_value, traceback)
def queue_manager(self, func, thread_count, *args, **kwargs):
connection_maker = kwargs.pop('connection_maker', None)
error_counter = kwargs.pop('error_counter', None)
return QueueFunctionManager(func, thread_count, self, thread_args=args,
thread_kwargs=kwargs,
connection_maker=connection_maker,
error_counter=error_counter)
def print_msg(self, msg, *fmt_args):
if fmt_args:
msg = msg % fmt_args
self.printer.queue.put(msg)
def error(self, msg, *fmt_args):
if fmt_args:
msg = msg % fmt_args
self.error_printer.queue.put(msg)
def _print(self, item, stream=None):
if stream is None:
stream = self.print_stream
if isinstance(item, unicode):
item = item.encode('utf8')
print >>stream, item
def _print_error(self, item):
self.error_count += 1
return self._print(item, stream=self.error_stream)

View File

@ -12,7 +12,6 @@
# implied.
# See the License for the specific language governing permissions and
# limitations under the License.
"""Miscellaneous utility functions for use with Swift."""
TRUE_VALUES = set(('true', '1', 'yes', 'on', 't', 'y'))

View File

@ -0,0 +1,334 @@
# Copyright (c) 2010-2013 OpenStack, 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 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 sys
import time
import mock
import testtools
import threading
from cStringIO import StringIO
from Queue import Queue, Empty
from swiftclient import multithreading as mt
from swiftclient.exceptions import ClientException
class ThreadTestCase(testtools.TestCase):
def setUp(self):
super(ThreadTestCase, self).setUp()
self.got_args_kwargs = Queue()
self.starting_thread_count = threading.active_count()
def _func(self, q_item, *args, **kwargs):
self.got_items.put(q_item)
self.got_args_kwargs.put((args, kwargs))
if q_item == 'go boom':
raise Exception('I went boom!')
if q_item == 'c boom':
raise ClientException(
'Client Boom', http_scheme='http', http_host='192.168.22.1',
http_port=80, http_path='/booze', http_status=404,
http_reason='to much', http_response_content='no sir!')
return 'best result EVAR!'
def assertQueueContains(self, queue, expected_contents):
got_contents = []
try:
while True:
got_contents.append(queue.get(timeout=0.1))
except Empty:
pass
if isinstance(expected_contents, set):
got_contents = set(got_contents)
self.assertEqual(expected_contents, got_contents)
class TestQueueFunctionThread(ThreadTestCase):
def setUp(self):
super(TestQueueFunctionThread, self).setUp()
self.input_queue = Queue()
self.got_items = Queue()
self.stored_results = []
self.qft = mt.QueueFunctionThread(self.input_queue, self._func,
'one_arg', 'two_arg',
red_fish='blue_arg',
store_results=self.stored_results)
self.qft.start()
def tearDown(self):
if self.qft.is_alive():
self.finish_up_thread()
super(TestQueueFunctionThread, self).tearDown()
def finish_up_thread(self):
self.input_queue.put(mt.StopWorkerThreadSignal())
while self.qft.is_alive():
time.sleep(0.05)
def test_plumbing_and_store_results(self):
self.input_queue.put('abc')
self.input_queue.put(123)
self.finish_up_thread()
self.assertQueueContains(self.got_items, ['abc', 123])
self.assertQueueContains(self.got_args_kwargs, [
(('one_arg', 'two_arg'), {'red_fish': 'blue_arg'}),
(('one_arg', 'two_arg'), {'red_fish': 'blue_arg'})])
self.assertEqual(self.stored_results,
['best result EVAR!', 'best result EVAR!'])
def test_exception_handling(self):
self.input_queue.put('go boom')
self.input_queue.put('ok')
self.input_queue.put('go boom')
self.finish_up_thread()
self.assertQueueContains(self.got_items,
['go boom', 'ok', 'go boom'])
self.assertEqual(len(self.qft.exc_infos), 2)
self.assertEqual(Exception, self.qft.exc_infos[0][0])
self.assertEqual(Exception, self.qft.exc_infos[1][0])
self.assertEqual(('I went boom!',), self.qft.exc_infos[0][1].args)
self.assertEqual(('I went boom!',), self.qft.exc_infos[1][1].args)
class TestQueueFunctionManager(ThreadTestCase):
def setUp(self):
super(TestQueueFunctionManager, self).setUp()
self.thread_manager = mock.create_autospec(
mt.MultiThreadingManager, spec_set=True, instance=True)
self.thread_count = 4
self.error_counter = [0]
self.got_items = Queue()
self.stored_results = []
self.qfq = mt.QueueFunctionManager(
self._func, self.thread_count, self.thread_manager,
thread_args=('1arg', '2arg'),
thread_kwargs={'a': 'b', 'store_results': self.stored_results},
error_counter=self.error_counter,
connection_maker=self.connection_maker)
def connection_maker(self):
return 'yup, I made a connection'
def test_context_manager_without_error_counter(self):
self.qfq = mt.QueueFunctionManager(
self._func, self.thread_count, self.thread_manager,
thread_args=('1arg', '2arg'),
thread_kwargs={'a': 'b', 'store_results': self.stored_results},
connection_maker=self.connection_maker)
with self.qfq as input_queue:
self.assertEqual(self.starting_thread_count + self.thread_count,
threading.active_count())
input_queue.put('go boom')
self.assertEqual(self.starting_thread_count, threading.active_count())
error_strs = map(str, self.thread_manager.error.call_args_list)
self.assertEqual(1, len(error_strs))
self.assertTrue('Exception: I went boom!' in error_strs[0])
def test_context_manager_without_conn_maker_or_error_counter(self):
self.qfq = mt.QueueFunctionManager(
self._func, self.thread_count, self.thread_manager,
thread_args=('1arg', '2arg'), thread_kwargs={'a': 'b'})
with self.qfq as input_queue:
self.assertEqual(self.starting_thread_count + self.thread_count,
threading.active_count())
for i in xrange(20):
input_queue.put('slap%d' % i)
self.assertEqual(self.starting_thread_count, threading.active_count())
self.assertEqual([], self.thread_manager.error.call_args_list)
self.assertEqual(0, self.error_counter[0])
self.assertQueueContains(self.got_items,
set(['slap%d' % i for i in xrange(20)]))
self.assertQueueContains(
self.got_args_kwargs,
[(('1arg', '2arg'), {'a': 'b'})] * 20)
self.assertEqual(self.stored_results, [])
def test_context_manager_with_exceptions(self):
with self.qfq as input_queue:
self.assertEqual(self.starting_thread_count + self.thread_count,
threading.active_count())
for i in xrange(20):
input_queue.put('item%d' % i if i % 2 == 0 else 'go boom')
self.assertEqual(self.starting_thread_count, threading.active_count())
error_strs = map(str, self.thread_manager.error.call_args_list)
self.assertEqual(10, len(error_strs))
self.assertTrue(all(['Exception: I went boom!' in s for s in
error_strs]))
self.assertEqual(10, self.error_counter[0])
expected_items = set(['go boom'] + ['item%d' % i for i in xrange(20)
if i % 2 == 0])
self.assertQueueContains(self.got_items, expected_items)
self.assertQueueContains(
self.got_args_kwargs,
[(('yup, I made a connection', '1arg', '2arg'), {'a': 'b'})] * 20)
self.assertEqual(self.stored_results, ['best result EVAR!'] * 10)
def test_context_manager_with_client_exceptions(self):
with self.qfq as input_queue:
self.assertEqual(self.starting_thread_count + self.thread_count,
threading.active_count())
for i in xrange(20):
input_queue.put('item%d' % i if i % 2 == 0 else 'c boom')
self.assertEqual(self.starting_thread_count, threading.active_count())
error_strs = map(str, self.thread_manager.error.call_args_list)
self.assertEqual(10, len(error_strs))
stringification = 'Client Boom: ' \
'http://192.168.22.1:80/booze 404 to much no sir!'
self.assertTrue(all([stringification in s for s in error_strs]))
self.assertEqual(10, self.error_counter[0])
expected_items = set(['c boom'] + ['item%d' % i for i in xrange(20)
if i % 2 == 0])
self.assertQueueContains(self.got_items, expected_items)
self.assertQueueContains(
self.got_args_kwargs,
[(('yup, I made a connection', '1arg', '2arg'), {'a': 'b'})] * 20)
self.assertEqual(self.stored_results, ['best result EVAR!'] * 10)
def test_context_manager_with_connection_maker(self):
with self.qfq as input_queue:
self.assertEqual(self.starting_thread_count + self.thread_count,
threading.active_count())
for i in xrange(20):
input_queue.put('item%d' % i)
self.assertEqual(self.starting_thread_count, threading.active_count())
self.assertEqual([], self.thread_manager.error.call_args_list)
self.assertEqual(0, self.error_counter[0])
self.assertQueueContains(self.got_items,
set(['item%d' % i for i in xrange(20)]))
self.assertQueueContains(
self.got_args_kwargs,
[(('yup, I made a connection', '1arg', '2arg'), {'a': 'b'})] * 20)
self.assertEqual(self.stored_results, ['best result EVAR!'] * 20)
class TestMultiThreadingManager(ThreadTestCase):
@mock.patch('swiftclient.multithreading.QueueFunctionManager')
def test_instantiation(self, mock_qfq):
thread_manager = mt.MultiThreadingManager()
self.assertEqual([
mock.call(thread_manager._print, 1, thread_manager),
mock.call(thread_manager._print_error, 1, thread_manager),
], mock_qfq.call_args_list)
# These contexts don't get entered into until the
# MultiThreadingManager's context is entered.
self.assertEqual([], thread_manager.printer.__enter__.call_args_list)
self.assertEqual([],
thread_manager.error_printer.__enter__.call_args_list)
# Test default values for the streams.
self.assertEqual(sys.stdout, thread_manager.print_stream)
self.assertEqual(sys.stderr, thread_manager.error_stream)
@mock.patch('swiftclient.multithreading.QueueFunctionManager')
def test_queue_manager_no_args(self, mock_qfq):
thread_manager = mt.MultiThreadingManager()
mock_qfq.reset_mock()
mock_qfq.return_value = 'slap happy!'
self.assertEqual(
'slap happy!',
thread_manager.queue_manager(self._func, 88))
self.assertEqual([
mock.call(self._func, 88, thread_manager, thread_args=(),
thread_kwargs={}, connection_maker=None,
error_counter=None)
], mock_qfq.call_args_list)
@mock.patch('swiftclient.multithreading.QueueFunctionManager')
def test_queue_manager_with_args(self, mock_qfq):
thread_manager = mt.MultiThreadingManager()
mock_qfq.reset_mock()
mock_qfq.return_value = 'do run run'
self.assertEqual(
'do run run',
thread_manager.queue_manager(self._func, 88, 'fun', times='are',
connection_maker='abc', to='be had',
error_counter='def'))
self.assertEqual([
mock.call(self._func, 88, thread_manager, thread_args=('fun',),
thread_kwargs={'times': 'are', 'to': 'be had'},
connection_maker='abc', error_counter='def')
], mock_qfq.call_args_list)
def test_printers(self):
out_stream = StringIO()
err_stream = StringIO()
with mt.MultiThreadingManager(
print_stream=out_stream,
error_stream=err_stream) as thread_manager:
# Sanity-checking these gives power to the previous test which
# looked at the default values of thread_manager.print/error_stream
self.assertEqual(out_stream, thread_manager.print_stream)
self.assertEqual(err_stream, thread_manager.error_stream)
self.assertEqual(self.starting_thread_count + 2,
threading.active_count())
thread_manager.print_msg('one-argument')
thread_manager.print_msg('one %s, %d fish', 'fish', 88)
thread_manager.error('I have %d problems, but a %s is not one',
99, u'\u062A\u062A')
thread_manager.print_msg('some\n%s\nover the %r', 'where',
u'\u062A\u062A')
thread_manager.error('one-error-argument')
thread_manager.error('Sometimes\n%.1f%% just\ndoes not\nwork!',
3.14159)
self.assertEqual(self.starting_thread_count, threading.active_count())
out_stream.seek(0)
self.assertEqual([
'one-argument\n',
'one fish, 88 fish\n',
'some\n', 'where\n', "over the u'\\u062a\\u062a'\n",
], list(out_stream.readlines()))
err_stream.seek(0)
self.assertEqual([
u'I have 99 problems, but a \u062A\u062A is not one\n'.encode(
'utf8'),
'one-error-argument\n',
'Sometimes\n', '3.1% just\n', 'does not\n', 'work!\n',
], list(err_stream.readlines()))
self.assertEqual(3, thread_manager.error_count)
if __name__ == '__main__':
testtools.main()