Merge "[AMQP 1.0] Propagate authentication errors to caller"
This commit is contained in:
commit
3db6c77675
@ -293,7 +293,7 @@ class Sender(pyngus.SenderEventHandler):
|
|||||||
if self._link:
|
if self._link:
|
||||||
self._link.close()
|
self._link.close()
|
||||||
|
|
||||||
def reset(self):
|
def reset(self, reason="Link reset"):
|
||||||
"""Called by the controller on connection failover. Release all link
|
"""Called by the controller on connection failover. Release all link
|
||||||
resources, abort any in-flight messages, and check the retry limit on
|
resources, abort any in-flight messages, and check the retry limit on
|
||||||
all pending send requests.
|
all pending send requests.
|
||||||
@ -304,16 +304,16 @@ class Sender(pyngus.SenderEventHandler):
|
|||||||
if self._link:
|
if self._link:
|
||||||
self._link.destroy()
|
self._link.destroy()
|
||||||
self._link = None
|
self._link = None
|
||||||
self._abort_unacked("Link reset")
|
self._abort_unacked(reason)
|
||||||
self._check_retry_limit()
|
self._check_retry_limit(reason)
|
||||||
|
|
||||||
def destroy(self):
|
def destroy(self, reason="Link destroyed"):
|
||||||
"""Destroy the sender and all pending messages. Called on driver
|
"""Destroy the sender and all pending messages. Called on driver
|
||||||
shutdown.
|
shutdown.
|
||||||
"""
|
"""
|
||||||
LOG.debug("Sender %s destroyed", self._address)
|
LOG.debug("Sender %s destroyed", self._address)
|
||||||
self.reset()
|
self.reset(reason)
|
||||||
self._abort_pending("Link destroyed")
|
self._abort_pending(reason)
|
||||||
|
|
||||||
def send_message(self, send_task):
|
def send_message(self, send_task):
|
||||||
"""Send a message out the link.
|
"""Send a message out the link.
|
||||||
@ -354,21 +354,24 @@ class Sender(pyngus.SenderEventHandler):
|
|||||||
# sender_closed() will be called once the link completes closing
|
# sender_closed() will be called once the link completes closing
|
||||||
|
|
||||||
def sender_closed(self, sender_link):
|
def sender_closed(self, sender_link):
|
||||||
self._abort_unacked("Sender closed")
|
self._handle_sender_closed()
|
||||||
if self._connection:
|
|
||||||
# still attached, so attempt to restart the link
|
|
||||||
self._check_retry_limit()
|
|
||||||
self._scheduler.defer(self._reopen_link, self._delay)
|
|
||||||
|
|
||||||
def sender_failed(self, sender_link, error):
|
def sender_failed(self, sender_link, error):
|
||||||
"""Protocol error occurred."""
|
"""Protocol error occurred."""
|
||||||
LOG.warning(_LW("sender %(addr)s failed error=%(error)s"),
|
LOG.warning(_LW("sender %(addr)s failed error=%(error)s"),
|
||||||
{'addr': self._address, 'error': error})
|
{'addr': self._address, 'error': error})
|
||||||
self.sender_closed(sender_link)
|
self._handle_sender_closed(str(error))
|
||||||
|
|
||||||
# end Pyngus callbacks
|
# end Pyngus callbacks
|
||||||
|
|
||||||
def _check_retry_limit(self):
|
def _handle_sender_closed(self, reason="Sender closed"):
|
||||||
|
self._abort_unacked(reason)
|
||||||
|
if self._connection:
|
||||||
|
# still attached, so attempt to restart the link
|
||||||
|
self._check_retry_limit(reason)
|
||||||
|
self._scheduler.defer(self._reopen_link, self._delay)
|
||||||
|
|
||||||
|
def _check_retry_limit(self, reason):
|
||||||
# Called on recoverable connection or link failure. Remove any pending
|
# Called on recoverable connection or link failure. Remove any pending
|
||||||
# sends that have exhausted their retry count:
|
# sends that have exhausted their retry count:
|
||||||
expired = set()
|
expired = set()
|
||||||
@ -377,7 +380,7 @@ class Sender(pyngus.SenderEventHandler):
|
|||||||
send_task.retry -= 1
|
send_task.retry -= 1
|
||||||
if send_task.retry <= 0:
|
if send_task.retry <= 0:
|
||||||
expired.add(send_task)
|
expired.add(send_task)
|
||||||
send_task._on_error("Send retries exhausted")
|
send_task._on_error("Message send failed: %s" % reason)
|
||||||
while expired:
|
while expired:
|
||||||
self._pending_sends.remove(expired.pop())
|
self._pending_sends.remove(expired.pop())
|
||||||
|
|
||||||
@ -813,7 +816,7 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
self._closing = False
|
self._closing = False
|
||||||
# only schedule one outstanding reconnect attempt at a time
|
# only schedule one outstanding reconnect attempt at a time
|
||||||
self._reconnecting = False
|
self._reconnecting = False
|
||||||
self._delay = 1 # seconds between retries
|
self._delay = self.conn_retry_interval # seconds between retries
|
||||||
# prevent queuing up multiple requests to run _process_tasks()
|
# prevent queuing up multiple requests to run _process_tasks()
|
||||||
self._process_tasks_scheduled = False
|
self._process_tasks_scheduled = False
|
||||||
self._process_tasks_lock = threading.Lock()
|
self._process_tasks_lock = threading.Lock()
|
||||||
@ -843,7 +846,7 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
self.processor.wakeup(self._start_shutdown)
|
self.processor.wakeup(self._start_shutdown)
|
||||||
LOG.debug("Waiting for eventloop to exit")
|
LOG.debug("Waiting for eventloop to exit")
|
||||||
self.processor.join(timeout)
|
self.processor.join(timeout)
|
||||||
self._hard_reset()
|
self._hard_reset("Shutting down")
|
||||||
for sender in itervalues(self._all_senders):
|
for sender in itervalues(self._all_senders):
|
||||||
sender.destroy()
|
sender.destroy()
|
||||||
self._all_senders.clear()
|
self._all_senders.clear()
|
||||||
@ -1013,7 +1016,7 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
def socket_error(self, error):
|
def socket_error(self, error):
|
||||||
"""Called by eventloop when a socket error occurs."""
|
"""Called by eventloop when a socket error occurs."""
|
||||||
LOG.error(_LE("Socket failure: %s"), error)
|
LOG.error(_LE("Socket failure: %s"), error)
|
||||||
self._handle_connection_loss()
|
self._handle_connection_loss(str(error))
|
||||||
|
|
||||||
# Pyngus connection event callbacks (and their helpers), all invoked from
|
# Pyngus connection event callbacks (and their helpers), all invoked from
|
||||||
# the eventloop thread:
|
# the eventloop thread:
|
||||||
@ -1026,7 +1029,7 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
# pyngus bug: ignore failure callback on destroyed connections
|
# pyngus bug: ignore failure callback on destroyed connections
|
||||||
return
|
return
|
||||||
LOG.debug("AMQP Connection failure: %s", error)
|
LOG.debug("AMQP Connection failure: %s", error)
|
||||||
self._handle_connection_loss()
|
self._handle_connection_loss(str(error))
|
||||||
|
|
||||||
def connection_active(self, connection):
|
def connection_active(self, connection):
|
||||||
"""This is a Pyngus callback, invoked by Pyngus when the connection to
|
"""This is a Pyngus callback, invoked by Pyngus when the connection to
|
||||||
@ -1048,7 +1051,7 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
self._reply_link_ready,
|
self._reply_link_ready,
|
||||||
self._reply_link_down,
|
self._reply_link_down,
|
||||||
self._reply_credit)
|
self._reply_credit)
|
||||||
self._delay = 1
|
self._delay = self.conn_retry_interval # reset
|
||||||
# schedule periodic maintenance of sender links
|
# schedule periodic maintenance of sender links
|
||||||
self._link_maint_timer = self.processor.defer(self._purge_sender_links,
|
self._link_maint_timer = self.processor.defer(self._purge_sender_links,
|
||||||
self._link_maint_timeout)
|
self._link_maint_timeout)
|
||||||
@ -1061,7 +1064,7 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
"""
|
"""
|
||||||
LOG.debug("AMQP connection closed.")
|
LOG.debug("AMQP connection closed.")
|
||||||
# if the driver isn't being shutdown, failover and reconnect
|
# if the driver isn't being shutdown, failover and reconnect
|
||||||
self._handle_connection_loss()
|
self._handle_connection_loss("AMQP connection closed.")
|
||||||
|
|
||||||
def connection_remote_closed(self, connection, reason):
|
def connection_remote_closed(self, connection, reason):
|
||||||
"""This is a Pyngus callback, invoked by Pyngus when the peer has
|
"""This is a Pyngus callback, invoked by Pyngus when the peer has
|
||||||
@ -1089,13 +1092,14 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
{'hostname': self.hosts.current.hostname,
|
{'hostname': self.hosts.current.hostname,
|
||||||
'port': self.hosts.current.port,
|
'port': self.hosts.current.port,
|
||||||
'username': self.hosts.current.username})
|
'username': self.hosts.current.username})
|
||||||
# connection failure will be handled later
|
# pyngus will invoke connection_failed() eventually
|
||||||
|
|
||||||
def _handle_connection_loss(self):
|
def _handle_connection_loss(self, reason):
|
||||||
"""The connection to the messaging service has been lost. Try to
|
"""The connection to the messaging service has been lost. Try to
|
||||||
reestablish the connection/failover if not shutting down the driver.
|
reestablish the connection/failover if not shutting down the driver.
|
||||||
"""
|
"""
|
||||||
self.addresser = None
|
self.addresser = None
|
||||||
|
self._socket_connection.close()
|
||||||
if self._closing:
|
if self._closing:
|
||||||
# we're in the middle of shutting down the driver anyways,
|
# we're in the middle of shutting down the driver anyways,
|
||||||
# just consider it done:
|
# just consider it done:
|
||||||
@ -1107,31 +1111,33 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
self._reconnecting = True
|
self._reconnecting = True
|
||||||
LOG.info(_LI("delaying reconnect attempt for %d seconds"),
|
LOG.info(_LI("delaying reconnect attempt for %d seconds"),
|
||||||
self._delay)
|
self._delay)
|
||||||
self.processor.defer(self._do_reconnect, self._delay)
|
self.processor.defer(lambda: self._do_reconnect(reason),
|
||||||
self._delay = min(self._delay * 2, 60)
|
self._delay)
|
||||||
|
self._delay = min(self._delay * self.conn_retry_backoff,
|
||||||
|
self.conn_retry_interval_max)
|
||||||
if self._link_maint_timer:
|
if self._link_maint_timer:
|
||||||
self._link_maint_timer.cancel()
|
self._link_maint_timer.cancel()
|
||||||
self._link_maint_timer = None
|
self._link_maint_timer = None
|
||||||
|
|
||||||
def _do_reconnect(self):
|
def _do_reconnect(self, reason):
|
||||||
"""Invoked on connection/socket failure, failover and re-connect to the
|
"""Invoked on connection/socket failure, failover and re-connect to the
|
||||||
messaging service.
|
messaging service.
|
||||||
"""
|
"""
|
||||||
self._reconnecting = False
|
self._reconnecting = False
|
||||||
if not self._closing:
|
if not self._closing:
|
||||||
self._hard_reset()
|
self._hard_reset(reason)
|
||||||
host = self.hosts.next()
|
host = self.hosts.next()
|
||||||
LOG.info(_LI("Reconnecting to: %(hostname)s:%(port)s"),
|
LOG.info(_LI("Reconnecting to: %(hostname)s:%(port)s"),
|
||||||
{'hostname': host.hostname, 'port': host.port})
|
{'hostname': host.hostname, 'port': host.port})
|
||||||
self._socket_connection.connect(host)
|
self._socket_connection.connect(host)
|
||||||
|
|
||||||
def _hard_reset(self):
|
def _hard_reset(self, reason):
|
||||||
"""Reset the controller to its pre-connection state"""
|
"""Reset the controller to its pre-connection state"""
|
||||||
# note well: since this method destroys the connection, it cannot be
|
# note well: since this method destroys the connection, it cannot be
|
||||||
# invoked directly from a pyngus callback. Use processor.defer() to
|
# invoked directly from a pyngus callback. Use processor.defer() to
|
||||||
# run this method on the main loop instead.
|
# run this method on the main loop instead.
|
||||||
for sender in self._purged_senders:
|
for sender in self._purged_senders:
|
||||||
sender.destroy()
|
sender.destroy(reason)
|
||||||
del self._purged_senders[:]
|
del self._purged_senders[:]
|
||||||
self._active_senders.clear()
|
self._active_senders.clear()
|
||||||
unused = []
|
unused = []
|
||||||
@ -1140,10 +1146,10 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
if sender.pending_messages == 0:
|
if sender.pending_messages == 0:
|
||||||
unused.append(key)
|
unused.append(key)
|
||||||
else:
|
else:
|
||||||
sender.reset()
|
sender.reset(reason)
|
||||||
self._active_senders.add(key)
|
self._active_senders.add(key)
|
||||||
for key in unused:
|
for key in unused:
|
||||||
self._all_senders[key].destroy()
|
self._all_senders[key].destroy(reason)
|
||||||
del self._all_senders[key]
|
del self._all_senders[key]
|
||||||
for servers in itervalues(self._servers):
|
for servers in itervalues(self._servers):
|
||||||
for server in itervalues(servers):
|
for server in itervalues(servers):
|
||||||
@ -1170,7 +1176,7 @@ class Controller(pyngus.ConnectionEventHandler):
|
|||||||
if not self._closing:
|
if not self._closing:
|
||||||
# destroy links that have already been closed
|
# destroy links that have already been closed
|
||||||
for sender in self._purged_senders:
|
for sender in self._purged_senders:
|
||||||
sender.destroy()
|
sender.destroy("Idle link purged")
|
||||||
del self._purged_senders[:]
|
del self._purged_senders[:]
|
||||||
|
|
||||||
# determine next set to purge
|
# determine next set to purge
|
||||||
|
@ -69,31 +69,27 @@ class _SocketConnection(object):
|
|||||||
|
|
||||||
def read_socket(self):
|
def read_socket(self):
|
||||||
"""Called to read from the socket."""
|
"""Called to read from the socket."""
|
||||||
while True:
|
if self.socket:
|
||||||
try:
|
try:
|
||||||
rc = pyngus.read_socket_input(self.pyngus_conn, self.socket)
|
pyngus.read_socket_input(self.pyngus_conn, self.socket)
|
||||||
self.pyngus_conn.process(now())
|
self.pyngus_conn.process(now())
|
||||||
return rc
|
|
||||||
except (socket.timeout, socket.error) as e:
|
except (socket.timeout, socket.error) as e:
|
||||||
# pyngus handles EAGAIN/EWOULDBLOCK and EINTER
|
# pyngus handles EAGAIN/EWOULDBLOCK and EINTER
|
||||||
self.pyngus_conn.close_input()
|
self.pyngus_conn.close_input()
|
||||||
self.pyngus_conn.close_output()
|
self.pyngus_conn.close_output()
|
||||||
self._handler.socket_error(str(e))
|
self._handler.socket_error(str(e))
|
||||||
return pyngus.Connection.EOS
|
|
||||||
|
|
||||||
def write_socket(self):
|
def write_socket(self):
|
||||||
"""Called to write to the socket."""
|
"""Called to write to the socket."""
|
||||||
while True:
|
if self.socket:
|
||||||
try:
|
try:
|
||||||
rc = pyngus.write_socket_output(self.pyngus_conn, self.socket)
|
pyngus.write_socket_output(self.pyngus_conn, self.socket)
|
||||||
self.pyngus_conn.process(now())
|
self.pyngus_conn.process(now())
|
||||||
return rc
|
|
||||||
except (socket.timeout, socket.error) as e:
|
except (socket.timeout, socket.error) as e:
|
||||||
# pyngus handles EAGAIN/EWOULDBLOCK and EINTER
|
# pyngus handles EAGAIN/EWOULDBLOCK and EINTER
|
||||||
self.pyngus_conn.close_output()
|
self.pyngus_conn.close_output()
|
||||||
self.pyngus_conn.close_input()
|
self.pyngus_conn.close_input()
|
||||||
self._handler.socket_error(str(e))
|
self._handler.socket_error(str(e))
|
||||||
return pyngus.Connection.EOS
|
|
||||||
|
|
||||||
def connect(self, host):
|
def connect(self, host):
|
||||||
"""Connect to host and start the AMQP protocol."""
|
"""Connect to host and start the AMQP protocol."""
|
||||||
@ -358,7 +354,7 @@ class Thread(threading.Thread):
|
|||||||
deadline = self._scheduler._next_deadline
|
deadline = self._scheduler._next_deadline
|
||||||
|
|
||||||
pyngus_conn = self._connection and self._connection.pyngus_conn
|
pyngus_conn = self._connection and self._connection.pyngus_conn
|
||||||
if pyngus_conn:
|
if pyngus_conn and self._connection.socket:
|
||||||
if pyngus_conn.needs_input:
|
if pyngus_conn.needs_input:
|
||||||
readfds.append(self._connection)
|
readfds.append(self._connection)
|
||||||
if pyngus_conn.has_output:
|
if pyngus_conn.has_output:
|
||||||
@ -388,9 +384,8 @@ class Thread(threading.Thread):
|
|||||||
# Testing shows that polling improves latency over checking the
|
# Testing shows that polling improves latency over checking the
|
||||||
# lists returned by select()
|
# lists returned by select()
|
||||||
self._requests.process_requests()
|
self._requests.process_requests()
|
||||||
if pyngus_conn:
|
|
||||||
self._connection.read_socket()
|
self._connection.read_socket()
|
||||||
if pyngus_conn.deadline:
|
if pyngus_conn and pyngus_conn.deadline:
|
||||||
_now = now()
|
_now = now()
|
||||||
if pyngus_conn.deadline <= _now:
|
if pyngus_conn.deadline <= _now:
|
||||||
pyngus_conn.process(_now)
|
pyngus_conn.process(_now)
|
||||||
|
@ -674,12 +674,12 @@ class TestAuthentication(test_utils.BaseTestCase):
|
|||||||
target = oslo_messaging.Target(topic="test-topic")
|
target = oslo_messaging.Target(topic="test-topic")
|
||||||
_ListenerThread(
|
_ListenerThread(
|
||||||
driver.listen(target, None, None)._poll_style_listener, 1)
|
driver.listen(target, None, None)._poll_style_listener, 1)
|
||||||
self.assertRaises(oslo_messaging.MessagingTimeout,
|
self.assertRaises(oslo_messaging.MessageDeliveryFailure,
|
||||||
driver.send,
|
driver.send,
|
||||||
target, {"context": True},
|
target, {"context": True},
|
||||||
{"method": "echo"},
|
{"method": "echo"},
|
||||||
wait_for_reply=True,
|
wait_for_reply=True,
|
||||||
timeout=2.0)
|
retry=2)
|
||||||
driver.cleanup()
|
driver.cleanup()
|
||||||
|
|
||||||
|
|
||||||
@ -771,7 +771,6 @@ mech_list: ${mechs}
|
|||||||
"""Verify that a bad password given in TransportHost is
|
"""Verify that a bad password given in TransportHost is
|
||||||
rejected by the broker.
|
rejected by the broker.
|
||||||
"""
|
"""
|
||||||
|
|
||||||
addr = "amqp://joe:badpass@%s:%d" % (self._broker.host,
|
addr = "amqp://joe:badpass@%s:%d" % (self._broker.host,
|
||||||
self._broker.port)
|
self._broker.port)
|
||||||
url = oslo_messaging.TransportURL.parse(self.conf, addr)
|
url = oslo_messaging.TransportURL.parse(self.conf, addr)
|
||||||
@ -779,12 +778,15 @@ mech_list: ${mechs}
|
|||||||
target = oslo_messaging.Target(topic="test-topic")
|
target = oslo_messaging.Target(topic="test-topic")
|
||||||
_ListenerThread(
|
_ListenerThread(
|
||||||
driver.listen(target, None, None)._poll_style_listener, 1)
|
driver.listen(target, None, None)._poll_style_listener, 1)
|
||||||
self.assertRaises(oslo_messaging.MessagingTimeout,
|
try:
|
||||||
driver.send,
|
driver.send(target, {"context": True}, {"method": "echo"},
|
||||||
target, {"context": True},
|
wait_for_reply=True, retry=2)
|
||||||
{"method": "echo"},
|
except oslo_messaging.MessageDeliveryFailure as e:
|
||||||
wait_for_reply=True,
|
# verify the exception indicates the failure was an authentication
|
||||||
timeout=2.0)
|
# error
|
||||||
|
self.assertTrue('amqp:unauthorized-access' in str(e))
|
||||||
|
else:
|
||||||
|
self.assertIsNone("Expected authentication failure")
|
||||||
driver.cleanup()
|
driver.cleanup()
|
||||||
|
|
||||||
def test_authentication_bad_mechs(self):
|
def test_authentication_bad_mechs(self):
|
||||||
@ -800,12 +802,12 @@ mech_list: ${mechs}
|
|||||||
target = oslo_messaging.Target(topic="test-topic")
|
target = oslo_messaging.Target(topic="test-topic")
|
||||||
_ListenerThread(
|
_ListenerThread(
|
||||||
driver.listen(target, None, None)._poll_style_listener, 1)
|
driver.listen(target, None, None)._poll_style_listener, 1)
|
||||||
self.assertRaises(oslo_messaging.MessagingTimeout,
|
self.assertRaises(oslo_messaging.MessageDeliveryFailure,
|
||||||
driver.send,
|
driver.send,
|
||||||
target, {"context": True},
|
target, {"context": True},
|
||||||
{"method": "echo"},
|
{"method": "echo"},
|
||||||
wait_for_reply=True,
|
wait_for_reply=True,
|
||||||
timeout=2.0)
|
retry=0)
|
||||||
driver.cleanup()
|
driver.cleanup()
|
||||||
|
|
||||||
def test_authentication_default_username(self):
|
def test_authentication_default_username(self):
|
||||||
|
Loading…
Reference in New Issue
Block a user