Whamcloud - gitweb
LU-19098 hsm: don't print progname twice with lhsmtool
[fs/lustre-release.git] / lustre / obdclass / lprocfs_jobstats.c
index 49bf950..30f7e48 100644 (file)
@@ -1,31 +1,14 @@
-/* GPL HEADER START
- *
- * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
- *
- * This program is free software; you can redistribute it and/or modify
- * it under the terms of the GNU General Public License version 2 only,
- * as published by the Free Software Foundation.
- *
- * This program is distributed in the hope that it will be useful, but
- * WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
- * General Public License version 2 for more details (a copy is included
- * in the LICENSE file that accompanied this code).
- *
- * You should have received a copy of the GNU General Public License
- * version 2 along with this program; If not, see
- * http://www.gnu.org/licenses/gpl-2.0.html
- *
- * GPL HEADER END
- */
+// SPDX-License-Identifier: GPL-2.0
+
 /*
  * Copyright (c) 2012, 2016, Intel Corporation.
  * Use is subject to license terms.
- *
- * Author: Niu Yawei <niu@whamcloud.com>
  */
+
 /*
- * lustre/obdclass/lprocfs_jobstats.c
+ * This file is part of Lustre, http://www.lustre.org/
+ *
+ * Author: Niu Yawei <niu@whamcloud.com>
  */
 
 #define DEBUG_SUBSYSTEM S_CLASS
 
 #ifdef CONFIG_PROC_FS
 
+enum js_info_flags {
+       JS_EXPIRED,             /* job is timed out and schedule for removal */
+};
+
+#define JOB_CLEANUP_BATCH 1024
 /*
  * JobID formats & JobID environment variable names for supported
  * job schedulers:
  */
 
 struct job_stat {
-       struct hlist_node       js_hash;        /* hash struct for this jobid */
-       struct list_head        js_list;        /* on ojs_list, with ojs_lock */
-       atomic_t                js_refcount;    /* num users of this struct */
+       struct rb_node          js_idnode;      /* js_jobid sorted node */
+       struct rb_node          js_posnode;     /* pos sorted node */
+       struct list_head        js_lru;         /* on ojs_lru, with ojs_lock */
+       unsigned long           js_flags;       /* JS_* flags */
+       struct llist_node       js_deleted;     /* on ojs_deleted w/ojs_lock */
+       u64                     js_pos_id;      /* pos for job stats seq file */
+       struct kref             js_refcount;    /* num users of this struct */
        char                    js_jobid[LUSTRE_JOBID_SIZE]; /* job name + NUL*/
        ktime_t                 js_time_init;   /* time of initial stat*/
        ktime_t                 js_time_latest; /* time of most recent stat*/
        struct lprocfs_stats    *js_stats;      /* per-job statistics */
        struct obd_job_stats    *js_jobstats;   /* for accessing ojs_lock */
+       struct rcu_head         js_rcu;         /* RCU head for job_reclaim_rcu*/
 };
 
-static unsigned
-job_stat_hash(struct cfs_hash *hs, const void *key, unsigned mask)
+static void job_reclaim_rcu(struct rcu_head *head)
 {
-       return cfs_hash_djb2_hash(key, strlen(key), mask);
-}
+       struct job_stat *job = container_of(head, typeof(*job), js_rcu);
+       struct obd_job_stats *stats;
 
-static void *job_stat_key(struct hlist_node *hnode)
-{
-       struct job_stat *job;
-       job = hlist_entry(hnode, struct job_stat, js_hash);
-       return job->js_jobid;
+       stats = job->js_jobstats;
+       lprocfs_stats_free(&job->js_stats);
+       OBD_FREE_PTR(job);
+       if (atomic64_dec_and_test(&stats->ojs_jobs))
+               clear_bit(OJS_ACTIVE_JOBS, &stats->ojs_flags);
 }
 
