asynpool.py 48 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304
  1. # -*- coding: utf-8 -*-
  2. """Version of multiprocessing.Pool using Async I/O.
  3. .. note::
  4. This module will be moved soon, so don't use it directly.
  5. This is a non-blocking version of :class:`multiprocessing.Pool`.
  6. This code deals with three major challenges:
  7. #. Starting up child processes and keeping them running.
  8. #. Sending jobs to the processes and receiving results back.
  9. #. Safely shutting down this system.
  10. """
  11. from __future__ import absolute_import, unicode_literals
  12. import errno
  13. import gc
  14. import os
  15. import select
  16. import socket
  17. import struct
  18. import sys
  19. import time
  20. from collections import deque, namedtuple
  21. from io import BytesIO
  22. from numbers import Integral
  23. from pickle import HIGHEST_PROTOCOL
  24. from time import sleep
  25. from weakref import WeakValueDictionary, ref
  26. from billiard import pool as _pool
  27. from billiard.compat import buf_t, isblocking, setblocking
  28. from billiard.pool import ACK, NACK, RUN, TERMINATE, WorkersJoined
  29. from billiard.queues import _SimpleQueue
  30. from kombu.async import ERR, WRITE
  31. from kombu.serialization import pickle as _pickle
  32. from kombu.utils.eventio import SELECT_BAD_FD
  33. from kombu.utils.functional import fxrange
  34. from vine import promise
  35. from celery.five import Counter, items, values
  36. from celery.utils.functional import noop
  37. from celery.utils.log import get_logger
  38. from celery.worker import state as worker_state
  39. # pylint: disable=redefined-outer-name
  40. # We cache globals and attribute lookups, so disable this warning.
  41. try:
  42. from _billiard import read as __read__
  43. from struct import unpack_from as _unpack_from
  44. memoryview = memoryview
  45. readcanbuf = True
  46. if sys.version_info[0] == 2 and sys.version_info < (2, 7, 6):
  47. def unpack_from(fmt, view, _unpack_from=_unpack_from): # noqa
  48. return _unpack_from(fmt, view.tobytes()) # <- memoryview
  49. else:
  50. # unpack_from supports memoryview in 2.7.6 and 3.3+
  51. unpack_from = _unpack_from # noqa
  52. except (ImportError, NameError): # pragma: no cover
  53. def __read__(fd, buf, size, read=os.read): # noqa
  54. chunk = read(fd, size)
  55. n = len(chunk)
  56. if n != 0:
  57. buf.write(chunk)
  58. return n
  59. readcanbuf = False # noqa
  60. def unpack_from(fmt, iobuf, unpack=struct.unpack): # noqa
  61. return unpack(fmt, iobuf.getvalue()) # <-- BytesIO
  62. __all__ = ('AsynPool',)
  63. logger = get_logger(__name__)
  64. error, debug = logger.error, logger.debug
  65. UNAVAIL = frozenset({errno.EAGAIN, errno.EINTR})
  66. #: Constant sent by child process when started (ready to accept work)
  67. WORKER_UP = 15
  68. #: A process must've started before this timeout (in secs.) expires.
  69. PROC_ALIVE_TIMEOUT = 4.0
  70. SCHED_STRATEGY_FCFS = 1
  71. SCHED_STRATEGY_FAIR = 4
  72. SCHED_STRATEGIES = {
  73. None: SCHED_STRATEGY_FAIR,
  74. 'fast': SCHED_STRATEGY_FCFS,
  75. 'fcfs': SCHED_STRATEGY_FCFS,
  76. 'fair': SCHED_STRATEGY_FAIR,
  77. }
  78. SCHED_STRATEGY_TO_NAME = {v: k for k, v in SCHED_STRATEGIES.items()}
  79. Ack = namedtuple('Ack', ('id', 'fd', 'payload'))
  80. def gen_not_started(gen):
  81. """Return true if generator is not started."""
  82. # gi_frame is None when generator stopped.
  83. return gen.gi_frame and gen.gi_frame.f_lasti == -1
  84. def _get_job_writer(job):
  85. try:
  86. writer = job._writer
  87. except AttributeError:
  88. pass
  89. else:
  90. return writer() # is a weakref
  91. if hasattr(select, 'poll'):
  92. def _select_imp(readers=None, writers=None, err=None, timeout=0,
  93. poll=select.poll, POLLIN=select.POLLIN,
  94. POLLOUT=select.POLLOUT, POLLERR=select.POLLERR):
  95. poller = poll()
  96. register = poller.register
  97. if readers:
  98. [register(fd, POLLIN) for fd in readers]
  99. if writers:
  100. [register(fd, POLLOUT) for fd in writers]
  101. if err:
  102. [register(fd, POLLERR) for fd in err]
  103. R, W = set(), set()
  104. timeout = 0 if timeout and timeout < 0 else round(timeout * 1e3)
  105. events = poller.poll(timeout)
  106. for fd, event in events:
  107. if not isinstance(fd, Integral):
  108. fd = fd.fileno()
  109. if event & POLLIN:
  110. R.add(fd)
  111. if event & POLLOUT:
  112. W.add(fd)
  113. if event & POLLERR:
  114. R.add(fd)
  115. return R, W, 0
  116. else:
  117. def _select_imp(readers=None, writers=None, err=None, timeout=0):
  118. r, w, e = select.select(readers, writers, err, timeout)
  119. if e:
  120. r = list(set(r) | set(e))
  121. return r, w, 0
  122. def _select(readers=None, writers=None, err=None, timeout=0,
  123. poll=_select_imp):
  124. """Simple wrapper to :class:`~select.select`, using :`~select.poll`.
  125. Arguments:
  126. readers (Set[Fd]): Set of reader fds to test if readable.
  127. writers (Set[Fd]): Set of writer fds to test if writable.
  128. err (Set[Fd]): Set of fds to test for error condition.
  129. All fd sets passed must be mutable as this function
  130. will remove non-working fds from them, this also means
  131. the caller must make sure there are still fds in the sets
  132. before calling us again.
  133. Returns:
  134. Tuple[Set, Set, Set]: of ``(readable, writable, again)``, where
  135. ``readable`` is a set of fds that have data available for read,
  136. ``writable`` is a set of fds that's ready to be written to
  137. and ``again`` is a flag that if set means the caller must
  138. throw away the result and call us again.
  139. """
  140. readers = set() if readers is None else readers
  141. writers = set() if writers is None else writers
  142. err = set() if err is None else err
  143. try:
  144. return poll(readers, writers, err, timeout)
  145. except (select.error, socket.error) as exc:
  146. # Workaround for celery/celery#4513
  147. try:
  148. _errno = exc.errno
  149. except AttributeError:
  150. _errno = exc.args[0]
  151. if _errno == errno.EINTR:
  152. return set(), set(), 1
  153. elif _errno in SELECT_BAD_FD:
  154. for fd in readers | writers | err:
  155. try:
  156. select.select([fd], [], [], 0)
  157. except (select.error, socket.error) as exc:
  158. try:
  159. _errno = exc.errno
  160. except AttributeError:
  161. _errno = exc.args[0]
  162. if _errno not in SELECT_BAD_FD:
  163. raise
  164. readers.discard(fd)
  165. writers.discard(fd)
  166. err.discard(fd)
  167. return set(), set(), 1
  168. else:
  169. raise
  170. class Worker(_pool.Worker):
  171. """Pool worker process."""
  172. def on_loop_start(self, pid):
  173. # our version sends a WORKER_UP message when the process is ready
  174. # to accept work, this will tell the parent that the inqueue fd
  175. # is writable.
  176. self.outq.put((WORKER_UP, (pid,)))
  177. class ResultHandler(_pool.ResultHandler):
  178. """Handles messages from the pool processes."""
  179. def __init__(self, *args, **kwargs):
  180. self.fileno_to_outq = kwargs.pop('fileno_to_outq')
  181. self.on_process_alive = kwargs.pop('on_process_alive')
  182. super(ResultHandler, self).__init__(*args, **kwargs)
  183. # add our custom message handler
  184. self.state_handlers[WORKER_UP] = self.on_process_alive
  185. def _recv_message(self, add_reader, fd, callback,
  186. __read__=__read__, readcanbuf=readcanbuf,
  187. BytesIO=BytesIO, unpack_from=unpack_from,
  188. load=_pickle.load):
  189. Hr = Br = 0
  190. if readcanbuf:
  191. buf = bytearray(4)
  192. bufv = memoryview(buf)
  193. else:
  194. buf = bufv = BytesIO()
  195. # header
  196. while Hr < 4:
  197. try:
  198. n = __read__(
  199. fd, bufv[Hr:] if readcanbuf else bufv, 4 - Hr,
  200. )
  201. except OSError as exc:
  202. if exc.errno not in UNAVAIL:
  203. raise
  204. yield
  205. else:
  206. if n == 0:
  207. raise (OSError('End of file during message') if Hr
  208. else EOFError())
  209. Hr += n
  210. body_size, = unpack_from(b'>i', bufv)
  211. if readcanbuf:
  212. buf = bytearray(body_size)
  213. bufv = memoryview(buf)
  214. else:
  215. buf = bufv = BytesIO()
  216. while Br < body_size:
  217. try:
  218. n = __read__(
  219. fd, bufv[Br:] if readcanbuf else bufv, body_size - Br,
  220. )
  221. except OSError as exc:
  222. if exc.errno not in UNAVAIL:
  223. raise
  224. yield
  225. else:
  226. if n == 0:
  227. raise (OSError('End of file during message') if Br
  228. else EOFError())
  229. Br += n
  230. add_reader(fd, self.handle_event, fd)
  231. if readcanbuf:
  232. message = load(BytesIO(bufv))
  233. else:
  234. bufv.seek(0)
  235. message = load(bufv)
  236. if message:
  237. callback(message)
  238. def _make_process_result(self, hub):
  239. """Coroutine reading messages from the pool processes."""
  240. fileno_to_outq = self.fileno_to_outq
  241. on_state_change = self.on_state_change
  242. add_reader = hub.add_reader
  243. remove_reader = hub.remove_reader
  244. recv_message = self._recv_message
  245. def on_result_readable(fileno):
  246. try:
  247. fileno_to_outq[fileno]
  248. except KeyError: # process gone
  249. return remove_reader(fileno)
  250. it = recv_message(add_reader, fileno, on_state_change)
  251. try:
  252. next(it)
  253. except StopIteration:
  254. pass
  255. except (IOError, OSError, EOFError):
  256. remove_reader(fileno)
  257. else:
  258. add_reader(fileno, it)
  259. return on_result_readable
  260. def register_with_event_loop(self, hub):
  261. self.handle_event = self._make_process_result(hub)
  262. def handle_event(self, *args):
  263. # pylint: disable=method-hidden
  264. # register_with_event_loop overrides this
  265. raise RuntimeError('Not registered with event loop')
  266. def on_stop_not_started(self):
  267. # This is always used, since we do not start any threads.
  268. cache = self.cache
  269. check_timeouts = self.check_timeouts
  270. fileno_to_outq = self.fileno_to_outq
  271. on_state_change = self.on_state_change
  272. join_exited_workers = self.join_exited_workers
  273. # flush the processes outqueues until they've all terminated.
  274. outqueues = set(fileno_to_outq)
  275. while cache and outqueues and self._state != TERMINATE:
  276. if check_timeouts is not None:
  277. # make sure tasks with a time limit will time out.
  278. check_timeouts()
  279. # cannot iterate and remove at the same time
  280. pending_remove_fd = set()
  281. for fd in outqueues:
  282. self._flush_outqueue(
  283. fd, pending_remove_fd.add, fileno_to_outq,
  284. on_state_change,
  285. )
  286. try:
  287. join_exited_workers(shutdown=True)
  288. except WorkersJoined:
  289. return debug('result handler: all workers terminated')
  290. outqueues.difference_update(pending_remove_fd)
  291. def _flush_outqueue(self, fd, remove, process_index, on_state_change):
  292. try:
  293. proc = process_index[fd]
  294. except KeyError:
  295. # process already found terminated
  296. # this means its outqueue has already been processed
  297. # by the worker lost handler.
  298. return remove(fd)
  299. reader = proc.outq._reader
  300. try:
  301. setblocking(reader, 1)
  302. except (OSError, IOError):
  303. return remove(fd)
  304. try:
  305. if reader.poll(0):
  306. task = reader.recv()
  307. else:
  308. task = None
  309. sleep(0.5)
  310. except (IOError, EOFError):
  311. return remove(fd)
  312. else:
  313. if task:
  314. on_state_change(task)
  315. finally:
  316. try:
  317. setblocking(reader, 0)
  318. except (OSError, IOError):
  319. return remove(fd)
  320. class AsynPool(_pool.Pool):
  321. """AsyncIO Pool (no threads)."""
  322. ResultHandler = ResultHandler
  323. Worker = Worker
  324. def WorkerProcess(self, worker):
  325. worker = super(AsynPool, self).WorkerProcess(worker)
  326. worker.dead = False
  327. return worker
  328. def __init__(self, processes=None, synack=False,
  329. sched_strategy=None, *args, **kwargs):
  330. self.sched_strategy = SCHED_STRATEGIES.get(sched_strategy,
  331. sched_strategy)
  332. processes = self.cpu_count() if processes is None else processes
  333. self.synack = synack
  334. # create queue-pairs for all our processes in advance.
  335. self._queues = {
  336. self.create_process_queues(): None for _ in range(processes)
  337. }
  338. # inqueue fileno -> process mapping
  339. self._fileno_to_inq = {}
  340. # outqueue fileno -> process mapping
  341. self._fileno_to_outq = {}
  342. # synqueue fileno -> process mapping
  343. self._fileno_to_synq = {}
  344. # We keep track of processes that haven't yet
  345. # sent a WORKER_UP message. If a process fails to send
  346. # this message within proc_up_timeout we terminate it
  347. # and hope the next process will recover.
  348. self._proc_alive_timeout = PROC_ALIVE_TIMEOUT
  349. self._waiting_to_start = set()
  350. # denormalized set of all inqueues.
  351. self._all_inqueues = set()
  352. # Set of fds being written to (busy)
  353. self._active_writes = set()
  354. # Set of active co-routines currently writing jobs.
  355. self._active_writers = set()
  356. # Set of fds that are busy (executing task)
  357. self._busy_workers = set()
  358. self._mark_worker_as_available = self._busy_workers.discard
  359. # Holds jobs waiting to be written to child processes.
  360. self.outbound_buffer = deque()
  361. self.write_stats = Counter()
  362. super(AsynPool, self).__init__(processes, *args, **kwargs)
  363. for proc in self._pool:
  364. # create initial mappings, these will be updated
  365. # as processes are recycled, or found lost elsewhere.
  366. self._fileno_to_outq[proc.outqR_fd] = proc
  367. self._fileno_to_synq[proc.synqW_fd] = proc
  368. self.on_soft_timeout = getattr(
  369. self._timeout_handler, 'on_soft_timeout', noop,
  370. )
  371. self.on_hard_timeout = getattr(
  372. self._timeout_handler, 'on_hard_timeout', noop,
  373. )
  374. def _create_worker_process(self, i):
  375. gc.collect() # Issue #2927
  376. return super(AsynPool, self)._create_worker_process(i)
  377. def _event_process_exit(self, hub, proc):
  378. # This method is called whenever the process sentinel is readable.
  379. self._untrack_child_process(proc, hub)
  380. self.maintain_pool()
  381. def _track_child_process(self, proc, hub):
  382. try:
  383. fd = proc._sentinel_poll
  384. except AttributeError:
  385. # we need to duplicate the fd here to carefully
  386. # control when the fd is removed from the process table,
  387. # as once the original fd is closed we cannot unregister
  388. # the fd from epoll(7) anymore, causing a 100% CPU poll loop.
  389. fd = proc._sentinel_poll = os.dup(proc._popen.sentinel)
  390. hub.add_reader(fd, self._event_process_exit, hub, proc)
  391. def _untrack_child_process(self, proc, hub):
  392. if proc._sentinel_poll is not None:
  393. fd, proc._sentinel_poll = proc._sentinel_poll, None
  394. hub.remove(fd)
  395. os.close(fd)
  396. def register_with_event_loop(self, hub):
  397. """Register the async pool with the current event loop."""
  398. self._result_handler.register_with_event_loop(hub)
  399. self.handle_result_event = self._result_handler.handle_event
  400. self._create_timelimit_handlers(hub)
  401. self._create_process_handlers(hub)
  402. self._create_write_handlers(hub)
  403. # Add handler for when a process exits (calls maintain_pool)
  404. [self._track_child_process(w, hub) for w in self._pool]
  405. # Handle_result_event is called whenever one of the
  406. # result queues are readable.
  407. [hub.add_reader(fd, self.handle_result_event, fd)
  408. for fd in self._fileno_to_outq]
  409. # Timers include calling maintain_pool at a regular interval
  410. # to be certain processes are restarted.
  411. for handler, interval in items(self.timers):
  412. hub.call_repeatedly(interval, handler)
  413. hub.on_tick.add(self.on_poll_start)
  414. def _create_timelimit_handlers(self, hub):
  415. """Create handlers used to implement time limits."""
  416. call_later = hub.call_later
  417. trefs = self._tref_for_id = WeakValueDictionary()
  418. def on_timeout_set(R, soft, hard):
  419. if soft:
  420. trefs[R._job] = call_later(
  421. soft, self._on_soft_timeout, R._job, soft, hard, hub,
  422. )
  423. elif hard:
  424. trefs[R._job] = call_later(
  425. hard, self._on_hard_timeout, R._job,
  426. )
  427. self.on_timeout_set = on_timeout_set
  428. def _discard_tref(job):
  429. try:
  430. tref = trefs.pop(job)
  431. tref.cancel()
  432. del tref
  433. except (KeyError, AttributeError):
  434. pass # out of scope
  435. self._discard_tref = _discard_tref
  436. def on_timeout_cancel(R):
  437. _discard_tref(R._job)
  438. self.on_timeout_cancel = on_timeout_cancel
  439. def _on_soft_timeout(self, job, soft, hard, hub):
  440. # only used by async pool.
  441. if hard:
  442. self._tref_for_id[job] = hub.call_later(
  443. hard - soft, self._on_hard_timeout, job,
  444. )
  445. try:
  446. result = self._cache[job]
  447. except KeyError:
  448. pass # job ready
  449. else:
  450. self.on_soft_timeout(result)
  451. finally:
  452. if not hard:
  453. # remove tref
  454. self._discard_tref(job)
  455. def _on_hard_timeout(self, job):
  456. # only used by async pool.
  457. try:
  458. result = self._cache[job]
  459. except KeyError:
  460. pass # job ready
  461. else:
  462. self.on_hard_timeout(result)
  463. finally:
  464. # remove tref
  465. self._discard_tref(job)
  466. def on_job_ready(self, job, i, obj, inqW_fd):
  467. self._mark_worker_as_available(inqW_fd)
  468. def _create_process_handlers(self, hub):
  469. """Create handlers called on process up/down, etc."""
  470. add_reader, remove_reader, remove_writer = (
  471. hub.add_reader, hub.remove_reader, hub.remove_writer,
  472. )
  473. cache = self._cache
  474. all_inqueues = self._all_inqueues
  475. fileno_to_inq = self._fileno_to_inq
  476. fileno_to_outq = self._fileno_to_outq
  477. fileno_to_synq = self._fileno_to_synq
  478. busy_workers = self._busy_workers
  479. handle_result_event = self.handle_result_event
  480. process_flush_queues = self.process_flush_queues
  481. waiting_to_start = self._waiting_to_start
  482. def verify_process_alive(proc):
  483. proc = proc() # is a weakref
  484. if (proc is not None and proc._is_alive() and
  485. proc in waiting_to_start):
  486. assert proc.outqR_fd in fileno_to_outq
  487. assert fileno_to_outq[proc.outqR_fd] is proc
  488. assert proc.outqR_fd in hub.readers
  489. error('Timed out waiting for UP message from %r', proc)
  490. os.kill(proc.pid, 9)
  491. def on_process_up(proc):
  492. """Called when a process has started."""
  493. # If we got the same fd as a previous process then we'll also
  494. # receive jobs in the old buffer, so we need to reset the
  495. # job._write_to and job._scheduled_for attributes used to recover
  496. # message boundaries when processes exit.
  497. infd = proc.inqW_fd
  498. for job in values(cache):
  499. if job._write_to and job._write_to.inqW_fd == infd:
  500. job._write_to = proc
  501. if job._scheduled_for and job._scheduled_for.inqW_fd == infd:
  502. job._scheduled_for = proc
  503. fileno_to_outq[proc.outqR_fd] = proc
  504. # maintain_pool is called whenever a process exits.
  505. self._track_child_process(proc, hub)
  506. assert not isblocking(proc.outq._reader)
  507. # handle_result_event is called when the processes outqueue is
  508. # readable.
  509. add_reader(proc.outqR_fd, handle_result_event, proc.outqR_fd)
  510. waiting_to_start.add(proc)
  511. hub.call_later(
  512. self._proc_alive_timeout, verify_process_alive, ref(proc),
  513. )
  514. self.on_process_up = on_process_up
  515. def _remove_from_index(obj, proc, index, remove_fun, callback=None):
  516. # this remove the file descriptors for a process from
  517. # the indices. we have to make sure we don't overwrite
  518. # another processes fds, as the fds may be reused.
  519. try:
  520. fd = obj.fileno()
  521. except (IOError, OSError):
  522. return
  523. try:
  524. if index[fd] is proc:
  525. # fd hasn't been reused so we can remove it from index.
  526. index.pop(fd, None)
  527. except KeyError:
  528. pass
  529. else:
  530. remove_fun(fd)
  531. if callback is not None:
  532. callback(fd)
  533. return fd
  534. def on_process_down(proc):
  535. """Called when a worker process exits."""
  536. if getattr(proc, 'dead', None):
  537. return
  538. process_flush_queues(proc)
  539. _remove_from_index(
  540. proc.outq._reader, proc, fileno_to_outq, remove_reader,
  541. )
  542. if proc.synq:
  543. _remove_from_index(
  544. proc.synq._writer, proc, fileno_to_synq, remove_writer,
  545. )
  546. inq = _remove_from_index(
  547. proc.inq._writer, proc, fileno_to_inq, remove_writer,
  548. callback=all_inqueues.discard,
  549. )
  550. if inq:
  551. busy_workers.discard(inq)
  552. self._untrack_child_process(proc, hub)
  553. waiting_to_start.discard(proc)
  554. self._active_writes.discard(proc.inqW_fd)
  555. remove_writer(proc.inq._writer)
  556. remove_reader(proc.outq._reader)
  557. if proc.synqR_fd:
  558. remove_reader(proc.synq._reader)
  559. if proc.synqW_fd:
  560. self._active_writes.discard(proc.synqW_fd)
  561. remove_reader(proc.synq._writer)
  562. self.on_process_down = on_process_down
  563. def _create_write_handlers(self, hub,
  564. pack=struct.pack, dumps=_pickle.dumps,
  565. protocol=HIGHEST_PROTOCOL):
  566. """Create handlers used to write data to child processes."""
  567. fileno_to_inq = self._fileno_to_inq
  568. fileno_to_synq = self._fileno_to_synq
  569. outbound = self.outbound_buffer
  570. pop_message = outbound.popleft
  571. put_message = outbound.append
  572. all_inqueues = self._all_inqueues
  573. active_writes = self._active_writes
  574. active_writers = self._active_writers
  575. busy_workers = self._busy_workers
  576. diff = all_inqueues.difference
  577. add_writer = hub.add_writer
  578. hub_add, hub_remove = hub.add, hub.remove
  579. mark_write_fd_as_active = active_writes.add
  580. mark_write_gen_as_active = active_writers.add
  581. mark_worker_as_busy = busy_workers.add
  582. write_generator_done = active_writers.discard
  583. get_job = self._cache.__getitem__
  584. write_stats = self.write_stats
  585. is_fair_strategy = self.sched_strategy == SCHED_STRATEGY_FAIR
  586. revoked_tasks = worker_state.revoked
  587. getpid = os.getpid
  588. precalc = {ACK: self._create_payload(ACK, (0,)),
  589. NACK: self._create_payload(NACK, (0,))}
  590. def _put_back(job, _time=time.time):
  591. # puts back at the end of the queue
  592. if job._terminated is not None or \
  593. job.correlation_id in revoked_tasks:
  594. if not job._accepted:
  595. job._ack(None, _time(), getpid(), None)
  596. job._set_terminated(job._terminated)
  597. else:
  598. # XXX linear lookup, should find a better way,
  599. # but this happens rarely and is here to protect against races.
  600. if job not in outbound:
  601. outbound.appendleft(job)
  602. self._put_back = _put_back
  603. # called for every event loop iteration, and if there
  604. # are messages pending this will schedule writing one message
  605. # by registering the 'schedule_writes' function for all currently
  606. # inactive inqueues (not already being written to)
  607. # consolidate means the event loop will merge them
  608. # and call the callback once with the list writable fds as
  609. # argument. Using this means we minimize the risk of having
  610. # the same fd receive every task if the pipe read buffer is not
  611. # full.
  612. if is_fair_strategy:
  613. def on_poll_start():
  614. if outbound and len(busy_workers) < len(all_inqueues):
  615. # print('ALL: %r ACTIVE: %r' % (len(all_inqueues),
  616. # len(active_writes)))
  617. inactive = diff(active_writes)
  618. [hub_add(fd, None, WRITE | ERR, consolidate=True)
  619. for fd in inactive]
  620. else:
  621. [hub_remove(fd) for fd in diff(active_writes)]
  622. else:
  623. def on_poll_start(): # noqa
  624. if outbound:
  625. [hub_add(fd, None, WRITE | ERR, consolidate=True)
  626. for fd in diff(active_writes)]
  627. else:
  628. [hub_remove(fd) for fd in diff(active_writes)]
  629. self.on_poll_start = on_poll_start
  630. def on_inqueue_close(fd, proc):
  631. # Makes sure the fd is removed from tracking when
  632. # the connection is closed, this is essential as fds may be reused.
  633. busy_workers.discard(fd)
  634. try:
  635. if fileno_to_inq[fd] is proc:
  636. fileno_to_inq.pop(fd, None)
  637. active_writes.discard(fd)
  638. all_inqueues.discard(fd)
  639. hub_remove(fd)
  640. except KeyError:
  641. pass
  642. self.on_inqueue_close = on_inqueue_close
  643. def schedule_writes(ready_fds, total_write_count=[0]):
  644. # Schedule write operation to ready file descriptor.
  645. # The file descriptor is writable, but that does not
  646. # mean the process is currently reading from the socket.
  647. # The socket is buffered so writable simply means that
  648. # the buffer can accept at least 1 byte of data.
  649. # This means we have to cycle between the ready fds.
  650. # the first version used shuffle, but this version
  651. # using `total_writes % ready_fds` is about 30% faster
  652. # with many processes, and also leans more towards fairness
  653. # in write stats when used with many processes
  654. # [XXX On macOS, this may vary depending
  655. # on event loop implementation (i.e, select/poll vs epoll), so
  656. # have to test further]
  657. num_ready = len(ready_fds)
  658. for _ in range(num_ready):
  659. ready_fd = ready_fds[total_write_count[0] % num_ready]
  660. total_write_count[0] += 1
  661. if ready_fd in active_writes:
  662. # already writing to this fd
  663. continue
  664. if is_fair_strategy and ready_fd in busy_workers:
  665. # worker is already busy with another task
  666. continue
  667. if ready_fd not in all_inqueues:
  668. hub_remove(ready_fd)
  669. continue
  670. try:
  671. job = pop_message()
  672. except IndexError:
  673. # no more messages, remove all inactive fds from the hub.
  674. # this is important since the fds are always writable
  675. # as long as there's 1 byte left in the buffer, and so
  676. # this may create a spinloop where the event loop
  677. # always wakes up.
  678. for inqfd in diff(active_writes):
  679. hub_remove(inqfd)
  680. break
  681. else:
  682. if not job._accepted: # job not accepted by another worker
  683. try:
  684. # keep track of what process the write operation
  685. # was scheduled for.
  686. proc = job._scheduled_for = fileno_to_inq[ready_fd]
  687. except KeyError:
  688. # write was scheduled for this fd but the process
  689. # has since exited and the message must be sent to
  690. # another process.
  691. put_message(job)
  692. continue
  693. cor = _write_job(proc, ready_fd, job)
  694. job._writer = ref(cor)
  695. mark_write_gen_as_active(cor)
  696. mark_write_fd_as_active(ready_fd)
  697. mark_worker_as_busy(ready_fd)
  698. # Try to write immediately, in case there's an error.
  699. try:
  700. next(cor)
  701. except StopIteration:
  702. pass
  703. except OSError as exc:
  704. if exc.errno != errno.EBADF:
  705. raise
  706. else:
  707. add_writer(ready_fd, cor)
  708. hub.consolidate_callback = schedule_writes
  709. def send_job(tup):
  710. # Schedule writing job request for when one of the process
  711. # inqueues are writable.
  712. body = dumps(tup, protocol=protocol)
  713. body_size = len(body)
  714. header = pack(b'>I', body_size)
  715. # index 1,0 is the job ID.
  716. job = get_job(tup[1][0])
  717. job._payload = buf_t(header), buf_t(body), body_size
  718. put_message(job)
  719. self._quick_put = send_job
  720. def on_not_recovering(proc, fd, job, exc):
  721. logger.exception(
  722. 'Process inqueue damaged: %r %r: %r', proc, proc.exitcode, exc)
  723. if proc._is_alive():
  724. proc.terminate()
  725. hub.remove(fd)
  726. self._put_back(job)
  727. def _write_job(proc, fd, job):
  728. # writes job to the worker process.
  729. # Operation must complete if more than one byte of data
  730. # was written. If the broker connection is lost
  731. # and no data was written the operation shall be canceled.
  732. header, body, body_size = job._payload
  733. errors = 0
  734. try:
  735. # job result keeps track of what process the job is sent to.
  736. job._write_to = proc
  737. send = proc.send_job_offset
  738. Hw = Bw = 0
  739. # write header
  740. while Hw < 4:
  741. try:
  742. Hw += send(header, Hw)
  743. except Exception as exc: # pylint: disable=broad-except
  744. if getattr(exc, 'errno', None) not in UNAVAIL:
  745. raise
  746. # suspend until more data
  747. errors += 1
  748. if errors > 100:
  749. on_not_recovering(proc, fd, job, exc)
  750. raise StopIteration()
  751. yield
  752. else:
  753. errors = 0
  754. # write body
  755. while Bw < body_size:
  756. try:
  757. Bw += send(body, Bw)
  758. except Exception as exc: # pylint: disable=broad-except
  759. if getattr(exc, 'errno', None) not in UNAVAIL:
  760. raise
  761. # suspend until more data
  762. errors += 1
  763. if errors > 100:
  764. on_not_recovering(proc, fd, job, exc)
  765. raise StopIteration()
  766. yield
  767. else:
  768. errors = 0
  769. finally:
  770. hub_remove(fd)
  771. write_stats[proc.index] += 1
  772. # message written, so this fd is now available
  773. active_writes.discard(fd)
  774. write_generator_done(job._writer()) # is a weakref
  775. def send_ack(response, pid, job, fd):
  776. # Only used when synack is enabled.
  777. # Schedule writing ack response for when the fd is writable.
  778. msg = Ack(job, fd, precalc[response])
  779. callback = promise(write_generator_done)
  780. cor = _write_ack(fd, msg, callback=callback)
  781. mark_write_gen_as_active(cor)
  782. mark_write_fd_as_active(fd)
  783. callback.args = (cor,)
  784. add_writer(fd, cor)
  785. self.send_ack = send_ack
  786. def _write_ack(fd, ack, callback=None):
  787. # writes ack back to the worker if synack enabled.
  788. # this operation *MUST* complete, otherwise
  789. # the worker process will hang waiting for the ack.
  790. header, body, body_size = ack[2]
  791. try:
  792. try:
  793. proc = fileno_to_synq[fd]
  794. except KeyError:
  795. # process died, we can safely discard the ack at this
  796. # point.
  797. raise StopIteration()
  798. send = proc.send_syn_offset
  799. Hw = Bw = 0
  800. # write header
  801. while Hw < 4:
  802. try:
  803. Hw += send(header, Hw)
  804. except Exception as exc: # pylint: disable=broad-except
  805. if getattr(exc, 'errno', None) not in UNAVAIL:
  806. raise
  807. yield
  808. # write body
  809. while Bw < body_size:
  810. try:
  811. Bw += send(body, Bw)
  812. except Exception as exc: # pylint: disable=broad-except
  813. if getattr(exc, 'errno', None) not in UNAVAIL:
  814. raise
  815. # suspend until more data
  816. yield
  817. finally:
  818. if callback:
  819. callback()
  820. # message written, so this fd is now available
  821. active_writes.discard(fd)
  822. def flush(self):
  823. if self._state == TERMINATE:
  824. return
  825. # cancel all tasks that haven't been accepted so that NACK is sent.
  826. for job in values(self._cache):
  827. if not job._accepted:
  828. job._cancel()
  829. # clear the outgoing buffer as the tasks will be redelivered by
  830. # the broker anyway.
  831. if self.outbound_buffer:
  832. self.outbound_buffer.clear()
  833. self.maintain_pool()
  834. try:
  835. # ...but we must continue writing the payloads we already started
  836. # to keep message boundaries.
  837. # The messages may be NACK'ed later if synack is enabled.
  838. if self._state == RUN:
  839. # flush outgoing buffers
  840. intervals = fxrange(0.01, 0.1, 0.01, repeatlast=True)
  841. owned_by = {}
  842. for job in values(self._cache):
  843. writer = _get_job_writer(job)
  844. if writer is not None:
  845. owned_by[writer] = job
  846. while self._active_writers:
  847. writers = list(self._active_writers)
  848. for gen in writers:
  849. if (gen.__name__ == '_write_job' and
  850. gen_not_started(gen)):
  851. # hasn't started writing the job so can
  852. # discard the task, but we must also remove
  853. # it from the Pool._cache.
  854. try:
  855. job = owned_by[gen]
  856. except KeyError:
  857. pass
  858. else:
  859. # removes from Pool._cache
  860. job.discard()
  861. self._active_writers.discard(gen)
  862. else:
  863. try:
  864. job = owned_by[gen]
  865. except KeyError:
  866. pass
  867. else:
  868. job_proc = job._write_to
  869. if job_proc._is_alive():
  870. self._flush_writer(job_proc, gen)
  871. # workers may have exited in the meantime.
  872. self.maintain_pool()
  873. sleep(next(intervals)) # don't busyloop
  874. finally:
  875. self.outbound_buffer.clear()
  876. self._active_writers.clear()
  877. self._active_writes.clear()
  878. self._busy_workers.clear()
  879. def _flush_writer(self, proc, writer):
  880. fds = {proc.inq._writer}
  881. try:
  882. while fds:
  883. if not proc._is_alive():
  884. break # process exited
  885. readable, writable, again = _select(
  886. writers=fds, err=fds, timeout=0.5,
  887. )
  888. if not again and (writable or readable):
  889. try:
  890. next(writer)
  891. except (StopIteration, OSError, IOError, EOFError):
  892. break
  893. finally:
  894. self._active_writers.discard(writer)
  895. def get_process_queues(self):
  896. """Get queues for a new process.
  897. Here we'll find an unused slot, as there should always
  898. be one available when we start a new process.
  899. """
  900. return next(q for q, owner in items(self._queues)
  901. if owner is None)
  902. def on_grow(self, n):
  903. """Grow the pool by ``n`` proceses."""
  904. diff = max(self._processes - len(self._queues), 0)
  905. if diff:
  906. self._queues.update({
  907. self.create_process_queues(): None for _ in range(diff)
  908. })
  909. def on_shrink(self, n):
  910. """Shrink the pool by ``n`` processes."""
  911. def create_process_queues(self):
  912. """Create new in, out, etc. queues, returned as a tuple."""
  913. # NOTE: Pipes must be set O_NONBLOCK at creation time (the original
  914. # fd), otherwise it won't be possible to change the flags until
  915. # there's an actual reader/writer on the other side.
  916. inq = _SimpleQueue(wnonblock=True)
  917. outq = _SimpleQueue(rnonblock=True)
  918. synq = None
  919. assert isblocking(inq._reader)
  920. assert not isblocking(inq._writer)
  921. assert not isblocking(outq._reader)
  922. assert isblocking(outq._writer)
  923. if self.synack:
  924. synq = _SimpleQueue(wnonblock=True)
  925. assert isblocking(synq._reader)
  926. assert not isblocking(synq._writer)
  927. return inq, outq, synq
  928. def on_process_alive(self, pid):
  929. """Called when reciving the :const:`WORKER_UP` message.
  930. Marks the process as ready to receive work.
  931. """
  932. try:
  933. proc = next(w for w in self._pool if w.pid == pid)
  934. except StopIteration:
  935. return logger.warning('process with pid=%s already exited', pid)
  936. assert proc.inqW_fd not in self._fileno_to_inq
  937. assert proc.inqW_fd not in self._all_inqueues
  938. self._waiting_to_start.discard(proc)
  939. self._fileno_to_inq[proc.inqW_fd] = proc
  940. self._fileno_to_synq[proc.synqW_fd] = proc
  941. self._all_inqueues.add(proc.inqW_fd)
  942. def on_job_process_down(self, job, pid_gone):
  943. """Called for each job when the process assigned to it exits."""
  944. if job._write_to and not job._write_to._is_alive():
  945. # job was partially written
  946. self.on_partial_read(job, job._write_to)
  947. elif job._scheduled_for and not job._scheduled_for._is_alive():
  948. # job was only scheduled to be written to this process,
  949. # but no data was sent so put it back on the outbound_buffer.
  950. self._put_back(job)
  951. def on_job_process_lost(self, job, pid, exitcode):
  952. """Called when the process executing job' exits.
  953. This happens when the process job'
  954. was assigned to exited by mysterious means (error exitcodes and
  955. signals).
  956. """
  957. self.mark_as_worker_lost(job, exitcode)
  958. def human_write_stats(self):
  959. if self.write_stats is None:
  960. return 'N/A'
  961. vals = list(values(self.write_stats))
  962. total = sum(vals)
  963. def per(v, total):
  964. return '{0:.2%}'.format((float(v) / total) if v else 0)
  965. return {
  966. 'total': total,
  967. 'avg': per(total / len(self.write_stats) if total else 0, total),
  968. 'all': ', '.join(per(v, total) for v in vals),
  969. 'raw': ', '.join(map(str, vals)),
  970. 'strategy': SCHED_STRATEGY_TO_NAME.get(
  971. self.sched_strategy, self.sched_strategy,
  972. ),
  973. 'inqueues': {
  974. 'total': len(self._all_inqueues),
  975. 'active': len(self._active_writes),
  976. }
  977. }
  978. def _process_cleanup_queues(self, proc):
  979. """Called to clean up queues after process exit."""
  980. if not proc.dead:
  981. try:
  982. self._queues[self._find_worker_queues(proc)] = None
  983. except (KeyError, ValueError):
  984. pass
  985. @staticmethod
  986. def _stop_task_handler(task_handler):
  987. """Called at shutdown to tell processes that we're shutting down."""
  988. for proc in task_handler.pool:
  989. try:
  990. setblocking(proc.inq._writer, 1)
  991. except (OSError, IOError):
  992. pass
  993. else:
  994. try:
  995. proc.inq.put(None)
  996. except OSError as exc:
  997. if exc.errno != errno.EBADF:
  998. raise
  999. def create_result_handler(self):
  1000. return super(AsynPool, self).create_result_handler(
  1001. fileno_to_outq=self._fileno_to_outq,
  1002. on_process_alive=self.on_process_alive,
  1003. )
  1004. def _process_register_queues(self, proc, queues):
  1005. """Mark new ownership for ``queues`` to update fileno indices."""
  1006. assert queues in self._queues
  1007. b = len(self._queues)
  1008. self._queues[queues] = proc
  1009. assert b == len(self._queues)
  1010. def _find_worker_queues(self, proc):
  1011. """Find the queues owned by ``proc``."""
  1012. try:
  1013. return next(q for q, owner in items(self._queues)
  1014. if owner == proc)
  1015. except StopIteration:
  1016. raise ValueError(proc)
  1017. def _setup_queues(self):
  1018. # this is only used by the original pool that used a shared
  1019. # queue for all processes.
  1020. self._quick_put = None
  1021. # these attributes are unused by this class, but we'll still
  1022. # have to initialize them for compatibility.
  1023. self._inqueue = self._outqueue = \
  1024. self._quick_get = self._poll_result = None
  1025. def process_flush_queues(self, proc):
  1026. """Flush all queues.
  1027. Including the outbound buffer, so that
  1028. all tasks that haven't been started will be discarded.
  1029. In Celery this is called whenever the transport connection is lost
  1030. (consumer restart), and when a process is terminated.
  1031. """
  1032. resq = proc.outq._reader
  1033. on_state_change = self._result_handler.on_state_change
  1034. fds = {resq}
  1035. while fds and not resq.closed and self._state != TERMINATE:
  1036. readable, _, _ = _select(fds, None, fds, timeout=0.01)
  1037. if readable:
  1038. try:
  1039. task = resq.recv()
  1040. except (OSError, IOError, EOFError) as exc:
  1041. _errno = getattr(exc, 'errno', None)
  1042. if _errno == errno.EINTR:
  1043. continue
  1044. elif _errno == errno.EAGAIN:
  1045. break
  1046. elif _errno not in UNAVAIL:
  1047. debug('got %r while flushing process %r',
  1048. exc, proc, exc_info=1)
  1049. break
  1050. else:
  1051. if task is None:
  1052. debug('got sentinel while flushing process %r', proc)
  1053. break
  1054. else:
  1055. on_state_change(task)
  1056. else:
  1057. break
  1058. def on_partial_read(self, job, proc):
  1059. """Called when a job was partially written to exited child."""
  1060. # worker terminated by signal:
  1061. # we cannot reuse the sockets again, because we don't know if
  1062. # the process wrote/read anything frmo them, and if so we cannot
  1063. # restore the message boundaries.
  1064. if not job._accepted:
  1065. # job was not acked, so find another worker to send it to.
  1066. self._put_back(job)
  1067. writer = _get_job_writer(job)
  1068. if writer:
  1069. self._active_writers.discard(writer)
  1070. del writer
  1071. if not proc.dead:
  1072. proc.dead = True
  1073. # Replace queues to avoid reuse
  1074. before = len(self._queues)
  1075. try:
  1076. queues = self._find_worker_queues(proc)
  1077. if self.destroy_queues(queues, proc):
  1078. self._queues[self.create_process_queues()] = None
  1079. except ValueError:
  1080. pass
  1081. assert len(self._queues) == before
  1082. def destroy_queues(self, queues, proc):
  1083. """Destroy queues that can no longer be used.
  1084. This way they can be replaced by new usable sockets.
  1085. """
  1086. assert not proc._is_alive()
  1087. self._waiting_to_start.discard(proc)
  1088. removed = 1
  1089. try:
  1090. self._queues.pop(queues)
  1091. except KeyError:
  1092. removed = 0
  1093. try:
  1094. self.on_inqueue_close(queues[0]._writer.fileno(), proc)
  1095. except IOError:
  1096. pass
  1097. for queue in queues:
  1098. if queue:
  1099. for sock in (queue._reader, queue._writer):
  1100. if not sock.closed:
  1101. try:
  1102. sock.close()
  1103. except (IOError, OSError):
  1104. pass
  1105. return removed
  1106. def _create_payload(self, type_, args,
  1107. dumps=_pickle.dumps, pack=struct.pack,
  1108. protocol=HIGHEST_PROTOCOL):
  1109. body = dumps((type_, args), protocol=protocol)
  1110. size = len(body)
  1111. header = pack(b'>I', size)
  1112. return header, body, size
  1113. @classmethod
  1114. def _set_result_sentinel(cls, _outqueue, _pool):
  1115. # unused
  1116. pass
  1117. def _help_stuff_finish_args(self):
  1118. # Pool._help_stuff_finished is a classmethod so we have to use this
  1119. # trick to modify the arguments passed to it.
  1120. return (self._pool,)
  1121. @classmethod
  1122. def _help_stuff_finish(cls, pool):
  1123. # pylint: disable=arguments-differ
  1124. debug(
  1125. 'removing tasks from inqueue until task handler finished',
  1126. )
  1127. fileno_to_proc = {}
  1128. inqR = set()
  1129. for w in pool:
  1130. try:
  1131. fd = w.inq._reader.fileno()
  1132. inqR.add(fd)
  1133. fileno_to_proc[fd] = w
  1134. except IOError:
  1135. pass
  1136. while inqR:
  1137. readable, _, again = _select(inqR, timeout=0.5)
  1138. if again:
  1139. continue
  1140. if not readable:
  1141. break
  1142. for fd in readable:
  1143. fileno_to_proc[fd].inq._reader.recv()
  1144. sleep(0)
  1145. @property
  1146. def timers(self):
  1147. return {self.maintain_pool: 5.0}