Whamcloud - gitweb
New tag 2.15.63
[fs/lustre-release.git] / lustre / obdclass / lprocfs_jobstats.c
index 518d9c3..c2d27e2 100644 (file)
 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 */
-       char                    js_jobid[LUSTRE_JOBID_SIZE]; /* job name */
-       time64_t                js_timestamp;   /* seconds of most recent stat*/
+       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 unsigned int
+job_stat_hash(struct cfs_hash *hs, const void *key, const unsigned int bits)
 {
-       return cfs_hash_djb2_hash(key, strlen(key), mask);
+       return cfs_hash_djb2_hash(key, strlen(key), bits);
 }
 
 static void *job_stat_key(struct hlist_node *hnode)
@@ -97,36 +99,49 @@ static void *job_stat_object(struct hlist_node *hnode)
        return hlist_entry(hnode, struct job_stat, js_hash);
 }
 
+static bool job_getref_try(struct job_stat *job)
+{
+       return kref_get_unless_zero(&job->js_refcount);
+}
+
 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);
+       kref_get(&job->js_refcount);
 }
 
-static void job_free(struct job_stat *job)
+static void job_reclaim_rcu(struct rcu_head *head)
 {
-       LASSERT(atomic_read(&job->js_refcount) == 0);
-       LASSERT(job->js_jobstats != NULL);
+       struct job_stat *job = container_of(head, typeof(*job), js_rcu);
 
-       write_lock(&job->js_jobstats->ojs_lock);
-       list_del_init(&job->js_list);
-       write_unlock(&job->js_jobstats->ojs_lock);
-
-       lprocfs_free_stats(&job->js_stats);
+       lprocfs_stats_free(&job->js_stats);
        OBD_FREE_PTR(job);
 }
 
+static void job_free(struct kref *kref)
+{
+       struct job_stat *job = container_of(kref, struct job_stat,
+                                           js_refcount);
+
+       LASSERT(job->js_jobstats != NULL);
+       spin_lock(&job->js_jobstats->ojs_lock);
+       list_del_rcu(&job->js_list);
+       spin_unlock(&job->js_jobstats->ojs_lock);
+
+       call_rcu(&job->js_rcu, job_reclaim_rcu);
+}
+
 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);
+       LASSERT(kref_read(&job->js_refcount) > 0);
+       kref_put(&job->js_refcount, job_free);
 }
 
 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);
 }