-static int job_stat_keycmp(const void *key, struct hlist_node *hnode)
+static void job_purge_locked(struct obd_job_stats *stats, unsigned int sched)
 {
-       struct job_stat *job;
-       job = hlist_entry(hnode, struct job_stat, js_hash);
-       return (strlen(job->js_jobid) == strlen(key)) &&
-              !strncmp(job->js_jobid, key, strlen(key));
-}
+       struct job_stat *job, *n;
+       struct llist_node *entry;
+       unsigned int count = 0;
 
-static void *job_stat_object(struct hlist_node *hnode)
-{
-       return hlist_entry(hnode, struct job_stat, js_hash);
-}
+       entry = llist_del_all(&stats->ojs_deleted);
+       if (!entry)
+               return;
 
-static void job_stat_get(struct cfs_hash *hs, struct hlist_node *hnode)
-{
-       struct job_stat *job;
-       job = hlist_entry(hnode, struct job_stat, js_hash);
-       atomic_inc(&job->js_refcount);
+       /* ojs_rwsem lock is needed to project rbtree re-balance on erase */
+       llist_for_each_entry_safe(job, n, entry, js_deleted) {
+               rb_erase(&job->js_posnode, &stats->ojs_postree);
+               rb_erase(&job->js_idnode, &stats->ojs_idtree);
+               call_rcu(&job->js_rcu, job_reclaim_rcu);
+
+               if (++count == sched) {
+                       sched = 0;
+                       up_write(&stats->ojs_rwsem);
+                       cond_resched();
+                       down_write(&stats->ojs_rwsem);
+               }
+       }
 }
 
-static void job_free(struct job_stat *job)
+static void job_free(struct kref *kref)
 {
-       LASSERT(atomic_read(&job->js_refcount) == 0);
-       LASSERT(job->js_jobstats != NULL);
+       struct job_stat *job = container_of(kref, struct job_stat, js_refcount);
+       struct obd_job_stats *stats;
 
-       write_lock(&job->js_jobstats->ojs_lock);
-       list_del_init(&job->js_list);
-       write_unlock(&job->js_jobstats->ojs_lock);
+       LASSERT(job->js_jobstats);
 
-       lprocfs_stats_free(&job->js_stats);
-       OBD_FREE_PTR(job);
+       stats = job->js_jobstats;
+       spin_lock(&stats->ojs_lock);
+       list_del_rcu(&job->js_lru);
+       llist_add(&job->js_deleted, &stats->ojs_deleted);
+       spin_unlock(&stats->ojs_lock);
 }
 
 static void job_putref(struct job_stat *job)
 {
-       LASSERT(atomic_read(&job->js_refcount) > 0);
-       if (atomic_dec_and_test(&job->js_refcount))
-               job_free(job);
-}
-
-static void job_stat_put_locked(struct cfs_hash *hs, struct hlist_node *hnode)
-{
-       struct job_stat *job;
-
-       job = hlist_entry(hnode, struct job_stat, js_hash);
-       job_putref(job);
-}
-
-static void job_stat_exit(struct cfs_hash *hs, struct hlist_node *hnode)
-{
-       CERROR("should not have any items\n");
-}
-
-static struct cfs_hash_ops job_stats_hash_ops = {
-       .hs_hash       = job_stat_hash,
-       .hs_key        = job_stat_key,
-       .hs_keycmp     = job_stat_keycmp,
-       .hs_object     = job_stat_object,
-       .hs_get        = job_stat_get,
-       .hs_put_locked = job_stat_put_locked,
-       .hs_exit       = job_stat_exit,
-};
-
-/**
- * Jobstats expiry iterator to clean up old jobids
- *
- * Called for each job_stat structure on this device, it should delete stats
- * older than the specified \a oldest_time in seconds.  If \a oldest_time is
- * in the future then this will delete all statistics (e.g. during shutdown).
- *
- * \param[in] hs       hash of all jobids on this device
- * \param[in] bd       hash bucket containing this jobid
- * \param[in] hnode    hash structure for this jobid
- * \param[in] data     pointer to stats expiry time in seconds
- */
-static int job_cleanup_iter_callback(struct cfs_hash *hs,
-                                    struct cfs_hash_bd *bd,
-                                    struct hlist_node *hnode, void *data)
-{
-       ktime_t oldest_time = *((ktime_t *)data);
-       struct job_stat *job;
-
-       job = hlist_entry(hnode, struct job_stat, js_hash);
-       if (ktime_before(job->js_time_latest, oldest_time))
-               cfs_hash_bd_del_locked(hs, bd, hnode);
-
-       return 0;
+       LASSERT(kref_read(&job->js_refcount) > 0);
+       kref_put(&job->js_refcount, job_free);
 }
 
 /**
@@ -199,11 +147,20 @@ static int job_cleanup_iter_callback(struct cfs_hash *hs,
  */
 static void lprocfs_job_cleanup(struct obd_job_stats *stats, bool clear)
 {
+       struct job_stat *job;
        ktime_t cleanup_interval = stats->ojs_cleanup_interval;
        ktime_t now = ktime_get_real();
        ktime_t oldest;
-
-       if (likely(!clear)) {
+       unsigned int sched = JOB_CLEANUP_BATCH;
+
+       if (unlikely(clear)) {
+               /* user request or shutdown: block until safe to clear */
+               do {
+                       wait_on_bit(&stats->ojs_flags, OJS_CLEANING,
+                                   TASK_UNINTERRUPTIBLE);
+               } while (test_and_set_bit(OJS_CLEANING, &stats->ojs_flags));
+               sched = UINT_MAX;
+       } else {
                /* ojs_cleanup_interval of zero means never clean up stats */
                if (ktime_to_ns(cleanup_interval) == 0)
                        return;
@@ -212,44 +169,35 @@ static void lprocfs_job_cleanup(struct obd_job_stats *stats, bool clear)
                                                cleanup_interval)))
                        return;
 
