Whamcloud - gitweb
LU-12624 lod: alloc dir stripes by QoS
[fs/lustre-release.git] / lustre / ldlm / ldlm_lockd.c
1 /*
2  * GPL HEADER START
3  *
4  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
5  *
6  * This program is free software; you can redistribute it and/or modify
7  * it under the terms of the GNU General Public License version 2 only,
8  * as published by the Free Software Foundation.
9  *
10  * This program is distributed in the hope that it will be useful, but
11  * WITHOUT ANY WARRANTY; without even the implied warranty of
12  * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
13  * General Public License version 2 for more details (a copy is included
14  * in the LICENSE file that accompanied this code).
15  *
16  * You should have received a copy of the GNU General Public License
17  * version 2 along with this program; If not, see
18  * http://www.gnu.org/licenses/gpl-2.0.html
19  *
20  * GPL HEADER END
21  */
22 /*
23  * Copyright (c) 2002, 2010, Oracle and/or its affiliates. All rights reserved.
24  * Use is subject to license terms.
25  *
26  * Copyright (c) 2010, 2017, Intel Corporation.
27  */
28 /*
29  * This file is part of Lustre, http://www.lustre.org/
30  * Lustre is a trademark of Sun Microsystems, Inc.
31  *
32  * lustre/ldlm/ldlm_lockd.c
33  *
34  * Author: Peter Braam <braam@clusterfs.com>
35  * Author: Phil Schwan <phil@clusterfs.com>
36  */
37
38 #define DEBUG_SUBSYSTEM S_LDLM
39
40 #include <linux/kthread.h>
41 #include <linux/list.h>
42 #include <libcfs/libcfs.h>
43 #include <lustre_errno.h>
44 #include <lustre_dlm.h>
45 #include <obd_class.h>
46 #include "ldlm_internal.h"
47
48 static int ldlm_num_threads;
49 module_param(ldlm_num_threads, int, 0444);
50 MODULE_PARM_DESC(ldlm_num_threads, "number of DLM service threads to start");
51
52 static unsigned int ldlm_cpu_bind = 1;
53 module_param(ldlm_cpu_bind, uint, 0444);
54 MODULE_PARM_DESC(ldlm_cpu_bind,
55                  "bind DLM service threads to particular CPU partitions");
56
57 static char *ldlm_cpts;
58 module_param(ldlm_cpts, charp, 0444);
59 MODULE_PARM_DESC(ldlm_cpts, "CPU partitions ldlm threads should run on");
60
61 static DEFINE_MUTEX(ldlm_ref_mutex);
62 static int ldlm_refcount;
63
64 struct kobject *ldlm_kobj;
65 struct kset *ldlm_ns_kset;
66 struct kset *ldlm_svc_kset;
67
68 /* LDLM state */
69
70 static struct ldlm_state *ldlm_state;
71
72 /*
73  * timeout for initial callback (AST) reply (bz10399)
74  * Due to having to send a 32 bit time value over the
75  * wire return it as time_t instead of time64_t
76  */
77 static inline time_t ldlm_get_rq_timeout(void)
78 {
79         /* Non-AT value */
80         time_t timeout = min(ldlm_timeout, obd_timeout / 3);
81
82         return timeout < 1 ? 1 : timeout;
83 }
84
85 struct ldlm_bl_pool {
86         spinlock_t blp_lock;
87
88         /*
89          * blp_prio_list is used for callbacks that should be handled
90          * as a priority. It is used for LDLM_FL_DISCARD_DATA requests.
91          * see b=13843
92          */
93         struct list_head blp_prio_list;
94
95         /*
96          * blp_list is used for all other callbacks which are likely
97          * to take longer to process.
98          */
99         struct list_head blp_list;
100
101         wait_queue_head_t blp_waitq;
102         struct completion blp_comp;
103         atomic_t blp_num_threads;
104         atomic_t blp_busy_threads;
105         int blp_min_threads;
106         int blp_max_threads;
107 };
108
109 struct ldlm_bl_work_item {
110         struct list_head        blwi_entry;
111         struct ldlm_namespace   *blwi_ns;
112         struct ldlm_lock_desc   blwi_ld;
113         struct ldlm_lock        *blwi_lock;
114         struct list_head        blwi_head;
115         int                     blwi_count;
116         struct completion       blwi_comp;
117         enum ldlm_cancel_flags  blwi_flags;
118         int                     blwi_mem_pressure;
119 };
120
121 #ifdef HAVE_SERVER_SUPPORT
122
123 /**
124  * Protects both waiting_locks_list and expired_lock_thread.
125  */
126 static DEFINE_SPINLOCK(waiting_locks_spinlock); /* BH lock (timer) */
127
128 /**
129  * List for contended locks.
130  *
131  * As soon as a lock is contended, it gets placed on this list and
132  * expected time to get a response is filled in the lock. A special
133  * thread walks the list looking for locks that should be released and
134  * schedules client evictions for those that have not been released in
135  * time.
136  *
137  * All access to it should be under waiting_locks_spinlock.
138  */
139 static LIST_HEAD(waiting_locks_list);
140 static void waiting_locks_callback(TIMER_DATA_TYPE unused);
141 static CFS_DEFINE_TIMER(waiting_locks_timer, waiting_locks_callback, 0, 0);
142
143 enum elt_state {
144         ELT_STOPPED,
145         ELT_READY,
146         ELT_TERMINATE,
147 };
148
149 static DECLARE_WAIT_QUEUE_HEAD(expired_lock_wait_queue);
150 static enum elt_state expired_lock_thread_state = ELT_STOPPED;
151 static int expired_lock_dump;
152 static LIST_HEAD(expired_lock_list);
153
154 static int ldlm_lock_busy(struct ldlm_lock *lock);
155 static int ldlm_add_waiting_lock(struct ldlm_lock *lock, time64_t timeout);
156 static int __ldlm_add_waiting_lock(struct ldlm_lock *lock, time64_t seconds);
157
158 static inline int have_expired_locks(void)
159 {
160         int need_to_run;
161
162         ENTRY;
163         spin_lock_bh(&waiting_locks_spinlock);
164         need_to_run = !list_empty(&expired_lock_list);
165         spin_unlock_bh(&waiting_locks_spinlock);
166
167         RETURN(need_to_run);
168 }
169
170 /**
171  * Check expired lock list for expired locks and time them out.
172  */
173 static int expired_lock_main(void *arg)
174 {
175         struct list_head *expired = &expired_lock_list;
176         struct l_wait_info lwi = { 0 };
177         int do_dump;
178
179         ENTRY;
180
181         expired_lock_thread_state = ELT_READY;
182         wake_up(&expired_lock_wait_queue);
183
184         while (1) {
185                 l_wait_event(expired_lock_wait_queue,
186                              have_expired_locks() ||
187                              expired_lock_thread_state == ELT_TERMINATE,
188                              &lwi);
189
190                 spin_lock_bh(&waiting_locks_spinlock);
191                 if (expired_lock_dump) {
192                         spin_unlock_bh(&waiting_locks_spinlock);
193
194                         /* from waiting_locks_callback, but not in timer */
195                         libcfs_debug_dumplog();
196
197                         spin_lock_bh(&waiting_locks_spinlock);
198                         expired_lock_dump = 0;
199                 }
200
201                 do_dump = 0;
202
203                 while (!list_empty(expired)) {
204                         struct obd_export *export;
205                         struct ldlm_lock *lock;
206
207                         lock = list_entry(expired->next, struct ldlm_lock,
208                                           l_pending_chain);
209                         if ((void *)lock < LP_POISON + PAGE_SIZE &&
210                             (void *)lock >= LP_POISON) {
211                                 spin_unlock_bh(&waiting_locks_spinlock);
212                                 CERROR("free lock on elt list %p\n", lock);
213                                 LBUG();
214                         }
215                         list_del_init(&lock->l_pending_chain);
216                         if ((void *)lock->l_export <
217                              LP_POISON + PAGE_SIZE &&
218                             (void *)lock->l_export >= LP_POISON) {
219                                 CERROR("lock with free export on elt list %p\n",
220                                        lock->l_export);
221                                 lock->l_export = NULL;
222                                 LDLM_ERROR(lock, "free export");
223                                 /*
224                                  * release extra ref grabbed by
225                                  * ldlm_add_waiting_lock() or
226                                  * ldlm_failed_ast()
227                                  */
228                                 LDLM_LOCK_RELEASE(lock);
229                                 continue;
230                         }
231
232                         if (ldlm_is_destroyed(lock)) {
233                                 /*
234                                  * release the lock refcount where
235                                  * waiting_locks_callback() founds
236                                  */
237                                 LDLM_LOCK_RELEASE(lock);
238                                 continue;
239                         }
240                         export = class_export_lock_get(lock->l_export, lock);
241                         spin_unlock_bh(&waiting_locks_spinlock);
242
243                         /* Check if we need to prolong timeout */
244                         if (!OBD_FAIL_CHECK(OBD_FAIL_PTLRPC_HPREQ_TIMEOUT) &&
245                             lock->l_callback_timeout != 0 && /* not AST error */
246                             ldlm_lock_busy(lock)) {
247                                 LDLM_DEBUG(lock, "prolong the busy lock");
248                                 lock_res_and_lock(lock);
249                                 ldlm_add_waiting_lock(lock,
250                                                 ldlm_bl_timeout(lock) >> 1);
251                                 unlock_res_and_lock(lock);
252                         } else {
253                                 spin_lock_bh(&export->exp_bl_list_lock);
254                                 list_del_init(&lock->l_exp_list);
255                                 spin_unlock_bh(&export->exp_bl_list_lock);
256
257                                 LDLM_ERROR(lock,
258                                            "lock callback timer expired after %llds: evicting client at %s ",
259                                            ktime_get_real_seconds() -
260                                            lock->l_blast_sent,
261                                            obd_export_nid2str(export));
262                                 ldlm_lock_to_ns(lock)->ns_timeouts++;
263                                 do_dump++;
264                                 class_fail_export(export);
265                         }
266                         class_export_lock_put(export, lock);
267                         /*
268                          * release extra ref grabbed by ldlm_add_waiting_lock()
269                          * or ldlm_failed_ast()
270                          */
271                         LDLM_LOCK_RELEASE(lock);
272
273                         spin_lock_bh(&waiting_locks_spinlock);
274                 }
275                 spin_unlock_bh(&waiting_locks_spinlock);
276
277                 if (do_dump && obd_dump_on_eviction) {
278                         CERROR("dump the log upon eviction\n");
279                         libcfs_debug_dumplog();
280                 }
281
282                 if (expired_lock_thread_state == ELT_TERMINATE)
283                         break;
284         }
285
286         expired_lock_thread_state = ELT_STOPPED;
287         wake_up(&expired_lock_wait_queue);
288         RETURN(0);
289 }
290
291 /**
292  * Check if there is a request in the export request list
293  * which prevents the lock canceling.
294  */
295 static int ldlm_lock_busy(struct ldlm_lock *lock)
296 {
297         struct ptlrpc_request *req;
298         int match = 0;
299
300         ENTRY;
301
302         if (lock->l_export == NULL)
303                 return 0;
304
305         spin_lock(&lock->l_export->exp_rpc_lock);
306         list_for_each_entry(req, &lock->l_export->exp_hp_rpcs,
307                                 rq_exp_list) {
308                 if (req->rq_ops->hpreq_lock_match) {
309                         match = req->rq_ops->hpreq_lock_match(req, lock);
310                         if (match)
311                                 break;
312                 }
313         }
314         spin_unlock(&lock->l_export->exp_rpc_lock);
315         RETURN(match);
316 }
317
318 /* This is called from within a timer interrupt and cannot schedule */
319 static void waiting_locks_callback(TIMER_DATA_TYPE unused)
320 {
321         struct ldlm_lock *lock;
322         int need_dump = 0;
323
324         spin_lock_bh(&waiting_locks_spinlock);
325         while (!list_empty(&waiting_locks_list)) {
326                 lock = list_entry(waiting_locks_list.next, struct ldlm_lock,
327                                   l_pending_chain);
328                 if (lock->l_callback_timeout > ktime_get_seconds() ||
329                     lock->l_req_mode == LCK_GROUP)
330                         break;
331
332                 /*
333                  * no needs to take an extra ref on the lock since it was in
334                  * the waiting_locks_list and ldlm_add_waiting_lock()
335                  * already grabbed a ref
336                  */
337                 list_del(&lock->l_pending_chain);
338                 list_add(&lock->l_pending_chain, &expired_lock_list);
339                 need_dump = 1;
340         }
341
342         if (!list_empty(&expired_lock_list)) {
343                 if (obd_dump_on_timeout && need_dump)
344                         expired_lock_dump = __LINE__;
345
346                 wake_up(&expired_lock_wait_queue);
347         }
348
349         /*
350          * Make sure the timer will fire again if we have any locks
351          * left.
352          */
353         if (!list_empty(&waiting_locks_list)) {
354                 unsigned long timeout_jiffies;
355
356                 lock = list_entry(waiting_locks_list.next, struct ldlm_lock,
357                                   l_pending_chain);
358                 timeout_jiffies = cfs_time_seconds(lock->l_callback_timeout);
359                 mod_timer(&waiting_locks_timer, timeout_jiffies);
360         }
361         spin_unlock_bh(&waiting_locks_spinlock);
362 }
363
364 /**
365  * Add lock to the list of contended locks.
366  *
367  * Indicate that we're waiting for a client to call us back cancelling a given
368  * lock.  We add it to the pending-callback chain, and schedule the lock-timeout
369  * timer to fire appropriately.  (We round up to the next second, to avoid
370  * floods of timer firings during periods of high lock contention and traffic).
371  * As done by ldlm_add_waiting_lock(), the caller must grab a lock reference
372  * if it has been added to the waiting list (1 is returned).
373  *
374  * Called with the namespace lock held.
375  */
376 static int __ldlm_add_waiting_lock(struct ldlm_lock *lock, time64_t seconds)
377 {
378         unsigned long timeout_jiffies;
379         time64_t timeout;
380
381         if (!list_empty(&lock->l_pending_chain))
382                 return 0;
383
384         if (OBD_FAIL_CHECK(OBD_FAIL_PTLRPC_HPREQ_NOTIMEOUT) ||
385             OBD_FAIL_CHECK(OBD_FAIL_PTLRPC_HPREQ_TIMEOUT))
386                 seconds = 1;
387
388         timeout = ktime_get_seconds() + seconds;
389         if (likely(timeout > lock->l_callback_timeout))
390                 lock->l_callback_timeout = timeout;
391
392         timeout_jiffies = cfs_time_seconds(lock->l_callback_timeout);
393
394         if (time_before(timeout_jiffies, waiting_locks_timer.expires) ||
395             !timer_pending(&waiting_locks_timer))
396                 mod_timer(&waiting_locks_timer, timeout_jiffies);
397
398         /*
399          * if the new lock has a shorter timeout than something earlier on
400          * the list, we'll wait the longer amount of time; no big deal.
401          */
402         /* FIFO */
403         list_add_tail(&lock->l_pending_chain, &waiting_locks_list);
404         return 1;
405 }
406
407 static void ldlm_add_blocked_lock(struct ldlm_lock *lock)
408 {
409         spin_lock_bh(&lock->l_export->exp_bl_list_lock);
410         if (list_empty(&lock->l_exp_list)) {
411                 if (!ldlm_is_granted(lock))
412                         list_add_tail(&lock->l_exp_list,
413                                       &lock->l_export->exp_bl_list);
414                 else
415                         list_add(&lock->l_exp_list,
416                                  &lock->l_export->exp_bl_list);
417         }
418         spin_unlock_bh(&lock->l_export->exp_bl_list_lock);
419
420         /*
421          * A blocked lock is added. Adjust the position in
422          * the stale list if the export is in the list.
423          * If export is stale and not in the list - it is being
424          * processed and will be placed on the right position
425          * on obd_stale_export_put().
426          */
427         if (!list_empty(&lock->l_export->exp_stale_list))
428                 obd_stale_export_adjust(lock->l_export);
429 }
430
431 static int ldlm_add_waiting_lock(struct ldlm_lock *lock, time64_t timeout)
432 {
433         int ret;
434
435         /* NB: must be called with hold of lock_res_and_lock() */
436         LASSERT(ldlm_is_res_locked(lock));
437         LASSERT(!ldlm_is_cancel_on_block(lock));
438
439         /*
440          * Do not put cross-MDT lock in the waiting list, since we
441          * will not evict it due to timeout for now
442          */
443         if (lock->l_export != NULL &&
444             (exp_connect_flags(lock->l_export) & OBD_CONNECT_MDS_MDS))
445                 return 0;
446
447         spin_lock_bh(&waiting_locks_spinlock);
448         if (ldlm_is_cancel(lock)) {
449                 spin_unlock_bh(&waiting_locks_spinlock);
450                 return 0;
451         }
452
453         if (ldlm_is_destroyed(lock)) {
454                 static time64_t next;
455
456                 spin_unlock_bh(&waiting_locks_spinlock);
457                 LDLM_ERROR(lock, "not waiting on destroyed lock (b=5653)");
458                 if (ktime_get_seconds() > next) {
459                         next = ktime_get_seconds() + 14400;
460                         libcfs_debug_dumpstack(NULL);
461                 }
462                 return 0;
463         }
464
465         ldlm_set_waited(lock);
466         lock->l_blast_sent = ktime_get_real_seconds();
467         ret = __ldlm_add_waiting_lock(lock, timeout);
468         if (ret) {
469                 /*
470                  * grab ref on the lock if it has been added to the
471                  * waiting list
472                  */
473                 LDLM_LOCK_GET(lock);
474         }
475         spin_unlock_bh(&waiting_locks_spinlock);
476
477         if (ret)
478                 ldlm_add_blocked_lock(lock);
479
480         LDLM_DEBUG(lock, "%sadding to wait list(timeout: %lld, AT: %s)",
481                    ret == 0 ? "not re-" : "", timeout,
482                    AT_OFF ? "off" : "on");
483         return ret;
484 }
485
486 /**
487  * Remove a lock from the pending list, likely because it had its cancellation
488  * callback arrive without incident.  This adjusts the lock-timeout timer if
489  * needed.  Returns 0 if the lock wasn't pending after all, 1 if it was.
490  * As done by ldlm_del_waiting_lock(), the caller must release the lock
491  * reference when the lock is removed from any list (1 is returned).
492  *
493  * Called with namespace lock held.
494  */
495 static int __ldlm_del_waiting_lock(struct ldlm_lock *lock)
496 {
497         struct list_head *list_next;
498
499         if (list_empty(&lock->l_pending_chain))
500                 return 0;
501
502         list_next = lock->l_pending_chain.next;
503         if (lock->l_pending_chain.prev == &waiting_locks_list) {
504                 /* Removing the head of the list, adjust timer. */
505                 if (list_next == &waiting_locks_list) {
506                         /* No more, just cancel. */
507                         del_timer(&waiting_locks_timer);
508                 } else {
509                         struct ldlm_lock *next;
510
511                         next = list_entry(list_next, struct ldlm_lock,
512                                           l_pending_chain);
513                         mod_timer(&waiting_locks_timer,
514                                   cfs_time_seconds(next->l_callback_timeout));
515                 }
516         }
517         list_del_init(&lock->l_pending_chain);
518
519         return 1;
520 }
521
522 int ldlm_del_waiting_lock(struct ldlm_lock *lock)
523 {
524         int ret;
525
526         if (lock->l_export == NULL) {
527                 /* We don't have a "waiting locks list" on clients. */
528                 CDEBUG(D_DLMTRACE, "Client lock %p : no-op\n", lock);
529                 return 0;
530         }
531
532         spin_lock_bh(&waiting_locks_spinlock);
533         ret = __ldlm_del_waiting_lock(lock);
534         ldlm_clear_waited(lock);
535         spin_unlock_bh(&waiting_locks_spinlock);
536
537         /* remove the lock out of export blocking list */
538         spin_lock_bh(&lock->l_export->exp_bl_list_lock);
539         list_del_init(&lock->l_exp_list);
540         spin_unlock_bh(&lock->l_export->exp_bl_list_lock);
541
542         if (ret) {
543                 /*
544                  * release lock ref if it has indeed been removed
545                  * from a list
546                  */
547                 LDLM_LOCK_RELEASE(lock);
548         }
549
550         LDLM_DEBUG(lock, "%s", ret == 0 ? "wasn't waiting" : "removed");
551         return ret;
552 }
553
554 /**
555  * Prolong the contended lock waiting time.
556  *
557  * Called with namespace lock held.
558  */
559 int ldlm_refresh_waiting_lock(struct ldlm_lock *lock, time64_t timeout)
560 {
561         if (lock->l_export == NULL) {
562                 /* We don't have a "waiting locks list" on clients. */
563                 LDLM_DEBUG(lock, "client lock: no-op");
564                 return 0;
565         }
566
567         if (exp_connect_flags(lock->l_export) & OBD_CONNECT_MDS_MDS) {
568                 /* We don't have a "waiting locks list" on OSP. */
569                 LDLM_DEBUG(lock, "MDS-MDS lock: no-op");
570                 return 0;
571         }
572
573         spin_lock_bh(&waiting_locks_spinlock);
574
575         if (list_empty(&lock->l_pending_chain)) {
576                 spin_unlock_bh(&waiting_locks_spinlock);
577                 LDLM_DEBUG(lock, "wasn't waiting");
578                 return 0;
579         }
580
581         /*
582          * we remove/add the lock to the waiting list, so no needs to
583          * release/take a lock reference
584          */
585         __ldlm_del_waiting_lock(lock);
586         __ldlm_add_waiting_lock(lock, timeout);
587         spin_unlock_bh(&waiting_locks_spinlock);
588
589         LDLM_DEBUG(lock, "refreshed");
590         return 1;
591 }
592 EXPORT_SYMBOL(ldlm_refresh_waiting_lock);
593
594 #else /* HAVE_SERVER_SUPPORT */
595
596 int ldlm_del_waiting_lock(struct ldlm_lock *lock)
597 {
598         RETURN(0);
599 }
600
601 int ldlm_refresh_waiting_lock(struct ldlm_lock *lock, time64_t timeout)
602 {
603         RETURN(0);
604 }
605
606 #endif /* !HAVE_SERVER_SUPPORT */
607
608 #ifdef HAVE_SERVER_SUPPORT
609
610 /**
611  * Calculate the per-export Blocking timeout (covering BL AST, data flush,
612  * lock cancel, and their replies). Used for lock callback timeout and AST
613  * re-send period.
614  *
615  * \param[in] lock        lock which is getting the blocking callback
616  *
617  * \retval            timeout in seconds to wait for the client reply
618  */
619 time64_t ldlm_bl_timeout(struct ldlm_lock *lock)
620 {
621         time64_t timeout;
622
623         if (AT_OFF)
624                 return obd_timeout / 2;
625
626         /*
627          * Since these are non-updating timeouts, we should be conservative.
628          * Take more than usually, 150%
629          * It would be nice to have some kind of "early reply" mechanism for
630          * lock callbacks too...
631          */
632         timeout = at_get(&lock->l_export->exp_bl_lock_at);
633         return max(timeout + (timeout >> 1), (time64_t)ldlm_enqueue_min);
634 }
635 EXPORT_SYMBOL(ldlm_bl_timeout);
636
637 /**
638  * Perform lock cleanup if AST sending failed.
639  */
640 static void ldlm_failed_ast(struct ldlm_lock *lock, int rc,
641                             const char *ast_type)
642 {
643         LCONSOLE_ERROR_MSG(0x138,
644                            "%s: A client on nid %s was evicted due to a lock %s callback time out: rc %d\n",
645                            lock->l_export->exp_obd->obd_name,
646                            obd_export_nid2str(lock->l_export), ast_type, rc);
647
648         if (obd_dump_on_timeout)
649                 libcfs_debug_dumplog();
650         spin_lock_bh(&waiting_locks_spinlock);
651         if (__ldlm_del_waiting_lock(lock) == 0)
652                 /*
653                  * the lock was not in any list, grab an extra ref before adding
654                  * the lock to the expired list
655                  */
656                 LDLM_LOCK_GET(lock);
657         lock->l_callback_timeout = 0; /* differentiate it from expired locks */
658         list_add(&lock->l_pending_chain, &expired_lock_list);
659         wake_up(&expired_lock_wait_queue);
660         spin_unlock_bh(&waiting_locks_spinlock);
661 }
662
663 /**
664  * Perform lock cleanup if AST reply came with error.
665  */
666 static int ldlm_handle_ast_error(struct ldlm_lock *lock,
667                                  struct ptlrpc_request *req, int rc,
668                                  const char *ast_type)
669 {
670         struct lnet_process_id peer = req->rq_import->imp_connection->c_peer;
671
672         if (!req->rq_replied || (rc && rc != -EINVAL)) {
673                 if (ldlm_is_cancel(lock)) {
674                         LDLM_DEBUG(lock,
675                                    "%s AST (req@%p x%llu) timeout from nid %s, but cancel was received (AST reply lost?)",
676                                    ast_type, req, req->rq_xid,
677                                    libcfs_nid2str(peer.nid));
678                         ldlm_lock_cancel(lock);
679                         rc = -ERESTART;
680                 } else if (rc == -ENODEV || rc == -ESHUTDOWN ||
681                            (rc == -EIO &&
682                             req->rq_import->imp_state == LUSTRE_IMP_CLOSED)) {
683                         /*
684                          * Upon umount process the AST fails because cannot be
685                          * sent. This shouldn't lead to the client eviction.
686                          * -ENODEV error is returned by ptl_send_rpc() for
687                          *  new request in such import.
688                          * -SHUTDOWN is returned by ptlrpc_import_delay_req()
689                          *  if imp_invalid is set or obd_no_recov.
690                          * Meanwhile there is also check for LUSTRE_IMP_CLOSED
691                          * in ptlrpc_import_delay_req() as well with -EIO code.
692                          * In all such cases errors are ignored.
693                          */
694                         LDLM_DEBUG(lock,
695                                    "%s AST can't be sent due to a server %s failure or umount process: rc = %d\n",
696                                     ast_type,
697                                      req->rq_import->imp_obd->obd_name, rc);
698                 } else {
699                         LDLM_ERROR(lock,
700                                    "client (nid %s) %s %s AST (req@%p x%llu status %d rc %d), evict it",
701                                    libcfs_nid2str(peer.nid),
702                                    req->rq_replied ? "returned error from" :
703                                    "failed to reply to",
704                                    ast_type, req, req->rq_xid,
705                                    (req->rq_repmsg != NULL) ?
706                                    lustre_msg_get_status(req->rq_repmsg) : 0,
707                                    rc);
708                         ldlm_failed_ast(lock, rc, ast_type);
709                 }
710                 return rc;
711         }
712
713         if (rc == -EINVAL) {
714                 struct ldlm_resource *res = lock->l_resource;
715
716                 LDLM_DEBUG(lock,
717                            "client (nid %s) returned %d from %s AST (req@%p x%llu) - normal race",
718                            libcfs_nid2str(peer.nid),
719                            req->rq_repmsg ?
720                            lustre_msg_get_status(req->rq_repmsg) : -1,
721                            ast_type, req, req->rq_xid);
722                 if (res) {
723                         /*
724                          * update lvbo to return proper attributes.
725                          * see b=23174
726                          */
727                         ldlm_resource_getref(res);
728                         ldlm_lvbo_update(res, lock, NULL, 1);
729                         ldlm_resource_putref(res);
730                 }
731                 ldlm_lock_cancel(lock);
732                 rc = -ERESTART;
733         }
734
735         return rc;
736 }
737
738 static int ldlm_cb_interpret(const struct lu_env *env,
739                              struct ptlrpc_request *req, void *args, int rc)
740 {
741         struct ldlm_cb_async_args *ca = args;
742         struct ldlm_lock *lock = ca->ca_lock;
743         struct ldlm_cb_set_arg *arg  = ca->ca_set_arg;
744
745         ENTRY;
746
747         LASSERT(lock != NULL);
748
749         switch (arg->type) {
750         case LDLM_GL_CALLBACK:
751                 /*
752                  * Update the LVB from disk if the AST failed
753                  * (this is a legal race)
754                  *
755                  * - Glimpse callback of local lock just returns
756                  *   -ELDLM_NO_LOCK_DATA.
757                  * - Glimpse callback of remote lock might return
758                  *   -ELDLM_NO_LOCK_DATA when inode is cleared. LU-274
759                  */
760                 if (unlikely(arg->gl_interpret_reply)) {
761                         rc = arg->gl_interpret_reply(NULL, req, args, rc);
762                 } else if (rc == -ELDLM_NO_LOCK_DATA) {
763                         LDLM_DEBUG(lock,
764                                    "lost race - client has a lock but no inode");
765                         ldlm_lvbo_update(lock->l_resource, lock, NULL, 1);
766                 } else if (rc != 0) {
767                         rc = ldlm_handle_ast_error(lock, req, rc, "glimpse");
768                 } else {
769                         rc = ldlm_lvbo_update(lock->l_resource,
770                                               lock, req, 1);
771                 }
772                 break;
773         case LDLM_BL_CALLBACK:
774                 if (rc != 0)
775                         rc = ldlm_handle_ast_error(lock, req, rc, "blocking");
776                 break;
777         case LDLM_CP_CALLBACK:
778                 if (rc != 0)
779                         rc = ldlm_handle_ast_error(lock, req, rc, "completion");
780                 break;
781         default:
782                 LDLM_ERROR(lock, "invalid opcode for lock callback %d",
783                            arg->type);
784                 LBUG();
785         }
786
787         /* release extra reference taken in ldlm_ast_fini() */
788         LDLM_LOCK_RELEASE(lock);
789
790         if (rc == -ERESTART)
791                 atomic_inc(&arg->restart);
792
793         RETURN(0);
794 }
795
796 static void ldlm_update_resend(struct ptlrpc_request *req, void *data)
797 {
798         struct ldlm_cb_async_args *ca = data;
799         struct ldlm_lock *lock = ca->ca_lock;
800
801         ldlm_refresh_waiting_lock(lock, ldlm_bl_timeout(lock));
802 }
803
804 static inline int ldlm_ast_fini(struct ptlrpc_request *req,
805                                 struct ldlm_cb_set_arg *arg,
806                                 struct ldlm_lock *lock,
807                                 int instant_cancel)
808 {
809         int rc = 0;
810
811         ENTRY;
812
813         if (unlikely(instant_cancel)) {
814                 rc = ptl_send_rpc(req, 1);
815                 ptlrpc_req_finished(req);
816                 if (rc == 0)
817                         atomic_inc(&arg->restart);
818         } else {
819                 LDLM_LOCK_GET(lock);
820                 ptlrpc_set_add_req(arg->set, req);
821         }
822
823         RETURN(rc);
824 }
825
826 /**
827  * Check if there are requests in the export request list which prevent
828  * the lock canceling and make these requests high priority ones.
829  */
830 static void ldlm_lock_reorder_req(struct ldlm_lock *lock)
831 {
832         struct ptlrpc_request *req;
833
834         ENTRY;
835
836         if (lock->l_export == NULL) {
837                 LDLM_DEBUG(lock, "client lock: no-op");
838                 RETURN_EXIT;
839         }
840
841         spin_lock(&lock->l_export->exp_rpc_lock);
842         list_for_each_entry(req, &lock->l_export->exp_hp_rpcs,
843                             rq_exp_list) {
844                 /*
845                  * Do not process requests that were not yet added to there
846                  * incoming queue or were already removed from there for
847                  * processing. We evaluate ptlrpc_nrs_req_can_move() without
848                  * holding svcpt->scp_req_lock, and then redo the check with
849                  * the lock held once we need to obtain a reliable result.
850                  */
851                 if (ptlrpc_nrs_req_can_move(req) &&
852                     req->rq_ops->hpreq_lock_match &&
853                     req->rq_ops->hpreq_lock_match(req, lock))
854                         ptlrpc_nrs_req_hp_move(req);
855         }
856         spin_unlock(&lock->l_export->exp_rpc_lock);
857         EXIT;
858 }
859
860 /**
861  * ->l_blocking_ast() method for server-side locks. This is invoked when newly
862  * enqueued server lock conflicts with given one.
863  *
864  * Sends blocking AST RPC to the client owning that lock; arms timeout timer
865  * to wait for client response.
866  */
867 int ldlm_server_blocking_ast(struct ldlm_lock *lock,
868                              struct ldlm_lock_desc *desc,
869                              void *data, int flag)
870 {
871         struct ldlm_cb_async_args *ca;
872         struct ldlm_cb_set_arg *arg = data;
873         struct ldlm_request *body;
874         struct ptlrpc_request  *req;
875         int instant_cancel = 0;
876         int rc = 0;
877
878         ENTRY;
879
880         if (flag == LDLM_CB_CANCELING)
881                 /* Don't need to do anything here. */
882                 RETURN(0);
883
884         if (OBD_FAIL_PRECHECK(OBD_FAIL_LDLM_SRV_BL_AST)) {
885                 LDLM_DEBUG(lock, "dropping BL AST");
886                 RETURN(0);
887         }
888
889         LASSERT(lock);
890         LASSERT(data != NULL);
891         if (lock->l_export->exp_obd->obd_recovering != 0)
892                 LDLM_ERROR(lock, "BUG 6063: lock collide during recovery");
893
894         ldlm_lock_reorder_req(lock);
895
896         req = ptlrpc_request_alloc_pack(lock->l_export->exp_imp_reverse,
897                                         &RQF_LDLM_BL_CALLBACK,
898                                         LUSTRE_DLM_VERSION, LDLM_BL_CALLBACK);
899         if (req == NULL)
900                 RETURN(-ENOMEM);
901
902         ca = ptlrpc_req_async_args(ca, req);
903         ca->ca_set_arg = arg;
904         ca->ca_lock = lock;
905
906         req->rq_interpret_reply = ldlm_cb_interpret;
907
908         lock_res_and_lock(lock);
909         if (ldlm_is_destroyed(lock)) {
910                 /* What's the point? */
911                 unlock_res_and_lock(lock);
912                 ptlrpc_req_finished(req);
913                 RETURN(0);
914         }
915
916         if (!ldlm_is_granted(lock)) {
917                 /*
918                  * this blocking AST will be communicated as part of the
919                  * completion AST instead
920                  */
921                 ldlm_add_blocked_lock(lock);
922                 ldlm_set_waited(lock);
923                 unlock_res_and_lock(lock);
924
925                 ptlrpc_req_finished(req);
926                 LDLM_DEBUG(lock, "lock not granted, not sending blocking AST");
927                 RETURN(0);
928         }
929
930         if (ldlm_is_cancel_on_block(lock))
931                 instant_cancel = 1;
932
933         body = req_capsule_client_get(&req->rq_pill, &RMF_DLM_REQ);
934         body->lock_handle[0] = lock->l_remote_handle;
935         body->lock_desc = *desc;
936         body->lock_flags |= ldlm_flags_to_wire(lock->l_flags & LDLM_FL_AST_MASK);
937
938         LDLM_DEBUG(lock, "server preparing blocking AST");
939
940         ptlrpc_request_set_replen(req);
941         ldlm_set_cbpending(lock);
942         if (instant_cancel) {
943                 unlock_res_and_lock(lock);
944                 ldlm_lock_cancel(lock);
945
946                 req->rq_no_resend = 1;
947         } else {
948                 LASSERT(ldlm_is_granted(lock));
949                 ldlm_add_waiting_lock(lock, ldlm_bl_timeout(lock));
950                 unlock_res_and_lock(lock);
951
952                 /* Do not resend after lock callback timeout */
953                 req->rq_delay_limit = ldlm_bl_timeout(lock);
954                 req->rq_resend_cb = ldlm_update_resend;
955         }
956
957         req->rq_send_state = LUSTRE_IMP_FULL;
958         /* ptlrpc_request_alloc_pack already set timeout */
959         if (AT_OFF)
960                 req->rq_timeout = ldlm_get_rq_timeout();
961
962         if (lock->l_export && lock->l_export->exp_nid_stats &&
963             lock->l_export->exp_nid_stats->nid_ldlm_stats)
964                 lprocfs_counter_incr(lock->l_export->exp_nid_stats->nid_ldlm_stats,
965                                      LDLM_BL_CALLBACK - LDLM_FIRST_OPC);
966
967         rc = ldlm_ast_fini(req, arg, lock, instant_cancel);
968
969         RETURN(rc);
970 }
971
972 /**
973  * ->l_completion_ast callback for a remote lock in server namespace.
974  *
975  *  Sends AST to the client notifying it of lock granting.  If initial
976  *  lock response was not sent yet, instead of sending another RPC, just
977  *  mark the lock as granted and client will understand
978  */
979 int ldlm_server_completion_ast(struct ldlm_lock *lock, __u64 flags, void *data)
980 {
981         struct ldlm_cb_set_arg *arg = data;
982         struct ldlm_request *body;
983         struct ptlrpc_request *req;
984         struct ldlm_cb_async_args *ca;
985         int instant_cancel = 0;
986         int rc = 0;
987         int lvb_len;
988
989         ENTRY;
990
991         LASSERT(lock != NULL);
992         LASSERT(data != NULL);
993
994         if (OBD_FAIL_PRECHECK(OBD_FAIL_LDLM_SRV_CP_AST)) {
995                 LDLM_DEBUG(lock, "dropping CP AST");
996                 RETURN(0);
997         }
998
999         req = ptlrpc_request_alloc(lock->l_export->exp_imp_reverse,
1000                                    &RQF_LDLM_CP_CALLBACK);
1001         if (req == NULL)
1002                 RETURN(-ENOMEM);
1003
1004         /* server namespace, doesn't need lock */
1005         lvb_len = ldlm_lvbo_size(lock);
1006         /*
1007          * LU-3124 & LU-2187: to not return layout in completion AST because
1008          * it may deadlock for LU-2187, or client may not have enough space
1009          * for large layout. The layout will be returned to client with an
1010          * extra RPC to fetch xattr.lov
1011          */
1012         if (ldlm_has_layout(lock))
1013                 lvb_len = 0;
1014
1015         req_capsule_set_size(&req->rq_pill, &RMF_DLM_LVB, RCL_CLIENT, lvb_len);
1016         rc = ptlrpc_request_pack(req, LUSTRE_DLM_VERSION, LDLM_CP_CALLBACK);
1017         if (rc) {
1018                 ptlrpc_request_free(req);
1019                 RETURN(rc);
1020         }
1021
1022         ca = ptlrpc_req_async_args(ca, req);
1023         ca->ca_set_arg = arg;
1024         ca->ca_lock = lock;
1025
1026         req->rq_interpret_reply = ldlm_cb_interpret;
1027         body = req_capsule_client_get(&req->rq_pill, &RMF_DLM_REQ);
1028
1029         body->lock_handle[0] = lock->l_remote_handle;
1030         body->lock_flags = ldlm_flags_to_wire(flags);
1031         ldlm_lock2desc(lock, &body->lock_desc);
1032         if (lvb_len > 0) {
1033                 void *lvb = req_capsule_client_get(&req->rq_pill, &RMF_DLM_LVB);
1034                 lvb_len = ldlm_lvbo_fill(lock, lvb, &lvb_len);
1035                 if (lvb_len < 0) {
1036                         /*
1037                          * We still need to send the RPC to wake up the blocked
1038                          * enqueue thread on the client.
1039                          *
1040                          * Consider old client, there is no better way to notify
1041                          * the failure, just zero-sized the LVB, then the client
1042                          * will fail out as "-EPROTO".
1043                          */
1044                         req_capsule_shrink(&req->rq_pill, &RMF_DLM_LVB, 0,
1045                                            RCL_CLIENT);
1046                         instant_cancel = 1;
1047                 } else {
1048                         req_capsule_shrink(&req->rq_pill, &RMF_DLM_LVB, lvb_len,
1049                                            RCL_CLIENT);
1050                 }
1051         }
1052
1053         LDLM_DEBUG(lock, "server preparing completion AST");
1054
1055         ptlrpc_request_set_replen(req);
1056
1057         req->rq_send_state = LUSTRE_IMP_FULL;
1058         /* ptlrpc_request_pack already set timeout */
1059         if (AT_OFF)
1060                 req->rq_timeout = ldlm_get_rq_timeout();
1061
1062         /* We only send real blocking ASTs after the lock is granted */
1063         lock_res_and_lock(lock);
1064         if (ldlm_is_ast_sent(lock)) {
1065                 body->lock_flags |= ldlm_flags_to_wire(LDLM_FL_AST_SENT);
1066                 /* Copy AST flags like LDLM_FL_DISCARD_DATA. */
1067                 body->lock_flags |= ldlm_flags_to_wire(lock->l_flags &
1068                                                        LDLM_FL_AST_MASK);
1069
1070                 /*
1071                  * We might get here prior to ldlm_handle_enqueue setting
1072                  * LDLM_FL_CANCEL_ON_BLOCK flag. Then we will put this lock
1073                  * into waiting list, but this is safe and similar code in
1074                  * ldlm_handle_enqueue will call ldlm_lock_cancel() still,
1075                  * that would not only cancel the lock, but will also remove
1076                  * it from waiting list
1077                  */
1078                 if (ldlm_is_cancel_on_block(lock)) {
1079                         unlock_res_and_lock(lock);
1080                         ldlm_lock_cancel(lock);
1081
1082                         instant_cancel = 1;
1083                         req->rq_no_resend = 1;
1084
1085                         lock_res_and_lock(lock);
1086                 } else {
1087                         /* start the lock-timeout clock */
1088                         ldlm_add_waiting_lock(lock, ldlm_bl_timeout(lock));
1089                         /* Do not resend after lock callback timeout */
1090                         req->rq_delay_limit = ldlm_bl_timeout(lock);
1091                         req->rq_resend_cb = ldlm_update_resend;
1092                 }
1093         }
1094         unlock_res_and_lock(lock);
1095
1096         if (lock->l_export && lock->l_export->exp_nid_stats &&
1097             lock->l_export->exp_nid_stats->nid_ldlm_stats)
1098                 lprocfs_counter_incr(lock->l_export->exp_nid_stats->nid_ldlm_stats,
1099                                      LDLM_CP_CALLBACK - LDLM_FIRST_OPC);
1100
1101         rc = ldlm_ast_fini(req, arg, lock, instant_cancel);
1102
1103         RETURN(lvb_len < 0 ? lvb_len : rc);
1104 }
1105
1106 /**
1107  * Server side ->l_glimpse_ast handler for client locks.
1108  *
1109  * Sends glimpse AST to the client and waits for reply. Then updates
1110  * lvbo with the result.
1111  */
1112 int ldlm_server_glimpse_ast(struct ldlm_lock *lock, void *data)
1113 {
1114         struct ldlm_cb_set_arg *arg = data;
1115         struct ldlm_request *body;
1116         struct ptlrpc_request *req;
1117         struct ldlm_cb_async_args *ca;
1118         int rc;
1119         struct req_format *req_fmt;
1120
1121         ENTRY;
1122
1123         LASSERT(lock != NULL);
1124
1125         if (arg->gl_desc != NULL)
1126                 /* There is a glimpse descriptor to pack */
1127                 req_fmt = &RQF_LDLM_GL_CALLBACK_DESC;
1128         else
1129                 req_fmt = &RQF_LDLM_GL_CALLBACK;
1130
1131         req = ptlrpc_request_alloc_pack(lock->l_export->exp_imp_reverse,
1132                                         req_fmt, LUSTRE_DLM_VERSION,
1133                                         LDLM_GL_CALLBACK);
1134
1135         if (req == NULL)
1136                 RETURN(-ENOMEM);
1137
1138         if (arg->gl_desc != NULL) {
1139                 /* copy the GL descriptor */
1140                 union ldlm_gl_desc      *desc;
1141
1142                 desc = req_capsule_client_get(&req->rq_pill, &RMF_DLM_GL_DESC);
1143                 *desc = *arg->gl_desc;
1144         }
1145
1146         body = req_capsule_client_get(&req->rq_pill, &RMF_DLM_REQ);
1147         body->lock_handle[0] = lock->l_remote_handle;
1148         ldlm_lock2desc(lock, &body->lock_desc);
1149
1150         ca = ptlrpc_req_async_args(ca, req);
1151         ca->ca_set_arg = arg;
1152         ca->ca_lock = lock;
1153
1154         /* server namespace, doesn't need lock */
1155         req_capsule_set_size(&req->rq_pill, &RMF_DLM_LVB, RCL_SERVER,
1156                              ldlm_lvbo_size(lock));
1157         ptlrpc_request_set_replen(req);
1158
1159         req->rq_send_state = LUSTRE_IMP_FULL;
1160         /* ptlrpc_request_alloc_pack already set timeout */
1161         if (AT_OFF)
1162                 req->rq_timeout = ldlm_get_rq_timeout();
1163
1164         req->rq_interpret_reply = ldlm_cb_interpret;
1165
1166         if (lock->l_export && lock->l_export->exp_nid_stats) {
1167                 struct nid_stat *nid_stats = lock->l_export->exp_nid_stats;
1168
1169                 lprocfs_counter_incr(nid_stats->nid_ldlm_stats,
1170                                      LDLM_GL_CALLBACK - LDLM_FIRST_OPC);
1171         }
1172
1173         rc = ldlm_ast_fini(req, arg, lock, 0);
1174
1175         RETURN(rc);
1176 }
1177 EXPORT_SYMBOL(ldlm_server_glimpse_ast);
1178
1179 int ldlm_glimpse_locks(struct ldlm_resource *res,
1180                        struct list_head *gl_work_list)
1181 {
1182         int rc;
1183
1184         ENTRY;
1185
1186         rc = ldlm_run_ast_work(ldlm_res_to_ns(res), gl_work_list,
1187                                LDLM_WORK_GL_AST);
1188         if (rc == -ERESTART)
1189                 ldlm_reprocess_all(res, NULL);
1190
1191         RETURN(rc);
1192 }
1193 EXPORT_SYMBOL(ldlm_glimpse_locks);
1194
1195 /* return LDLM lock associated with a lock callback request */
1196 struct ldlm_lock *ldlm_request_lock(struct ptlrpc_request *req)
1197 {
1198         struct ldlm_cb_async_args *ca;
1199         struct ldlm_lock *lock;
1200
1201         ENTRY;
1202
1203         ca = ptlrpc_req_async_args(ca, req);
1204         lock = ca->ca_lock;
1205         if (lock == NULL)
1206                 RETURN(ERR_PTR(-EFAULT));
1207
1208         RETURN(lock);
1209 }
1210 EXPORT_SYMBOL(ldlm_request_lock);
1211
1212 /**
1213  * Main server-side entry point into LDLM for enqueue. This is called by ptlrpc
1214  * service threads to carry out client lock enqueueing requests.
1215  */
1216 int ldlm_handle_enqueue0(struct ldlm_namespace *ns,
1217                          struct ptlrpc_request *req,
1218                          const struct ldlm_request *dlm_req,
1219                          const struct ldlm_callback_suite *cbs)
1220 {
1221         struct ldlm_reply *dlm_rep;
1222         __u64 flags;
1223         enum ldlm_error err = ELDLM_OK;
1224         struct ldlm_lock *lock = NULL;
1225         void *cookie = NULL;
1226         int rc = 0;
1227         struct ldlm_resource *res = NULL;
1228         const struct lu_env *env = req->rq_svc_thread->t_env;
1229
1230         ENTRY;
1231
1232         LDLM_DEBUG_NOLOCK("server-side enqueue handler START");
1233
1234         ldlm_request_cancel(req, dlm_req, LDLM_ENQUEUE_CANCEL_OFF, LATF_SKIP);
1235         flags = ldlm_flags_from_wire(dlm_req->lock_flags);
1236
1237         LASSERT(req->rq_export);
1238
1239         /* for intent enqueue the stat will be updated inside intent policy */
1240         if (ptlrpc_req2svc(req)->srv_stats != NULL &&
1241             !(dlm_req->lock_flags & LDLM_FL_HAS_INTENT))
1242                 ldlm_svc_get_eopc(dlm_req, ptlrpc_req2svc(req)->srv_stats);
1243
1244         if (req->rq_export && req->rq_export->exp_nid_stats &&
1245             req->rq_export->exp_nid_stats->nid_ldlm_stats)
1246                 lprocfs_counter_incr(req->rq_export->exp_nid_stats->nid_ldlm_stats,
1247                                      LDLM_ENQUEUE - LDLM_FIRST_OPC);
1248
1249         if (unlikely(dlm_req->lock_desc.l_resource.lr_type < LDLM_MIN_TYPE ||
1250                      dlm_req->lock_desc.l_resource.lr_type >= LDLM_MAX_TYPE)) {
1251                 DEBUG_REQ(D_ERROR, req, "invalid lock request type %d",
1252                           dlm_req->lock_desc.l_resource.lr_type);
1253                 GOTO(out, rc = -EFAULT);
1254         }
1255
1256         if (unlikely(dlm_req->lock_desc.l_req_mode <= LCK_MINMODE ||
1257                      dlm_req->lock_desc.l_req_mode >= LCK_MAXMODE ||
1258                      dlm_req->lock_desc.l_req_mode &
1259                      (dlm_req->lock_desc.l_req_mode-1))) {
1260                 DEBUG_REQ(D_ERROR, req, "invalid lock request mode %d",
1261                           dlm_req->lock_desc.l_req_mode);
1262                 GOTO(out, rc = -EFAULT);
1263         }
1264
1265         if (unlikely((flags & LDLM_FL_REPLAY) ||
1266                      (lustre_msg_get_flags(req->rq_reqmsg) & MSG_RESENT))) {
1267                 /* Find an existing lock in the per-export lock hash */
1268                 /*
1269                  * In the function below, .hs_keycmp resolves to
1270                  * ldlm_export_lock_keycmp()
1271                  */
1272                 /* coverity[overrun-buffer-val] */
1273                 lock = cfs_hash_lookup(req->rq_export->exp_lock_hash,
1274                                        (void *)&dlm_req->lock_handle[0]);
1275                 if (lock != NULL) {
1276                         DEBUG_REQ(D_DLMTRACE, req,
1277                                   "found existing lock cookie %#llx",
1278                                   lock->l_handle.h_cookie);
1279                         flags |= LDLM_FL_RESENT;
1280                         GOTO(existing_lock, rc = 0);
1281                 }
1282         } else {
1283                 if (ldlm_reclaim_full()) {
1284                         DEBUG_REQ(D_DLMTRACE, req,
1285                                   "Too many granted locks, reject current enqueue request and let the client retry later");
1286                         GOTO(out, rc = -EINPROGRESS);
1287                 }
1288         }
1289
1290         /* The lock's callback data might be set in the policy function */
1291         lock = ldlm_lock_create(ns, &dlm_req->lock_desc.l_resource.lr_name,
1292                                 dlm_req->lock_desc.l_resource.lr_type,
1293                                 dlm_req->lock_desc.l_req_mode,
1294                                 cbs, NULL, 0, LVB_T_NONE);
1295         if (IS_ERR(lock)) {
1296                 rc = PTR_ERR(lock);
1297                 lock = NULL;
1298                 GOTO(out, rc);
1299         }
1300
1301         lock->l_remote_handle = dlm_req->lock_handle[0];
1302         LDLM_DEBUG(lock, "server-side enqueue handler, new lock created");
1303
1304         /*
1305          * Initialize resource lvb but not for a lock being replayed since
1306          * Client already got lvb sent in this case.
1307          * This must occur early since some policy methods assume resource
1308          * lvb is available (lr_lvb_data != NULL).
1309          */
1310         res = lock->l_resource;
1311         if (!(flags & LDLM_FL_REPLAY)) {
1312                 /* non-replayed lock, delayed lvb init may need to be done */
1313                 rc = ldlm_lvbo_init(res);
1314                 if (rc < 0) {
1315                         LDLM_DEBUG(lock, "delayed lvb init failed (rc %d)", rc);
1316                         GOTO(out, rc);
1317                 }
1318         }
1319
1320         OBD_FAIL_TIMEOUT(OBD_FAIL_LDLM_ENQUEUE_BLOCKED, obd_timeout * 2);
1321         /*
1322          * Don't enqueue a lock onto the export if it is been disonnected
1323          * due to eviction (b=3822) or server umount (b=24324).
1324          * Cancel it now instead.
1325          */
1326         if (req->rq_export->exp_disconnected) {
1327                 LDLM_ERROR(lock, "lock on disconnected export %p",
1328                            req->rq_export);
1329                 GOTO(out, rc = -ENOTCONN);
1330         }
1331
1332         lock->l_export = class_export_lock_get(req->rq_export, lock);
1333         if (lock->l_export->exp_lock_hash)
1334                 cfs_hash_add(lock->l_export->exp_lock_hash,
1335                              &lock->l_remote_handle,
1336                              &lock->l_exp_hash);
1337
1338         /*
1339          * Inherit the enqueue flags before the operation, because we do not
1340          * keep the res lock on return and next operations (BL AST) may proceed
1341          * without them.
1342          */
1343         lock->l_flags |= ldlm_flags_from_wire(dlm_req->lock_flags &
1344                                               LDLM_FL_INHERIT_MASK);
1345
1346         ldlm_convert_policy_to_local(req->rq_export,
1347                                      dlm_req->lock_desc.l_resource.lr_type,
1348                                      &dlm_req->lock_desc.l_policy_data,
1349                                      &lock->l_policy_data);
1350         if (dlm_req->lock_desc.l_resource.lr_type == LDLM_EXTENT)
1351                 lock->l_req_extent = lock->l_policy_data.l_extent;
1352
1353 existing_lock:
1354         if (flags & LDLM_FL_HAS_INTENT) {
1355                 /*
1356                  * In this case, the reply buffer is allocated deep in
1357                  * local_lock_enqueue by the policy function.
1358                  */
1359                 cookie = req;
1360         } else {
1361                 /*
1362                  * based on the assumption that lvb size never changes during
1363                  * resource life time otherwise it need resource->lr_lock's
1364                  * protection
1365                  */
1366                 req_capsule_set_size(&req->rq_pill, &RMF_DLM_LVB,
1367                                      RCL_SERVER, ldlm_lvbo_size(lock));
1368
1369                 if (OBD_FAIL_CHECK(OBD_FAIL_LDLM_ENQUEUE_EXTENT_ERR))
1370                         GOTO(out, rc = -ENOMEM);
1371
1372                 rc = req_capsule_server_pack(&req->rq_pill);
1373                 if (rc)
1374                         GOTO(out, rc);
1375         }
1376
1377         err = ldlm_lock_enqueue(env, ns, &lock, cookie, &flags);
1378         if (err) {
1379                 if ((int)err < 0)
1380                         rc = (int)err;
1381                 GOTO(out, err);
1382         }
1383
1384         dlm_rep = req_capsule_server_get(&req->rq_pill, &RMF_DLM_REP);
1385
1386         ldlm_lock2desc(lock, &dlm_rep->lock_desc);
1387         ldlm_lock2handle(lock, &dlm_rep->lock_handle);
1388
1389         if (lock && lock->l_resource->lr_type == LDLM_EXTENT)
1390                 OBD_FAIL_TIMEOUT(OBD_FAIL_LDLM_BL_EVICT, 6);
1391
1392         /*
1393          * We never send a blocking AST until the lock is granted, but
1394          * we can tell it right now
1395          */
1396         lock_res_and_lock(lock);
1397
1398         /*
1399          * Now take into account flags to be inherited from original lock
1400          * request both in reply to client and in our own lock flags.
1401          */
1402         dlm_rep->lock_flags = ldlm_flags_to_wire(flags);
1403         lock->l_flags |= flags & LDLM_FL_INHERIT_MASK;
1404
1405         /*
1406          * Don't move a pending lock onto the export if it has already been
1407          * disconnected due to eviction (b=5683) or server umount (b=24324).
1408          * Cancel it now instead.
1409          */
1410         if (unlikely(req->rq_export->exp_disconnected ||
1411                      OBD_FAIL_CHECK(OBD_FAIL_LDLM_ENQUEUE_OLD_EXPORT))) {
1412                 LDLM_ERROR(lock, "lock on destroyed export %p", req->rq_export);
1413                 rc = -ENOTCONN;
1414         } else if (ldlm_is_ast_sent(lock)) {
1415                 /* fill lock desc for possible lock convert */
1416                 if (lock->l_blocking_lock &&
1417                     lock->l_resource->lr_type == LDLM_IBITS) {
1418                         struct ldlm_lock *bl_lock = lock->l_blocking_lock;
1419                         struct ldlm_lock_desc *rep_desc = &dlm_rep->lock_desc;
1420
1421                         LDLM_DEBUG(lock,
1422                                    "save blocking bits %llx in granted lock",
1423                                    bl_lock->l_policy_data.l_inodebits.bits);
1424                         /*
1425                          * If lock is blocked then save blocking ibits
1426                          * in returned lock policy for the possible lock
1427                          * convert on a client.
1428                          */
1429                         rep_desc->l_policy_data.l_inodebits.cancel_bits =
1430                                 bl_lock->l_policy_data.l_inodebits.bits;
1431                 }
1432                 dlm_rep->lock_flags |= ldlm_flags_to_wire(LDLM_FL_AST_SENT);
1433                 if (ldlm_is_granted(lock)) {
1434                         /*
1435                          * Only cancel lock if it was granted, because it would
1436                          * be destroyed immediately and would never be granted
1437                          * in the future, causing timeouts on client.  Not
1438                          * granted lock will be cancelled immediately after
1439                          * sending completion AST.
1440                          */
1441                         if (ldlm_is_cancel_on_block(lock)) {
1442                                 unlock_res_and_lock(lock);
1443                                 ldlm_lock_cancel(lock);
1444                                 lock_res_and_lock(lock);
1445                         } else {
1446                                 ldlm_add_waiting_lock(lock,
1447                                                       ldlm_bl_timeout(lock));
1448                         }
1449                 }
1450         }
1451         unlock_res_and_lock(lock);
1452
1453         EXIT;
1454 out:
1455         req->rq_status = rc ?: err; /* return either error - b=11190 */
1456         if (!req->rq_packed_final) {
1457                 err = lustre_pack_reply(req, 1, NULL, NULL);
1458                 if (rc == 0)
1459                         rc = err;
1460         }
1461
1462         /*
1463          * The LOCK_CHANGED code in ldlm_lock_enqueue depends on this
1464          * ldlm_reprocess_all.  If this moves, revisit that code. -phil
1465          */
1466         if (lock != NULL) {
1467                 LDLM_DEBUG(lock,
1468                            "server-side enqueue handler, sending reply (err=%d, rc=%d)",
1469                            err, rc);
1470
1471                 if (rc == 0 &&
1472                     req_capsule_has_field(&req->rq_pill, &RMF_DLM_LVB,
1473                                           RCL_SERVER) &&
1474                     ldlm_lvbo_size(lock) > 0) {
1475                         void *buf;
1476                         int buflen;
1477
1478 retry:
1479                         buf = req_capsule_server_get(&req->rq_pill,
1480                                                      &RMF_DLM_LVB);
1481                         LASSERTF(buf != NULL, "req %p, lock %p\n", req, lock);
1482                         buflen = req_capsule_get_size(&req->rq_pill,
1483                                         &RMF_DLM_LVB, RCL_SERVER);
1484                         /*
1485                          * non-replayed lock, delayed lvb init may
1486                          * need to be occur now
1487                          */
1488                         if ((buflen > 0) && !(flags & LDLM_FL_REPLAY)) {
1489                                 int rc2;
1490
1491                                 rc2 = ldlm_lvbo_fill(lock, buf, &buflen);
1492                                 if (rc2 >= 0) {
1493                                         req_capsule_shrink(&req->rq_pill,
1494                                                            &RMF_DLM_LVB,
1495                                                            rc2, RCL_SERVER);
1496                                 } else if (rc2 == -ERANGE) {
1497                                         rc2 = req_capsule_server_grow(
1498                                                         &req->rq_pill,
1499                                                         &RMF_DLM_LVB, buflen);
1500                                         if (!rc2) {
1501                                                 goto retry;
1502                                         } else {
1503                                                 /*
1504                                                  * if we can't grow the buffer,
1505                                                  * it's ok to return empty lvb
1506                                                  * to client.
1507                                                  */
1508                                                 req_capsule_shrink(
1509                                                         &req->rq_pill,
1510                                                         &RMF_DLM_LVB, 0,
1511                                                         RCL_SERVER);
1512                                         }
1513                                 } else {
1514                                         rc = rc2;
1515                                 }
1516                         } else if (flags & LDLM_FL_REPLAY) {
1517                                 /* no LVB resend upon replay */
1518                                 if (buflen > 0)
1519                                         req_capsule_shrink(&req->rq_pill,
1520                                                            &RMF_DLM_LVB,
1521                                                            0, RCL_SERVER);
1522                                 else
1523                                         rc = buflen;
1524                         } else {
1525                                 rc = buflen;
1526                         }
1527                 }
1528
1529                 if (rc != 0 && !(flags & LDLM_FL_RESENT)) {
1530                         if (lock->l_export) {
1531                                 ldlm_lock_cancel(lock);
1532                         } else {
1533                                 lock_res_and_lock(lock);
1534                                 ldlm_resource_unlink_lock(lock);
1535                                 ldlm_lock_destroy_nolock(lock);
1536                                 unlock_res_and_lock(lock);
1537
1538                         }
1539                 }
1540
1541                 if (!err && !ldlm_is_cbpending(lock) &&
1542                     dlm_req->lock_desc.l_resource.lr_type != LDLM_FLOCK)
1543                         ldlm_reprocess_all(lock->l_resource, lock);
1544
1545                 LDLM_LOCK_RELEASE(lock);
1546         }
1547
1548         LDLM_DEBUG_NOLOCK("server-side enqueue handler END (lock %p, rc %d)",
1549                           lock, rc);
1550
1551         return rc;
1552 }
1553
1554 /*
1555  * Clear the blocking lock, the race is possible between ldlm_handle_convert0()
1556  * and ldlm_work_bl_ast_lock(), so this is done under lock with check for NULL.
1557  */
1558 void ldlm_clear_blocking_lock(struct ldlm_lock *lock)
1559 {
1560         if (lock->l_blocking_lock) {
1561                 LDLM_LOCK_RELEASE(lock->l_blocking_lock);
1562                 lock->l_blocking_lock = NULL;
1563         }
1564 }
1565
1566 /* A lock can be converted to new ibits or mode and should be considered
1567  * as new lock. Clear all states related to a previous blocking AST
1568  * processing so new conflicts will cause new blocking ASTs.
1569  *
1570  * This is used during lock convert below and lock downgrade to COS mode in
1571  * ldlm_lock_mode_downgrade().
1572  */
1573 void ldlm_clear_blocking_data(struct ldlm_lock *lock)
1574 {
1575         ldlm_clear_ast_sent(lock);
1576         lock->l_bl_ast_run = 0;
1577         ldlm_clear_blocking_lock(lock);
1578 }
1579
1580 /**
1581  * Main LDLM entry point for server code to process lock conversion requests.
1582  */
1583 int ldlm_handle_convert0(struct ptlrpc_request *req,
1584                          const struct ldlm_request *dlm_req)
1585 {
1586         struct obd_export *exp = req->rq_export;
1587         struct ldlm_reply *dlm_rep;
1588         struct ldlm_lock *lock;
1589         int rc;
1590
1591         ENTRY;
1592
1593         if (exp && exp->exp_nid_stats && exp->exp_nid_stats->nid_ldlm_stats)
1594                 lprocfs_counter_incr(exp->exp_nid_stats->nid_ldlm_stats,
1595                                      LDLM_CONVERT - LDLM_FIRST_OPC);
1596
1597         rc = req_capsule_server_pack(&req->rq_pill);
1598         if (rc)
1599                 RETURN(rc);
1600
1601         dlm_rep = req_capsule_server_get(&req->rq_pill, &RMF_DLM_REP);
1602         dlm_rep->lock_flags = dlm_req->lock_flags;
1603
1604         lock = ldlm_handle2lock(&dlm_req->lock_handle[0]);
1605         if (lock) {
1606                 __u64 bits;
1607                 __u64 new;
1608
1609                 bits = lock->l_policy_data.l_inodebits.bits;
1610                 new = dlm_req->lock_desc.l_policy_data.l_inodebits.bits;
1611                 LDLM_DEBUG(lock, "server-side convert handler START");
1612
1613                 if (ldlm_is_cancel(lock)) {
1614                         LDLM_ERROR(lock, "convert on canceled lock!");
1615                         rc = ELDLM_NO_LOCK_DATA;
1616                 } else if (dlm_req->lock_desc.l_req_mode !=
1617                            lock->l_granted_mode) {
1618                         LDLM_ERROR(lock, "lock mode differs!");
1619                         rc = ELDLM_NO_LOCK_DATA;
1620                 } else if (bits == new) {
1621                         /*
1622                          * This can be valid situation if CONVERT RPCs are
1623                          * re-ordered. Just finish silently
1624                          */
1625                         LDLM_DEBUG(lock, "lock is converted already!");
1626                         rc = ELDLM_OK;
1627                 } else {
1628                         lock_res_and_lock(lock);
1629                         if (ldlm_is_waited(lock))
1630                                 ldlm_del_waiting_lock(lock);
1631
1632                         ldlm_clear_cbpending(lock);
1633                         lock->l_policy_data.l_inodebits.cancel_bits = 0;
1634                         ldlm_inodebits_drop(lock, bits & ~new);
1635
1636                         ldlm_clear_blocking_data(lock);
1637                         unlock_res_and_lock(lock);
1638
1639                         ldlm_reprocess_all(lock->l_resource, NULL);
1640                         rc = ELDLM_OK;
1641                 }
1642
1643                 if (rc == ELDLM_OK) {
1644                         dlm_rep->lock_handle = lock->l_remote_handle;
1645                         ldlm_ibits_policy_local_to_wire(&lock->l_policy_data,
1646                                         &dlm_rep->lock_desc.l_policy_data);
1647                 }
1648
1649                 LDLM_DEBUG(lock, "server-side convert handler END, rc = %d",
1650                            rc);
1651                 LDLM_LOCK_PUT(lock);
1652         } else {
1653                 rc = ELDLM_NO_LOCK_DATA;
1654                 LDLM_DEBUG_NOLOCK("server-side convert handler END, rc = %d",
1655                                   rc);
1656         }
1657
1658         req->rq_status = rc;
1659
1660         RETURN(0);
1661 }
1662
1663 /**
1664  * Cancel all the locks whose handles are packed into ldlm_request
1665  *
1666  * Called by server code expecting such combined cancel activity
1667  * requests.
1668  */
1669 int ldlm_request_cancel(struct ptlrpc_request *req,
1670                         const struct ldlm_request *dlm_req,
1671                         int first, enum lustre_at_flags flags)
1672 {
1673         struct ldlm_resource *res, *pres = NULL;
1674         struct ldlm_lock *lock;
1675         int i, count, done = 0;
1676         unsigned int size;
1677
1678         ENTRY;
1679
1680         size = req_capsule_get_size(&req->rq_pill, &RMF_DLM_REQ, RCL_CLIENT);
1681         if (size <= offsetof(struct ldlm_request, lock_handle) ||
1682             (size - offsetof(struct ldlm_request, lock_handle)) /
1683              sizeof(struct lustre_handle) < dlm_req->lock_count)
1684                 RETURN(0);
1685
1686         count = dlm_req->lock_count ? dlm_req->lock_count : 1;
1687         if (first >= count)
1688                 RETURN(0);
1689
1690         if (count == 1 && dlm_req->lock_handle[0].cookie == 0)
1691                 RETURN(0);
1692
1693         /*
1694          * There is no lock on the server at the replay time,
1695          * skip lock cancelling to make replay tests to pass.
1696          */
1697         if (lustre_msg_get_flags(req->rq_reqmsg) & MSG_REPLAY)
1698                 RETURN(0);
1699
1700         LDLM_DEBUG_NOLOCK("server-side cancel handler START: %d locks, starting at %d",
1701                           count, first);
1702
1703         for (i = first; i < count; i++) {
1704                 lock = ldlm_handle2lock(&dlm_req->lock_handle[i]);
1705                 if (!lock) {
1706                         /* below message checked in replay-single.sh test_36 */
1707                         LDLM_DEBUG_NOLOCK("server-side cancel handler stale lock (cookie %llu)",
1708                                           dlm_req->lock_handle[i].cookie);
1709                         continue;
1710                 }
1711
1712                 res = lock->l_resource;
1713                 done++;
1714
1715                 /*
1716                  * This code is an optimization to only attempt lock
1717                  * granting on the resource (that could be CPU-expensive)
1718                  * after we are done cancelling lock in that resource.
1719                  */
1720                 if (res != pres) {
1721                         if (pres != NULL) {
1722                                 ldlm_reprocess_all(pres, NULL);
1723                                 LDLM_RESOURCE_DELREF(pres);
1724                                 ldlm_resource_putref(pres);
1725                         }
1726                         if (res != NULL) {
1727                                 ldlm_resource_getref(res);
1728                                 LDLM_RESOURCE_ADDREF(res);
1729
1730                                 if (!ldlm_is_discard_data(lock))
1731                                         ldlm_lvbo_update(res, lock,
1732                                                          NULL, 1);
1733                         }
1734                         pres = res;
1735                 }
1736
1737                 if ((flags & LATF_STATS) && ldlm_is_ast_sent(lock) &&
1738                     lock->l_blast_sent != 0) {
1739                         time64_t delay = ktime_get_real_seconds() -
1740                                          lock->l_blast_sent;
1741                         LDLM_DEBUG(lock,
1742                                    "server cancels blocked lock after %llds",
1743                                    (s64)delay);
1744                         at_measured(&lock->l_export->exp_bl_lock_at, delay);
1745                 }
1746                 ldlm_lock_cancel(lock);
1747                 LDLM_LOCK_PUT(lock);
1748         }
1749         if (pres != NULL) {
1750                 ldlm_reprocess_all(pres, NULL);
1751                 LDLM_RESOURCE_DELREF(pres);
1752                 ldlm_resource_putref(pres);
1753         }
1754         LDLM_DEBUG_NOLOCK("server-side cancel handler END");
1755         RETURN(done);
1756 }
1757 EXPORT_SYMBOL(ldlm_request_cancel);
1758
1759 /**
1760  * Main LDLM entry point for server code to cancel locks.
1761  *
1762  * Typically gets called from service handler on LDLM_CANCEL opc.
1763  */
1764 int ldlm_handle_cancel(struct ptlrpc_request *req)
1765 {
1766         struct ldlm_request *dlm_req;
1767         int rc;
1768
1769         ENTRY;
1770
1771         dlm_req = req_capsule_client_get(&req->rq_pill, &RMF_DLM_REQ);
1772         if (dlm_req == NULL) {
1773                 CDEBUG(D_INFO, "bad request buffer for cancel\n");
1774                 RETURN(-EFAULT);
1775         }
1776
1777         if (req_capsule_get_size(&req->rq_pill, &RMF_DLM_REQ, RCL_CLIENT) <
1778             offsetof(struct ldlm_request, lock_handle[1]))
1779                 RETURN(-EPROTO);
1780
1781         if (req->rq_export && req->rq_export->exp_nid_stats &&
1782             req->rq_export->exp_nid_stats->nid_ldlm_stats)
1783                 lprocfs_counter_incr(req->rq_export->exp_nid_stats->nid_ldlm_stats,
1784                                      LDLM_CANCEL - LDLM_FIRST_OPC);
1785
1786         rc = req_capsule_server_pack(&req->rq_pill);
1787         if (rc)
1788                 RETURN(rc);
1789
1790         if (!ldlm_request_cancel(req, dlm_req, 0, LATF_STATS))
1791                 req->rq_status = LUSTRE_ESTALE;
1792
1793         RETURN(ptlrpc_reply(req));
1794 }
1795 #endif /* HAVE_SERVER_SUPPORT */
1796
1797 /**
1798  * Callback handler for receiving incoming blocking ASTs.
1799  *
1800  * This can only happen on client side.
1801  */
1802 void ldlm_handle_bl_callback(struct ldlm_namespace *ns,
1803                              struct ldlm_lock_desc *ld, struct ldlm_lock *lock)
1804 {
1805         int do_ast;
1806
1807         ENTRY;
1808
1809         LDLM_DEBUG(lock, "client blocking AST callback handler");
1810
1811         lock_res_and_lock(lock);
1812
1813         /* set bits to cancel for this lock for possible lock convert */
1814         if (ns_is_client(ns) && (lock->l_resource->lr_type == LDLM_IBITS)) {
1815                 /*
1816                  * Lock description contains policy of blocking lock,
1817                  * and its cancel_bits is used to pass conflicting bits.
1818                  * NOTE: ld can be NULL or can be not NULL but zeroed if
1819                  * passed from ldlm_bl_thread_blwi(), check below used bits
1820                  * in ld to make sure it is valid description.
1821                  *
1822                  * If server may replace lock resource keeping the same cookie,
1823                  * never use cancel bits from different resource, full cancel
1824                  * is to be used.
1825                  */
1826                 if (ld && ld->l_policy_data.l_inodebits.bits &&
1827                     ldlm_res_eq(&ld->l_resource.lr_name,
1828                                 &lock->l_resource->lr_name))
1829                         lock->l_policy_data.l_inodebits.cancel_bits =
1830                                 ld->l_policy_data.l_inodebits.cancel_bits;
1831                 /*
1832                  * if there is no valid ld and lock is cbpending already
1833                  * then cancel_bits should be kept, otherwise it is zeroed.
1834                  */
1835                 else if (!ldlm_is_cbpending(lock))
1836                         lock->l_policy_data.l_inodebits.cancel_bits = 0;
1837         }
1838         ldlm_set_cbpending(lock);
1839
1840         do_ast = (!lock->l_readers && !lock->l_writers);
1841         unlock_res_and_lock(lock);
1842
1843         if (do_ast) {
1844                 CDEBUG(D_DLMTRACE,
1845                        "Lock %p already unused, calling callback (%p)\n",
1846                        lock, lock->l_blocking_ast);
1847                 if (lock->l_blocking_ast != NULL)
1848                         lock->l_blocking_ast(lock, ld, lock->l_ast_data,
1849                                              LDLM_CB_BLOCKING);
1850         } else {
1851                 CDEBUG(D_DLMTRACE,
1852                        "Lock %p is referenced, will be cancelled later\n",
1853                        lock);
1854         }
1855
1856         LDLM_DEBUG(lock, "client blocking callback handler END");
1857         LDLM_LOCK_RELEASE(lock);
1858         EXIT;
1859 }
1860
1861 /**
1862  * Callback handler for receiving incoming completion ASTs.
1863  *
1864  * This only can happen on client side.
1865  */
1866 static void ldlm_handle_cp_callback(struct ptlrpc_request *req,
1867                                     struct ldlm_namespace *ns,
1868                                     struct ldlm_request *dlm_req,
1869                                     struct ldlm_lock *lock)
1870 {
1871         struct list_head ast_list;
1872         int lvb_len;
1873         int rc = 0;
1874
1875         ENTRY;
1876
1877         LDLM_DEBUG(lock, "client completion callback handler START");
1878
1879         INIT_LIST_HEAD(&ast_list);
1880         if (OBD_FAIL_CHECK(OBD_FAIL_LDLM_CANCEL_BL_CB_RACE)) {
1881                 long to = cfs_time_seconds(1);
1882
1883                 while (to > 0) {
1884                         set_current_state(TASK_INTERRUPTIBLE);
1885                         schedule_timeout(to);
1886                         if (ldlm_is_granted(lock) ||
1887                             ldlm_is_destroyed(lock))
1888                                 break;
1889                 }
1890         }
1891
1892         lvb_len = req_capsule_get_size(&req->rq_pill, &RMF_DLM_LVB, RCL_CLIENT);
1893         if (lvb_len < 0) {
1894                 LDLM_ERROR(lock, "Fail to get lvb_len, rc = %d", lvb_len);
1895                 GOTO(out, rc = lvb_len);
1896         } else if (lvb_len > 0) {
1897                 if (lock->l_lvb_len > 0) {
1898                         /* for extent lock, lvb contains ost_lvb{}. */
1899                         LASSERT(lock->l_lvb_data != NULL);
1900
1901                         if (unlikely(lock->l_lvb_len < lvb_len)) {
1902                                 LDLM_ERROR(lock,
1903                                            "Replied LVB is larger than expectation, expected = %d, replied = %d",
1904                                            lock->l_lvb_len, lvb_len);
1905                                 GOTO(out, rc = -EINVAL);
1906                         }
1907                 }
1908         }
1909
1910         lock_res_and_lock(lock);
1911
1912         if (!ldlm_res_eq(&dlm_req->lock_desc.l_resource.lr_name,
1913                          &lock->l_resource->lr_name)) {
1914                 ldlm_resource_unlink_lock(lock);
1915                 unlock_res_and_lock(lock);
1916                 rc = ldlm_lock_change_resource(ns, lock,
1917                                 &dlm_req->lock_desc.l_resource.lr_name);
1918                 if (rc < 0) {
1919                         LDLM_ERROR(lock, "Failed to allocate resource");
1920                         GOTO(out, rc);
1921                 }
1922                 LDLM_DEBUG(lock, "completion AST, new resource");
1923                 lock_res_and_lock(lock);
1924         }
1925
1926         if (ldlm_is_destroyed(lock) ||
1927             ldlm_is_granted(lock)) {
1928                 /* b=11300: the lock has already been granted */
1929                 unlock_res_and_lock(lock);
1930                 LDLM_DEBUG(lock, "Double grant race happened");
1931                 GOTO(out, rc = 0);
1932         }
1933
1934         /*
1935          * If we receive the completion AST before the actual enqueue returned,
1936          * then we might need to switch lock modes, resources, or extents.
1937          */
1938         if (dlm_req->lock_desc.l_granted_mode != lock->l_req_mode) {
1939                 lock->l_req_mode = dlm_req->lock_desc.l_granted_mode;
1940                 LDLM_DEBUG(lock, "completion AST, new lock mode");
1941         }
1942
1943         if (lock->l_resource->lr_type != LDLM_PLAIN) {
1944                 ldlm_convert_policy_to_local(req->rq_export,
1945                                           dlm_req->lock_desc.l_resource.lr_type,
1946                                           &dlm_req->lock_desc.l_policy_data,
1947                                           &lock->l_policy_data);
1948                 LDLM_DEBUG(lock, "completion AST, new policy data");
1949         }
1950
1951         ldlm_resource_unlink_lock(lock);
1952
1953         if (dlm_req->lock_flags & LDLM_FL_AST_SENT) {
1954                 /*
1955                  * BL_AST locks are not needed in LRU.
1956                  * Let ldlm_cancel_lru() be fast.
1957                  */
1958                 ldlm_lock_remove_from_lru(lock);
1959                 lock->l_flags |= LDLM_FL_CBPENDING | LDLM_FL_BL_AST;
1960                 LDLM_DEBUG(lock, "completion AST includes blocking AST");
1961         }
1962
1963         if (lock->l_lvb_len > 0) {
1964                 rc = ldlm_fill_lvb(lock, &req->rq_pill, RCL_CLIENT,
1965                                    lock->l_lvb_data, lvb_len);
1966                 if (rc < 0) {
1967                         unlock_res_and_lock(lock);
1968                         GOTO(out, rc);
1969                 }
1970         }
1971
1972         ldlm_grant_lock(lock, &ast_list);
1973         unlock_res_and_lock(lock);
1974
1975         LDLM_DEBUG(lock, "callback handler finished, about to run_ast_work");
1976
1977         /*
1978          * Let Enqueue to call osc_lock_upcall() and initialize
1979          * l_ast_data
1980          */
1981         OBD_FAIL_TIMEOUT(OBD_FAIL_OSC_CP_ENQ_RACE, 2);
1982
1983         ldlm_run_ast_work(ns, &ast_list, LDLM_WORK_CP_AST);
1984
1985         LDLM_DEBUG_NOLOCK("client completion callback handler END (lock %p)",
1986                           lock);
1987         GOTO(out, rc);
1988
1989 out:
1990         if (rc < 0) {
1991                 lock_res_and_lock(lock);
1992                 ldlm_set_failed(lock);
1993                 unlock_res_and_lock(lock);
1994                 wake_up(&lock->l_waitq);
1995         }
1996         LDLM_LOCK_RELEASE(lock);
1997 }
1998
1999 /**
2000  * Callback handler for receiving incoming glimpse ASTs.
2001  *
2002  * This only can happen on client side.  After handling the glimpse AST
2003  * we also consider dropping the lock here if it is unused locally for a
2004  * long time.
2005  */
2006 static void ldlm_handle_gl_callback(struct ptlrpc_request *req,
2007                                     struct ldlm_namespace *ns,
2008                                     struct ldlm_request *dlm_req,
2009                                     struct ldlm_lock *lock)
2010 {
2011         int rc = -ENOSYS;
2012
2013         ENTRY;
2014
2015         LDLM_DEBUG(lock, "client glimpse AST callback handler");
2016
2017         if (lock->l_glimpse_ast != NULL)
2018                 rc = lock->l_glimpse_ast(lock, req);
2019
2020         if (req->rq_repmsg != NULL) {
2021                 ptlrpc_reply(req);
2022         } else {
2023                 req->rq_status = rc;
2024                 ptlrpc_error(req);
2025         }
2026
2027         lock_res_and_lock(lock);
2028         if (lock->l_granted_mode == LCK_PW &&
2029             !lock->l_readers && !lock->l_writers &&
2030             ktime_after(ktime_get(),
2031                         ktime_add(lock->l_last_used,
2032                                   ktime_set(ns->ns_dirty_age_limit, 0)))) {
2033                 unlock_res_and_lock(lock);
2034                 if (ldlm_bl_to_thread_lock(ns, NULL, lock))
2035                         ldlm_handle_bl_callback(ns, NULL, lock);
2036
2037                 EXIT;
2038                 return;
2039         }
2040         unlock_res_and_lock(lock);
2041         LDLM_LOCK_RELEASE(lock);
2042         EXIT;
2043 }
2044
2045 static int ldlm_callback_reply(struct ptlrpc_request *req, int rc)
2046 {
2047         if (req->rq_no_reply)
2048                 return 0;
2049
2050         req->rq_status = rc;
2051         if (!req->rq_packed_final) {
2052                 rc = lustre_pack_reply(req, 1, NULL, NULL);
2053                 if (rc)
2054                         return rc;
2055         }
2056         return ptlrpc_reply(req);
2057 }
2058
2059 static int __ldlm_bl_to_thread(struct ldlm_bl_work_item *blwi,
2060                                enum ldlm_cancel_flags cancel_flags)
2061 {
2062         struct ldlm_bl_pool *blp = ldlm_state->ldlm_bl_pool;
2063
2064         ENTRY;
2065
2066         spin_lock(&blp->blp_lock);
2067         if (blwi->blwi_lock &&
2068             ldlm_is_discard_data(blwi->blwi_lock)) {
2069                 /* add LDLM_FL_DISCARD_DATA requests to the priority list */
2070                 list_add_tail(&blwi->blwi_entry, &blp->blp_prio_list);
2071         } else {
2072                 /* other blocking callbacks are added to the regular list */
2073                 list_add_tail(&blwi->blwi_entry, &blp->blp_list);
2074         }
2075         spin_unlock(&blp->blp_lock);
2076
2077         wake_up(&blp->blp_waitq);
2078
2079         /*
2080          * can not check blwi->blwi_flags as blwi could be already freed in
2081          * LCF_ASYNC mode
2082          */
2083         if (!(cancel_flags & LCF_ASYNC))
2084                 wait_for_completion(&blwi->blwi_comp);
2085
2086         RETURN(0);
2087 }
2088
2089 static inline void init_blwi(struct ldlm_bl_work_item *blwi,
2090                              struct ldlm_namespace *ns,
2091                              struct ldlm_lock_desc *ld,
2092                              struct list_head *cancels, int count,
2093                              struct ldlm_lock *lock,
2094                              enum ldlm_cancel_flags cancel_flags)
2095 {
2096         init_completion(&blwi->blwi_comp);
2097         INIT_LIST_HEAD(&blwi->blwi_head);
2098
2099         if (memory_pressure_get())
2100                 blwi->blwi_mem_pressure = 1;
2101
2102         blwi->blwi_ns = ns;
2103         blwi->blwi_flags = cancel_flags;
2104         if (ld != NULL)
2105                 blwi->blwi_ld = *ld;
2106         if (count) {
2107                 list_add(&blwi->blwi_head, cancels);
2108                 list_del_init(cancels);
2109                 blwi->blwi_count = count;
2110         } else {
2111                 blwi->blwi_lock = lock;
2112         }
2113 }
2114
2115 /**
2116  * Queues a list of locks \a cancels containing \a count locks
2117  * for later processing by a blocking thread.  If \a count is zero,
2118  * then the lock referenced as \a lock is queued instead.
2119  *
2120  * The blocking thread would then call ->l_blocking_ast callback in the lock.
2121  * If list addition fails an error is returned and caller is supposed to
2122  * call ->l_blocking_ast itself.
2123  */
2124 static int ldlm_bl_to_thread(struct ldlm_namespace *ns,
2125                              struct ldlm_lock_desc *ld,
2126                              struct ldlm_lock *lock,
2127                              struct list_head *cancels, int count,
2128                              enum ldlm_cancel_flags cancel_flags)
2129 {
2130         ENTRY;
2131
2132         if (cancels && count == 0)
2133                 RETURN(0);
2134
2135         if (cancel_flags & LCF_ASYNC) {
2136                 struct ldlm_bl_work_item *blwi;
2137
2138                 OBD_ALLOC(blwi, sizeof(*blwi));
2139                 if (blwi == NULL)
2140                         RETURN(-ENOMEM);
2141                 init_blwi(blwi, ns, ld, cancels, count, lock, cancel_flags);
2142
2143                 RETURN(__ldlm_bl_to_thread(blwi, cancel_flags));
2144         } else {
2145                 /*
2146                  * if it is synchronous call do minimum mem alloc, as it could
2147                  * be triggered from kernel shrinker
2148                  */
2149                 struct ldlm_bl_work_item blwi;
2150
2151                 memset(&blwi, 0, sizeof(blwi));
2152                 init_blwi(&blwi, ns, ld, cancels, count, lock, cancel_flags);
2153                 RETURN(__ldlm_bl_to_thread(&blwi, cancel_flags));
2154         }
2155 }
2156
2157
2158 int ldlm_bl_to_thread_lock(struct ldlm_namespace *ns, struct ldlm_lock_desc *ld,
2159                            struct ldlm_lock *lock)
2160 {
2161         return ldlm_bl_to_thread(ns, ld, lock, NULL, 0, LCF_ASYNC);
2162 }
2163
2164 int ldlm_bl_to_thread_list(struct ldlm_namespace *ns, struct ldlm_lock_desc *ld,
2165                            struct list_head *cancels, int count,
2166                            enum ldlm_cancel_flags cancel_flags)
2167 {
2168         return ldlm_bl_to_thread(ns, ld, NULL, cancels, count, cancel_flags);
2169 }
2170
2171 int ldlm_bl_thread_wakeup(void)
2172 {
2173         wake_up(&ldlm_state->ldlm_bl_pool->blp_waitq);
2174         return 0;
2175 }
2176
2177 /* Setinfo coming from Server (eg MDT) to Client (eg MDC)! */
2178 static int ldlm_handle_setinfo(struct ptlrpc_request *req)
2179 {
2180         struct obd_device *obd = req->rq_export->exp_obd;
2181         char *key;
2182         void *val;
2183         int keylen, vallen;
2184         int rc = -ENOSYS;
2185
2186         ENTRY;
2187
2188         DEBUG_REQ(D_HSM, req, "%s: handle setinfo", obd->obd_name);
2189
2190         req_capsule_set(&req->rq_pill, &RQF_OBD_SET_INFO);
2191
2192         key = req_capsule_client_get(&req->rq_pill, &RMF_SETINFO_KEY);
2193         if (key == NULL) {
2194                 DEBUG_REQ(D_IOCTL, req, "no set_info key");
2195                 RETURN(-EFAULT);
2196         }
2197         keylen = req_capsule_get_size(&req->rq_pill, &RMF_SETINFO_KEY,
2198                                       RCL_CLIENT);
2199         val = req_capsule_client_get(&req->rq_pill, &RMF_SETINFO_VAL);
2200         if (val == NULL) {
2201                 DEBUG_REQ(D_IOCTL, req, "no set_info val");
2202                 RETURN(-EFAULT);
2203         }
2204         vallen = req_capsule_get_size(&req->rq_pill, &RMF_SETINFO_VAL,
2205                                       RCL_CLIENT);
2206
2207         /* We are responsible for swabbing contents of val */
2208
2209         if (KEY_IS(KEY_HSM_COPYTOOL_SEND))
2210                 /* Pass it on to mdc (the "export" in this case) */
2211                 rc = obd_set_info_async(req->rq_svc_thread->t_env,
2212                                         req->rq_export,
2213                                         sizeof(KEY_HSM_COPYTOOL_SEND),
2214                                         KEY_HSM_COPYTOOL_SEND,
2215                                         vallen, val, NULL);
2216         else
2217                 DEBUG_REQ(D_WARNING, req, "ignoring unknown key '%s'", key);
2218
2219         return rc;
2220 }
2221
2222 static inline void ldlm_callback_errmsg(struct ptlrpc_request *req,
2223                                         const char *msg, int rc,
2224                                         const struct lustre_handle *handle)
2225 {
2226         DEBUG_REQ((req->rq_no_reply || rc) ? D_WARNING : D_DLMTRACE, req,
2227                   "%s, NID=%s lock=%#llx: rc = %d",
2228                   msg, libcfs_id2str(req->rq_peer),
2229                   handle ? handle->cookie : 0, rc);
2230         if (req->rq_no_reply)
2231                 CWARN("No reply was sent, maybe cause b=21636.\n");
2232         else if (rc)
2233                 CWARN("Send reply failed, maybe cause b=21636.\n");
2234 }
2235
2236 /* TODO: handle requests in a similar way as MDT: see mdt_handle_common() */
2237 static int ldlm_callback_handler(struct ptlrpc_request *req)
2238 {
2239         struct ldlm_namespace *ns;
2240         struct ldlm_request *dlm_req;
2241         struct ldlm_lock *lock;
2242         int rc;
2243
2244         ENTRY;
2245
2246         /*
2247          * Requests arrive in sender's byte order.  The ptlrpc service
2248          * handler has already checked and, if necessary, byte-swapped the
2249          * incoming request message body, but I am responsible for the
2250          * message buffers.
2251          */
2252
2253         /* do nothing for sec context finalize */
2254         if (lustre_msg_get_opc(req->rq_reqmsg) == SEC_CTX_FINI)
2255                 RETURN(0);
2256
2257         req_capsule_init(&req->rq_pill, req, RCL_SERVER);
2258
2259         if (req->rq_export == NULL) {
2260                 rc = ldlm_callback_reply(req, -ENOTCONN);
2261                 ldlm_callback_errmsg(req, "Operate on unconnected server",
2262                                      rc, NULL);
2263                 RETURN(0);
2264         }
2265
2266         LASSERT(req->rq_export != NULL);
2267         LASSERT(req->rq_export->exp_obd != NULL);
2268
2269         switch (lustre_msg_get_opc(req->rq_reqmsg)) {
2270         case LDLM_BL_CALLBACK:
2271                 if (OBD_FAIL_CHECK(OBD_FAIL_LDLM_BL_CALLBACK_NET)) {
2272                         if (cfs_fail_err)
2273                                 ldlm_callback_reply(req, -(int)cfs_fail_err);
2274                         RETURN(0);
2275                 }
2276                 break;
2277         case LDLM_CP_CALLBACK:
2278                 if (OBD_FAIL_CHECK(OBD_FAIL_LDLM_CP_CALLBACK_NET))
2279                         RETURN(0);
2280                 break;
2281         case LDLM_GL_CALLBACK:
2282                 if (OBD_FAIL_CHECK(OBD_FAIL_LDLM_GL_CALLBACK_NET))
2283                         RETURN(0);
2284                 break;
2285         case LDLM_SET_INFO:
2286                 rc = ldlm_handle_setinfo(req);
2287                 ldlm_callback_reply(req, rc);
2288                 RETURN(0);
2289         default:
2290                 CERROR("unknown opcode %u\n",
2291                        lustre_msg_get_opc(req->rq_reqmsg));
2292                 ldlm_callback_reply(req, -EPROTO);
2293                 RETURN(0);
2294         }
2295
2296         ns = req->rq_export->exp_obd->obd_namespace;
2297         LASSERT(ns != NULL);
2298
2299         req_capsule_set(&req->rq_pill, &RQF_LDLM_CALLBACK);
2300
2301         dlm_req = req_capsule_client_get(&req->rq_pill, &RMF_DLM_REQ);
2302         if (dlm_req == NULL) {
2303                 rc = ldlm_callback_reply(req, -EPROTO);
2304                 ldlm_callback_errmsg(req, "Operate without parameter", rc,
2305                                      NULL);
2306                 RETURN(0);
2307         }
2308
2309         /*
2310          * Force a known safe race, send a cancel to the server for a lock
2311          * which the server has already started a blocking callback on.
2312          */
2313         if (OBD_FAIL_CHECK(OBD_FAIL_LDLM_CANCEL_BL_CB_RACE) &&
2314             lustre_msg_get_opc(req->rq_reqmsg) == LDLM_BL_CALLBACK) {
2315                 rc = ldlm_cli_cancel(&dlm_req->lock_handle[0], 0);
2316                 if (rc < 0)
2317                         CERROR("ldlm_cli_cancel: %d\n", rc);
2318         }
2319
2320         lock = ldlm_handle2lock_long(&dlm_req->lock_handle[0], 0);
2321         if (!lock) {
2322                 CDEBUG(D_DLMTRACE,
2323                        "callback on lock %#llx - lock disappeared\n",
2324                        dlm_req->lock_handle[0].cookie);
2325                 rc = ldlm_callback_reply(req, -EINVAL);
2326                 ldlm_callback_errmsg(req, "Operate with invalid parameter", rc,
2327                                      &dlm_req->lock_handle[0]);
2328                 RETURN(0);
2329         }
2330
2331         if (ldlm_is_fail_loc(lock) &&
2332             lustre_msg_get_opc(req->rq_reqmsg) == LDLM_BL_CALLBACK)
2333                 OBD_RACE(OBD_FAIL_LDLM_CP_BL_RACE);
2334
2335         /* Copy hints/flags (e.g. LDLM_FL_DISCARD_DATA) from AST. */
2336         lock_res_and_lock(lock);
2337         lock->l_flags |= ldlm_flags_from_wire(dlm_req->lock_flags &
2338                                               LDLM_FL_AST_MASK);
2339         if (lustre_msg_get_opc(req->rq_reqmsg) == LDLM_BL_CALLBACK) {
2340                 /*
2341                  * If somebody cancels lock and cache is already dropped,
2342                  * or lock is failed before cp_ast received on client,
2343                  * we can tell the server we have no lock. Otherwise, we
2344                  * should send cancel after dropping the cache.
2345                  */
2346                 if ((ldlm_is_canceling(lock) && ldlm_is_bl_done(lock)) ||
2347                      ldlm_is_failed(lock)) {
2348                         LDLM_DEBUG(lock,
2349                                    "callback on lock %llx - lock disappeared",
2350                                    dlm_req->lock_handle[0].cookie);
2351                         unlock_res_and_lock(lock);
2352                         LDLM_LOCK_RELEASE(lock);
2353                         rc = ldlm_callback_reply(req, -EINVAL);
2354                         ldlm_callback_errmsg(req, "Operate on stale lock", rc,
2355                                              &dlm_req->lock_handle[0]);
2356                         RETURN(0);
2357                 }
2358                 /*
2359                  * BL_AST locks are not needed in LRU.
2360                  * Let ldlm_cancel_lru() be fast.
2361                  */
2362                 ldlm_lock_remove_from_lru(lock);
2363                 ldlm_set_bl_ast(lock);
2364         }
2365         unlock_res_and_lock(lock);
2366
2367         /*
2368          * We want the ost thread to get this reply so that it can respond
2369          * to ost requests (write cache writeback) that might be triggered
2370          * in the callback.
2371          *
2372          * But we'd also like to be able to indicate in the reply that we're
2373          * cancelling right now, because it's unused, or have an intent result
2374          * in the reply, so we might have to push the responsibility for sending
2375          * the reply down into the AST handlers, alas.
2376          */
2377
2378         switch (lustre_msg_get_opc(req->rq_reqmsg)) {
2379         case LDLM_BL_CALLBACK:
2380                 CDEBUG(D_INODE, "blocking ast\n");
2381                 req_capsule_extend(&req->rq_pill, &RQF_LDLM_BL_CALLBACK);
2382                 if (!ldlm_is_cancel_on_block(lock)) {
2383                         rc = ldlm_callback_reply(req, 0);
2384                         if (req->rq_no_reply || rc)
2385                                 ldlm_callback_errmsg(req, "Normal process", rc,
2386                                                      &dlm_req->lock_handle[0]);
2387                 }
2388                 if (ldlm_bl_to_thread_lock(ns, &dlm_req->lock_desc, lock))
2389                         ldlm_handle_bl_callback(ns, &dlm_req->lock_desc, lock);
2390                 break;
2391         case LDLM_CP_CALLBACK:
2392                 CDEBUG(D_INODE, "completion ast\n");
2393                 req_capsule_extend(&req->rq_pill, &RQF_LDLM_CP_CALLBACK);
2394                 ldlm_callback_reply(req, 0);
2395                 ldlm_handle_cp_callback(req, ns, dlm_req, lock);
2396                 break;
2397         case LDLM_GL_CALLBACK:
2398                 CDEBUG(D_INODE, "glimpse ast\n");
2399                 req_capsule_extend(&req->rq_pill, &RQF_LDLM_GL_CALLBACK);
2400                 ldlm_handle_gl_callback(req, ns, dlm_req, lock);
2401                 break;
2402         default:
2403                 LBUG(); /* checked above */
2404         }
2405
2406         RETURN(0);
2407 }
2408
2409 #ifdef HAVE_SERVER_SUPPORT
2410 /**
2411  * Main handler for canceld thread.
2412  *
2413  * Separated into its own thread to avoid deadlocks.
2414  */
2415 static int ldlm_cancel_handler(struct ptlrpc_request *req)
2416 {
2417         int rc;
2418
2419         ENTRY;
2420
2421         /*
2422          * Requests arrive in sender's byte order.  The ptlrpc service
2423          * handler has already checked and, if necessary, byte-swapped the
2424          * incoming request message body, but I am responsible for the
2425          * message buffers.
2426          */
2427
2428         req_capsule_init(&req->rq_pill, req, RCL_SERVER);
2429
2430         if (req->rq_export == NULL) {
2431                 struct ldlm_request *dlm_req;
2432
2433                 CERROR("%s from %s arrived at %llu with bad export cookie %llu\n",
2434                        ll_opcode2str(lustre_msg_get_opc(req->rq_reqmsg)),
2435                        libcfs_nid2str(req->rq_peer.nid),
2436                        (unsigned long long)req->rq_arrival_time.tv_sec,
2437                        lustre_msg_get_handle(req->rq_reqmsg)->cookie);
2438
2439                 if (lustre_msg_get_opc(req->rq_reqmsg) == LDLM_CANCEL) {
2440                         req_capsule_set(&req->rq_pill, &RQF_LDLM_CALLBACK);
2441                         dlm_req = req_capsule_client_get(&req->rq_pill,
2442                                                          &RMF_DLM_REQ);
2443                         if (dlm_req != NULL)
2444                                 ldlm_lock_dump_handle(D_ERROR,
2445                                                       &dlm_req->lock_handle[0]);
2446                 }
2447                 ldlm_callback_reply(req, -ENOTCONN);
2448                 RETURN(0);
2449         }
2450
2451         switch (lustre_msg_get_opc(req->rq_reqmsg)) {
2452         /* XXX FIXME move this back to mds/handler.c, b=249 */
2453         case LDLM_CANCEL:
2454                 req_capsule_set(&req->rq_pill, &RQF_LDLM_CANCEL);
2455                 CDEBUG(D_INODE, "cancel\n");
2456                 if (CFS_FAIL_CHECK(OBD_FAIL_LDLM_CANCEL_NET) ||
2457                     CFS_FAIL_CHECK(OBD_FAIL_PTLRPC_CANCEL_RESEND) ||
2458                     CFS_FAIL_CHECK(OBD_FAIL_LDLM_BL_EVICT))
2459                         RETURN(0);
2460                 rc = ldlm_handle_cancel(req);
2461                 break;
2462         case LDLM_CONVERT:
2463         {
2464                 struct ldlm_request *dlm_req;
2465
2466                 req_capsule_set(&req->rq_pill, &RQF_LDLM_CONVERT);
2467                 CDEBUG(D_INODE, "convert\n");
2468
2469                 dlm_req = req_capsule_client_get(&req->rq_pill, &RMF_DLM_REQ);
2470                 if (dlm_req == NULL) {
2471                         CDEBUG(D_INFO, "bad request buffer for cancel\n");
2472                         rc = ldlm_callback_reply(req, -EPROTO);
2473                 } else {
2474                         req->rq_status = ldlm_handle_convert0(req, dlm_req);
2475                         rc = ptlrpc_reply(req);
2476                 }
2477                 break;
2478         }
2479         default:
2480                 CERROR("invalid opcode %d\n",
2481                        lustre_msg_get_opc(req->rq_reqmsg));
2482                 req_capsule_set(&req->rq_pill, &RQF_LDLM_CALLBACK);
2483                 rc = ldlm_callback_reply(req, -EINVAL);
2484         }
2485
2486         RETURN(rc);
2487 }
2488
2489 static int ldlm_cancel_hpreq_lock_match(struct ptlrpc_request *req,
2490                                         struct ldlm_lock *lock)
2491 {
2492         struct ldlm_request *dlm_req;
2493         struct lustre_handle lockh;
2494         int rc = 0;
2495         int i;
2496
2497         ENTRY;
2498
2499         dlm_req = req_capsule_client_get(&req->rq_pill, &RMF_DLM_REQ);
2500         if (dlm_req == NULL)
2501                 RETURN(0);
2502
2503         ldlm_lock2handle(lock, &lockh);
2504         for (i = 0; i < dlm_req->lock_count; i++) {
2505                 if (lustre_handle_equal(&dlm_req->lock_handle[i],
2506                                         &lockh)) {
2507                         DEBUG_REQ(D_RPCTRACE, req,
2508                                   "Prio raised by lock %#llx", lockh.cookie);
2509                         rc = 1;
2510                         break;
2511                 }
2512         }
2513
2514         RETURN(rc);
2515 }
2516
2517 static int ldlm_cancel_hpreq_check(struct ptlrpc_request *req)
2518 {
2519         struct ldlm_request *dlm_req;
2520         int rc = 0;
2521         int i;
2522         unsigned int size;
2523
2524         ENTRY;
2525
2526         /* no prolong in recovery */
2527         if (lustre_msg_get_flags(req->rq_reqmsg) & MSG_REPLAY)
2528                 RETURN(0);
2529
2530         dlm_req = req_capsule_client_get(&req->rq_pill, &RMF_DLM_REQ);
2531         if (dlm_req == NULL)
2532                 RETURN(-EFAULT);
2533
2534         size = req_capsule_get_size(&req->rq_pill, &RMF_DLM_REQ, RCL_CLIENT);
2535         if (size <= offsetof(struct ldlm_request, lock_handle) ||
2536             (size - offsetof(struct ldlm_request, lock_handle)) /
2537              sizeof(struct lustre_handle) < dlm_req->lock_count)
2538                 RETURN(-EPROTO);
2539
2540         for (i = 0; i < dlm_req->lock_count; i++) {
2541                 struct ldlm_lock *lock;
2542
2543                 lock = ldlm_handle2lock(&dlm_req->lock_handle[i]);
2544                 if (lock == NULL)
2545                         continue;
2546
2547                 rc = ldlm_is_ast_sent(lock) ? 1 : 0;
2548                 if (rc)
2549                         LDLM_DEBUG(lock, "hpreq cancel/convert lock");
2550                 LDLM_LOCK_PUT(lock);
2551
2552                 if (rc)
2553                         break;
2554         }
2555
2556         RETURN(rc);
2557 }
2558
2559 static struct ptlrpc_hpreq_ops ldlm_cancel_hpreq_ops = {
2560         .hpreq_lock_match = ldlm_cancel_hpreq_lock_match,
2561         .hpreq_check      = ldlm_cancel_hpreq_check,
2562         .hpreq_fini       = NULL,
2563 };
2564
2565 static int ldlm_hpreq_handler(struct ptlrpc_request *req)
2566 {
2567         ENTRY;
2568
2569         req_capsule_init(&req->rq_pill, req, RCL_SERVER);
2570
2571         if (req->rq_export == NULL)
2572                 RETURN(0);
2573
2574         if (LDLM_CANCEL == lustre_msg_get_opc(req->rq_reqmsg)) {
2575                 req_capsule_set(&req->rq_pill, &RQF_LDLM_CANCEL);
2576                 req->rq_ops = &ldlm_cancel_hpreq_ops;
2577         } else if (LDLM_CONVERT == lustre_msg_get_opc(req->rq_reqmsg)) {
2578                 req_capsule_set(&req->rq_pill, &RQF_LDLM_CONVERT);
2579                 req->rq_ops = &ldlm_cancel_hpreq_ops;
2580         }
2581         RETURN(0);
2582 }
2583
2584 static int ldlm_revoke_lock_cb(struct cfs_hash *hs, struct cfs_hash_bd *bd,
2585                                struct hlist_node *hnode, void *data)
2586
2587 {
2588         struct list_head *rpc_list = data;
2589         struct ldlm_lock *lock = cfs_hash_object(hs, hnode);
2590
2591         lock_res_and_lock(lock);
2592
2593         if (!ldlm_is_granted(lock)) {
2594                 unlock_res_and_lock(lock);
2595                 return 0;
2596         }
2597
2598         LASSERT(lock->l_resource);
2599         if (lock->l_resource->lr_type != LDLM_IBITS &&
2600             lock->l_resource->lr_type != LDLM_PLAIN) {
2601                 unlock_res_and_lock(lock);
2602                 return 0;
2603         }
2604
2605         if (ldlm_is_ast_sent(lock)) {
2606                 unlock_res_and_lock(lock);
2607                 return 0;
2608         }
2609
2610         LASSERT(lock->l_blocking_ast);
2611         LASSERT(!lock->l_blocking_lock);
2612
2613         ldlm_set_ast_sent(lock);
2614         if (lock->l_export && lock->l_export->exp_lock_hash) {
2615                 /*
2616                  * NB: it's safe to call cfs_hash_del() even lock isn't
2617                  * in exp_lock_hash.
2618                  */
2619                 /*
2620                  * In the function below, .hs_keycmp resolves to
2621                  * ldlm_export_lock_keycmp()
2622                  */
2623                 /* coverity[overrun-buffer-val] */
2624                 cfs_hash_del(lock->l_export->exp_lock_hash,
2625                              &lock->l_remote_handle, &lock->l_exp_hash);
2626         }
2627
2628         list_add_tail(&lock->l_rk_ast, rpc_list);
2629         LDLM_LOCK_GET(lock);
2630
2631         unlock_res_and_lock(lock);
2632         return 0;
2633 }
2634
2635 void ldlm_revoke_export_locks(struct obd_export *exp)
2636 {
2637         struct list_head rpc_list;
2638
2639         ENTRY;
2640
2641         INIT_LIST_HEAD(&rpc_list);
2642         cfs_hash_for_each_nolock(exp->exp_lock_hash,
2643                                  ldlm_revoke_lock_cb, &rpc_list, 0);
2644         ldlm_run_ast_work(exp->exp_obd->obd_namespace, &rpc_list,
2645                           LDLM_WORK_REVOKE_AST);
2646
2647         EXIT;
2648 }
2649 EXPORT_SYMBOL(ldlm_revoke_export_locks);
2650 #endif /* HAVE_SERVER_SUPPORT */
2651
2652 static int ldlm_bl_get_work(struct ldlm_bl_pool *blp,
2653                             struct ldlm_bl_work_item **p_blwi,
2654                             struct obd_export **p_exp)
2655 {
2656         struct ldlm_bl_work_item *blwi = NULL;
2657         static unsigned int num_bl;
2658         static unsigned int num_stale;
2659         int num_th = atomic_read(&blp->blp_num_threads);
2660
2661         *p_exp = obd_stale_export_get();
2662
2663         spin_lock(&blp->blp_lock);
2664         if (*p_exp != NULL) {
2665                 if (num_th == 1 || ++num_stale < num_th) {
2666                         spin_unlock(&blp->blp_lock);
2667                         return 1;
2668                 }
2669                 num_stale = 0;
2670         }
2671
2672         /* process a request from the blp_list at least every blp_num_threads */
2673         if (!list_empty(&blp->blp_list) &&
2674             (list_empty(&blp->blp_prio_list) || num_bl == 0))
2675                 blwi = list_entry(blp->blp_list.next,
2676                                   struct ldlm_bl_work_item, blwi_entry);
2677         else
2678                 if (!list_empty(&blp->blp_prio_list))
2679                         blwi = list_entry(blp->blp_prio_list.next,
2680                                           struct ldlm_bl_work_item,
2681                                           blwi_entry);
2682
2683         if (blwi) {
2684                 if (++num_bl >= num_th)
2685                         num_bl = 0;
2686                 list_del(&blwi->blwi_entry);
2687         }
2688         spin_unlock(&blp->blp_lock);
2689         *p_blwi = blwi;
2690
2691         if (*p_exp != NULL && *p_blwi != NULL) {
2692                 obd_stale_export_put(*p_exp);
2693                 *p_exp = NULL;
2694         }
2695
2696         return (*p_blwi != NULL || *p_exp != NULL) ? 1 : 0;
2697 }
2698
2699 /* This only contains temporary data until the thread starts */
2700 struct ldlm_bl_thread_data {
2701         struct ldlm_bl_pool     *bltd_blp;
2702         struct completion       bltd_comp;
2703         int                     bltd_num;
2704 };
2705
2706 static int ldlm_bl_thread_main(void *arg);
2707
2708 static int ldlm_bl_thread_start(struct ldlm_bl_pool *blp, bool check_busy)
2709 {
2710         struct ldlm_bl_thread_data bltd = { .bltd_blp = blp };
2711         struct task_struct *task;
2712
2713         init_completion(&bltd.bltd_comp);
2714
2715         bltd.bltd_num = atomic_inc_return(&blp->blp_num_threads);
2716         if (bltd.bltd_num >= blp->blp_max_threads) {
2717                 atomic_dec(&blp->blp_num_threads);
2718                 return 0;
2719         }
2720
2721         LASSERTF(bltd.bltd_num > 0, "thread num:%d\n", bltd.bltd_num);
2722         if (check_busy &&
2723             atomic_read(&blp->blp_busy_threads) < (bltd.bltd_num - 1)) {
2724                 atomic_dec(&blp->blp_num_threads);
2725                 return 0;
2726         }
2727
2728         task = kthread_run(ldlm_bl_thread_main, &bltd, "ldlm_bl_%02d",
2729                            bltd.bltd_num);
2730         if (IS_ERR(task)) {
2731                 CERROR("cannot start LDLM thread ldlm_bl_%02d: rc %ld\n",
2732                        bltd.bltd_num, PTR_ERR(task));
2733                 atomic_dec(&blp->blp_num_threads);
2734                 return PTR_ERR(task);
2735         }
2736         wait_for_completion(&bltd.bltd_comp);
2737
2738         return 0;
2739 }
2740
2741 /* Not fatal if racy and have a few too many threads */
2742 static int ldlm_bl_thread_need_create(struct ldlm_bl_pool *blp,
2743                                       struct ldlm_bl_work_item *blwi)
2744 {
2745         if (atomic_read(&blp->blp_num_threads) >= blp->blp_max_threads)
2746                 return 0;
2747
2748         if (atomic_read(&blp->blp_busy_threads) <
2749             atomic_read(&blp->blp_num_threads))
2750                 return 0;
2751
2752         if (blwi != NULL && (blwi->blwi_ns == NULL ||
2753                              blwi->blwi_mem_pressure))
2754                 return 0;
2755
2756         return 1;
2757 }
2758
2759 static int ldlm_bl_thread_blwi(struct ldlm_bl_pool *blp,
2760                                struct ldlm_bl_work_item *blwi)
2761 {
2762         ENTRY;
2763
2764         if (blwi->blwi_ns == NULL)
2765                 /* added by ldlm_cleanup() */
2766                 RETURN(LDLM_ITER_STOP);
2767
2768         if (blwi->blwi_mem_pressure)
2769                 memory_pressure_set();
2770
2771         OBD_FAIL_TIMEOUT(OBD_FAIL_LDLM_PAUSE_CANCEL2, 4);
2772
2773         if (blwi->blwi_count) {
2774                 int count;
2775                 /*
2776                  * The special case when we cancel locks in lru
2777                  * asynchronously, we pass the list of locks here.
2778                  * Thus locks are marked LDLM_FL_CANCELING, but NOT
2779                  * canceled locally yet.
2780                  */
2781                 count = ldlm_cli_cancel_list_local(&blwi->blwi_head,
2782                                                    blwi->blwi_count,
2783                                                    LCF_BL_AST);
2784                 ldlm_cli_cancel_list(&blwi->blwi_head, count, NULL,
2785                                      blwi->blwi_flags);
2786         } else {
2787                 ldlm_handle_bl_callback(blwi->blwi_ns, &blwi->blwi_ld,
2788                                         blwi->blwi_lock);
2789         }
2790         if (blwi->blwi_mem_pressure)
2791                 memory_pressure_clr();
2792
2793         if (blwi->blwi_flags & LCF_ASYNC)
2794                 OBD_FREE(blwi, sizeof(*blwi));
2795         else
2796                 complete(&blwi->blwi_comp);
2797
2798         RETURN(0);
2799 }
2800
2801 /**
2802  * Cancel stale locks on export. Cancel blocked locks first.
2803  * If the given export has blocked locks, the next in the list may have
2804  * them too, thus cancel not blocked locks only if the current export has
2805  * no blocked locks.
2806  **/
2807 static int ldlm_bl_thread_exports(struct ldlm_bl_pool *blp,
2808                                   struct obd_export *exp)
2809 {
2810         int num;
2811
2812         ENTRY;
2813
2814         OBD_FAIL_TIMEOUT(OBD_FAIL_LDLM_BL_EVICT, 4);
2815
2816         num = ldlm_export_cancel_blocked_locks(exp);
2817         if (num == 0)
2818                 ldlm_export_cancel_locks(exp);
2819
2820         obd_stale_export_put(exp);
2821
2822         RETURN(0);
2823 }
2824
2825
2826 /**
2827  * Main blocking requests processing thread.
2828  *
2829  * Callers put locks into its queue by calling ldlm_bl_to_thread.
2830  * This thread in the end ends up doing actual call to ->l_blocking_ast
2831  * for queued locks.
2832  */
2833 static int ldlm_bl_thread_main(void *arg)
2834 {
2835         struct lu_env *env;
2836         struct ldlm_bl_pool *blp;
2837         struct ldlm_bl_thread_data *bltd = arg;
2838         int rc;
2839
2840         ENTRY;
2841
2842         OBD_ALLOC_PTR(env);
2843         if (!env)
2844                 RETURN(-ENOMEM);
2845         rc = lu_env_init(env, LCT_DT_THREAD);
2846         if (rc)
2847                 GOTO(out_env, rc);
2848         rc = lu_env_add(env);
2849         if (rc)
2850                 GOTO(out_env_fini, rc);
2851
2852         blp = bltd->bltd_blp;
2853
2854         complete(&bltd->bltd_comp);
2855         /* cannot use bltd after this, it is only on caller's stack */
2856
2857         while (1) {
2858                 struct l_wait_info lwi = { 0 };
2859                 struct ldlm_bl_work_item *blwi = NULL;
2860                 struct obd_export *exp = NULL;
2861                 int rc;
2862
2863                 rc = ldlm_bl_get_work(blp, &blwi, &exp);
2864
2865                 if (rc == 0)
2866                         l_wait_event_exclusive(blp->blp_waitq,
2867                                                ldlm_bl_get_work(blp, &blwi,
2868                                                                 &exp),
2869                                                &lwi);
2870                 atomic_inc(&blp->blp_busy_threads);
2871
2872                 if (ldlm_bl_thread_need_create(blp, blwi))
2873                         /* discard the return value, we tried */
2874                         ldlm_bl_thread_start(blp, true);
2875
2876                 if (exp)
2877                         rc = ldlm_bl_thread_exports(blp, exp);
2878                 else if (blwi)
2879                         rc = ldlm_bl_thread_blwi(blp, blwi);
2880
2881                 atomic_dec(&blp->blp_busy_threads);
2882
2883                 if (rc == LDLM_ITER_STOP)
2884                         break;
2885
2886                 /*
2887                  * If there are many namespaces, we will not sleep waiting for
2888                  * work, and must do a cond_resched to avoid holding the CPU
2889                  * for too long
2890                  */
2891                 cond_resched();
2892         }
2893
2894         atomic_dec(&blp->blp_num_threads);
2895         complete(&blp->blp_comp);
2896
2897         lu_env_remove(env);
2898 out_env_fini:
2899         lu_env_fini(env);
2900 out_env:
2901         OBD_FREE_PTR(env);
2902         RETURN(rc);
2903 }
2904
2905
2906 static int ldlm_setup(void);
2907 static int ldlm_cleanup(void);
2908
2909 int ldlm_get_ref(void)
2910 {
2911         int rc = 0;
2912
2913         ENTRY;
2914         mutex_lock(&ldlm_ref_mutex);
2915         if (++ldlm_refcount == 1) {
2916                 rc = ldlm_setup();
2917                 if (rc)
2918                         ldlm_refcount--;
2919         }
2920         mutex_unlock(&ldlm_ref_mutex);
2921
2922         RETURN(rc);
2923 }
2924
2925 void ldlm_put_ref(void)
2926 {
2927         ENTRY;
2928         mutex_lock(&ldlm_ref_mutex);
2929         if (ldlm_refcount == 1) {
2930                 int rc = ldlm_cleanup();
2931
2932                 if (rc)
2933                         CERROR("ldlm_cleanup failed: %d\n", rc);
2934                 else
2935                         ldlm_refcount--;
2936         } else {
2937                 ldlm_refcount--;
2938         }
2939         mutex_unlock(&ldlm_ref_mutex);
2940
2941         EXIT;
2942 }
2943
2944 /*
2945  * Export handle<->lock hash operations.
2946  */
2947 static unsigned
2948 ldlm_export_lock_hash(struct cfs_hash *hs, const void *key, unsigned int mask)
2949 {
2950         return cfs_hash_u64_hash(((struct lustre_handle *)key)->cookie, mask);
2951 }
2952
2953 static void *
2954 ldlm_export_lock_key(struct hlist_node *hnode)
2955 {
2956         struct ldlm_lock *lock;
2957
2958         lock = hlist_entry(hnode, struct ldlm_lock, l_exp_hash);
2959         return &lock->l_remote_handle;
2960 }
2961
2962 static void
2963 ldlm_export_lock_keycpy(struct hlist_node *hnode, void *key)
2964 {
2965         struct ldlm_lock     *lock;
2966
2967         lock = hlist_entry(hnode, struct ldlm_lock, l_exp_hash);
2968         lock->l_remote_handle = *(struct lustre_handle *)key;
2969 }
2970
2971 static int
2972 ldlm_export_lock_keycmp(const void *key, struct hlist_node *hnode)
2973 {
2974         return lustre_handle_equal(ldlm_export_lock_key(hnode), key);
2975 }
2976
2977 static void *
2978 ldlm_export_lock_object(struct hlist_node *hnode)
2979 {
2980         return hlist_entry(hnode, struct ldlm_lock, l_exp_hash);
2981 }
2982
2983 static void
2984 ldlm_export_lock_get(struct cfs_hash *hs, struct hlist_node *hnode)
2985 {
2986         struct ldlm_lock *lock;
2987
2988         lock = hlist_entry(hnode, struct ldlm_lock, l_exp_hash);
2989         LDLM_LOCK_GET(lock);
2990 }
2991
2992 static void
2993 ldlm_export_lock_put(struct cfs_hash *hs, struct hlist_node *hnode)
2994 {
2995         struct ldlm_lock *lock;
2996
2997         lock = hlist_entry(hnode, struct ldlm_lock, l_exp_hash);
2998         LDLM_LOCK_RELEASE(lock);
2999 }
3000
3001 static struct cfs_hash_ops ldlm_export_lock_ops = {
3002         .hs_hash        = ldlm_export_lock_hash,
3003         .hs_key         = ldlm_export_lock_key,
3004         .hs_keycmp      = ldlm_export_lock_keycmp,
3005         .hs_keycpy      = ldlm_export_lock_keycpy,
3006         .hs_object      = ldlm_export_lock_object,
3007         .hs_get         = ldlm_export_lock_get,
3008         .hs_put         = ldlm_export_lock_put,
3009         .hs_put_locked  = ldlm_export_lock_put,
3010 };
3011
3012 int ldlm_init_export(struct obd_export *exp)
3013 {
3014         int rc;
3015
3016         ENTRY;
3017
3018         exp->exp_lock_hash =
3019                 cfs_hash_create(obd_uuid2str(&exp->exp_client_uuid),
3020                                 HASH_EXP_LOCK_CUR_BITS,
3021                                 HASH_EXP_LOCK_MAX_BITS,
3022                                 HASH_EXP_LOCK_BKT_BITS, 0,
3023                                 CFS_HASH_MIN_THETA, CFS_HASH_MAX_THETA,
3024                                 &ldlm_export_lock_ops,
3025                                 CFS_HASH_DEFAULT | CFS_HASH_REHASH_KEY |
3026                                 CFS_HASH_NBLK_CHANGE);
3027
3028         if (!exp->exp_lock_hash)
3029                 RETURN(-ENOMEM);
3030
3031         rc = ldlm_init_flock_export(exp);
3032         if (rc)
3033                 GOTO(err, rc);
3034
3035         RETURN(0);
3036 err:
3037         ldlm_destroy_export(exp);
3038         RETURN(rc);
3039 }
3040 EXPORT_SYMBOL(ldlm_init_export);
3041
3042 void ldlm_destroy_export(struct obd_export *exp)
3043 {
3044         ENTRY;
3045         cfs_hash_putref(exp->exp_lock_hash);
3046         exp->exp_lock_hash = NULL;
3047
3048         ldlm_destroy_flock_export(exp);
3049         EXIT;
3050 }
3051 EXPORT_SYMBOL(ldlm_destroy_export);
3052
3053 static ssize_t cancel_unused_locks_before_replay_show(struct kobject *kobj,
3054                                                       struct attribute *attr,
3055                                                       char *buf)
3056 {
3057         return sprintf(buf, "%d\n", ldlm_cancel_unused_locks_before_replay);
3058 }
3059
3060 static ssize_t cancel_unused_locks_before_replay_store(struct kobject *kobj,
3061                                                        struct attribute *attr,
3062                                                        const char *buffer,
3063                                                        size_t count)
3064 {
3065         int rc;
3066         unsigned long val;
3067
3068         rc = kstrtoul(buffer, 10, &val);
3069         if (rc)
3070                 return rc;
3071
3072         ldlm_cancel_unused_locks_before_replay = val;
3073
3074         return count;
3075 }
3076 LUSTRE_RW_ATTR(cancel_unused_locks_before_replay);
3077
3078 static struct attribute *ldlm_attrs[] = {
3079         &lustre_attr_cancel_unused_locks_before_replay.attr,
3080         NULL,
3081 };
3082
3083 static struct attribute_group ldlm_attr_group = {
3084         .attrs = ldlm_attrs,
3085 };
3086
3087 static int ldlm_setup(void)
3088 {
3089         static struct ptlrpc_service_conf       conf;
3090         struct ldlm_bl_pool                    *blp = NULL;
3091 #ifdef HAVE_SERVER_SUPPORT
3092         struct task_struct *task;
3093 #endif /* HAVE_SERVER_SUPPORT */
3094         int i;
3095         int rc = 0;
3096
3097         ENTRY;
3098
3099         if (ldlm_state != NULL)
3100                 RETURN(-EALREADY);
3101
3102         OBD_ALLOC(ldlm_state, sizeof(*ldlm_state));
3103         if (ldlm_state == NULL)
3104                 RETURN(-ENOMEM);
3105
3106         ldlm_kobj = kobject_create_and_add("ldlm", &lustre_kset->kobj);
3107         if (!ldlm_kobj)
3108                 GOTO(out, -ENOMEM);
3109
3110         rc = sysfs_create_group(ldlm_kobj, &ldlm_attr_group);
3111         if (rc)
3112                 GOTO(out, rc);
3113
3114         ldlm_ns_kset = kset_create_and_add("namespaces", NULL, ldlm_kobj);
3115         if (!ldlm_ns_kset)
3116                 GOTO(out, -ENOMEM);
3117
3118         ldlm_svc_kset = kset_create_and_add("services", NULL, ldlm_kobj);
3119         if (!ldlm_svc_kset)
3120                 GOTO(out, -ENOMEM);
3121
3122         rc = ldlm_debugfs_setup();
3123         if (rc != 0)
3124                 GOTO(out, rc);
3125
3126         memset(&conf, 0, sizeof(conf));
3127         conf = (typeof(conf)) {
3128                 .psc_name               = "ldlm_cbd",
3129                 .psc_watchdog_factor    = 2,
3130                 .psc_buf                = {
3131                         .bc_nbufs               = LDLM_CLIENT_NBUFS,
3132                         .bc_buf_size            = LDLM_BUFSIZE,
3133                         .bc_req_max_size        = LDLM_MAXREQSIZE,
3134                         .bc_rep_max_size        = LDLM_MAXREPSIZE,
3135                         .bc_req_portal          = LDLM_CB_REQUEST_PORTAL,
3136                         .bc_rep_portal          = LDLM_CB_REPLY_PORTAL,
3137                 },
3138                 .psc_thr                = {
3139                         .tc_thr_name            = "ldlm_cb",
3140                         .tc_thr_factor          = LDLM_THR_FACTOR,
3141                         .tc_nthrs_init          = LDLM_NTHRS_INIT,
3142                         .tc_nthrs_base          = LDLM_NTHRS_BASE,
3143                         .tc_nthrs_max           = LDLM_NTHRS_MAX,
3144                         .tc_nthrs_user          = ldlm_num_threads,
3145                         .tc_cpu_bind            = ldlm_cpu_bind,
3146                         .tc_ctx_tags            = LCT_MD_THREAD | LCT_DT_THREAD,
3147                 },
3148                 .psc_cpt                = {
3149                         .cc_pattern             = ldlm_cpts,
3150                         .cc_affinity            = true,
3151                 },
3152                 .psc_ops                = {
3153                         .so_req_handler         = ldlm_callback_handler,
3154                 },
3155         };
3156         ldlm_state->ldlm_cb_service = \
3157                         ptlrpc_register_service(&conf, ldlm_svc_kset,
3158                                                 ldlm_svc_debugfs_dir);
3159         if (IS_ERR(ldlm_state->ldlm_cb_service)) {
3160                 CERROR("failed to start service\n");
3161                 rc = PTR_ERR(ldlm_state->ldlm_cb_service);
3162                 ldlm_state->ldlm_cb_service = NULL;
3163                 GOTO(out, rc);
3164         }
3165
3166 #ifdef HAVE_SERVER_SUPPORT
3167         memset(&conf, 0, sizeof(conf));
3168         conf = (typeof(conf)) {
3169                 .psc_name               = "ldlm_canceld",
3170                 .psc_watchdog_factor    = 6,
3171                 .psc_buf                = {
3172                         .bc_nbufs               = LDLM_SERVER_NBUFS,
3173                         .bc_buf_size            = LDLM_BUFSIZE,
3174                         .bc_req_max_size        = LDLM_MAXREQSIZE,
3175                         .bc_rep_max_size        = LDLM_MAXREPSIZE,
3176                         .bc_req_portal          = LDLM_CANCEL_REQUEST_PORTAL,
3177                         .bc_rep_portal          = LDLM_CANCEL_REPLY_PORTAL,
3178
3179                 },
3180                 .psc_thr                = {
3181                         .tc_thr_name            = "ldlm_cn",
3182                         .tc_thr_factor          = LDLM_THR_FACTOR,
3183                         .tc_nthrs_init          = LDLM_NTHRS_INIT,
3184                         .tc_nthrs_base          = LDLM_NTHRS_BASE,
3185                         .tc_nthrs_max           = LDLM_NTHRS_MAX,
3186                         .tc_nthrs_user          = ldlm_num_threads,
3187                         .tc_cpu_bind            = ldlm_cpu_bind,
3188                         .tc_ctx_tags            = LCT_MD_THREAD | \
3189                                                   LCT_DT_THREAD | \
3190                                                   LCT_CL_THREAD,
3191                 },
3192                 .psc_cpt                = {
3193                         .cc_pattern             = ldlm_cpts,
3194                         .cc_affinity            = true,
3195                 },
3196                 .psc_ops                = {
3197                         .so_req_handler         = ldlm_cancel_handler,
3198                         .so_hpreq_handler       = ldlm_hpreq_handler,
3199                 },
3200         };
3201         ldlm_state->ldlm_cancel_service = \
3202                         ptlrpc_register_service(&conf, ldlm_svc_kset,
3203                                                 ldlm_svc_debugfs_dir);
3204         if (IS_ERR(ldlm_state->ldlm_cancel_service)) {
3205                 CERROR("failed to start service\n");
3206                 rc = PTR_ERR(ldlm_state->ldlm_cancel_service);
3207                 ldlm_state->ldlm_cancel_service = NULL;
3208                 GOTO(out, rc);
3209         }
3210 #endif /* HAVE_SERVER_SUPPORT */
3211
3212         OBD_ALLOC(blp, sizeof(*blp));
3213         if (blp == NULL)
3214                 GOTO(out, rc = -ENOMEM);
3215         ldlm_state->ldlm_bl_pool = blp;
3216
3217         spin_lock_init(&blp->blp_lock);
3218         INIT_LIST_HEAD(&blp->blp_list);
3219         INIT_LIST_HEAD(&blp->blp_prio_list);
3220         init_waitqueue_head(&blp->blp_waitq);
3221         atomic_set(&blp->blp_num_threads, 0);
3222         atomic_set(&blp->blp_busy_threads, 0);
3223
3224         if (ldlm_num_threads == 0) {
3225                 blp->blp_min_threads = LDLM_NTHRS_INIT;
3226                 blp->blp_max_threads = LDLM_NTHRS_MAX;
3227         } else {
3228                 blp->blp_min_threads = blp->blp_max_threads = \
3229                         min_t(int, LDLM_NTHRS_MAX, max_t(int, LDLM_NTHRS_INIT,
3230                                                          ldlm_num_threads));
3231         }
3232
3233         for (i = 0; i < blp->blp_min_threads; i++) {
3234                 rc = ldlm_bl_thread_start(blp, false);
3235                 if (rc < 0)
3236                         GOTO(out, rc);
3237         }
3238
3239 #ifdef HAVE_SERVER_SUPPORT
3240         task = kthread_run(expired_lock_main, NULL, "ldlm_elt");
3241         if (IS_ERR(task)) {
3242                 rc = PTR_ERR(task);
3243                 CERROR("Cannot start ldlm expired-lock thread: %d\n", rc);
3244                 GOTO(out, rc);
3245         }
3246
3247         wait_event(expired_lock_wait_queue,
3248                    expired_lock_thread_state == ELT_READY);
3249 #endif /* HAVE_SERVER_SUPPORT */
3250
3251         rc = ldlm_pools_init();
3252         if (rc) {
3253                 CERROR("Failed to initialize LDLM pools: %d\n", rc);
3254                 GOTO(out, rc);
3255         }
3256
3257         rc = ldlm_reclaim_setup();
3258         if (rc) {
3259                 CERROR("Failed to setup reclaim thread: rc = %d\n", rc);
3260                 GOTO(out, rc);
3261         }
3262         RETURN(0);
3263
3264  out:
3265         ldlm_cleanup();
3266         RETURN(rc);
3267 }
3268
3269 static int ldlm_cleanup(void)
3270 {
3271         ENTRY;
3272
3273         if (!list_empty(ldlm_namespace_list(LDLM_NAMESPACE_SERVER)) ||
3274             !list_empty(ldlm_namespace_list(LDLM_NAMESPACE_CLIENT))) {
3275                 CERROR("ldlm still has namespaces; clean these up first.\n");
3276                 ldlm_dump_all_namespaces(LDLM_NAMESPACE_SERVER, D_DLMTRACE);
3277                 ldlm_dump_all_namespaces(LDLM_NAMESPACE_CLIENT, D_DLMTRACE);
3278                 RETURN(-EBUSY);
3279         }
3280
3281         ldlm_reclaim_cleanup();
3282         ldlm_pools_fini();
3283
3284         if (ldlm_state->ldlm_bl_pool != NULL) {
3285                 struct ldlm_bl_pool *blp = ldlm_state->ldlm_bl_pool;
3286
3287                 while (atomic_read(&blp->blp_num_threads) > 0) {
3288                         struct ldlm_bl_work_item blwi = { .blwi_ns = NULL };
3289
3290                         init_completion(&blp->blp_comp);
3291
3292                         spin_lock(&blp->blp_lock);
3293                         list_add_tail(&blwi.blwi_entry, &blp->blp_list);
3294                         wake_up(&blp->blp_waitq);
3295                         spin_unlock(&blp->blp_lock);
3296
3297                         wait_for_completion(&blp->blp_comp);
3298                 }
3299
3300                 OBD_FREE(blp, sizeof(*blp));
3301         }
3302
3303         if (ldlm_state->ldlm_cb_service != NULL)
3304                 ptlrpc_unregister_service(ldlm_state->ldlm_cb_service);
3305 #ifdef HAVE_SERVER_SUPPORT
3306         if (ldlm_state->ldlm_cancel_service != NULL)
3307                 ptlrpc_unregister_service(ldlm_state->ldlm_cancel_service);
3308 #endif
3309
3310         if (ldlm_ns_kset)
3311                 kset_unregister(ldlm_ns_kset);
3312         if (ldlm_svc_kset)
3313                 kset_unregister(ldlm_svc_kset);
3314         if (ldlm_kobj) {
3315                 sysfs_remove_group(ldlm_kobj, &ldlm_attr_group);
3316                 kobject_put(ldlm_kobj);
3317         }
3318
3319         ldlm_debugfs_cleanup();
3320
3321 #ifdef HAVE_SERVER_SUPPORT
3322         if (expired_lock_thread_state != ELT_STOPPED) {
3323                 expired_lock_thread_state = ELT_TERMINATE;
3324                 wake_up(&expired_lock_wait_queue);
3325                 wait_event(expired_lock_wait_queue,
3326                            expired_lock_thread_state == ELT_STOPPED);
3327         }
3328 #endif
3329
3330         OBD_FREE(ldlm_state, sizeof(*ldlm_state));
3331         ldlm_state = NULL;
3332
3333         RETURN(0);
3334 }
3335
3336 int ldlm_init(void)
3337 {
3338         ldlm_resource_slab = kmem_cache_create("ldlm_resources",
3339                                                sizeof(struct ldlm_resource), 0,
3340                                                SLAB_HWCACHE_ALIGN, NULL);
3341         if (ldlm_resource_slab == NULL)
3342                 return -ENOMEM;
3343
3344         ldlm_lock_slab = kmem_cache_create("ldlm_locks",
3345                               sizeof(struct ldlm_lock), 0,
3346                               SLAB_HWCACHE_ALIGN, NULL);
3347         if (ldlm_lock_slab == NULL)
3348                 goto out_resource;
3349
3350         ldlm_interval_slab = kmem_cache_create("interval_node",
3351                                         sizeof(struct ldlm_interval),
3352                                         0, SLAB_HWCACHE_ALIGN, NULL);
3353         if (ldlm_interval_slab == NULL)
3354                 goto out_lock;
3355
3356         ldlm_interval_tree_slab = kmem_cache_create("interval_tree",
3357                         sizeof(struct ldlm_interval_tree) * LCK_MODE_NUM,
3358                         0, SLAB_HWCACHE_ALIGN, NULL);
3359         if (ldlm_interval_tree_slab == NULL)
3360                 goto out_interval;
3361
3362 #ifdef HAVE_SERVER_SUPPORT
3363         ldlm_inodebits_slab = kmem_cache_create("ldlm_ibits_node",
3364                                                 sizeof(struct ldlm_ibits_node),
3365                                                 0, SLAB_HWCACHE_ALIGN, NULL);
3366         if (ldlm_inodebits_slab == NULL)
3367                 goto out_interval_tree;
3368
3369         ldlm_glimpse_work_kmem = kmem_cache_create("ldlm_glimpse_work_kmem",
3370                                         sizeof(struct ldlm_glimpse_work),
3371                                         0, 0, NULL);
3372         if (ldlm_glimpse_work_kmem == NULL)
3373                 goto out_inodebits;
3374 #endif
3375
3376 #if LUSTRE_TRACKS_LOCK_EXP_REFS
3377         class_export_dump_hook = ldlm_dump_export_locks;
3378 #endif
3379         return 0;
3380 #ifdef HAVE_SERVER_SUPPORT
3381 out_inodebits:
3382         kmem_cache_destroy(ldlm_inodebits_slab);
3383 out_interval_tree:
3384         kmem_cache_destroy(ldlm_interval_tree_slab);
3385 #endif
3386 out_interval:
3387         kmem_cache_destroy(ldlm_interval_slab);
3388 out_lock:
3389         kmem_cache_destroy(ldlm_lock_slab);
3390 out_resource:
3391         kmem_cache_destroy(ldlm_resource_slab);
3392
3393         return -ENOMEM;
3394 }
3395
3396 void ldlm_exit(void)
3397 {
3398         if (ldlm_refcount)
3399                 CERROR("ldlm_refcount is %d in ldlm_exit!\n", ldlm_refcount);
3400         kmem_cache_destroy(ldlm_resource_slab);
3401         /*
3402          * ldlm_lock_put() use RCU to call ldlm_lock_free, so need call
3403          * rcu_barrier() to wait all outstanding RCU callbacks to complete,
3404          * so that ldlm_lock_free() get a chance to be called.
3405          */
3406         rcu_barrier();
3407         kmem_cache_destroy(ldlm_lock_slab);
3408         kmem_cache_destroy(ldlm_interval_slab);
3409         kmem_cache_destroy(ldlm_interval_tree_slab);
3410 #ifdef HAVE_SERVER_SUPPORT
3411         kmem_cache_destroy(ldlm_inodebits_slab);
3412         kmem_cache_destroy(ldlm_glimpse_work_kmem);
3413 #endif
3414 }