2013-05-06 16:05:10 -04:00
|
|
|
# Copyright 2013 Red Hat, Inc.
|
|
|
|
# Copyright 2013 New Dream Network, LLC (DreamHost)
|
|
|
|
#
|
|
|
|
# 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.
|
|
|
|
|
2015-01-16 15:17:56 +01:00
|
|
|
import futurist
|
Fix a race calling blocking MessageHandlingServer.start()
This fixes a race due to the quirkiness of the blocking executor. The
blocking executor does not create a separate thread, but is instead
explicitly executed in the calling thread. Other threads will,
however, continue to interact with it.
In the non-blocking case, the executor will have done certain
initialisation in start() before starting a worker thread and
returning control to the caller. That is, the caller can be sure that
this initialisation has occurred when control is returned. However, in
the blocking case, control is never returned. We currently work round
this by setting self._running to True before executing executor.start,
and by not doing any locking whatsoever in MessageHandlingServer.
However, this current means there is a race whereby executor.stop()
can run before executor.start(). This is fragile and extremely
difficult to reason about robustly, if not currently broken.
The solution is to split the initialisation from the execution in the
blocking case. executor.start() is no longer a blocking operation for
the blocking executor. As for the non-blocking case, executor.start()
returns as soon as initialisation is complete, indicating that it is
safe to subsequently call stop(). Actual execution is done explicitly
via the new execute() method, which blocks.
In doing this, we also make FakeBlockingThread a more complete
implementation of threading.Thread. This fixes a related issue in
that, previously, calling server.wait() on a blocking executor from
another thread would not wait for the completion of the executor. This
has a knock-on effect in test_server's ServerSetupMixin. This mixin
created an endpoint with a stop method which called server.stop().
However, as this is executed by the executor, and also joins the
executor thread, which is now blocking, this results in a deadlock. I
am satisfied that, in general, this is not a sane thing to do.
However, it is useful for these tests. We fix the tests by making the
stop method non-blocking, and do the actual stop and wait calls from
the main thread.
Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45
2015-10-19 13:04:37 +01:00
|
|
|
import threading
|
2014-11-27 11:13:50 +01:00
|
|
|
|
2015-01-16 15:17:56 +01:00
|
|
|
from oslo_messaging._executors import impl_pooledexecutor
|
Fix a race calling blocking MessageHandlingServer.start()
This fixes a race due to the quirkiness of the blocking executor. The
blocking executor does not create a separate thread, but is instead
explicitly executed in the calling thread. Other threads will,
however, continue to interact with it.
In the non-blocking case, the executor will have done certain
initialisation in start() before starting a worker thread and
returning control to the caller. That is, the caller can be sure that
this initialisation has occurred when control is returned. However, in
the blocking case, control is never returned. We currently work round
this by setting self._running to True before executing executor.start,
and by not doing any locking whatsoever in MessageHandlingServer.
However, this current means there is a race whereby executor.stop()
can run before executor.start(). This is fragile and extremely
difficult to reason about robustly, if not currently broken.
The solution is to split the initialisation from the execution in the
blocking case. executor.start() is no longer a blocking operation for
the blocking executor. As for the non-blocking case, executor.start()
returns as soon as initialisation is complete, indicating that it is
safe to subsequently call stop(). Actual execution is done explicitly
via the new execute() method, which blocks.
In doing this, we also make FakeBlockingThread a more complete
implementation of threading.Thread. This fixes a related issue in
that, previously, calling server.wait() on a blocking executor from
another thread would not wait for the completion of the executor. This
has a knock-on effect in test_server's ServerSetupMixin. This mixin
created an endpoint with a stop method which called server.stop().
However, as this is executed by the executor, and also joins the
executor thread, which is now blocking, this results in a deadlock. I
am satisfied that, in general, this is not a sane thing to do.
However, it is useful for these tests. We fix the tests by making the
stop method non-blocking, and do the actual stop and wait calls from
the main thread.
Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45
2015-10-19 13:04:37 +01:00
|
|
|
from oslo_utils import timeutils
|
2014-11-27 11:13:50 +01:00
|
|
|
|
2013-05-06 16:05:10 -04:00
|
|
|
|
2015-01-16 15:17:56 +01:00
|
|
|
class FakeBlockingThread(object):
|
Fix a race calling blocking MessageHandlingServer.start()
This fixes a race due to the quirkiness of the blocking executor. The
blocking executor does not create a separate thread, but is instead
explicitly executed in the calling thread. Other threads will,
however, continue to interact with it.
In the non-blocking case, the executor will have done certain
initialisation in start() before starting a worker thread and
returning control to the caller. That is, the caller can be sure that
this initialisation has occurred when control is returned. However, in
the blocking case, control is never returned. We currently work round
this by setting self._running to True before executing executor.start,
and by not doing any locking whatsoever in MessageHandlingServer.
However, this current means there is a race whereby executor.stop()
can run before executor.start(). This is fragile and extremely
difficult to reason about robustly, if not currently broken.
The solution is to split the initialisation from the execution in the
blocking case. executor.start() is no longer a blocking operation for
the blocking executor. As for the non-blocking case, executor.start()
returns as soon as initialisation is complete, indicating that it is
safe to subsequently call stop(). Actual execution is done explicitly
via the new execute() method, which blocks.
In doing this, we also make FakeBlockingThread a more complete
implementation of threading.Thread. This fixes a related issue in
that, previously, calling server.wait() on a blocking executor from
another thread would not wait for the completion of the executor. This
has a knock-on effect in test_server's ServerSetupMixin. This mixin
created an endpoint with a stop method which called server.stop().
However, as this is executed by the executor, and also joins the
executor thread, which is now blocking, this results in a deadlock. I
am satisfied that, in general, this is not a sane thing to do.
However, it is useful for these tests. We fix the tests by making the
stop method non-blocking, and do the actual stop and wait calls from
the main thread.
Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45
2015-10-19 13:04:37 +01:00
|
|
|
'''A minimal implementation of threading.Thread which does not create a
|
|
|
|
thread or start executing the target when start() is called. Instead, the
|
|
|
|
caller must explicitly execute the non-blocking thread.execute() method
|
|
|
|
after start() has been called.
|
|
|
|
'''
|
|
|
|
|
2015-01-16 15:17:56 +01:00
|
|
|
def __init__(self, target):
|
|
|
|
self._target = target
|
Fix a race calling blocking MessageHandlingServer.start()
This fixes a race due to the quirkiness of the blocking executor. The
blocking executor does not create a separate thread, but is instead
explicitly executed in the calling thread. Other threads will,
however, continue to interact with it.
In the non-blocking case, the executor will have done certain
initialisation in start() before starting a worker thread and
returning control to the caller. That is, the caller can be sure that
this initialisation has occurred when control is returned. However, in
the blocking case, control is never returned. We currently work round
this by setting self._running to True before executing executor.start,
and by not doing any locking whatsoever in MessageHandlingServer.
However, this current means there is a race whereby executor.stop()
can run before executor.start(). This is fragile and extremely
difficult to reason about robustly, if not currently broken.
The solution is to split the initialisation from the execution in the
blocking case. executor.start() is no longer a blocking operation for
the blocking executor. As for the non-blocking case, executor.start()
returns as soon as initialisation is complete, indicating that it is
safe to subsequently call stop(). Actual execution is done explicitly
via the new execute() method, which blocks.
In doing this, we also make FakeBlockingThread a more complete
implementation of threading.Thread. This fixes a related issue in
that, previously, calling server.wait() on a blocking executor from
another thread would not wait for the completion of the executor. This
has a knock-on effect in test_server's ServerSetupMixin. This mixin
created an endpoint with a stop method which called server.stop().
However, as this is executed by the executor, and also joins the
executor thread, which is now blocking, this results in a deadlock. I
am satisfied that, in general, this is not a sane thing to do.
However, it is useful for these tests. We fix the tests by making the
stop method non-blocking, and do the actual stop and wait calls from
the main thread.
Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45
2015-10-19 13:04:37 +01:00
|
|
|
self._running = False
|
|
|
|
self._running_cond = threading.Condition()
|
2013-05-06 16:05:10 -04:00
|
|
|
|
2015-01-16 15:17:56 +01:00
|
|
|
def start(self):
|
Fix a race calling blocking MessageHandlingServer.start()
This fixes a race due to the quirkiness of the blocking executor. The
blocking executor does not create a separate thread, but is instead
explicitly executed in the calling thread. Other threads will,
however, continue to interact with it.
In the non-blocking case, the executor will have done certain
initialisation in start() before starting a worker thread and
returning control to the caller. That is, the caller can be sure that
this initialisation has occurred when control is returned. However, in
the blocking case, control is never returned. We currently work round
this by setting self._running to True before executing executor.start,
and by not doing any locking whatsoever in MessageHandlingServer.
However, this current means there is a race whereby executor.stop()
can run before executor.start(). This is fragile and extremely
difficult to reason about robustly, if not currently broken.
The solution is to split the initialisation from the execution in the
blocking case. executor.start() is no longer a blocking operation for
the blocking executor. As for the non-blocking case, executor.start()
returns as soon as initialisation is complete, indicating that it is
safe to subsequently call stop(). Actual execution is done explicitly
via the new execute() method, which blocks.
In doing this, we also make FakeBlockingThread a more complete
implementation of threading.Thread. This fixes a related issue in
that, previously, calling server.wait() on a blocking executor from
another thread would not wait for the completion of the executor. This
has a knock-on effect in test_server's ServerSetupMixin. This mixin
created an endpoint with a stop method which called server.stop().
However, as this is executed by the executor, and also joins the
executor thread, which is now blocking, this results in a deadlock. I
am satisfied that, in general, this is not a sane thing to do.
However, it is useful for these tests. We fix the tests by making the
stop method non-blocking, and do the actual stop and wait calls from
the main thread.
Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45
2015-10-19 13:04:37 +01:00
|
|
|
if self._running:
|
|
|
|
# Not a user error. No need to translate.
|
|
|
|
raise RuntimeError('FakeBlockingThread already started')
|
|
|
|
|
|
|
|
with self._running_cond:
|
|
|
|
self._running = True
|
|
|
|
self._running_cond.notify_all()
|
|
|
|
|
|
|
|
def join(self, timeout=None):
|
|
|
|
with timeutils.StopWatch(duration=timeout) as w, self._running_cond:
|
|
|
|
while self._running:
|
|
|
|
self._running_cond.wait(w.leftover(return_none=True))
|
2015-01-16 15:17:56 +01:00
|
|
|
|
Fix a race calling blocking MessageHandlingServer.start()
This fixes a race due to the quirkiness of the blocking executor. The
blocking executor does not create a separate thread, but is instead
explicitly executed in the calling thread. Other threads will,
however, continue to interact with it.
In the non-blocking case, the executor will have done certain
initialisation in start() before starting a worker thread and
returning control to the caller. That is, the caller can be sure that
this initialisation has occurred when control is returned. However, in
the blocking case, control is never returned. We currently work round
this by setting self._running to True before executing executor.start,
and by not doing any locking whatsoever in MessageHandlingServer.
However, this current means there is a race whereby executor.stop()
can run before executor.start(). This is fragile and extremely
difficult to reason about robustly, if not currently broken.
The solution is to split the initialisation from the execution in the
blocking case. executor.start() is no longer a blocking operation for
the blocking executor. As for the non-blocking case, executor.start()
returns as soon as initialisation is complete, indicating that it is
safe to subsequently call stop(). Actual execution is done explicitly
via the new execute() method, which blocks.
In doing this, we also make FakeBlockingThread a more complete
implementation of threading.Thread. This fixes a related issue in
that, previously, calling server.wait() on a blocking executor from
another thread would not wait for the completion of the executor. This
has a knock-on effect in test_server's ServerSetupMixin. This mixin
created an endpoint with a stop method which called server.stop().
However, as this is executed by the executor, and also joins the
executor thread, which is now blocking, this results in a deadlock. I
am satisfied that, in general, this is not a sane thing to do.
However, it is useful for these tests. We fix the tests by making the
stop method non-blocking, and do the actual stop and wait calls from
the main thread.
Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45
2015-10-19 13:04:37 +01:00
|
|
|
# Thread.join() does not raise an exception on timeout. It is
|
|
|
|
# the caller's responsibility to check is_alive().
|
|
|
|
if w.expired():
|
|
|
|
return
|
2015-01-16 15:17:56 +01:00
|
|
|
|
Fix a race calling blocking MessageHandlingServer.start()
This fixes a race due to the quirkiness of the blocking executor. The
blocking executor does not create a separate thread, but is instead
explicitly executed in the calling thread. Other threads will,
however, continue to interact with it.
In the non-blocking case, the executor will have done certain
initialisation in start() before starting a worker thread and
returning control to the caller. That is, the caller can be sure that
this initialisation has occurred when control is returned. However, in
the blocking case, control is never returned. We currently work round
this by setting self._running to True before executing executor.start,
and by not doing any locking whatsoever in MessageHandlingServer.
However, this current means there is a race whereby executor.stop()
can run before executor.start(). This is fragile and extremely
difficult to reason about robustly, if not currently broken.
The solution is to split the initialisation from the execution in the
blocking case. executor.start() is no longer a blocking operation for
the blocking executor. As for the non-blocking case, executor.start()
returns as soon as initialisation is complete, indicating that it is
safe to subsequently call stop(). Actual execution is done explicitly
via the new execute() method, which blocks.
In doing this, we also make FakeBlockingThread a more complete
implementation of threading.Thread. This fixes a related issue in
that, previously, calling server.wait() on a blocking executor from
another thread would not wait for the completion of the executor. This
has a knock-on effect in test_server's ServerSetupMixin. This mixin
created an endpoint with a stop method which called server.stop().
However, as this is executed by the executor, and also joins the
executor thread, which is now blocking, this results in a deadlock. I
am satisfied that, in general, this is not a sane thing to do.
However, it is useful for these tests. We fix the tests by making the
stop method non-blocking, and do the actual stop and wait calls from
the main thread.
Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45
2015-10-19 13:04:37 +01:00
|
|
|
def is_alive(self):
|
|
|
|
return self._running
|
2015-01-16 15:17:56 +01:00
|
|
|
|
Fix a race calling blocking MessageHandlingServer.start()
This fixes a race due to the quirkiness of the blocking executor. The
blocking executor does not create a separate thread, but is instead
explicitly executed in the calling thread. Other threads will,
however, continue to interact with it.
In the non-blocking case, the executor will have done certain
initialisation in start() before starting a worker thread and
returning control to the caller. That is, the caller can be sure that
this initialisation has occurred when control is returned. However, in
the blocking case, control is never returned. We currently work round
this by setting self._running to True before executing executor.start,
and by not doing any locking whatsoever in MessageHandlingServer.
However, this current means there is a race whereby executor.stop()
can run before executor.start(). This is fragile and extremely
difficult to reason about robustly, if not currently broken.
The solution is to split the initialisation from the execution in the
blocking case. executor.start() is no longer a blocking operation for
the blocking executor. As for the non-blocking case, executor.start()
returns as soon as initialisation is complete, indicating that it is
safe to subsequently call stop(). Actual execution is done explicitly
via the new execute() method, which blocks.
In doing this, we also make FakeBlockingThread a more complete
implementation of threading.Thread. This fixes a related issue in
that, previously, calling server.wait() on a blocking executor from
another thread would not wait for the completion of the executor. This
has a knock-on effect in test_server's ServerSetupMixin. This mixin
created an endpoint with a stop method which called server.stop().
However, as this is executed by the executor, and also joins the
executor thread, which is now blocking, this results in a deadlock. I
am satisfied that, in general, this is not a sane thing to do.
However, it is useful for these tests. We fix the tests by making the
stop method non-blocking, and do the actual stop and wait calls from
the main thread.
Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45
2015-10-19 13:04:37 +01:00
|
|
|
def execute(self):
|
|
|
|
if not self._running:
|
|
|
|
# Not a user error. No need to translate.
|
|
|
|
raise RuntimeError('FakeBlockingThread not started')
|
|
|
|
|
|
|
|
try:
|
|
|
|
self._target()
|
|
|
|
finally:
|
|
|
|
with self._running_cond:
|
|
|
|
self._running = False
|
|
|
|
self._running_cond.notify_all()
|
2015-08-28 11:42:36 -07:00
|
|
|
|
2015-01-16 15:17:56 +01:00
|
|
|
|
|
|
|
class BlockingExecutor(impl_pooledexecutor.PooledExecutor):
|
Re-work server API to eliminate server subclasses
This is something I think Doug has been trying to tell me to do from the
start :-)
The main idea is to remove all the MessageHandlingServer subclasses and,
instead, if you want a server which is hooked up with the RPC dispatcher
you just use this convenience function:
server = rpc_server.get_rpc_server(transport, target, endpoints)
This means the dispatcher interface is now part of the public API, but
that should be fine since it's very simple - it's a callable that takes
a request context and message.
However, we also need to be able to construct a MessageHandlingServer
with a specific executor. By having an executor_cls parameter to the
constructor as part of the public API, we'd be exposing the executor
interface which is quite likely to change. Instead - and this seems
obvious in retrospect - just use stevedore to load executors and allow
them to be requested by name:
server = rpc_server.get_rpc_server(transport, target, endpoints,
executor='eventlet')
This also means we can get rid of openstack.common.messaging.eventlet.
2013-06-14 11:41:49 +01:00
|
|
|
"""A message executor which blocks the current thread.
|
|
|
|
|
|
|
|
The blocking executor's start() method functions as a request processing
|
|
|
|
loop - i.e. it blocks, processes messages and only returns when stop() is
|
|
|
|
called from a dispatched method.
|
|
|
|
|
|
|
|
Method calls are dispatched in the current thread, so only a single method
|
|
|
|
call can be executing at once. This executor is likely to only be useful
|
|
|
|
for simple demo programs.
|
|
|
|
"""
|
|
|
|
|
2015-01-16 15:17:56 +01:00
|
|
|
_executor_cls = lambda __, ___: futurist.SynchronousExecutor()
|
|
|
|
_thread_cls = FakeBlockingThread
|
Fix a race calling blocking MessageHandlingServer.start()
This fixes a race due to the quirkiness of the blocking executor. The
blocking executor does not create a separate thread, but is instead
explicitly executed in the calling thread. Other threads will,
however, continue to interact with it.
In the non-blocking case, the executor will have done certain
initialisation in start() before starting a worker thread and
returning control to the caller. That is, the caller can be sure that
this initialisation has occurred when control is returned. However, in
the blocking case, control is never returned. We currently work round
this by setting self._running to True before executing executor.start,
and by not doing any locking whatsoever in MessageHandlingServer.
However, this current means there is a race whereby executor.stop()
can run before executor.start(). This is fragile and extremely
difficult to reason about robustly, if not currently broken.
The solution is to split the initialisation from the execution in the
blocking case. executor.start() is no longer a blocking operation for
the blocking executor. As for the non-blocking case, executor.start()
returns as soon as initialisation is complete, indicating that it is
safe to subsequently call stop(). Actual execution is done explicitly
via the new execute() method, which blocks.
In doing this, we also make FakeBlockingThread a more complete
implementation of threading.Thread. This fixes a related issue in
that, previously, calling server.wait() on a blocking executor from
another thread would not wait for the completion of the executor. This
has a knock-on effect in test_server's ServerSetupMixin. This mixin
created an endpoint with a stop method which called server.stop().
However, as this is executed by the executor, and also joins the
executor thread, which is now blocking, this results in a deadlock. I
am satisfied that, in general, this is not a sane thing to do.
However, it is useful for these tests. We fix the tests by making the
stop method non-blocking, and do the actual stop and wait calls from
the main thread.
Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45
2015-10-19 13:04:37 +01:00
|
|
|
|
|
|
|
def __init__(self, *args, **kwargs):
|
|
|
|
super(BlockingExecutor, self).__init__(*args, **kwargs)
|
|
|
|
|
|
|
|
def execute(self):
|
|
|
|
'''Explicitly run the executor in the current context.'''
|
|
|
|
# NOTE(mdbooth): Splitting start into start and execute for the
|
|
|
|
# blocking executor closes a potential race. On a non-blocking
|
|
|
|
# executor, calling start performs some initialisation synchronously
|
|
|
|
# before starting the executor and returning control to the caller. In
|
|
|
|
# the non-blocking caller there was no externally visible boundary
|
|
|
|
# between the completion of initialisation and the start of execution,
|
|
|
|
# meaning the caller cannot indicate to another thread that
|
|
|
|
# initialisation is complete. With the split, the start call for the
|
|
|
|
# blocking executor becomes analogous to the non-blocking case,
|
|
|
|
# indicating that initialisation is complete. The caller can then
|
|
|
|
# synchronously call execute.
|
|
|
|
if self._poller is not None:
|
|
|
|
self._poller.execute()
|