@@ -162,11 +177,11 @@ static int job_cleanup_iter_callback(struct cfs_hash *hs,
                                     struct cfs_hash_bd *bd,
                                     struct hlist_node *hnode, void *data)
 {
-       time64_t oldest_time = *((time64_t *)data);
+       ktime_t oldest_time = *((ktime_t *)data);
        struct job_stat *job;
 
        job = hlist_entry(hnode, struct job_stat, js_hash);
-       if (job->js_timestamp < oldest_time)
+       if (ktime_before(job->js_time_latest, oldest_time))
                cfs_hash_bd_del_locked(hs, bd, hnode);
 
        return 0;
@@ -177,44 +192,51 @@ static int job_cleanup_iter_callback(struct cfs_hash *hs,
  *
  * Since this function may be called frequently, do not scan all of the
  * jobstats on each call, only twice per cleanup interval.  That means stats
- * may be around on average cleanup_interval / 4 longer than necessary,
- * but that is not considered harmful.
+ * may be on average around cleanup_interval / 4 older than the cleanup
+ * interval, but that is not considered harmful.
+ *
+ * The value stored in ojs_cleanup_interval is how often to perform a cleanup
+ * scan, and 1/2 of the maximum age of the individual statistics.  This is
+ * done rather than dividing the interval by two each time, because it is
+ * much easier to do the division when the value is initially set (in seconds)
+ * rather than after it has been converted to ktime_t, and maybe a bit faster.
  *
- * If \a before is negative then this will force clean up all jobstats due
- * to the expiry time being in the future (e.g. at shutdown).
+ * If \a clear is true then this will force clean up all jobstats
+ * (e.g. at shutdown).
  *
  * If there is already another thread doing jobstats cleanup, don't try to
  * do this again in the current thread unless this is a force cleanup.
  *
  * \param[in] stats    stucture tracking all job stats for this device
- * \param[in] before   expire jobstats updated more than this many seconds ago
+ * \param[in] clear    clear all job stats if true
  */
-static void lprocfs_job_cleanup(struct obd_job_stats *stats, int before)
+static void lprocfs_job_cleanup(struct obd_job_stats *stats, bool clear)
 {
-       time64_t now = ktime_get_real_seconds();
-       time64_t oldest;
+       ktime_t cleanup_interval = stats->ojs_cleanup_interval;
+       ktime_t now = ktime_get_real();
+       ktime_t oldest;
 
-       if (likely(before >= 0)) {
-               unsigned int cleanup_interval = stats->ojs_cleanup_interval;
-
-               if (cleanup_interval == 0 || before == 0)
+       if (likely(!clear)) {
+               /* ojs_cleanup_interval of zero means never clean up stats */
+               if (ktime_to_ns(cleanup_interval) == 0)
                        return;
 
-               if (now < stats->ojs_last_cleanup + cleanup_interval / 2)
+               if (ktime_before(now, ktime_add(stats->ojs_cleanup_last,
+                                               cleanup_interval)))
                        return;
 
                if (stats->ojs_cleaning)
                        return;
        }
 
-       write_lock(&stats->ojs_lock);
-       if (before >= 0 && stats->ojs_cleaning) {
-               write_unlock(&stats->ojs_lock);
+       spin_lock(&stats->ojs_lock);
+       if (!clear && stats->ojs_cleaning) {
+               spin_unlock(&stats->ojs_lock);
                return;
        }
 
        stats->ojs_cleaning = true;
-       write_unlock(&stats->ojs_lock);
+       spin_unlock(&stats->ojs_lock);
 
        /* Can't hold ojs_lock over hash iteration, since it is grabbed by
         * job_cleanup_iter_callback()
@@ -225,15 +247,22 @@ static void lprocfs_job_cleanup(struct obd_job_stats *stats, int before)
         * 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.
         */
-       oldest = now - before;
+       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);
+       spin_lock(&stats->ojs_lock);
        stats->ojs_cleaning = false;
-       stats->ojs_last_cleanup = ktime_get_real_seconds();
-       write_unlock(&stats->ojs_lock);
+       stats->ojs_cleanup_last = ktime_get_real();
+       spin_unlock(&stats->ojs_lock);
 }
 
 static struct job_stat *job_alloc(char *jobid, struct obd_job_stats *jobs)
@@ -244,20 +273,20 @@ static struct job_stat *job_alloc(char *jobid, struct obd_job_stats *jobs)
        if (job == NULL)
                return NULL;
 
-       job->js_stats = lprocfs_alloc_stats(jobs->ojs_cntr_num, 0);
+       job->js_stats = lprocfs_stats_alloc(jobs->ojs_cntr_num, 0);
        if (job->js_stats == NULL) {
                OBD_FREE_PTR(job);
                return NULL;
        }
 
-       jobs->ojs_cntr_init_fn(job->js_stats);
+       jobs->ojs_cntr_init_fn(job->js_stats, 0, 0);
 
-       memcpy(job->js_jobid, jobid, LUSTRE_JOBID_SIZE);
-       job->js_timestamp = ktime_get_real_seconds();
+       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);
+       kref_init(&job->js_refcount);
 
        return job;
 }
@@ -265,7 +294,7 @@ static struct job_stat *job_alloc(char *jobid, struct obd_job_stats *jobs)
 int lprocfs_job_stats_log(struct obd_device *obd, char *jobid,
                          int event, long amount)
 {
-       struct obd_job_stats *stats = &obd->u.obt.obt_jobstats;
+       struct obd_job_stats *stats = &obd2obt(obd)->obt_jobstats;
        struct job_stat *job, *job2;
        ENTRY;
 
@@ -276,10 +305,11 @@ int lprocfs_job_stats_log(struct obd_device *obd, char *jobid,
                RETURN(-EINVAL);
 
        if (jobid == NULL || strlen(jobid) == 0)
-               RETURN(-EINVAL);
+               RETURN(0);
 
+       /* unterminated jobid should be handled in lustre_msg_get_jobid() */
        if (strlen(jobid) >= LUSTRE_JOBID_SIZE) {
-               CERROR("Invalid jobid size (%lu), expect(%d)\n",
+               CERROR("%s: invalid jobid size %lu, expect %d\n", obd->obd_name,
                       (unsigned long)strlen(jobid) + 1, LUSTRE_JOBID_SIZE);
                RETURN(-EINVAL);
        }
@@ -288,7 +318,7 @@ int lprocfs_job_stats_log(struct obd_device *obd, char *jobid,
        if (job)
                goto found;
 
-       lprocfs_job_cleanup(stats, stats->ojs_cleanup_interval);
+       lprocfs_job_cleanup(stats, false);
 
        job = job_alloc(jobid, stats);
        if (job == NULL)
@@ -306,14 +336,14 @@ int lprocfs_job_stats_log(struct obd_device *obd, char *jobid,
                 * "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);
+               spin_lock(&stats->ojs_lock);
+               list_add_tail_rcu(&job->js_list, &stats->ojs_list);
+               spin_unlock(&stats->ojs_lock);
        }
 
 found:
        LASSERT(stats == job->js_jobstats);
-       job->js_timestamp = ktime_get_real_seconds();
+       job->js_time_latest = ktime_get_real();
        lprocfs_counter_add(job->js_stats, event, amount);
 
        job_putref(job);
@@ -324,29 +354,42 @@ EXPORT_SYMBOL(lprocfs_job_stats_log);
 
 void lprocfs_job_stats_fini(struct obd_device *obd)
 {
-       struct obd_job_stats *stats = &obd->u.obt.obt_jobstats;
+       struct obd_job_stats *stats = &obd2obt(obd)->obt_jobstats;
 
        if (stats->ojs_hash == NULL)
                return;
 
-       lprocfs_job_cleanup(stats, -99);
+       lprocfs_job_cleanup(stats, true);
        cfs_hash_putref(stats->ojs_hash);
        stats->ojs_hash = NULL;
        LASSERT(list_empty(&stats->ojs_list));
 }
 EXPORT_SYMBOL(lprocfs_job_stats_fini);
 
+
+struct lprocfs_jobstats_data {
+       struct obd_job_stats *pjd_stats;
+       loff_t pjd_last_pos;
+       struct job_stat *pjd_last_job;
+};
+
 static void *lprocfs_jobstats_seq_start(struct seq_file *p, loff_t *pos)
 {
-       struct obd_job_stats *stats = p->private;
+       struct lprocfs_jobstats_data *data = p->private;
+       struct obd_job_stats *stats = data->pjd_stats;
        loff_t off = *pos;
        struct job_stat *job;
 
-       read_lock(&stats->ojs_lock);
+       rcu_read_lock();
        if (off == 0)
                return SEQ_START_TOKEN;
+
+       /* if pos matches the offset of last saved job, start from saved job */
+       if (data->pjd_last_job && data->pjd_last_pos == off)
+               return data->pjd_last_job;
+
        off--;
-       list_for_each_entry(job, &stats->ojs_list, js_list) {
+       list_for_each_entry_rcu(job, &stats->ojs_list, js_list) {
                if (!off--)
                        return job;
        }
@@ -355,27 +398,47 @@ static void *lprocfs_jobstats_seq_start(struct seq_file *p, loff_t *pos)
 
 static void lprocfs_jobstats_seq_stop(struct seq_file *p, void *v)
 {
-       struct obd_job_stats *stats = p->private;
+       struct lprocfs_jobstats_data *data = p->private;
+       struct job_stat *job = NULL;
+
+       /* try to get a ref on current job (not deleted) */
+       if (v && v != SEQ_START_TOKEN && job_getref_try(v))
+               job = v;
+
+       rcu_read_unlock();
 
-       read_unlock(&stats->ojs_lock);
+       /* drop the ref on the old saved job */
+       if (data->pjd_last_job) {
+               job_putref(data->pjd_last_job);
+               data->pjd_last_job = NULL;
+       }
+
+       /* save the current job for the next read */
+       if (job)
+               data->pjd_last_job = job;
 }
 
 static void *lprocfs_jobstats_seq_next(struct seq_file *p, void *v, loff_t *pos)
 {
-       struct obd_job_stats *stats = p->private;
+       struct lprocfs_jobstats_data *data = p->private;
+       struct obd_job_stats *stats = data->pjd_stats;
        struct job_stat *job;
-       struct list_head *next;
+       struct list_head *cur;
 
        ++*pos;
+       data->pjd_last_pos = *pos;
        if (v == SEQ_START_TOKEN) {
-               next = stats->ojs_list.next;
+               cur = &stats->ojs_list;
        } else {
                job = (struct job_stat *)v;
-               next = job->js_list.next;
+               cur = &job->js_list;
        }
 
-       return next == &stats->ojs_list ? NULL :
-               list_entry(next, struct job_stat, js_list);
+       job = list_entry_rcu(cur->next, struct job_stat, js_list);
+       if (&job->js_list == &stats->ojs_list)
+               return NULL;
+
+       return job;
 }
 
 /*
@@ -383,7 +446,9 @@ static void *lprocfs_jobstats_seq_next(struct seq_file *p, void *v, loff_t *pos)
  *
  * job_stats:
  * - job_id:        dd.4854
- *   snapshot_time: 1322494486
+ *   snapshot_time: 1322494486.123456789
+ *   start_time:    1322494476.012345678
+ *   elapsed_time:  10.111111111
  *   open:          { samples:        1, unit: reqs }
  *   close:         { samples:        1, unit: reqs }
  *   mknod:         { samples:        0, unit: reqs }
@@ -403,7 +468,9 @@ static void *lprocfs_jobstats_seq_next(struct seq_file *p, void *v, loff_t *pos)
  *
  * job_stats:
  * - job_id         dd.4854
- *   snapshot_time: 1322494602
+ *   snapshot_time: 1322494602.123456789
+ *   start_time:    1322494592.987654321
+ *   elapsed_time:  9.135802468
  *   read:          { samples: 0, unit: bytes, min:  0, max:  0, sum:  0 }
  *   write:         { samples: 1, unit: bytes, min: 4096, max: 4096, sum: 4096 }
  *   setattr:       { samples: 0, unit: reqs }
@@ -420,32 +487,51 @@ static int inline width(const char *str, int len)
 
 static int lprocfs_jobstats_seq_show(struct seq_file *p, void *v)
 {
-       struct job_stat                 *job = v;
-       struct lprocfs_stats            *s;
-       struct lprocfs_counter          ret;
-       struct lprocfs_counter_header   *cntr_header;
-       int                             i;
+       struct job_stat *job = v;
+       struct lprocfs_stats *s;
+       struct lprocfs_counter ret;
+       struct lprocfs_counter_header *cntr_header;
+       char escaped[LUSTRE_JOBID_SIZE * 4] = "";
+       char *quote = "", *c, *end;
+       int i, joblen = 0;
 
        if (v == SEQ_START_TOKEN) {
-               seq_printf(p, "job_stats:\n");
+               seq_puts(p, "job_stats:\n");
                return 0;
        }
 
-       /* Replace the non-printable character in jobid with '?', so
-        * that the output of jobid will be confined in single line. */
-       seq_printf(p, "- %-16s ", "job_id:");
-       for (i = 0; i < strlen(job->js_jobid); i++) {
-               if (isprint(job->js_jobid[i]) != 0)
-                       seq_putc(p, job->js_jobid[i]);
-               else
-                       seq_putc(p, '?');
+       /* Quote and escape jobid characters to escape hex codes "\xHH" if
+        * it contains any non-standard characters (space, newline, etc),
+        * so it will be confined to single line and not break parsing.
+        */
+       for (c = job->js_jobid, end = job->js_jobid + sizeof(job->js_jobid);
+            c < end && *c != '\0';
+            c++, joblen++) {
+               if (!isalnum(*c) && strchr(".@-_:/", *c) == NULL) {
+                       quote = "\"";
+                       snprintf(escaped + joblen, sizeof(escaped), "\\x%02X",
+                                (unsigned char)*c);
+                       joblen += 3;
+               } else {
+                       escaped[joblen] = *c;
+                       /* if jobid has ':', it should be quoted too */
+                       if (*c == ':')
+                               quote = "\"";
+               }
        }
-       seq_putc(p, '\n');
+       /* '@' is reserved in YAML, so it cannot start a bare string. */
+       if (escaped[0] == '@')
+               quote = "\"";
 
-       seq_printf(p, "  %-16s %lld\n", "snapshot_time:", job->js_timestamp);
+       seq_printf(p, "- %-16s %s%*s%s\n",
+                  "job_id:", quote, joblen, escaped, quote);
+       lprocfs_stats_header(p, job->js_time_latest, job->js_stats->ls_init,
+                            16, ":", true, "  ");
 
        s = job->js_stats;
        for (i = 0; i < s->ls_num; i++) {
+               struct obd_histogram *hist;
+
                cntr_header = &s->ls_cnt_header[i];
                lprocfs_stats_collect(s, i, &ret);
 
@@ -457,8 +543,7 @@ static int lprocfs_jobstats_seq_show(struct seq_file *p, void *v)
                        seq_printf(p, ", unit: %5s", cntr_header->lc_units);
 
                if (cntr_header->lc_config & LPROCFS_CNTR_AVGMINMAX) {
-                       seq_printf(p, ", min:%8llu, max:%8llu,"
-                                  " sum:%16llu",
+                       seq_printf(p, ", min: %8llu, max: %8llu, sum: %16llu",
                                   ret.lc_count ? ret.lc_min : 0,
                                   ret.lc_count ? ret.lc_max : 0,
                                   ret.lc_count ? ret.lc_sum : 0);
@@ -468,9 +553,40 @@ static int lprocfs_jobstats_seq_show(struct seq_file *p, void *v)
                                   ret.lc_count ? ret.lc_sumsquare : 0);
                }
 
-               seq_printf(p, " }\n");
-
+               /* show obd_histogram */
+               hist = s->ls_cnt_header[i].lc_hist;
+               if (hist != NULL) {
+                       bool first = true;
+                       int j;
+
+                       seq_puts(p, ", hist: { ");
+                       for (j = 0; j < ARRAY_SIZE(hist->oh_buckets); j++) {
+                               unsigned long val = hist->oh_buckets[j];
+
+                               if (val == 0)
+                                       continue;
+                               if (first)
+                                       first = false;
+                               else
+                                       seq_puts(p, ", ");
+
+                               if (j < 10)
+                                       seq_printf(p, "%lu: %lu", BIT(j), val);
+                               else if (j < 20)
+                                       seq_printf(p, "%luK: %lu", BIT(j - 10),
+                                                  val);
+                               else if (j < 30)
+                                       seq_printf(p, "%luM: %lu", BIT(j - 20),
+                                                  val);
+                               else
+                                       seq_printf(p, "%luG: %lu", BIT(j - 30),
+                                                  val);
+                       }
+                       seq_puts(p, " }");
+               }
+               seq_puts(p, " }\n");
        }
+
        return 0;
 }
 
@@ -483,18 +599,23 @@ static const struct seq_operations lprocfs_jobstats_seq_sops = {
 
 static int lprocfs_jobstats_seq_open(struct inode *inode, struct file *file)
 {
+       struct lprocfs_jobstats_data *data = NULL;
        struct seq_file *seq;
        int rc;
 
-       rc = LPROCFS_ENTRY_CHECK(inode);
-       if (rc < 0)
-               return rc;
-
        rc = seq_open(file, &lprocfs_jobstats_seq_sops);
        if (rc)
                return rc;
+
+       OBD_ALLOC_PTR(data);
+       if (!data)
+               return -ENOMEM;
+
+       data->pjd_stats = pde_data(inode);
+       data->pjd_last_job = NULL;
+       data->pjd_last_pos = 0;
        seq = file->private_data;
-       seq->private = PDE_DATA(inode);
+       seq->private = data;
        return 0;
 }
 
@@ -503,11 +624,14 @@ static ssize_t lprocfs_jobstats_seq_write(struct file *file,
                                          size_t len, loff_t *off)
 {
        struct seq_file *seq = file->private_data;
-       struct obd_job_stats *stats = seq->private;
-       char jobid[LUSTRE_JOBID_SIZE];
+       struct lprocfs_jobstats_data *data = seq->private;
+       struct obd_job_stats *stats = data->pjd_stats;
+       char jobid[4 * LUSTRE_JOBID_SIZE]; /* all escaped chars, plus ""\n\0 */
+       char *p1, *p2, *last;
+       unsigned int c;
        struct job_stat *job;
 
-       if (len == 0 || len >= LUSTRE_JOBID_SIZE)
+       if (len == 0 || len >= 4 * LUSTRE_JOBID_SIZE)
                return -EINVAL;
 
        if (stats->ojs_hash == NULL)
@@ -516,13 +640,34 @@ static ssize_t lprocfs_jobstats_seq_write(struct file *file,
        if (copy_from_user(jobid, buf, len))
                return -EFAULT;
        jobid[len] = 0;
+       last = jobid + len - 1;
 
        /* Trim '\n' if any */
-       if (jobid[len - 1] == '\n')
-               jobid[len - 1] = 0;
+       if (*last == '\n')
+               *(last--) = 0;
+
+       /* decode escaped chars if jobid is a quoted string */
+       if (jobid[0] == '"' && *last == '"') {
+               last--;
+
+               for (p1 = jobid, p2 = jobid + 1; p2 <= last; p1++, p2++) {
+                       if (*p2 != '\\') {
+                               *p1 = *p2;
+                       } else if (p2 + 3 <= last && *(p2 + 1) == 'x' &&
+                                sscanf(p2 + 2, "%02X", &c) == 1) {
+                               *p1 = c;
+                               p2 += 3;
+                       } else {
+                               return -EINVAL;
+                       }
+               }
+               *p1 = 0;
+
+       }
+       jobid[LUSTRE_JOBID_SIZE - 1] = 0;
 
        if (strcmp(jobid, "clear") == 0) {
-               lprocfs_job_cleanup(stats, -99);
+               lprocfs_job_cleanup(stats, true);
 
                return len;
        }
@@ -555,20 +700,28 @@ static ssize_t lprocfs_jobstats_seq_write(struct file *file,
 static int lprocfs_jobstats_seq_release(struct inode *inode, struct file *file)
 {
        struct seq_file *seq = file->private_data;
-       struct obd_job_stats *stats = seq->private;
+       struct lprocfs_jobstats_data *data = seq->private;
 
-       lprocfs_job_cleanup(stats, stats->ojs_cleanup_interval);
+       /* drop the ref of last saved job */
+       if (data->pjd_last_job) {
+               job_putref(data->pjd_last_job);
+               data->pjd_last_pos = 0;
+               data->pjd_last_job = NULL;
+       }
+
+       lprocfs_job_cleanup(data->pjd_stats, false);
+       OBD_FREE_PTR(data);
 
        return lprocfs_seq_release(inode, file);
 }
 
-static const struct file_operations lprocfs_jobstats_seq_fops = {
-       .owner   = THIS_MODULE,
-       .open    = lprocfs_jobstats_seq_open,
-       .read    = seq_read,
-       .write   = lprocfs_jobstats_seq_write,
-       .llseek  = seq_lseek,
-       .release = lprocfs_jobstats_seq_release,
+static const struct proc_ops lprocfs_jobstats_seq_fops = {
+       PROC_OWNER(THIS_MODULE)
+       .proc_open      = lprocfs_jobstats_seq_open,
+       .proc_read      = seq_read,
+       .proc_write     = lprocfs_jobstats_seq_write,
+       .proc_lseek     = seq_lseek,
+       .proc_release   = lprocfs_jobstats_seq_release,
 };
 
 int lprocfs_job_stats_init(struct obd_device *obd, int cntr_num,
@@ -594,7 +747,7 @@ int lprocfs_job_stats_init(struct obd_device *obd, int cntr_num,
                       obd->obd_name, obd->obd_type->typ_name, -EINVAL);
                RETURN(-EINVAL);
        }
-       stats = &obd->u.obt.obt_jobstats;
+       stats = &obd2obt(obd)->obt_jobstats;
 
        LASSERT(stats->ojs_hash == NULL);
        stats->ojs_hash = cfs_hash_create("JOB_STATS",
@@ -609,11 +762,14 @@ int lprocfs_job_stats_init(struct obd_device *obd, int cntr_num,
                RETURN(-ENOMEM);
 
        INIT_LIST_HEAD(&stats->ojs_list);
-       rwlock_init(&stats->ojs_lock);
+       spin_lock_init(&stats->ojs_lock);
        stats->ojs_cntr_num = cntr_num;
        stats->ojs_cntr_init_fn = init_fn;
-       stats->ojs_cleanup_interval = 600; /* 10 mins by default */
-       stats->ojs_last_cleanup = ktime_get_real_seconds();
+       /* 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();
 
        entry = lprocfs_add_simple(obd->obd_proc_entry, "job_stats", stats,
                                   &lprocfs_jobstats_seq_fops);
@@ -624,43 +780,42 @@ int lprocfs_job_stats_init(struct obd_device *obd, int cntr_num,
        RETURN(0);
 }
 EXPORT_SYMBOL(lprocfs_job_stats_init);
+#endif /* CONFIG_PROC_FS*/
 
-int lprocfs_job_interval_seq_show(struct seq_file *m, void *data)
+ssize_t job_cleanup_interval_show(struct kobject *kobj, struct attribute *attr,
+                                 char *buf)
 {
-       struct obd_device *obd = m->private;
+       struct obd_device *obd = container_of(kobj, struct obd_device,
+                                             obd_kset.kobj);
        struct obd_job_stats *stats;
+       struct timespec64 ts;
 
-       if (obd == NULL)
-               return -ENODEV;
+       stats = &obd2obt(obd)->obt_jobstats;
+       ts = ktime_to_timespec64(stats->ojs_cleanup_interval);
 
-       stats = &obd->u.obt.obt_jobstats;
-       seq_printf(m, "%d\n", stats->ojs_cleanup_interval);
-       return 0;
+       return scnprintf(buf, PAGE_SIZE, "%lld\n", (long long)ts.tv_sec * 2);
 }
-EXPORT_SYMBOL(lprocfs_job_interval_seq_show);
+EXPORT_SYMBOL(job_cleanup_interval_show);
 
-ssize_t
-lprocfs_job_interval_seq_write(struct file *file, const char __user *buffer,
-                               size_t count, loff_t *off)
+ssize_t job_cleanup_interval_store(struct kobject *kobj,
+                                  struct attribute *attr,
+                                  const char *buffer, size_t count)
 {
-       struct obd_device *obd;
+       struct obd_device *obd = container_of(kobj, struct obd_device,
+                                             obd_kset.kobj);
        struct obd_job_stats *stats;
        unsigned int val;
        int rc;
 
-       obd = ((struct seq_file *)file->private_data)->private;
-       if (obd == NULL)
-               return -ENODEV;
-
-       stats = &obd->u.obt.obt_jobstats;
+       stats = &obd2obt(obd)->obt_jobstats;
 
-       rc = kstrtouint_from_user(buffer, count, 0, &val);
+       rc = kstrtouint(buffer, 0, &val);
        if (rc)
                return rc;
 
-       stats->ojs_cleanup_interval = val;
-       lprocfs_job_cleanup(stats, stats->ojs_cleanup_interval);
+       stats->ojs_cleanup_interval = ktime_set(val / 2, 0);
+       lprocfs_job_cleanup(stats, false);
+
        return count;
 }
-EXPORT_SYMBOL(lprocfs_job_interval_seq_write);
-#endif /* CONFIG_PROC_FS*/
+EXPORT_SYMBOL(job_cleanup_interval_store);