-               if (stats->ojs_cleaning)
+               /* skip if clean is in progress */
+               if (test_and_set_bit(OJS_CLEANING, &stats->ojs_flags))
                        return;
        }
 
-       write_lock(&stats->ojs_lock);
-       if (!clear && stats->ojs_cleaning) {
-               write_unlock(&stats->ojs_lock);
-               return;
-       }
-
-       stats->ojs_cleaning = true;
-       write_unlock(&stats->ojs_lock);
-
-       /* Can't hold ojs_lock over hash iteration, since it is grabbed by
-        * job_cleanup_iter_callback()
-        *   ->cfs_hash_bd_del_locked()
-        *     ->job_putref()
-        *       ->job_free()
-        *
-        * Holding ojs_lock isn't necessary for safety of the hash iteration,
-        * since locking of the hash is handled internally, but there isn't
-        * any benefit to having multiple threads doing cleanup at one time.
-        *
-        * Subtract or add twice the cleanup_interval, since it is 1/2 the
-        * maximum age.  When clearing all stats, push oldest into the future.
-        */
        cleanup_interval = ktime_add(cleanup_interval, cleanup_interval);
        if (likely(!clear))
                oldest = ktime_sub(now, cleanup_interval);
        else
                oldest = ktime_add(now, cleanup_interval);
-       cfs_hash_for_each_safe(stats->ojs_hash, job_cleanup_iter_callback,
-                              &oldest);
 
-       write_lock(&stats->ojs_lock);
-       stats->ojs_cleaning = false;
+       /* remove all jobs older oldest */
+       rcu_read_lock();
+       list_for_each_entry_rcu(job, &stats->ojs_lru, js_lru) {
+               if (!ktime_before(job->js_time_latest, oldest))
+                       break;
+               /* only put jobs that have not expired */
+               if (test_and_set_bit(JS_EXPIRED, &job->js_flags))
+                       continue;
+               job_putref(job); /* drop ref to initiate removal */
+       }
+       rcu_read_unlock();
        stats->ojs_cleanup_last = ktime_get_real();
