|
| 1 | +from cassandra.connection import Connection |
| 2 | + |
| 3 | +import asyncio |
| 4 | +import logging |
| 5 | +import os |
| 6 | +import socket |
| 7 | +import ssl |
| 8 | +from threading import Lock, Thread |
| 9 | + |
| 10 | + |
| 11 | +log = logging.getLogger(__name__) |
| 12 | + |
| 13 | + |
| 14 | +# This module uses ``yield from`` and ``@asyncio.coroutine`` over ``await`` and |
| 15 | +# ``async def`` for pre-Python-3.5 compatibility, so keep in mind that the |
| 16 | +# managed coroutines are generator-based, not native coroutines. See PEP 492: |
| 17 | +# https://www.python.org/dev/peps/pep-0492/#coroutine-objects |
| 18 | + |
| 19 | + |
| 20 | +try: |
| 21 | + asyncio.run_coroutine_threadsafe |
| 22 | +except AttributeError: |
| 23 | + raise ImportError( |
| 24 | + 'Cannot use asyncioreactor without access to ' |
| 25 | + 'asyncio.run_coroutine_threadsafe (added in 3.4.6 and 3.5.1)' |
| 26 | + ) |
| 27 | + |
| 28 | + |
| 29 | +class AsyncioTimer(object): |
| 30 | + """ |
| 31 | + An ``asyncioreactor``-specific Timer. Similar to :class:`.connection.Timer, |
| 32 | + but with a slightly different API due to limitations in the underlying |
| 33 | + ``call_later`` interface. Not meant to be used with a |
| 34 | + :class:`.connection.TimerManager`. |
| 35 | + """ |
| 36 | + |
| 37 | + @property |
| 38 | + def end(self): |
| 39 | + raise NotImplementedError('{} is not compatible with TimerManager and ' |
| 40 | + 'does not implement .end()') |
| 41 | + |
| 42 | + def __init__(self, timeout, callback, loop): |
| 43 | + delayed = self._call_delayed_coro(timeout=timeout, |
| 44 | + callback=callback, |
| 45 | + loop=loop) |
| 46 | + self._handle = asyncio.run_coroutine_threadsafe(delayed, loop=loop) |
| 47 | + |
| 48 | + @staticmethod |
| 49 | + @asyncio.coroutine |
| 50 | + def _call_delayed_coro(timeout, callback, loop): |
| 51 | + yield from asyncio.sleep(timeout, loop=loop) |
| 52 | + return callback() |
| 53 | + |
| 54 | + def __lt__(self, other): |
| 55 | + try: |
| 56 | + return self._handle < other._handle |
| 57 | + except AttributeError: |
| 58 | + raise NotImplemented |
| 59 | + |
| 60 | + def cancel(self): |
| 61 | + self._handle.cancel() |
| 62 | + |
| 63 | + def finish(self): |
| 64 | + # connection.Timer method not implemented here because we can't inspect |
| 65 | + # the Handle returned from call_later |
| 66 | + raise NotImplementedError('{} is not compatible with TimerManager and ' |
| 67 | + 'does not implement .finish()') |
| 68 | + |
| 69 | + |
| 70 | +class AsyncioConnection(Connection): |
| 71 | + """ |
| 72 | + An experimental implementation of :class:`.Connection` that uses the |
| 73 | + ``asyncio`` module in the Python standard library for its event loop. |
| 74 | +
|
| 75 | + Note that it requires ``asyncio`` features that were only introduced in the |
| 76 | + 3.4 line in 3.4.6, and in the 3.5 line in 3.5.1. |
| 77 | + """ |
| 78 | + |
| 79 | + _loop = None |
| 80 | + _pid = os.getpid() |
| 81 | + |
| 82 | + _lock = Lock() |
| 83 | + _loop_thread = None |
| 84 | + |
| 85 | + _write_queue = None |
| 86 | + |
| 87 | + def __init__(self, *args, **kwargs): |
| 88 | + Connection.__init__(self, *args, **kwargs) |
| 89 | + |
| 90 | + self._connect_socket() |
| 91 | + self._socket.setblocking(0) |
| 92 | + |
| 93 | + self._write_queue = asyncio.Queue(loop=self._loop) |
| 94 | + |
| 95 | + # see initialize_reactor -- loop is running in a separate thread, so we |
| 96 | + # have to use a threadsafe call |
| 97 | + self._read_watcher = asyncio.run_coroutine_threadsafe( |
| 98 | + self.handle_read(), loop=self._loop |
| 99 | + ) |
| 100 | + self._write_watcher = asyncio.run_coroutine_threadsafe( |
| 101 | + self.handle_write(), loop=self._loop |
| 102 | + ) |
| 103 | + self._send_options_message() |
| 104 | + |
| 105 | + @classmethod |
| 106 | + def initialize_reactor(cls): |
| 107 | + with cls._lock: |
| 108 | + if cls._pid != os.getpid(): |
| 109 | + cls._loop = None |
| 110 | + if cls._loop is None: |
| 111 | + cls._loop = asyncio.get_event_loop() |
| 112 | + |
| 113 | + if not cls._loop_thread: |
| 114 | + # daemonize so the loop will be shut down on interpreter |
| 115 | + # shutdown |
| 116 | + cls._loop_thread = Thread(target=cls._loop.run_forever, |
| 117 | + daemon=True) |
| 118 | + cls._loop_thread.start() |
| 119 | + |
| 120 | + @classmethod |
| 121 | + def create_timer(cls, timeout, callback): |
| 122 | + return AsyncioTimer(timeout, callback, loop=cls._loop) |
| 123 | + |
| 124 | + def close(self): |
| 125 | + log.debug("Closing connection (%s) to %s" % (id(self), self.host)) |
| 126 | + with self.lock: |
| 127 | + if self.is_closed: |
| 128 | + return |
| 129 | + self.is_closed = True |
| 130 | + |
| 131 | + self._write_watcher.cancel() |
| 132 | + self._read_watcher.cancel() |
| 133 | + self.connected_event.set() |
| 134 | + |
| 135 | + def push(self, data): |
| 136 | + buff_size = self.out_buffer_size |
| 137 | + if len(data) > buff_size: |
| 138 | + for i in range(0, len(data), buff_size): |
| 139 | + self._push_chunk(data[i:i + buff_size]) |
| 140 | + else: |
| 141 | + self._push_chunk(data) |
| 142 | + |
| 143 | + def _push_chunk(self, chunk): |
| 144 | + asyncio.run_coroutine_threadsafe( |
| 145 | + self._write_queue.put(chunk), |
| 146 | + loop=self._loop |
| 147 | + ) |
| 148 | + |
| 149 | + @asyncio.coroutine |
| 150 | + def handle_write(self): |
| 151 | + while True: |
| 152 | + try: |
| 153 | + next_msg = yield from self._write_queue.get() |
| 154 | + if next_msg: |
| 155 | + yield from self._loop.sock_sendall(self._socket, next_msg) |
| 156 | + except socket.error as err: |
| 157 | + log.debug("Exception in send for %s: %s", self, err) |
| 158 | + self.defunct(err) |
| 159 | + return |
| 160 | + |
| 161 | + @asyncio.coroutine |
| 162 | + def handle_read(self): |
| 163 | + while True: |
| 164 | + try: |
| 165 | + buf = yield from self._loop.sock_recv(self._socket, self.in_buffer_size) |
| 166 | + self._iobuf.write(buf) |
| 167 | + # sock_recv expects EWOULDBLOCK if socket provides no data, but |
| 168 | + # nonblocking ssl sockets raise these instead, so we handle them |
| 169 | + # ourselves by yielding to the event loop, where the socket will |
| 170 | + # get the reading/writing it "wants" before retrying |
| 171 | + except (ssl.SSLWantWriteError, ssl.SSLWantReadError): |
| 172 | + yield |
| 173 | + continue |
| 174 | + except socket.error as err: |
| 175 | + log.debug("Exception during socket recv for %s: %s", |
| 176 | + self, err) |
| 177 | + self.defunct(err) |
| 178 | + return # leave the read loop |
| 179 | + |
| 180 | + if buf and self._iobuf.tell(): |
| 181 | + self.process_io_buffer() |
| 182 | + else: |
| 183 | + log.debug("Connection %s closed by server", self) |
| 184 | + self.close() |
| 185 | + return |
0 commit comments