poll.c 29 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061
  1. // SPDX-License-Identifier: GPL-2.0
  2. #include <linux/kernel.h>
  3. #include <linux/errno.h>
  4. #include <linux/fs.h>
  5. #include <linux/file.h>
  6. #include <linux/mm.h>
  7. #include <linux/slab.h>
  8. #include <linux/poll.h>
  9. #include <linux/hashtable.h>
  10. #include <linux/io_uring.h>
  11. #include <trace/events/io_uring.h>
  12. #include <uapi/linux/io_uring.h>
  13. #include "io_uring.h"
  14. #include "alloc_cache.h"
  15. #include "refs.h"
  16. #include "napi.h"
  17. #include "opdef.h"
  18. #include "kbuf.h"
  19. #include "poll.h"
  20. #include "cancel.h"
  21. struct io_poll_update {
  22. struct file *file;
  23. u64 old_user_data;
  24. u64 new_user_data;
  25. __poll_t events;
  26. bool update_events;
  27. bool update_user_data;
  28. };
  29. struct io_poll_table {
  30. struct poll_table_struct pt;
  31. struct io_kiocb *req;
  32. int nr_entries;
  33. int error;
  34. bool owning;
  35. /* output value, set only if arm poll returns >0 */
  36. __poll_t result_mask;
  37. };
  38. #define IO_POLL_CANCEL_FLAG BIT(31)
  39. #define IO_POLL_RETRY_FLAG BIT(30)
  40. #define IO_POLL_REF_MASK GENMASK(29, 0)
  41. /*
  42. * We usually have 1-2 refs taken, 128 is more than enough and we want to
  43. * maximise the margin between this amount and the moment when it overflows.
  44. */
  45. #define IO_POLL_REF_BIAS 128
  46. #define IO_WQE_F_DOUBLE 1
  47. static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
  48. void *key);
  49. static inline struct io_kiocb *wqe_to_req(struct wait_queue_entry *wqe)
  50. {
  51. unsigned long priv = (unsigned long)wqe->private;
  52. return (struct io_kiocb *)(priv & ~IO_WQE_F_DOUBLE);
  53. }
  54. static inline bool wqe_is_double(struct wait_queue_entry *wqe)
  55. {
  56. unsigned long priv = (unsigned long)wqe->private;
  57. return priv & IO_WQE_F_DOUBLE;
  58. }
  59. static bool io_poll_get_ownership_slowpath(struct io_kiocb *req)
  60. {
  61. int v;
  62. /*
  63. * poll_refs are already elevated and we don't have much hope for
  64. * grabbing the ownership. Instead of incrementing set a retry flag
  65. * to notify the loop that there might have been some change.
  66. */
  67. v = atomic_fetch_or(IO_POLL_RETRY_FLAG, &req->poll_refs);
  68. if (v & IO_POLL_REF_MASK)
  69. return false;
  70. return !(atomic_fetch_inc(&req->poll_refs) & IO_POLL_REF_MASK);
  71. }
  72. /*
  73. * If refs part of ->poll_refs (see IO_POLL_REF_MASK) is 0, it's free. We can
  74. * bump it and acquire ownership. It's disallowed to modify requests while not
  75. * owning it, that prevents from races for enqueueing task_work's and b/w
  76. * arming poll and wakeups.
  77. */
  78. static inline bool io_poll_get_ownership(struct io_kiocb *req)
  79. {
  80. if (unlikely(atomic_read(&req->poll_refs) >= IO_POLL_REF_BIAS))
  81. return io_poll_get_ownership_slowpath(req);
  82. return !(atomic_fetch_inc(&req->poll_refs) & IO_POLL_REF_MASK);
  83. }
  84. static void io_poll_mark_cancelled(struct io_kiocb *req)
  85. {
  86. atomic_or(IO_POLL_CANCEL_FLAG, &req->poll_refs);
  87. }
  88. static struct io_poll *io_poll_get_double(struct io_kiocb *req)
  89. {
  90. /* pure poll stashes this in ->async_data, poll driven retry elsewhere */
  91. if (req->opcode == IORING_OP_POLL_ADD)
  92. return req->async_data;
  93. return req->apoll->double_poll;
  94. }
  95. static struct io_poll *io_poll_get_single(struct io_kiocb *req)
  96. {
  97. if (req->opcode == IORING_OP_POLL_ADD)
  98. return io_kiocb_to_cmd(req, struct io_poll);
  99. return &req->apoll->poll;
  100. }
  101. static void io_poll_req_insert(struct io_kiocb *req)
  102. {
  103. struct io_hash_table *table = &req->ctx->cancel_table;
  104. u32 index = hash_long(req->cqe.user_data, table->hash_bits);
  105. struct io_hash_bucket *hb = &table->hbs[index];
  106. spin_lock(&hb->lock);
  107. hlist_add_head(&req->hash_node, &hb->list);
  108. spin_unlock(&hb->lock);
  109. }
  110. static void io_poll_req_delete(struct io_kiocb *req, struct io_ring_ctx *ctx)
  111. {
  112. struct io_hash_table *table = &req->ctx->cancel_table;
  113. u32 index = hash_long(req->cqe.user_data, table->hash_bits);
  114. spinlock_t *lock = &table->hbs[index].lock;
  115. spin_lock(lock);
  116. hash_del(&req->hash_node);
  117. spin_unlock(lock);
  118. }
  119. static void io_poll_req_insert_locked(struct io_kiocb *req)
  120. {
  121. struct io_hash_table *table = &req->ctx->cancel_table_locked;
  122. u32 index = hash_long(req->cqe.user_data, table->hash_bits);
  123. lockdep_assert_held(&req->ctx->uring_lock);
  124. hlist_add_head(&req->hash_node, &table->hbs[index].list);
  125. }
  126. static void io_poll_tw_hash_eject(struct io_kiocb *req, struct io_tw_state *ts)
  127. {
  128. struct io_ring_ctx *ctx = req->ctx;
  129. if (req->flags & REQ_F_HASH_LOCKED) {
  130. /*
  131. * ->cancel_table_locked is protected by ->uring_lock in
  132. * contrast to per bucket spinlocks. Likely, tctx_task_work()
  133. * already grabbed the mutex for us, but there is a chance it
  134. * failed.
  135. */
  136. io_tw_lock(ctx, ts);
  137. hash_del(&req->hash_node);
  138. req->flags &= ~REQ_F_HASH_LOCKED;
  139. } else {
  140. io_poll_req_delete(req, ctx);
  141. }
  142. }
  143. static void io_init_poll_iocb(struct io_poll *poll, __poll_t events)
  144. {
  145. poll->head = NULL;
  146. #define IO_POLL_UNMASK (EPOLLERR|EPOLLHUP|EPOLLNVAL|EPOLLRDHUP)
  147. /* mask in events that we always want/need */
  148. poll->events = events | IO_POLL_UNMASK;
  149. INIT_LIST_HEAD(&poll->wait.entry);
  150. init_waitqueue_func_entry(&poll->wait, io_poll_wake);
  151. }
  152. static inline void io_poll_remove_entry(struct io_poll *poll)
  153. {
  154. struct wait_queue_head *head = smp_load_acquire(&poll->head);
  155. if (head) {
  156. spin_lock_irq(&head->lock);
  157. list_del_init(&poll->wait.entry);
  158. poll->head = NULL;
  159. spin_unlock_irq(&head->lock);
  160. }
  161. }
  162. static void io_poll_remove_entries(struct io_kiocb *req)
  163. {
  164. /*
  165. * Nothing to do if neither of those flags are set. Avoid dipping
  166. * into the poll/apoll/double cachelines if we can.
  167. */
  168. if (!(req->flags & (REQ_F_SINGLE_POLL | REQ_F_DOUBLE_POLL)))
  169. return;
  170. /*
  171. * While we hold the waitqueue lock and the waitqueue is nonempty,
  172. * wake_up_pollfree() will wait for us. However, taking the waitqueue
  173. * lock in the first place can race with the waitqueue being freed.
  174. *
  175. * We solve this as eventpoll does: by taking advantage of the fact that
  176. * all users of wake_up_pollfree() will RCU-delay the actual free. If
  177. * we enter rcu_read_lock() and see that the pointer to the queue is
  178. * non-NULL, we can then lock it without the memory being freed out from
  179. * under us.
  180. *
  181. * Keep holding rcu_read_lock() as long as we hold the queue lock, in
  182. * case the caller deletes the entry from the queue, leaving it empty.
  183. * In that case, only RCU prevents the queue memory from being freed.
  184. */
  185. rcu_read_lock();
  186. if (req->flags & REQ_F_SINGLE_POLL)
  187. io_poll_remove_entry(io_poll_get_single(req));
  188. if (req->flags & REQ_F_DOUBLE_POLL)
  189. io_poll_remove_entry(io_poll_get_double(req));
  190. rcu_read_unlock();
  191. }
  192. enum {
  193. IOU_POLL_DONE = 0,
  194. IOU_POLL_NO_ACTION = 1,
  195. IOU_POLL_REMOVE_POLL_USE_RES = 2,
  196. IOU_POLL_REISSUE = 3,
  197. IOU_POLL_REQUEUE = 4,
  198. };
  199. static void __io_poll_execute(struct io_kiocb *req, int mask)
  200. {
  201. unsigned flags = 0;
  202. io_req_set_res(req, mask, 0);
  203. req->io_task_work.func = io_poll_task_func;
  204. trace_io_uring_task_add(req, mask);
  205. if (!(req->flags & REQ_F_POLL_NO_LAZY))
  206. flags = IOU_F_TWQ_LAZY_WAKE;
  207. __io_req_task_work_add(req, flags);
  208. }
  209. static inline void io_poll_execute(struct io_kiocb *req, int res)
  210. {
  211. if (io_poll_get_ownership(req))
  212. __io_poll_execute(req, res);
  213. }
  214. /*
  215. * All poll tw should go through this. Checks for poll events, manages
  216. * references, does rewait, etc.
  217. *
  218. * Returns a negative error on failure. IOU_POLL_NO_ACTION when no action
  219. * require, which is either spurious wakeup or multishot CQE is served.
  220. * IOU_POLL_DONE when it's done with the request, then the mask is stored in
  221. * req->cqe.res. IOU_POLL_REMOVE_POLL_USE_RES indicates to remove multishot
  222. * poll and that the result is stored in req->cqe.
  223. */
  224. static int io_poll_check_events(struct io_kiocb *req, struct io_tw_state *ts)
  225. {
  226. int v;
  227. /* req->task == current here, checking PF_EXITING is safe */
  228. if (unlikely(req->task->flags & PF_EXITING))
  229. return -ECANCELED;
  230. do {
  231. v = atomic_read(&req->poll_refs);
  232. if (unlikely(v != 1)) {
  233. /* tw should be the owner and so have some refs */
  234. if (WARN_ON_ONCE(!(v & IO_POLL_REF_MASK)))
  235. return IOU_POLL_NO_ACTION;
  236. if (v & IO_POLL_CANCEL_FLAG)
  237. return -ECANCELED;
  238. /*
  239. * cqe.res contains only events of the first wake up
  240. * and all others are to be lost. Redo vfs_poll() to get
  241. * up to date state.
  242. */
  243. if ((v & IO_POLL_REF_MASK) != 1)
  244. req->cqe.res = 0;
  245. if (v & IO_POLL_RETRY_FLAG) {
  246. req->cqe.res = 0;
  247. /*
  248. * We won't find new events that came in between
  249. * vfs_poll and the ref put unless we clear the
  250. * flag in advance.
  251. */
  252. atomic_andnot(IO_POLL_RETRY_FLAG, &req->poll_refs);
  253. v &= ~IO_POLL_RETRY_FLAG;
  254. }
  255. }
  256. /* the mask was stashed in __io_poll_execute */
  257. if (!req->cqe.res) {
  258. struct poll_table_struct pt = { ._key = req->apoll_events };
  259. req->cqe.res = vfs_poll(req->file, &pt) & req->apoll_events;
  260. /*
  261. * We got woken with a mask, but someone else got to
  262. * it first. The above vfs_poll() doesn't add us back
  263. * to the waitqueue, so if we get nothing back, we
  264. * should be safe and attempt a reissue.
  265. */
  266. if (unlikely(!req->cqe.res)) {
  267. /* Multishot armed need not reissue */
  268. if (!(req->apoll_events & EPOLLONESHOT))
  269. continue;
  270. return IOU_POLL_REISSUE;
  271. }
  272. }
  273. if (unlikely(req->cqe.res & EPOLLERR))
  274. req_set_fail(req);
  275. if (req->apoll_events & EPOLLONESHOT)
  276. return IOU_POLL_DONE;
  277. /* multishot, just fill a CQE and proceed */
  278. if (!(req->flags & REQ_F_APOLL_MULTISHOT)) {
  279. __poll_t mask = mangle_poll(req->cqe.res &
  280. req->apoll_events);
  281. if (!io_req_post_cqe(req, mask, IORING_CQE_F_MORE)) {
  282. io_req_set_res(req, mask, 0);
  283. return IOU_POLL_REMOVE_POLL_USE_RES;
  284. }
  285. } else {
  286. int ret = io_poll_issue(req, ts);
  287. if (ret == IOU_STOP_MULTISHOT)
  288. return IOU_POLL_REMOVE_POLL_USE_RES;
  289. else if (ret == IOU_REQUEUE)
  290. return IOU_POLL_REQUEUE;
  291. if (ret < 0)
  292. return ret;
  293. }
  294. /* force the next iteration to vfs_poll() */
  295. req->cqe.res = 0;
  296. /*
  297. * Release all references, retry if someone tried to restart
  298. * task_work while we were executing it.
  299. */
  300. v &= IO_POLL_REF_MASK;
  301. } while (atomic_sub_return(v, &req->poll_refs) & IO_POLL_REF_MASK);
  302. io_napi_add(req);
  303. return IOU_POLL_NO_ACTION;
  304. }
  305. void io_poll_task_func(struct io_kiocb *req, struct io_tw_state *ts)
  306. {
  307. int ret;
  308. ret = io_poll_check_events(req, ts);
  309. if (ret == IOU_POLL_NO_ACTION) {
  310. io_kbuf_recycle(req, 0);
  311. return;
  312. } else if (ret == IOU_POLL_REQUEUE) {
  313. io_kbuf_recycle(req, 0);
  314. __io_poll_execute(req, 0);
  315. return;
  316. }
  317. io_poll_remove_entries(req);
  318. io_poll_tw_hash_eject(req, ts);
  319. if (req->opcode == IORING_OP_POLL_ADD) {
  320. if (ret == IOU_POLL_DONE) {
  321. struct io_poll *poll;
  322. poll = io_kiocb_to_cmd(req, struct io_poll);
  323. req->cqe.res = mangle_poll(req->cqe.res & poll->events);
  324. } else if (ret == IOU_POLL_REISSUE) {
  325. io_req_task_submit(req, ts);
  326. return;
  327. } else if (ret != IOU_POLL_REMOVE_POLL_USE_RES) {
  328. req->cqe.res = ret;
  329. req_set_fail(req);
  330. }
  331. io_req_set_res(req, req->cqe.res, 0);
  332. io_req_task_complete(req, ts);
  333. } else {
  334. io_tw_lock(req->ctx, ts);
  335. if (ret == IOU_POLL_REMOVE_POLL_USE_RES)
  336. io_req_task_complete(req, ts);
  337. else if (ret == IOU_POLL_DONE || ret == IOU_POLL_REISSUE)
  338. io_req_task_submit(req, ts);
  339. else
  340. io_req_defer_failed(req, ret);
  341. }
  342. }
  343. static void io_poll_cancel_req(struct io_kiocb *req)
  344. {
  345. io_poll_mark_cancelled(req);
  346. /* kick tw, which should complete the request */
  347. io_poll_execute(req, 0);
  348. }
  349. #define IO_ASYNC_POLL_COMMON (EPOLLONESHOT | EPOLLPRI)
  350. static __cold int io_pollfree_wake(struct io_kiocb *req, struct io_poll *poll)
  351. {
  352. io_poll_mark_cancelled(req);
  353. /* we have to kick tw in case it's not already */
  354. io_poll_execute(req, 0);
  355. /*
  356. * If the waitqueue is being freed early but someone is already
  357. * holds ownership over it, we have to tear down the request as
  358. * best we can. That means immediately removing the request from
  359. * its waitqueue and preventing all further accesses to the
  360. * waitqueue via the request.
  361. */
  362. list_del_init(&poll->wait.entry);
  363. /*
  364. * Careful: this *must* be the last step, since as soon
  365. * as req->head is NULL'ed out, the request can be
  366. * completed and freed, since aio_poll_complete_work()
  367. * will no longer need to take the waitqueue lock.
  368. */
  369. smp_store_release(&poll->head, NULL);
  370. return 1;
  371. }
  372. static int io_poll_wake(struct wait_queue_entry *wait, unsigned mode, int sync,
  373. void *key)
  374. {
  375. struct io_kiocb *req = wqe_to_req(wait);
  376. struct io_poll *poll = container_of(wait, struct io_poll, wait);
  377. __poll_t mask = key_to_poll(key);
  378. if (unlikely(mask & POLLFREE))
  379. return io_pollfree_wake(req, poll);
  380. /* for instances that support it check for an event match first */
  381. if (mask && !(mask & (poll->events & ~IO_ASYNC_POLL_COMMON)))
  382. return 0;
  383. if (io_poll_get_ownership(req)) {
  384. /*
  385. * If we trigger a multishot poll off our own wakeup path,
  386. * disable multishot as there is a circular dependency between
  387. * CQ posting and triggering the event.
  388. */
  389. if (mask & EPOLL_URING_WAKE)
  390. poll->events |= EPOLLONESHOT;
  391. /* optional, saves extra locking for removal in tw handler */
  392. if (mask && poll->events & EPOLLONESHOT) {
  393. list_del_init(&poll->wait.entry);
  394. poll->head = NULL;
  395. if (wqe_is_double(wait))
  396. req->flags &= ~REQ_F_DOUBLE_POLL;
  397. else
  398. req->flags &= ~REQ_F_SINGLE_POLL;
  399. }
  400. __io_poll_execute(req, mask);
  401. }
  402. return 1;
  403. }
  404. /* fails only when polling is already completing by the first entry */
  405. static bool io_poll_double_prepare(struct io_kiocb *req)
  406. {
  407. struct wait_queue_head *head;
  408. struct io_poll *poll = io_poll_get_single(req);
  409. /* head is RCU protected, see io_poll_remove_entries() comments */
  410. rcu_read_lock();
  411. head = smp_load_acquire(&poll->head);
  412. /*
  413. * poll arm might not hold ownership and so race for req->flags with
  414. * io_poll_wake(). There is only one poll entry queued, serialise with
  415. * it by taking its head lock. As we're still arming the tw hanlder
  416. * is not going to be run, so there are no races with it.
  417. */
  418. if (head) {
  419. spin_lock_irq(&head->lock);
  420. req->flags |= REQ_F_DOUBLE_POLL;
  421. if (req->opcode == IORING_OP_POLL_ADD)
  422. req->flags |= REQ_F_ASYNC_DATA;
  423. spin_unlock_irq(&head->lock);
  424. }
  425. rcu_read_unlock();
  426. return !!head;
  427. }
  428. static void __io_queue_proc(struct io_poll *poll, struct io_poll_table *pt,
  429. struct wait_queue_head *head,
  430. struct io_poll **poll_ptr)
  431. {
  432. struct io_kiocb *req = pt->req;
  433. unsigned long wqe_private = (unsigned long) req;
  434. /*
  435. * The file being polled uses multiple waitqueues for poll handling
  436. * (e.g. one for read, one for write). Setup a separate io_poll
  437. * if this happens.
  438. */
  439. if (unlikely(pt->nr_entries)) {
  440. struct io_poll *first = poll;
  441. /* double add on the same waitqueue head, ignore */
  442. if (first->head == head)
  443. return;
  444. /* already have a 2nd entry, fail a third attempt */
  445. if (*poll_ptr) {
  446. if ((*poll_ptr)->head == head)
  447. return;
  448. pt->error = -EINVAL;
  449. return;
  450. }
  451. poll = kmalloc(sizeof(*poll), GFP_ATOMIC);
  452. if (!poll) {
  453. pt->error = -ENOMEM;
  454. return;
  455. }
  456. /* mark as double wq entry */
  457. wqe_private |= IO_WQE_F_DOUBLE;
  458. io_init_poll_iocb(poll, first->events);
  459. if (!io_poll_double_prepare(req)) {
  460. /* the request is completing, just back off */
  461. kfree(poll);
  462. return;
  463. }
  464. *poll_ptr = poll;
  465. } else {
  466. /* fine to modify, there is no poll queued to race with us */
  467. req->flags |= REQ_F_SINGLE_POLL;
  468. }
  469. pt->nr_entries++;
  470. poll->head = head;
  471. poll->wait.private = (void *) wqe_private;
  472. if (poll->events & EPOLLEXCLUSIVE) {
  473. add_wait_queue_exclusive(head, &poll->wait);
  474. } else {
  475. add_wait_queue(head, &poll->wait);
  476. }
  477. }
  478. static void io_poll_queue_proc(struct file *file, struct wait_queue_head *head,
  479. struct poll_table_struct *p)
  480. {
  481. struct io_poll_table *pt = container_of(p, struct io_poll_table, pt);
  482. struct io_poll *poll = io_kiocb_to_cmd(pt->req, struct io_poll);
  483. __io_queue_proc(poll, pt, head,
  484. (struct io_poll **) &pt->req->async_data);
  485. }
  486. static bool io_poll_can_finish_inline(struct io_kiocb *req,
  487. struct io_poll_table *pt)
  488. {
  489. return pt->owning || io_poll_get_ownership(req);
  490. }
  491. static void io_poll_add_hash(struct io_kiocb *req)
  492. {
  493. if (req->flags & REQ_F_HASH_LOCKED)
  494. io_poll_req_insert_locked(req);
  495. else
  496. io_poll_req_insert(req);
  497. }
  498. /*
  499. * Returns 0 when it's handed over for polling. The caller owns the requests if
  500. * it returns non-zero, but otherwise should not touch it. Negative values
  501. * contain an error code. When the result is >0, the polling has completed
  502. * inline and ipt.result_mask is set to the mask.
  503. */
  504. static int __io_arm_poll_handler(struct io_kiocb *req,
  505. struct io_poll *poll,
  506. struct io_poll_table *ipt, __poll_t mask,
  507. unsigned issue_flags)
  508. {
  509. INIT_HLIST_NODE(&req->hash_node);
  510. io_init_poll_iocb(poll, mask);
  511. poll->file = req->file;
  512. req->apoll_events = poll->events;
  513. ipt->pt._key = mask;
  514. ipt->req = req;
  515. ipt->error = 0;
  516. ipt->nr_entries = 0;
  517. /*
  518. * Polling is either completed here or via task_work, so if we're in the
  519. * task context we're naturally serialised with tw by merit of running
  520. * the same task. When it's io-wq, take the ownership to prevent tw
  521. * from running. However, when we're in the task context, skip taking
  522. * it as an optimisation.
  523. *
  524. * Note: even though the request won't be completed/freed, without
  525. * ownership we still can race with io_poll_wake().
  526. * io_poll_can_finish_inline() tries to deal with that.
  527. */
  528. ipt->owning = issue_flags & IO_URING_F_UNLOCKED;
  529. atomic_set(&req->poll_refs, (int)ipt->owning);
  530. /* io-wq doesn't hold uring_lock */
  531. if (issue_flags & IO_URING_F_UNLOCKED)
  532. req->flags &= ~REQ_F_HASH_LOCKED;
  533. /*
  534. * Exclusive waits may only wake a limited amount of entries
  535. * rather than all of them, this may interfere with lazy
  536. * wake if someone does wait(events > 1). Ensure we don't do
  537. * lazy wake for those, as we need to process each one as they
  538. * come in.
  539. */
  540. if (poll->events & EPOLLEXCLUSIVE)
  541. req->flags |= REQ_F_POLL_NO_LAZY;
  542. mask = vfs_poll(req->file, &ipt->pt) & poll->events;
  543. if (unlikely(ipt->error || !ipt->nr_entries)) {
  544. io_poll_remove_entries(req);
  545. if (!io_poll_can_finish_inline(req, ipt)) {
  546. io_poll_mark_cancelled(req);
  547. return 0;
  548. } else if (mask && (poll->events & EPOLLET)) {
  549. ipt->result_mask = mask;
  550. return 1;
  551. }
  552. return ipt->error ?: -EINVAL;
  553. }
  554. if (mask &&
  555. ((poll->events & (EPOLLET|EPOLLONESHOT)) == (EPOLLET|EPOLLONESHOT))) {
  556. if (!io_poll_can_finish_inline(req, ipt)) {
  557. io_poll_add_hash(req);
  558. return 0;
  559. }
  560. io_poll_remove_entries(req);
  561. ipt->result_mask = mask;
  562. /* no one else has access to the req, forget about the ref */
  563. return 1;
  564. }
  565. io_poll_add_hash(req);
  566. if (mask && (poll->events & EPOLLET) &&
  567. io_poll_can_finish_inline(req, ipt)) {
  568. __io_poll_execute(req, mask);
  569. return 0;
  570. }
  571. io_napi_add(req);
  572. if (ipt->owning) {
  573. /*
  574. * Try to release ownership. If we see a change of state, e.g.
  575. * poll was waken up, queue up a tw, it'll deal with it.
  576. */
  577. if (atomic_cmpxchg(&req->poll_refs, 1, 0) != 1)
  578. __io_poll_execute(req, 0);
  579. }
  580. return 0;
  581. }
  582. static void io_async_queue_proc(struct file *file, struct wait_queue_head *head,
  583. struct poll_table_struct *p)
  584. {
  585. struct io_poll_table *pt = container_of(p, struct io_poll_table, pt);
  586. struct async_poll *apoll = pt->req->apoll;
  587. __io_queue_proc(&apoll->poll, pt, head, &apoll->double_poll);
  588. }
  589. /*
  590. * We can't reliably detect loops in repeated poll triggers and issue
  591. * subsequently failing. But rather than fail these immediately, allow a
  592. * certain amount of retries before we give up. Given that this condition
  593. * should _rarely_ trigger even once, we should be fine with a larger value.
  594. */
  595. #define APOLL_MAX_RETRY 128
  596. static struct async_poll *io_req_alloc_apoll(struct io_kiocb *req,
  597. unsigned issue_flags)
  598. {
  599. struct io_ring_ctx *ctx = req->ctx;
  600. struct async_poll *apoll;
  601. if (req->flags & REQ_F_POLLED) {
  602. apoll = req->apoll;
  603. kfree(apoll->double_poll);
  604. } else if (!(issue_flags & IO_URING_F_UNLOCKED)) {
  605. apoll = io_alloc_cache_get(&ctx->apoll_cache);
  606. if (!apoll)
  607. goto alloc_apoll;
  608. apoll->poll.retries = APOLL_MAX_RETRY;
  609. } else {
  610. alloc_apoll:
  611. apoll = kmalloc(sizeof(*apoll), GFP_ATOMIC);
  612. if (unlikely(!apoll))
  613. return NULL;
  614. apoll->poll.retries = APOLL_MAX_RETRY;
  615. }
  616. apoll->double_poll = NULL;
  617. req->apoll = apoll;
  618. if (unlikely(!--apoll->poll.retries))
  619. return NULL;
  620. return apoll;
  621. }
  622. int io_arm_poll_handler(struct io_kiocb *req, unsigned issue_flags)
  623. {
  624. const struct io_issue_def *def = &io_issue_defs[req->opcode];
  625. struct async_poll *apoll;
  626. struct io_poll_table ipt;
  627. __poll_t mask = POLLPRI | POLLERR | EPOLLET;
  628. int ret;
  629. /*
  630. * apoll requests already grab the mutex to complete in the tw handler,
  631. * so removal from the mutex-backed hash is free, use it by default.
  632. */
  633. req->flags |= REQ_F_HASH_LOCKED;
  634. if (!def->pollin && !def->pollout)
  635. return IO_APOLL_ABORTED;
  636. if (!io_file_can_poll(req))
  637. return IO_APOLL_ABORTED;
  638. if (!(req->flags & REQ_F_APOLL_MULTISHOT))
  639. mask |= EPOLLONESHOT;
  640. if (def->pollin) {
  641. mask |= EPOLLIN | EPOLLRDNORM;
  642. /* If reading from MSG_ERRQUEUE using recvmsg, ignore POLLIN */
  643. if (req->flags & REQ_F_CLEAR_POLLIN)
  644. mask &= ~EPOLLIN;
  645. } else {
  646. mask |= EPOLLOUT | EPOLLWRNORM;
  647. }
  648. if (def->poll_exclusive)
  649. mask |= EPOLLEXCLUSIVE;
  650. apoll = io_req_alloc_apoll(req, issue_flags);
  651. if (!apoll)
  652. return IO_APOLL_ABORTED;
  653. req->flags &= ~(REQ_F_SINGLE_POLL | REQ_F_DOUBLE_POLL);
  654. req->flags |= REQ_F_POLLED;
  655. ipt.pt._qproc = io_async_queue_proc;
  656. io_kbuf_recycle(req, issue_flags);
  657. ret = __io_arm_poll_handler(req, &apoll->poll, &ipt, mask, issue_flags);
  658. if (ret)
  659. return ret > 0 ? IO_APOLL_READY : IO_APOLL_ABORTED;
  660. trace_io_uring_poll_arm(req, mask, apoll->poll.events);
  661. return IO_APOLL_OK;
  662. }
  663. static __cold bool io_poll_remove_all_table(struct task_struct *tsk,
  664. struct io_hash_table *table,
  665. bool cancel_all)
  666. {
  667. unsigned nr_buckets = 1U << table->hash_bits;
  668. struct hlist_node *tmp;
  669. struct io_kiocb *req;
  670. bool found = false;
  671. int i;
  672. for (i = 0; i < nr_buckets; i++) {
  673. struct io_hash_bucket *hb = &table->hbs[i];
  674. spin_lock(&hb->lock);
  675. hlist_for_each_entry_safe(req, tmp, &hb->list, hash_node) {
  676. if (io_match_task_safe(req, tsk, cancel_all)) {
  677. hlist_del_init(&req->hash_node);
  678. io_poll_cancel_req(req);
  679. found = true;
  680. }
  681. }
  682. spin_unlock(&hb->lock);
  683. }
  684. return found;
  685. }
  686. /*
  687. * Returns true if we found and killed one or more poll requests
  688. */
  689. __cold bool io_poll_remove_all(struct io_ring_ctx *ctx, struct task_struct *tsk,
  690. bool cancel_all)
  691. __must_hold(&ctx->uring_lock)
  692. {
  693. bool ret;
  694. ret = io_poll_remove_all_table(tsk, &ctx->cancel_table, cancel_all);
  695. ret |= io_poll_remove_all_table(tsk, &ctx->cancel_table_locked, cancel_all);
  696. return ret;
  697. }
  698. static struct io_kiocb *io_poll_find(struct io_ring_ctx *ctx, bool poll_only,
  699. struct io_cancel_data *cd,
  700. struct io_hash_table *table,
  701. struct io_hash_bucket **out_bucket)
  702. {
  703. struct io_kiocb *req;
  704. u32 index = hash_long(cd->data, table->hash_bits);
  705. struct io_hash_bucket *hb = &table->hbs[index];
  706. *out_bucket = NULL;
  707. spin_lock(&hb->lock);
  708. hlist_for_each_entry(req, &hb->list, hash_node) {
  709. if (cd->data != req->cqe.user_data)
  710. continue;
  711. if (poll_only && req->opcode != IORING_OP_POLL_ADD)
  712. continue;
  713. if (cd->flags & IORING_ASYNC_CANCEL_ALL) {
  714. if (io_cancel_match_sequence(req, cd->seq))
  715. continue;
  716. }
  717. *out_bucket = hb;
  718. return req;
  719. }
  720. spin_unlock(&hb->lock);
  721. return NULL;
  722. }
  723. static struct io_kiocb *io_poll_file_find(struct io_ring_ctx *ctx,
  724. struct io_cancel_data *cd,
  725. struct io_hash_table *table,
  726. struct io_hash_bucket **out_bucket)
  727. {
  728. unsigned nr_buckets = 1U << table->hash_bits;
  729. struct io_kiocb *req;
  730. int i;
  731. *out_bucket = NULL;
  732. for (i = 0; i < nr_buckets; i++) {
  733. struct io_hash_bucket *hb = &table->hbs[i];
  734. spin_lock(&hb->lock);
  735. hlist_for_each_entry(req, &hb->list, hash_node) {
  736. if (io_cancel_req_match(req, cd)) {
  737. *out_bucket = hb;
  738. return req;
  739. }
  740. }
  741. spin_unlock(&hb->lock);
  742. }
  743. return NULL;
  744. }
  745. static int io_poll_disarm(struct io_kiocb *req)
  746. {
  747. if (!req)
  748. return -ENOENT;
  749. if (!io_poll_get_ownership(req))
  750. return -EALREADY;
  751. io_poll_remove_entries(req);
  752. hash_del(&req->hash_node);
  753. return 0;
  754. }
  755. static int __io_poll_cancel(struct io_ring_ctx *ctx, struct io_cancel_data *cd,
  756. struct io_hash_table *table)
  757. {
  758. struct io_hash_bucket *bucket;
  759. struct io_kiocb *req;
  760. if (cd->flags & (IORING_ASYNC_CANCEL_FD | IORING_ASYNC_CANCEL_OP |
  761. IORING_ASYNC_CANCEL_ANY))
  762. req = io_poll_file_find(ctx, cd, table, &bucket);
  763. else
  764. req = io_poll_find(ctx, false, cd, table, &bucket);
  765. if (req)
  766. io_poll_cancel_req(req);
  767. if (bucket)
  768. spin_unlock(&bucket->lock);
  769. return req ? 0 : -ENOENT;
  770. }
  771. int io_poll_cancel(struct io_ring_ctx *ctx, struct io_cancel_data *cd,
  772. unsigned issue_flags)
  773. {
  774. int ret;
  775. ret = __io_poll_cancel(ctx, cd, &ctx->cancel_table);
  776. if (ret != -ENOENT)
  777. return ret;
  778. io_ring_submit_lock(ctx, issue_flags);
  779. ret = __io_poll_cancel(ctx, cd, &ctx->cancel_table_locked);
  780. io_ring_submit_unlock(ctx, issue_flags);
  781. return ret;
  782. }
  783. static __poll_t io_poll_parse_events(const struct io_uring_sqe *sqe,
  784. unsigned int flags)
  785. {
  786. u32 events;
  787. events = READ_ONCE(sqe->poll32_events);
  788. #ifdef __BIG_ENDIAN
  789. events = swahw32(events);
  790. #endif
  791. if (!(flags & IORING_POLL_ADD_MULTI))
  792. events |= EPOLLONESHOT;
  793. if (!(flags & IORING_POLL_ADD_LEVEL))
  794. events |= EPOLLET;
  795. return demangle_poll(events) |
  796. (events & (EPOLLEXCLUSIVE|EPOLLONESHOT|EPOLLET));
  797. }
  798. int io_poll_remove_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
  799. {
  800. struct io_poll_update *upd = io_kiocb_to_cmd(req, struct io_poll_update);
  801. u32 flags;
  802. if (sqe->buf_index || sqe->splice_fd_in)
  803. return -EINVAL;
  804. flags = READ_ONCE(sqe->len);
  805. if (flags & ~(IORING_POLL_UPDATE_EVENTS | IORING_POLL_UPDATE_USER_DATA |
  806. IORING_POLL_ADD_MULTI))
  807. return -EINVAL;
  808. /* meaningless without update */
  809. if (flags == IORING_POLL_ADD_MULTI)
  810. return -EINVAL;
  811. upd->old_user_data = READ_ONCE(sqe->addr);
  812. upd->update_events = flags & IORING_POLL_UPDATE_EVENTS;
  813. upd->update_user_data = flags & IORING_POLL_UPDATE_USER_DATA;
  814. upd->new_user_data = READ_ONCE(sqe->off);
  815. if (!upd->update_user_data && upd->new_user_data)
  816. return -EINVAL;
  817. if (upd->update_events)
  818. upd->events = io_poll_parse_events(sqe, flags);
  819. else if (sqe->poll32_events)
  820. return -EINVAL;
  821. return 0;
  822. }
  823. int io_poll_add_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
  824. {
  825. struct io_poll *poll = io_kiocb_to_cmd(req, struct io_poll);
  826. u32 flags;
  827. if (sqe->buf_index || sqe->off || sqe->addr)
  828. return -EINVAL;
  829. flags = READ_ONCE(sqe->len);
  830. if (flags & ~IORING_POLL_ADD_MULTI)
  831. return -EINVAL;
  832. if ((flags & IORING_POLL_ADD_MULTI) && (req->flags & REQ_F_CQE_SKIP))
  833. return -EINVAL;
  834. poll->events = io_poll_parse_events(sqe, flags);
  835. return 0;
  836. }
  837. int io_poll_add(struct io_kiocb *req, unsigned int issue_flags)
  838. {
  839. struct io_poll *poll = io_kiocb_to_cmd(req, struct io_poll);
  840. struct io_poll_table ipt;
  841. int ret;
  842. ipt.pt._qproc = io_poll_queue_proc;
  843. /*
  844. * If sqpoll or single issuer, there is no contention for ->uring_lock
  845. * and we'll end up holding it in tw handlers anyway.
  846. */
  847. if (req->ctx->flags & (IORING_SETUP_SQPOLL|IORING_SETUP_SINGLE_ISSUER))
  848. req->flags |= REQ_F_HASH_LOCKED;
  849. ret = __io_arm_poll_handler(req, poll, &ipt, poll->events, issue_flags);
  850. if (ret > 0) {
  851. io_req_set_res(req, ipt.result_mask, 0);
  852. return IOU_OK;
  853. }
  854. return ret ?: IOU_ISSUE_SKIP_COMPLETE;
  855. }
  856. int io_poll_remove(struct io_kiocb *req, unsigned int issue_flags)
  857. {
  858. struct io_poll_update *poll_update = io_kiocb_to_cmd(req, struct io_poll_update);
  859. struct io_ring_ctx *ctx = req->ctx;
  860. struct io_cancel_data cd = { .ctx = ctx, .data = poll_update->old_user_data, };
  861. struct io_hash_bucket *bucket;
  862. struct io_kiocb *preq;
  863. int ret2, ret = 0;
  864. io_ring_submit_lock(ctx, issue_flags);
  865. preq = io_poll_find(ctx, true, &cd, &ctx->cancel_table, &bucket);
  866. ret2 = io_poll_disarm(preq);
  867. if (bucket)
  868. spin_unlock(&bucket->lock);
  869. if (!ret2)
  870. goto found;
  871. if (ret2 != -ENOENT) {
  872. ret = ret2;
  873. goto out;
  874. }
  875. preq = io_poll_find(ctx, true, &cd, &ctx->cancel_table_locked, &bucket);
  876. ret2 = io_poll_disarm(preq);
  877. if (bucket)
  878. spin_unlock(&bucket->lock);
  879. if (ret2) {
  880. ret = ret2;
  881. goto out;
  882. }
  883. found:
  884. if (WARN_ON_ONCE(preq->opcode != IORING_OP_POLL_ADD)) {
  885. ret = -EFAULT;
  886. goto out;
  887. }
  888. if (poll_update->update_events || poll_update->update_user_data) {
  889. /* only mask one event flags, keep behavior flags */
  890. if (poll_update->update_events) {
  891. struct io_poll *poll = io_kiocb_to_cmd(preq, struct io_poll);
  892. poll->events &= ~0xffff;
  893. poll->events |= poll_update->events & 0xffff;
  894. poll->events |= IO_POLL_UNMASK;
  895. }
  896. if (poll_update->update_user_data)
  897. preq->cqe.user_data = poll_update->new_user_data;
  898. ret2 = io_poll_add(preq, issue_flags & ~IO_URING_F_UNLOCKED);
  899. /* successfully updated, don't complete poll request */
  900. if (!ret2 || ret2 == -EIOCBQUEUED)
  901. goto out;
  902. }
  903. req_set_fail(preq);
  904. io_req_set_res(preq, -ECANCELED, 0);
  905. preq->io_task_work.func = io_req_task_complete;
  906. io_req_task_work_add(preq);
  907. out:
  908. io_ring_submit_unlock(ctx, issue_flags);
  909. if (ret < 0) {
  910. req_set_fail(req);
  911. return ret;
  912. }
  913. /* complete update request, we're done with it */
  914. io_req_set_res(req, ret, 0);
  915. return IOU_OK;
  916. }