-       write_unlock(&stats->ojs_lock);
+
+       if (down_write_trylock(&stats->ojs_rwsem)) {
+               job_purge_locked(stats, sched);
+               up_write(&stats->ojs_rwsem);
+       }
+       clear_bit(OJS_CLEANING, &stats->ojs_flags);
 }
 
 static struct job_stat *job_alloc(char *jobid, struct obd_job_stats *jobs)
@@ -271,22 +219,176 @@ static struct job_stat *job_alloc(char *jobid, struct obd_job_stats *jobs)
        memcpy(job->js_jobid, jobid, sizeof(job->js_jobid));
        job->js_time_latest = job->js_stats->ls_init;
        job->js_jobstats = jobs;
-       INIT_HLIST_NODE(&job->js_hash);
-       INIT_LIST_HEAD(&job->js_list);
-       atomic_set(&job->js_refcount, 1);
+       RB_CLEAR_NODE(&job->js_idnode);
+       INIT_LIST_HEAD(&job->js_lru);
+       clear_bit(JS_EXPIRED, &job->js_flags);
+       /* open code init_llist_node */
+       job->js_deleted.next = &job->js_deleted;
+       kref_init(&job->js_refcount);
+       if (atomic64_inc_return(&jobs->ojs_jobs) == 1)
+               set_bit(OJS_ACTIVE_JOBS, &jobs->ojs_flags);
 
        return job;
 }
 
