Whamcloud - gitweb
Update based on lustre-iokit-20051107.tar.gz from the customer download area.
[fs/lustre-release.git] / lustre / ldlm / ldlm_extent.c
index dd04c1c..8240eab 100644 (file)
@@ -45,6 +45,7 @@ ldlm_extent_internal_policy(struct list_head *queue, struct ldlm_lock *req,
         ldlm_mode_t req_mode = req->l_req_mode;
         __u64 req_start = req->l_req_extent.start;
         __u64 req_end = req->l_req_extent.end;
+        int conflicting = 0;
         ENTRY;
 
         lockmode_verify(req_mode);
@@ -65,15 +66,24 @@ ldlm_extent_internal_policy(struct list_head *queue, struct ldlm_lock *req,
                 if (req == lock)
                         continue;
 
+                /* Locks are compatible, overlap doesn't matter */
+                /* Until bug 20 is fixed, try to avoid granting overlapping
+                 * locks on one client (they take a long time to cancel) */
+                if (lockmode_compat(lock->l_req_mode, req_mode) &&
+                    lock->l_export != req->l_export)
+                        continue;
+
+                /* If this is a high-traffic lock, don't grow downwards at all
+                 * or grow upwards too much */
+                ++conflicting;
+                if (conflicting > 4)
+                        new_ex->start = req_start;
+
                 /* If lock doesn't overlap new_ex, skip it. */
                 if (l_extent->end < new_ex->start ||
                     l_extent->start > new_ex->end)
                         continue;
 
-                /* Locks are compatible, overlap doesn't matter */
-                if (lockmode_compat(lock->l_req_mode, req_mode))
-                        continue;
-
                 /* Locks conflicting in requested extents and we can't satisfy
                  * both locks, so ignore it.  Either we will ping-pong this
                  * extent (we would regardless of what extent we granted) or
@@ -85,10 +95,10 @@ ldlm_extent_internal_policy(struct list_head *queue, struct ldlm_lock *req,
                 /* We grow extents downwards only as far as they don't overlap
                  * with already-granted locks, on the assumtion that clients
                  * will be writing beyond the initial requested end and would
-                 * then need to enqueue a new lock beyond the previous request.
-                 * We don't grow downwards if there are lots of lockers. */
-                if (l_extent->start < req_start) {
-                        if (atomic_read(&req->l_resource->lr_refcount) > 20)
+                 * then need to enqueue a new lock beyond previous request.
+                 * l_req_extent->end strictly < req_start, checked above. */
+                if (l_extent->start < req_start && new_ex->start != req_start) {
+                        if (l_extent->end >= req_start)
                                 new_ex->start = req_start;
                         else
                                 new_ex->start = min(l_extent->end+1, req_start);
@@ -107,6 +117,13 @@ ldlm_extent_internal_policy(struct list_head *queue, struct ldlm_lock *req,
                                 new_ex->end = max(l_extent->start - 1, req_end);
                 }
         }
+
+#define LDLM_MAX_GROWN_EXTENT (32 * 1024 * 1024 - 1)
+        if (conflicting > 32 && (req_mode == LCK_PW || req_mode == LCK_CW)) {
+                if (req_end < req_start + LDLM_MAX_GROWN_EXTENT)
+                        new_ex->end = min(req_start + LDLM_MAX_GROWN_EXTENT,
+                                          new_ex->end);
+        }
         EXIT;
 }
 
@@ -117,6 +134,9 @@ static void ldlm_extent_policy(struct ldlm_resource *res,
 {
         struct ldlm_extent new_ex = { .start = 0, .end = ~0};
 
+        if (lock->l_req_mode == LCK_GROUP)
+                return;
+
         ldlm_extent_internal_policy(&res->lr_granted, lock, &new_ex);
         ldlm_extent_internal_policy(&res->lr_waiting, lock, &new_ex);
 
@@ -139,87 +159,207 @@ static void ldlm_extent_policy(struct ldlm_resource *res,
  */
 static int
 ldlm_extent_compat_queue(struct list_head *queue, struct ldlm_lock *req,
-                         int send_cbs, int *flags, ldlm_error_t *err)
+                         int *flags, struct list_head *work_list,
+                         struct list_head **insertp)
 {
         struct list_head *tmp;
-        struct ldlm_lock *lock;
+        struct list_head *save = NULL;
+        struct ldlm_lock *lock = NULL;
         ldlm_mode_t req_mode = req->l_req_mode;
-        __u64 req_start = req->l_req_extent.start;
-        __u64 req_end = req->l_req_extent.end;
         int compat = 1;
+        int found = 0;
         ENTRY;
 
         lockmode_verify(req_mode);
 
+        /* Extent locks are only queued once. We can get back here with
+         * insertp != NULL if the blocking ASTs returned -ERESTART. */ 
+        if (!list_empty(&req->l_res_link))
+                insertp = NULL;
+
+        if (req->l_req_mode != LCK_GROUP) {
+                __u64 req_start = req->l_req_extent.start;
+                __u64 req_end = req->l_req_extent.end;
+
+                list_for_each(tmp, queue) {
+                        lock = list_entry(tmp, struct ldlm_lock, l_res_link);
+                        if (req == lock)
+                                break;
+
+                        if (lock->l_req_mode == LCK_GROUP) {
+                                if (*flags & LDLM_FL_BLOCK_NOWAIT)
+                                        RETURN(-EWOULDBLOCK);
+
+                                /* No blocking ASTs are sent for group locks. */
+                                compat = 0;
+
+                                /* there's a blocking group lock in front
+                                 * of us on the queue.  It can be held
+                                 * indefinitely, so don't timeout. */
+                                if (insertp) {
+                                        *flags |= LDLM_FL_NO_TIMEOUT;
+                                        /* lock_bitlock(req) is held here. */
+                                        req->l_flags |= LDLM_FL_NO_TIMEOUT;
+                                }
+
+                                if (work_list)
+                                        continue;
+                                else
+                                        break;
+                        }
+
+                        /* locks are compatible, overlap doesn't matter */
+                        if (lockmode_compat(lock->l_req_mode, req_mode))
+                                continue;
+
+                        if (lock->l_policy_data.l_extent.end < req_start ||
+                            lock->l_policy_data.l_extent.start > req_end)
+                                continue;
+
+                        compat = 0;
+
+                        if (!work_list)
+                                break;
+
+                        if (lock->l_blocking_ast)
+                                ldlm_add_ast_work_item(lock, req, work_list);
+                }
+
+                if (insertp)
+                        *insertp = queue;
+
+                RETURN(compat);
+        }
+
         list_for_each(tmp, queue) {
                 lock = list_entry(tmp, struct ldlm_lock, l_res_link);
-
                 if (req == lock)
-                        RETURN(compat);
+                        break;
+
+                if (lock->l_req_mode != LCK_GROUP) {
+                        if (lock->l_req_mode != lock->l_granted_mode) {
+                                /* we must be traversing the waitq. */
+
+                                /* If a group lock was already found then
+                                 * req can be queued before any extent locks
+                                 * that come after the found group lock. */
+                                if (found)
+                                        break;
+
+                                if (!insertp) {
+                                        /* We've hit a conflicting extent lock
+                                         * on the waitq before hitting the req
+                                         * group lock. See comments below. */
+                                        compat = 0;
+                                        break;
+                                }
 
-                /* locks are compatible, overlap doesn't matter */
-                if (lockmode_compat(lock->l_req_mode, req_mode)) {
-                        /* nonCW locks are compatible, overlap doesn't matter */
-                        if (req_mode != LCK_CW)
-                                continue;
-                                
-                        /* If we are trying to get a CW lock and there is
-                           another one of this kind, we need to compare gid */
-                        if (req->l_policy_data.l_extent.gid ==
-                             lock->l_policy_data.l_extent.gid) {
-                                if (lock->l_req_mode == lock->l_granted_mode)
-                                        RETURN(2);
-
-                                /* If we are in nonblocking mode - return
-                                   immediately */
-                                if (*flags & LDLM_FL_BLOCK_NOWAIT) {
-                                        compat = -EWOULDBLOCK;
-                                        goto destroylock;
+                                /* Group locks are not normally blocked by
+                                 * waiting PR|PW locks. */
+
+                                /* If NO_TIMEOUT was sent back to the client
+                                 * we can queue the group lock in front of
+                                 * this extent lock. */
+                                if (lock->l_flags & LDLM_FL_NO_TIMEOUT) {
+                                        if (save == NULL)
+                                                save = tmp;
+                                        continue;
                                 }
-                                /* If this group lock is compatible with another
-                                 * group lock on the waiting list, they must be
-                                 * together in the list, so they can be granted
-                                 * at the same time.  Otherwise the later lock
-                                 * can get stuck behind another, incompatible,
-                                 * lock. */
-                                ldlm_resource_insert_lock_after(lock, req);
-                                /* Because 'lock' is not granted, we can stop
-                                 * processing this queue and return immediately.
-                                 * There is no need to check the rest of the
-                                 * list. */
-                                RETURN(0);
+
+                                /* If we did NOT send NO_TIMEOUT back to the
+                                 * client for this extent lock then the client
+                                 * could possibly timeout if we queue this
+                                 * group lock before it, so don't. This is the
+                                 * only way to get a conflicting extent lock
+                                 * in front of a group lock on the waitq. */
+                        }
+
+                        compat = 0;
+                        if (!work_list) {
+                                LASSERT(save == NULL);
+                                break;
                         }
-                }
 
-                if (lock->l_req_mode == LCK_CW) {
-                        /* If compared lock is CW, then requested is PR/PW/ =>
-                         * this is not compatible; extent range does not
-                         * matter */
-                        if (*flags & LDLM_FL_BLOCK_NOWAIT) {
-                                compat = -EWOULDBLOCK;
-                                goto destroylock;
-                        } else {
-                                *flags |= LDLM_FL_NO_TIMEOUT;
+                        /* If we previously skipped over some extent locks
+                         * because we thought we were going to queue the 
+                         * group lock in front of them then we need to go back
+                         * and send blocking ASTs for the locks we skipped. */
+                        if (save != NULL) {
+                                struct ldlm_lock *lck2;
+
+                                for (; save != tmp; save = save->next) {
+                                        lck2 = list_entry(save,
+                                                          struct ldlm_lock,
+                                                          l_res_link);
+
+                                        /* If there was a group lock after save
+                                         * then we would have exited this loop
+                                         * above. */
+                                        LASSERT(lck2->l_req_mode!=LCK_GROUP);
+
+                                        if (lck2->l_blocking_ast) {
+                                                ldlm_add_ast_work_item(lck2,req,
+                                                                     work_list);
+                                        }
+                                }
+                                save = NULL;
                         }
-                } else if (lock->l_policy_data.l_extent.end < req_start ||
-                           lock->l_policy_data.l_extent.start > req_end) {
-                        /* if a non-CW lock doesn't overlap skip it */
+
+                        if (lock->l_blocking_ast)
+                                ldlm_add_ast_work_item(lock, req, work_list);
                         continue;
                 }
 
-                if (!send_cbs)
-                        RETURN(0);
+                /* If it was safe to insert a group lock at save,
+                 * i.e. save != NULL, then this group lock already
+                 * on the queue would have been inserted before save. */
+                LASSERT(save == NULL);
+
+                /* Note: no blocking ASTs are sent for group locks. */
+
+                if (lock->l_policy_data.l_extent.gid ==
+                    req->l_policy_data.l_extent.gid) {
+                        /* group locks with this gid already on the waitq. */
+                        found = 2;
+
+                        if (lock->l_req_mode == lock->l_granted_mode) {
+                                /* if a group lock with this gid has already
+                                 * been granted then grant this one. */
+                                compat = 2;
+                                break;
+                        }
+                } else {
+                        if (found == 2)
+                                break;
+
+                        /* group locks already exist on the queue. */
+                        found = 1;
 
-                compat = 0;
-                if (lock->l_blocking_ast)
-                        ldlm_add_ast_work_item(lock, req, NULL, 0);
+                        if (*flags & LDLM_FL_BLOCK_NOWAIT)
+                                RETURN(-EWOULDBLOCK);
+
+                        compat = 0;
+
+                        /* there's a blocking group lock in front
+                         * of us on the queue.  It can be held
+                         * indefinitely, so don't timeout. */
+                        *flags |= LDLM_FL_NO_TIMEOUT;
+
+                        /* the only reason to continue traversing the
+                         * list at this point is to find the proper
+                         * place to insert the lock in the waitq. */
+                        if (!insertp)
+                                break;
+                }
+        }
+
+        if (insertp != NULL) {
+                if (save != NULL)
+                        *insertp = save;
+                else
+                        *insertp = tmp;
         }
 
-        return(compat);
-destroylock:
-        list_del_init(&req->l_res_link);
-        ldlm_lock_destroy(req);
-        *err = compat;
         RETURN(compat);
 }
 
@@ -233,10 +373,11 @@ destroylock:
   *   - the caller has NOT initialized req->lr_tmp, so we must
   *   - must call this function with the ns lock held once */
 int ldlm_process_extent_lock(struct ldlm_lock *lock, int *flags, int first_enq,
-                             ldlm_error_t *err)
+                             ldlm_error_t *err, struct list_head *work_list)
 {
         struct ldlm_resource *res = lock->l_resource;
         struct list_head rpc_list = LIST_HEAD_INIT(rpc_list);
+        struct list_head *insertp = NULL;
         int rc, rc2;
         ENTRY;
 
@@ -244,48 +385,49 @@ int ldlm_process_extent_lock(struct ldlm_lock *lock, int *flags, int first_enq,
         *err = ELDLM_OK;
 
         if (!first_enq) {
-                /* Careful observers will note that we don't handle -EWOULDBLOCK
-                 * here, but it's ok for a non-obvious reason -- compat_queue
-                 * can only return -EWOULDBLOCK if (flags & BLOCK_NOWAIT).
-                 * flags should always be zero here, and if that ever stops
-                 * being true, we want to find out. */
+                /* -EWOULDBLOCK can't occur here since (flags & BLOCK_NOWAIT)
+                 * lock requests would either be granted or fail on their
+                 * first_enq. flags should always be zero here, and if that
+                 * ever changes we want to find out. */
                 LASSERT(*flags == 0);
-                LASSERT(res->lr_tmp != NULL);
-                rc = ldlm_extent_compat_queue(&res->lr_granted, lock, 0, flags,
-                                              err);
+                rc = ldlm_extent_compat_queue(&res->lr_granted, lock,
+                                              flags, NULL, NULL);
                 if (rc == 1) {
-                        rc = ldlm_extent_compat_queue(&res->lr_waiting, lock, 0,
-                                                      flags, err);
+                        rc = ldlm_extent_compat_queue(&res->lr_waiting, lock,
+                                                      flags, NULL, NULL);
                 }
                 if (rc == 0)
                         RETURN(LDLM_ITER_STOP);
 
                 ldlm_resource_unlink_lock(lock);
-
                 ldlm_extent_policy(res, lock, flags);
-                ldlm_grant_lock(lock, NULL, 0, 1);
+                lock_bitlock(lock);
+                lock->l_flags &= ~LDLM_FL_NO_TIMEOUT;
+                unlock_bitlock(lock);
+                ldlm_grant_lock(lock, work_list);
                 RETURN(LDLM_ITER_CONTINUE);
         }
 
  restart:
-        LASSERT(res->lr_tmp == NULL);
-        res->lr_tmp = &rpc_list;
-        rc = ldlm_extent_compat_queue(&res->lr_granted, lock, 1, flags, err);
+        rc = ldlm_extent_compat_queue(&res->lr_granted, lock, flags, &rpc_list,
+                                      NULL);
         if (rc < 0)
-                RETURN(rc); /* lock was destroyed */
+                GOTO(destroylock, rc);
         if (rc == 2)
                 goto grant;
 
-        rc2 = ldlm_extent_compat_queue(&res->lr_waiting, lock, 1, flags, err);
+        /* Traverse the waiting list in case there are other conflicting
+         * lock requests ahead of us in the queue and send blocking ASTs */
+        rc2 = ldlm_extent_compat_queue(&res->lr_waiting, lock, flags, &rpc_list,
+                                       &insertp);
         if (rc2 < 0)
-                RETURN(rc2); /* lock was destroyed */
-        res->lr_tmp = NULL;
-
+                GOTO(destroylock, rc);
         if (rc + rc2 == 2) {
       grant:
+ grant:
                 ldlm_extent_policy(res, lock, flags);
                 ldlm_resource_unlink_lock(lock);
-                ldlm_grant_lock(lock, NULL, 0, 0);
+                lock->l_flags &= ~LDLM_FL_NO_TIMEOUT;
+                ldlm_grant_lock(lock, NULL);
         } else {
                 /* If either of the compat_queue()s returned failure, then we
                  * have ASTs to send and must go onto the waiting list.
@@ -294,19 +436,29 @@ int ldlm_process_extent_lock(struct ldlm_lock *lock, int *flags, int first_enq,
                  * terrible folly -- if we goto restart, we could get
                  * re-ordered!  Causes deadlock, because ASTs aren't sent! */
                 if (list_empty(&lock->l_res_link))
-                        ldlm_resource_add_lock(res, &res->lr_waiting, lock);
-                l_unlock(&res->lr_namespace->ns_lock);
-                rc = ldlm_run_ast_work(res->lr_namespace, &rpc_list);
-                l_lock(&res->lr_namespace->ns_lock);
+                        ldlm_resource_add_lock(res, insertp, lock);
+                unlock_res(res);
+                rc = ldlm_run_bl_ast_work(&rpc_list);
+                lock_res(res);
                 if (rc == -ERESTART)
                         GOTO(restart, -ERESTART);
                 *flags |= LDLM_FL_BLOCK_GRANTED;
         }
+
         RETURN(0);
+
+ destroylock:
+        list_del_init(&lock->l_res_link);
+        unlock_res(res);
+        ldlm_lock_destroy(lock);
+        lock_res(res);
+        *err = rc;
+        RETURN(rc);
 }
 
 /* When a lock is cancelled by a client, the KMS may undergo change if this
  * is the "highest lock".  This function returns the new KMS value.
+ * Caller must hold ns_lock already. 
  *
  * NB: A lock on [x,y] protects a KMS of up to y + 1 bytes! */
 __u64 ldlm_extent_shift_kms(struct ldlm_lock *lock, __u64 old_kms)
@@ -317,19 +469,27 @@ __u64 ldlm_extent_shift_kms(struct ldlm_lock *lock, __u64 old_kms)
         __u64 kms = 0;
         ENTRY;
 
-        l_lock(&res->lr_namespace->ns_lock);
+        /* don't let another thread in ldlm_extent_shift_kms race in
+         * just after we finish and take our lock into account in its
+         * calculation of the kms */
+
+        lock->l_flags |= LDLM_FL_KMS_IGNORE;
+
         list_for_each(tmp, &res->lr_granted) {
                 lck = list_entry(tmp, struct ldlm_lock, l_res_link);
 
-                if (lock == lck)
+                if (lck->l_flags & LDLM_FL_KMS_IGNORE)
                         continue;
+
                 if (lck->l_policy_data.l_extent.end >= old_kms)
-                        GOTO(out, kms = old_kms);
-                kms = lck->l_policy_data.l_extent.end + 1;
+                        RETURN(old_kms);
+
+                /* This extent _has_ to be smaller than old_kms (checked above)
+                 * so kms can only ever be smaller or the same as old_kms. */
+                if (lck->l_policy_data.l_extent.end + 1 > kms)
+                        kms = lck->l_policy_data.l_extent.end + 1;
         }
+        LASSERTF(kms <= old_kms, "kms "LPU64" old_kms "LPU64"\n", kms, old_kms);
 
-        GOTO(out, kms);
- out:
-        l_unlock(&res->lr_namespace->ns_lock);
-        return kms;
+        RETURN(kms);
 }