+static inline int cmp_key_jobid(const void *_key, const struct rb_node *node)
+{
+       struct job_stat *job = container_of(node, struct job_stat, js_idnode);
+       const char *key = (const char *)_key;
+
+       return strcmp(key, job->js_jobid);
+}
+
+/* return the next job in pos_id order or NULL*/
+static struct job_stat *job_get_next_pos(struct job_stat *job)
+{
+       struct rb_node *next = rb_next(&job->js_posnode);
+
+       while (next) {
+               struct job_stat *next_job;
+
+               next_job = container_of(next, struct job_stat, js_posnode);
+               if (kref_get_unless_zero(&next_job->js_refcount))
+                       return next_job;
+
+               /* 'next_job' is going away, try again */
+               if (next)
+                       next = rb_next(next);
+       }
+
+       return NULL;
+}
+
+/* find and add a ref to a job with pos_id <= pos or NULL */
+static struct job_stat *job_find_first_pos(struct obd_job_stats *stats, u64 pos)
+{
+       struct rb_node *node = stats->ojs_postree.rb_node;
+       struct job_stat *found = NULL;
+
+       while (node) {
+               struct job_stat *job;
+
+               job = container_of(node, struct job_stat, js_posnode);
+               if (pos <= job->js_pos_id) {
+                       found = job;
+                       if (pos == job->js_pos_id)
+                               break;
+                       node = node->rb_left;
+               } else {
+                       node = node->rb_right;
+               }
+       }
+       if (found) {
+               if (kref_get_unless_zero(&found->js_refcount))
+                       return found;
+               return job_get_next_pos(found);
+       }
+       return NULL;
+}
+
+/* find and add a ref to a job, returns NULL if the job is being deleted */
+static struct job_stat *job_find(struct obd_job_stats *stats,
+                                const char *key)
+{
+       struct rb_node *node;
+       struct job_stat *job;
+
+       node = rb_find((void *)key, &stats->ojs_idtree, cmp_key_jobid);
+       if (node) {
+               job = container_of(node, struct job_stat, js_idnode);
+               if (kref_get_unless_zero(&job->js_refcount))
+                       return job;
+       }
+       return NULL;
+}
+
+static inline int cmp_node_jobid(struct rb_node *left,
+                                const struct rb_node *node)
+{
+       struct job_stat *key = container_of(left, struct job_stat, js_idnode);
+       struct job_stat *job = container_of(node, struct job_stat, js_idnode);
+
+       return strcmp(key->js_jobid, job->js_jobid);
+}
+
+/* insert a (newly allocated) job into the rbtree
+ * In the case of a collision handle and existing job
+ *  - is being deleted return -EAGAIN
+ *  - is active increment the ref count and return it.
+ * otherwise no collision and the job as added, add reference the new job
+ * and return NULL.
+ */
+static struct job_stat *job_insert(struct obd_job_stats *stats,
+                                  struct job_stat *job)
+{
+       struct rb_node *node;
+
+       node = rb_find_add(&job->js_idnode, &stats->ojs_idtree, cmp_node_jobid);
+       if (node) {
+               struct job_stat *existing_job;
+
+               existing_job = container_of(node, struct job_stat, js_idnode);
+               if (test_bit(JS_EXPIRED, &existing_job->js_flags))
+                       return ERR_PTR(-EAGAIN);
+               if (kref_get_unless_zero(&existing_job->js_refcount))
+                       return existing_job;
+               /* entry is being deleted */
+               return ERR_PTR(-EAGAIN);
+       }
+       kref_get(&job->js_refcount);
+
+       return NULL;
+}
+
+static inline int cmp_node_pos(struct rb_node *left, const struct rb_node *node)
+{
+       struct job_stat *key = container_of(left, struct job_stat, js_posnode);
+       struct job_stat *job = container_of(node, struct job_stat, js_posnode);
+
+       if (key->js_pos_id < job->js_pos_id)
+               return -1;
+       else if (key->js_pos_id > job->js_pos_id)
+               return 1;
+       return 0;
+}
+
+static inline void _next_pos_id(struct obd_job_stats *stats,
+                               struct job_stat *job)
+{
+       /* avoid pos clash with 'SEQ_START_TOKEN' */
+       do {
+               job->js_pos_id = atomic64_inc_return(&stats->ojs_next_pos);
+       } while (job->js_pos_id < 2);
+}
+
+/* insert a job into the rbtree, return NULL if added otherwise existing job */
+static void job_insert_pos(struct obd_job_stats *stats, struct job_stat *job)
+{
+       struct rb_node *node;
+
+       /* on wrapping u64 insert could fail so advance pos_id need
+        * to fill in gaps
+        */
+       do {
+               _next_pos_id(stats, job);
+               node = rb_find_add(&job->js_posnode, &stats->ojs_postree,
+                                  cmp_node_pos);
+       } while (node);
+}
+
 int lprocfs_job_stats_log(struct obd_device *obd, char *jobid,
                          int event, long amount)
 {
        struct obd_job_stats *stats = &obd2obt(obd)->obt_jobstats;
-       struct job_stat *job, *job2;
+       struct job_stat *job, *existing_job;
+       bool mru_last = false;
        ENTRY;
 
-       LASSERT(stats != NULL);
-       LASSERT(stats->ojs_hash != NULL);
+       LASSERT(stats);
+
+       /* do not add jobs while shutting down */
+       if (test_bit(OJS_FINI, &stats->ojs_flags))
+               RETURN(0);
 
        if (event >= stats->ojs_cntr_num)
                RETURN(-EINVAL);
@@ -301,38 +403,53 @@ int lprocfs_job_stats_log(struct obd_device *obd, char *jobid,
                RETURN(-EINVAL);
        }
 
-       job = cfs_hash_lookup(stats->ojs_hash, jobid);
+       down_read(&stats->ojs_rwsem);
+       job = job_find(stats, jobid);
+       up_read(&stats->ojs_rwsem);
        if (job)
                goto found;
 
        lprocfs_job_cleanup(stats, false);
 
        job = job_alloc(jobid, stats);
-       if (job == NULL)
+       if (!job)
                RETURN(-ENOMEM);
 
-       job2 = cfs_hash_findadd_unique(stats->ojs_hash, job->js_jobid,
-                                      &job->js_hash);
-       if (job2 != job) {
-               job_putref(job);
-               job = job2;
-               /* We cannot LASSERT(!list_empty(&job->js_list)) here,
-                * since we just lost the race for inserting "job" into the
-                * ojs_list, and some other thread is doing it _right_now_.
-                * Instead, be content the other thread is doing this, since
-                * "job2" was initialized in job_alloc() already. LU-2163 */
-       } else {
-               LASSERT(list_empty(&job->js_list));
-               write_lock(&stats->ojs_lock);
-               list_add_tail(&job->js_list, &stats->ojs_list);
-               write_unlock(&stats->ojs_lock);
+try_insert:
+       down_write(&stats->ojs_rwsem);
+       job_purge_locked(stats, UINT_MAX);
+       existing_job = job_insert(stats, job);
+       if (IS_ERR(existing_job) && PTR_ERR(existing_job) == -EAGAIN) {
+               up_write(&stats->ojs_rwsem);
+               goto try_insert;
+       }
+       /* on collision drop the old job and proceed with the existing job */
+       if (existing_job) {
+               job_putref(job); /* duplicate job, remove */
+               job = existing_job;
+               up_write(&stats->ojs_rwsem);
+               goto found;
        }
+       job_insert_pos(stats, job);
+       LASSERT(list_empty(&job->js_lru));
+       spin_lock(&stats->ojs_lock);
+       list_add_tail_rcu(&job->js_lru, &stats->ojs_lru);
+       mru_last = true;
+       spin_unlock(&stats->ojs_lock);
+       up_write(&stats->ojs_rwsem);
 
 found:
        LASSERT(stats == job->js_jobstats);
        job->js_time_latest = ktime_get_real();
+       if (!mru_last) {
+               spin_lock(&stats->ojs_lock);
+               list_del_rcu(&job->js_lru);
+               list_add_tail_rcu(&job->js_lru, &stats->ojs_lru);
+               spin_unlock(&stats->ojs_lock);
+       }
        lprocfs_counter_add(job->js_stats, event, amount);
 
+       /* drop the extra ref from find | insert */
        job_putref(job);
 
        RETURN(0);
@@ -342,57 +459,85 @@ EXPORT_SYMBOL(lprocfs_job_stats_log);
 void lprocfs_job_stats_fini(struct obd_device *obd)
 {
        struct obd_job_stats *stats = &obd2obt(obd)->obt_jobstats;
+       struct job_stat *job, *n;
+       int retry = 0;
+       bool purge = false;
 
-       if (stats->ojs_hash == NULL)
-               return;
+       set_bit(OJS_FINI, &stats->ojs_flags);
+       do {
+               lprocfs_job_cleanup(stats, true);
+               down_write(&stats->ojs_rwsem);
+               job_purge_locked(stats, UINT_MAX);
+               up_write(&stats->ojs_rwsem);
+               rcu_barrier();
+               purge = false;
+
+               rbtree_postorder_for_each_entry_safe(job, n,
+                                                    &stats->ojs_idtree,
+                                                    js_idnode) {
+                       if (kref_read(&job->js_refcount) > 0) {
+                               job_putref(job); /* drop ref */
+                               purge = true;
+                       }
+               }
+
+               rbtree_postorder_for_each_entry_safe(job, n,
+                                                    &stats->ojs_postree,
+                                                    js_posnode) {
+                       if (kref_read(&job->js_refcount) > 0) {
+                               job_putref(job); /* drop ref */
+                               purge = true;
+                       }
+               }
 
-       lprocfs_job_cleanup(stats, true);
-       cfs_hash_putref(stats->ojs_hash);
-       stats->ojs_hash = NULL;
-       LASSERT(list_empty(&stats->ojs_list));
+               if (atomic64_read(&stats->ojs_jobs))
+                       purge = true;
+       } while (purge && retry++ < 3);
+       wait_on_bit_timeout(&stats->ojs_flags, OJS_ACTIVE_JOBS,
+                           TASK_UNINTERRUPTIBLE, cfs_time_seconds(30));
+       rcu_barrier();
+       LASSERTF(atomic64_read(&stats->ojs_jobs) == 0, "jobs:%llu flags:%lx\n",
+                (long long)atomic64_read(&stats->ojs_jobs), stats->ojs_flags);
+       LASSERT(RB_EMPTY_ROOT(&stats->ojs_idtree));
+       LASSERT(list_empty(&stats->ojs_lru));
+       LASSERT(llist_empty(&stats->ojs_deleted));
 }
 EXPORT_SYMBOL(lprocfs_job_stats_fini);
 
 static void *lprocfs_jobstats_seq_start(struct seq_file *p, loff_t *pos)
 {
        struct obd_job_stats *stats = p->private;
-       loff_t off = *pos;
-       struct job_stat *job;
+       struct job_stat *start;
 
-       read_lock(&stats->ojs_lock);
-       if (off == 0)
-               return SEQ_START_TOKEN;
-       off--;
-       list_for_each_entry(job, &stats->ojs_list, js_list) {
-               if (!off--)
-                       return job;
-       }
-       return NULL;
+       down_read(&stats->ojs_rwsem);
+       if (*pos == 0)
+               set_bit(OJS_HEADER, &stats->ojs_flags);
+       start = job_find_first_pos(stats, *pos);
+       if (start)
+               *pos = start->js_pos_id;
+
+       return start;
 }
 
-static void lprocfs_jobstats_seq_stop(struct seq_file *p, void *v)
+static void *lprocfs_jobstats_seq_next(struct seq_file *p, void *v, loff_t *pos)
 {
-       struct obd_job_stats *stats = p->private;
+       struct job_stat *job = v, *next = NULL;
+
+       ++*pos;
+       if (!job)
+               return next;
+       next = job_get_next_pos(job);
+       if (next)
+               *pos = next->js_pos_id;
 
-       read_unlock(&stats->ojs_lock);
+       return next;
 }
 
-static void *lprocfs_jobstats_seq_next(struct seq_file *p, void *v, loff_t *pos)
+static void lprocfs_jobstats_seq_stop(struct seq_file *p, void *v)
 {
        struct obd_job_stats *stats = p->private;
-       struct job_stat *job;
-       struct list_head *next;
-
-       ++*pos;
-       if (v == SEQ_START_TOKEN) {
-               next = stats->ojs_list.next;
-       } else {
-               job = (struct job_stat *)v;
-               next = job->js_list.next;
-       }
 
-       return next == &stats->ojs_list ? NULL :
-               list_entry(next, struct job_stat, js_list);
+       up_read(&stats->ojs_rwsem);
 }
 
 /*
@@ -434,13 +579,14 @@ static void *lprocfs_jobstats_seq_next(struct seq_file *p, void *v, loff_t *pos)
 
 static const char spaces[] = "                    ";
 
-static int inline width(const char *str, int len)
+static inline int width(const char *str, int len)
 {
        return len - min((int)strlen(str), 15);
 }
 
 static int lprocfs_jobstats_seq_show(struct seq_file *p, void *v)
 {
+       struct obd_job_stats *stats = p->private;
        struct job_stat *job = v;
        struct lprocfs_stats *s;
        struct lprocfs_counter ret;
@@ -449,10 +595,11 @@ static int lprocfs_jobstats_seq_show(struct seq_file *p, void *v)
        char *quote = "", *c, *end;
        int i, joblen = 0;
 
-       if (v == SEQ_START_TOKEN) {
-               seq_puts(p, "job_stats:\n");
+       if (v == SEQ_START_TOKEN)
                return 0;
-       }
+
+       if (test_and_clear_bit(OJS_HEADER, &stats->ojs_flags))
+               seq_puts(p, "job_stats:\n");
 
        /* Quote and escape jobid characters to escape hex codes "\xHH" if
         * it contains any non-standard characters (space, newline, etc),
@@ -473,6 +620,9 @@ static int lprocfs_jobstats_seq_show(struct seq_file *p, void *v)
                                quote = "\"";
                }
        }
+       /* '@' is reserved in YAML, so it cannot start a bare string. */
+       if (escaped[0] == '@')
+               quote = "\"";
 
        seq_printf(p, "- %-16s %s%*s%s\n",
                   "job_id:", quote, joblen, escaped, quote);
@@ -537,6 +687,7 @@ static int lprocfs_jobstats_seq_show(struct seq_file *p, void *v)
                }
                seq_puts(p, " }\n");
        }
+       job_putref(job);
 
        return 0;
 }
@@ -551,13 +702,19 @@ static const struct seq_operations lprocfs_jobstats_seq_sops = {
 static int lprocfs_jobstats_seq_open(struct inode *inode, struct file *file)
 {
        struct seq_file *seq;
+       struct obd_job_stats *stats;
        int rc;
 
        rc = seq_open(file, &lprocfs_jobstats_seq_sops);
        if (rc)
                return rc;
+
+       stats = pde_data(inode);
+       /* wait for any active cleaning to finish */
+       set_bit(OJS_HEADER, &stats->ojs_flags);
        seq = file->private_data;
-       seq->private = pde_data(inode);
+       seq->private = stats;
+
        return 0;
 }
 
@@ -575,7 +732,7 @@ static ssize_t lprocfs_jobstats_seq_write(struct file *file,
        if (len == 0 || len >= 4 * LUSTRE_JOBID_SIZE)
                return -EINVAL;
 
-       if (stats->ojs_hash == NULL)
+       if (!stats->ojs_cntr_num)
                return -ENODEV;
 
        if (copy_from_user(jobid, buf, len))
@@ -616,13 +773,14 @@ static ssize_t lprocfs_jobstats_seq_write(struct file *file,
        if (strlen(jobid) == 0)
                return -EINVAL;
 
-       job = cfs_hash_lookup(stats->ojs_hash, jobid);
+       down_read(&stats->ojs_rwsem);
+       job = job_find(stats, jobid);
+       up_read(&stats->ojs_rwsem);
        if (!job)
                return -EINVAL;
+       job_putref(job); /* drop ref from job_find() */
+       job_putref(job); /* drop ref to initiate removal */
 
-       cfs_hash_del_key(stats->ojs_hash, jobid);
-
-       job_putref(job);
        return len;
 }
 
@@ -681,28 +839,23 @@ int lprocfs_job_stats_init(struct obd_device *obd, int cntr_num,
                RETURN(-EINVAL);
        }
        stats = &obd2obt(obd)->obt_jobstats;
-
-       LASSERT(stats->ojs_hash == NULL);
-       stats->ojs_hash = cfs_hash_create("JOB_STATS",
-                                         HASH_JOB_STATS_CUR_BITS,
-                                         HASH_JOB_STATS_MAX_BITS,
-                                         HASH_JOB_STATS_BKT_BITS, 0,
-                                         CFS_HASH_MIN_THETA,
-                                         CFS_HASH_MAX_THETA,
-                                         &job_stats_hash_ops,
-                                         CFS_HASH_DEFAULT);
-       if (stats->ojs_hash == NULL)
-               RETURN(-ENOMEM);
-
-       INIT_LIST_HEAD(&stats->ojs_list);
-       rwlock_init(&stats->ojs_lock);
-       stats->ojs_cntr_num = cntr_num;
-       stats->ojs_cntr_init_fn = init_fn;
+       stats->ojs_idtree = RB_ROOT;
+       stats->ojs_postree = RB_ROOT;
+       atomic64_set(&stats->ojs_next_pos, 2);
+       init_rwsem(&stats->ojs_rwsem);
+       INIT_LIST_HEAD(&stats->ojs_lru);
+       init_llist_head(&stats->ojs_deleted);
+       stats->ojs_flags = 0;
+       atomic_set(&stats->ojs_readers, 0);
+       spin_lock_init(&stats->ojs_lock);
        /* Store 1/2 the actual interval, since we use that the most, and
         * it is easier to work with.
         */
        stats->ojs_cleanup_interval = ktime_set(600 / 2, 0); /* default 10 min*/
        stats->ojs_cleanup_last = ktime_get_real();
+       stats->ojs_cntr_num = cntr_num;
+       stats->ojs_cntr_init_fn = init_fn;
+       atomic64_set(&stats->ojs_jobs, 0);
 
        entry = lprocfs_add_simple(obd->obd_proc_entry, "job_stats", stats,
                                   &lprocfs_jobstats_seq_fops);