Whamcloud - gitweb
LU-14911 osp: release thandle if it was created
[fs/lustre-release.git] / lustre / osp / osp_precreate.c
index 4d867fa..805f87d 100644 (file)
  * Copyright (c) 2007, 2010, Oracle and/or its affiliates. All rights reserved.
  * Use is subject to license terms.
  *
- * Copyright (c) 2012, 2016, Intel Corporation.
+ * Copyright (c) 2012, 2017, Intel Corporation.
  */
 /*
  * This file is part of Lustre, http://www.lustre.org/
- * Lustre is a trademark of Sun Microsystems, Inc.
  *
- * lustre/osp/osp_sync.c
+ * lustre/osp/osp_precreate.c
  *
  * Lustre OST Proxy Device
  *
  * = import is disconnected =
  *
  * = import is inactive =
- *   in this case osp_declare_object_create() returns an error
+ *   in this case osp_declare_create() returns an error
  *
  */
 
-/*
- **
+/**
  * Check whether statfs data is expired
  *
  * OSP device caches statfs data for the target, the function checks
@@ -69,8 +67,7 @@
  */
 static inline int osp_statfs_need_update(struct osp_device *d)
 {
-       return !cfs_time_before(cfs_time_current(),
-                               d->opd_statfs_fresh_till);
+       return !ktime_before(ktime_get(), d->opd_statfs_fresh_till);
 }
 
 /*
@@ -79,25 +76,46 @@ static inline int osp_statfs_need_update(struct osp_device *d)
  *
  * each time OSP gets connected to OST, we should start from precreation cleanup
  */
-static inline bool osp_precreate_running(struct osp_device *d)
+static void osp_statfs_timer_cb(cfs_timer_cb_arg_t data)
 {
-       return !!(d->opd_pre_thread.t_flags & SVC_RUNNING);
+       struct osp_device *d = cfs_from_timer(d, data, opd_statfs_timer);
+
+       LASSERT(d);
+       if (d->opd_pre_task)
+               wake_up(&d->opd_pre_waitq);
 }
 
-static inline bool osp_precreate_stopped(struct osp_device *d)
+static void osp_pre_update_msfs(struct osp_device *d, struct obd_statfs *msfs);
+
+/*
+ * The function updates current precreation status if broken, and
+ * updates that cached statfs state if functional, then wakes up waiters.
+ * We don't clear opd_pre_status directly here, but rather leave this
+ * to osp_pre_update_msfs() to do if everything is OK so that we don't
+ * have a race to clear opd_pre_status and then set it to -ENOSPC again.
+ *
+ * \param[in] d                OSP device
+ * \param[in] msfs     statfs data
+ * \param[in] rc       new precreate status for device \a d
+ */
+static void osp_pre_update_status_msfs(struct osp_device *d,
+                                      struct obd_statfs *msfs, int rc)
 {
-       return !!(d->opd_pre_thread.t_flags & SVC_STOPPED);
+       if (rc)
+               d->opd_pre_status = rc;
+       else
+               osp_pre_update_msfs(d, msfs);
+
+       wake_up(&d->opd_pre_user_waitq);
 }
 
-static void osp_statfs_timer_cb(unsigned long _d)
+/* Pass in the old statfs data in case the limits have changed */
+void osp_pre_update_status(struct osp_device *d, int rc)
 {
-       struct osp_device *d = (struct osp_device *) _d;
-
-       LASSERT(d);
-       if (d->opd_pre != NULL && osp_precreate_running(d))
-               wake_up(&d->opd_pre_waitq);
+       osp_pre_update_status_msfs(d, &d->opd_statfs, rc);
 }
 
+
 /**
  * RPC interpret callback for OST_STATFS RPC
  *
@@ -115,16 +133,18 @@ static void osp_statfs_timer_cb(unsigned long _d)
  * \retval negative    negated errno on error
  */
 static int osp_statfs_interpret(const struct lu_env *env,
-                               struct ptlrpc_request *req,
-                               union ptlrpc_async_args *aa, int rc)
+                               struct ptlrpc_request *req, void *args, int rc)
 {
-       struct obd_import       *imp = req->rq_import;
-       struct obd_statfs       *msfs;
-       struct osp_device       *d;
+       union ptlrpc_async_args *aa = args;
+       struct obd_import *imp = req->rq_import;
+       struct obd_statfs *msfs;
+       struct obd_statfs *sfs;
+       struct osp_device *d;
+       u64 maxage_ns;
 
        ENTRY;
 
-       aa = ptlrpc_req_async_args(req);
+       aa = ptlrpc_req_async_args(aa, req);
        d = aa->pointer_arg[0];
        LASSERT(d);
 
@@ -135,21 +155,32 @@ static int osp_statfs_interpret(const struct lu_env *env,
        if (msfs == NULL)
                GOTO(out, rc = -EPROTO);
 
-       d->opd_statfs = *msfs;
-
-       osp_pre_update_status(d, rc);
+       if (d->opd_pre)
+               osp_pre_update_status_msfs(d, msfs, 0);
+       else
+               d->opd_statfs = *msfs;
 
        /* schedule next update */
-       d->opd_statfs_fresh_till = cfs_time_shift(d->opd_statfs_maxage);
-       mod_timer(&d->opd_statfs_timer, d->opd_statfs_fresh_till);
+       maxage_ns = d->opd_statfs_maxage * NSEC_PER_SEC;
+       d->opd_statfs_fresh_till = ktime_add_ns(ktime_get(), maxage_ns);
+       mod_timer(&d->opd_statfs_timer,
+                 jiffies + cfs_time_seconds(d->opd_statfs_maxage));
        d->opd_statfs_update_in_progress = 0;
 
-       CDEBUG(D_CACHE, "updated statfs %p\n", d);
+       sfs = &d->opd_statfs;
+       CDEBUG(D_CACHE, "%s (%p): %llu blocks, %llu free, %llu avail, "
+              "%u bsize, %u reserved mb low, %u reserved mb high,"
+              "%llu files, %llu free files\n", d->opd_obd->obd_name, d,
+              sfs->os_blocks, sfs->os_bfree, sfs->os_bavail, sfs->os_bsize,
+              d->opd_reserved_mb_low, d->opd_reserved_mb_high,
+              sfs->os_files, sfs->os_ffree);
 
        RETURN(0);
 out:
-       /* couldn't update statfs, try again as soon as possible */
-       if (d->opd_pre != NULL && osp_precreate_running(d))
+       /* couldn't update statfs, try again with a small delay */
+       d->opd_statfs_fresh_till = ktime_add_ns(ktime_get(), 10 * NSEC_PER_SEC);
+       d->opd_statfs_update_in_progress = 0;
+       if (d->opd_pre && d->opd_pre_task)
                wake_up(&d->opd_pre_waitq);
 
        if (req->rq_import_generation == imp->imp_generation)
@@ -168,12 +199,13 @@ out:
  *
  * \param[in] d                OSP device
  */
-static int osp_statfs_update(struct osp_device *d)
+static int osp_statfs_update(const struct lu_env *env, struct osp_device *d)
 {
+       u64 expire = obd_timeout * 1000 * NSEC_PER_SEC;
        struct ptlrpc_request   *req;
        struct obd_import       *imp;
        union ptlrpc_async_args *aa;
-       int                      rc;
+       int rc;
 
        ENTRY;
 
@@ -182,32 +214,59 @@ static int osp_statfs_update(struct osp_device *d)
        imp = d->opd_obd->u.cli.cl_import;
        LASSERT(imp);
 
-       req = ptlrpc_request_alloc(imp, &RQF_OST_STATFS);
+       req = ptlrpc_request_alloc(imp,
+                          d->opd_pre ? &RQF_OST_STATFS : &RQF_MDS_STATFS);
        if (req == NULL)
                RETURN(-ENOMEM);
 
-       rc = ptlrpc_request_pack(req, LUSTRE_OST_VERSION, OST_STATFS);
+       rc = ptlrpc_request_pack(req,
+                        d->opd_pre ? LUSTRE_OST_VERSION : LUSTRE_MDS_VERSION,
+                        d->opd_pre ? OST_STATFS : MDS_STATFS);
        if (rc) {
                ptlrpc_request_free(req);
                RETURN(rc);
        }
        ptlrpc_request_set_replen(req);
-       req->rq_request_portal = OST_CREATE_PORTAL;
+       if (d->opd_pre)
+               req->rq_request_portal = OST_CREATE_PORTAL;
        ptlrpc_at_set_req_timeout(req);
 
-       req->rq_interpret_reply = (ptlrpc_interpterer_t)osp_statfs_interpret;
-       aa = ptlrpc_req_async_args(req);
+       req->rq_interpret_reply = osp_statfs_interpret;
+       aa = ptlrpc_req_async_args(aa, req);
        aa->pointer_arg[0] = d;
 
        /*
         * no updates till reply
         */
        del_timer(&d->opd_statfs_timer);
-       d->opd_statfs_fresh_till = cfs_time_shift(obd_timeout * 1000);
+       d->opd_statfs_fresh_till = ktime_add_ns(ktime_get(), expire);
        d->opd_statfs_update_in_progress = 1;
 
        ptlrpcd_add_req(req);
 
+       /* we still want to sync changes if no new changes are coming */
+       if (ktime_before(ktime_get(), d->opd_sync_next_commit_cb))
+               GOTO(out, rc);
+
+       if (atomic_read(&d->opd_sync_changes)) {
+               struct thandle *th;
+
+               th = dt_trans_create(env, d->opd_storage);
+               if (IS_ERR(th)) {
+                       CERROR("%s: can't sync\n", d->opd_obd->obd_name);
+                       GOTO(out, rc);
+               }
+               rc = dt_trans_start_local(env, d->opd_storage, th);
+               if (rc == 0) {
+                       CDEBUG(D_OTHER, "%s: sync forced, %d changes\n",
+                              d->opd_obd->obd_name,
+                              atomic_read(&d->opd_sync_changes));
+                       osp_sync_add_commit_cb_1s(env, d, th);
+               }
+               dt_trans_stop(env, d->opd_storage, th);
+       }
+
+out:
        RETURN(0);
 }
 
@@ -232,7 +291,7 @@ void osp_statfs_need_now(struct osp_device *d)
                 * then we should poll OST immediately once object destroy
                 * is replied
                 */
-               d->opd_statfs_fresh_till = cfs_time_shift(-1);
+               d->opd_statfs_fresh_till = ktime_sub_ns(ktime_get(), NSEC_PER_SEC);
                del_timer(&d->opd_statfs_timer);
                wake_up(&d->opd_pre_waitq);
        }
@@ -296,6 +355,9 @@ static inline int osp_precreate_near_empty(const struct lu_env *env,
 {
        int rc;
 
+       if (d->opd_pre == NULL)
+               return 0;
+
        /* XXX: do we really need locking here? */
        spin_lock(&d->opd_pre_lock);
        rc = osp_precreate_near_empty_nolock(env, d);
@@ -349,6 +411,7 @@ int osp_write_last_oid_seq_files(struct lu_env *env, struct osp_device *osp,
        struct lu_buf      *lb_oid = &oti->osi_lb;
        struct lu_buf      *lb_oseq = &oti->osi_lb2;
        loff_t             oid_off;
+       u64                oid;
        loff_t             oseq_off;
        struct thandle    *th;
        int                   rc;
@@ -359,13 +422,12 @@ int osp_write_last_oid_seq_files(struct lu_env *env, struct osp_device *osp,
 
        /* Note: through f_oid is only 32 bits, it will also write 64 bits
         * for oid to keep compatibility with the previous version. */
-       lb_oid->lb_buf = &fid->f_oid;
-       lb_oid->lb_len = sizeof(u64);
-       oid_off = sizeof(u64) * osp->opd_index;
+       oid = fid->f_oid;
+       osp_objid_buf_prep(lb_oid, &oid_off,
+                          &oid, osp->opd_index);
 
-       lb_oseq->lb_buf = &fid->f_seq;
-       lb_oseq->lb_len = sizeof(u64);
-       oseq_off = sizeof(u64) * osp->opd_index;
+       osp_objseq_buf_prep(lb_oseq, &oseq_off,
+                           &fid->f_seq, osp->opd_index);
 
        th = dt_trans_create(env, osp->opd_storage);
        if (IS_ERR(th))
@@ -458,6 +520,7 @@ static int osp_precreate_rollover_new_seq(struct lu_env *env,
        /* Update last_xxx to the new seq */
        spin_lock(&osp->opd_pre_lock);
        osp->opd_last_used_fid = *fid;
+       osp_fid_to_obdid(fid, &osp->opd_last_id);
        osp->opd_gap_start_fid = *fid;
        osp->opd_pre_used_fid = *fid;
        osp->opd_pre_last_created_fid = *fid;
@@ -530,9 +593,9 @@ static int osp_precreate_fids(const struct lu_env *env, struct osp_device *osp,
  *
  * The function finds how many objects should be precreated.  Then allocates,
  * prepares and schedules precreate RPC synchronously. Upon reply the function
- * wake ups the threads waiting for the new objects on this target. If the
+ * wakes up the threads waiting for the new objects on this target. If the
  * target wasn't able to create all the objects requested, then the next
- * precreate will be asking less objects (i.e. slow precreate down).
+ * precreate will be asking for fewer objects (i.e. slow precreate down).
  *
  * \param[in] env      LU environment provided by the caller
  * \param[in] d                OSP device
@@ -588,15 +651,9 @@ static int osp_precreate_send(const struct lu_env *env, struct osp_device *d)
 
        *fid = d->opd_pre_last_created_fid;
        rc = osp_precreate_fids(env, d, fid, &grow);
-       if (rc == 1) {
+       if (rc == 1)
                /* Current seq has been used up*/
-               if (!osp_is_fid_client(d)) {
-                       osp_pre_update_status(d, -ENOSPC);
-                       rc = -ENOSPC;
-               }
-               wake_up(&d->opd_pre_waitq);
-               GOTO(out_req, rc);
-       }
+               GOTO(out_req, rc = -ENOSPC);
 
        if (!osp_is_fid_client(d)) {
                /* Non-FID client will always send seq 0 because of
@@ -617,6 +674,9 @@ static int osp_precreate_send(const struct lu_env *env, struct osp_device *d)
        if (rc) {
                CERROR("%s: can't precreate: rc = %d\n", d->opd_obd->obd_name,
                       rc);
+               if (req->rq_net_err)
+                       /* have osp_precreate_reserve() to wait for repeat */
+                       rc = -ENOTCONN;
                GOTO(out_req, rc);
        }
        LASSERT(req->rq_transno == 0);
@@ -629,7 +689,7 @@ static int osp_precreate_send(const struct lu_env *env, struct osp_device *d)
 
 ready:
        if (osp_fid_diff(fid, &d->opd_pre_used_fid) <= 0) {
-               CERROR("%s: precreate fid "DFID" < local used fid "DFID
+               CERROR("%s: precreate fid "DFID" <= local used fid "DFID
                       ": rc = %d\n", d->opd_obd->obd_name,
                       PFID(fid), PFID(&d->opd_pre_used_fid), -ESTALE);
                GOTO(out_req, rc = -ESTALE);
@@ -664,6 +724,9 @@ out_req:
        osp_pre_update_status(d, rc);
        wake_up(&d->opd_pre_user_waitq);
 
+       /* pause to let osp_precreate_reserve to go first */
+       CFS_FAIL_TIMEOUT(OBD_FAIL_OSP_PRECREATE_PAUSE, 2);
+
        ptlrpc_req_finished(req);
        RETURN(rc);
 }
@@ -778,7 +841,6 @@ static int osp_precreate_cleanup_orphans(struct lu_env *env,
        struct ptlrpc_request   *req = NULL;
        struct obd_import       *imp;
        struct ost_body         *body;
-       struct l_wait_info       lwi = { 0 };
        int                      update_status = 0;
        int                      rc;
        int                      diff;
@@ -803,11 +865,10 @@ static int osp_precreate_cleanup_orphans(struct lu_env *env,
         * catch all osp_precreate_reserve() calls who find
         * "!opd_pre_recovering".
         */
-       l_wait_event(d->opd_pre_waitq,
-                    (!d->opd_pre_reserved && d->opd_recovery_completed) ||
-                    !osp_precreate_running(d) || d->opd_got_disconnected,
-                    &lwi);
-       if (!osp_precreate_running(d) || d->opd_got_disconnected)
+       wait_event_idle(d->opd_pre_waitq,
+                       (!d->opd_pre_reserved && d->opd_recovery_completed) ||
+                       !d->opd_pre_task || d->opd_got_disconnected);
+       if (!d->opd_pre_task || d->opd_got_disconnected)
                GOTO(out, rc = -EAGAIN);
 
        CDEBUG(D_HA, "%s: going to cleanup orphans since "DFID"\n",
@@ -927,92 +988,106 @@ out:
  * data is used to make this decision. If the latest result of statfs
  * request (rc argument) is not success, then just mark OSP unavailable
  * right away.
-
- * Add a bit of hysteresis so this flag isn't continually flapping,
- * and ensure that new files don't get extremely fragmented due to
- * only a small amount of available space in the filesystem.
- * We want to set the ENOSPC when there is less than reserved size
- * free and clear it when there is at least 2*reserved size free space.
- * the function updates current precreation status used: functional or not
  *
- * \param[in] d                OSP device
- * \param[in] rc       new precreate status for device \a d
+ * The new statfs data is passed in \a msfs and needs to be stored into
+ * opd_statfs, but only after the various flags in os_state are set, so
+ * that the new statfs data is not visible without appropriate flags set.
+ * As such, there is no need to clear the flags here, since this is called
+ * with new statfs data, and they should not be cleared if sent from OST.
  *
- * \retval 0           on success
- * \retval negative    negated errno on error
+ * Add a bit of hysteresis so this flag isn't continually flapping, and
+ * ensure that new files don't get extremely fragmented due to only a
+ * small amount of available space in the filesystem.  We want to set
+ * the ENOSPC/ENOINO flags unconditionally when there is less than the
+ * reserved size free, and still copy them from the old state when there
+ * is less than 2*reserved size free space or inodes.
+ *
+ * \param[in] d                OSP device
+ * \param[in] msfs     statfs data
  */
-void osp_pre_update_status(struct osp_device *d, int rc)
+static void osp_pre_update_msfs(struct osp_device *d, struct obd_statfs *msfs)
 {
-       struct obd_statfs       *msfs = &d->opd_statfs;
-       int                      old = d->opd_pre_status;
-       __u64                    available;
+       u32 old_state = d->opd_statfs.os_state;
+       u32 reserved_ino_low = 32;      /* could be tunable in the future */
+       u32 reserved_ino_high = reserved_ino_low * 2;
+       u64 available_mb;
+
+       /* statfs structure not initialized yet */
+       if (unlikely(!msfs->os_type))
+               return;
+
+       /* if the low and high watermarks have not been initialized yet */
+       if (unlikely(d->opd_reserved_mb_high == 0 &&
+                    d->opd_reserved_mb_low == 0)) {
+               /* Use ~0.1% by default to disable object allocation,
+                * and ~0.2% to enable, size in MB, set both watermark
+                */
+               spin_lock(&d->opd_pre_lock);
+               if (d->opd_reserved_mb_high == 0 &&
+                   d->opd_reserved_mb_low == 0) {
+                       d->opd_reserved_mb_low = ((msfs->os_bsize >> 10) *
+                                                 msfs->os_blocks) >> 20;
+                       if (d->opd_reserved_mb_low == 0)
+                               d->opd_reserved_mb_low = 1;
+                       d->opd_reserved_mb_high =
+                               (d->opd_reserved_mb_low << 1) + 1;
+               }
+               spin_unlock(&d->opd_pre_lock);
+       }
 
-       d->opd_pre_status = rc;
-       if (rc)
-               goto out;
+       available_mb = (msfs->os_bavail * (msfs->os_bsize >> 10)) >> 10;
+       if (msfs->os_ffree < reserved_ino_low)
+               msfs->os_state |= OS_STATFS_ENOINO;
+       else if (msfs->os_ffree <= reserved_ino_high)
+               msfs->os_state |= old_state & OS_STATFS_ENOINO;
+       /* else don't clear flags in new msfs->os_state sent from OST */
+
+       CDEBUG(D_INFO,
+              "%s: blocks=%llu free=%llu avail=%llu avail_mb=%llu hwm_mb=%u files=%llu ffree=%llu state=%x: rc = %d\n",
+              d->opd_obd->obd_name, msfs->os_blocks, msfs->os_bfree,
+              msfs->os_bavail, available_mb, d->opd_reserved_mb_high,
+              msfs->os_files, msfs->os_ffree, msfs->os_state,
+              d->opd_pre_status);
+       if (available_mb < d->opd_reserved_mb_low)
+               msfs->os_state |= OS_STATFS_ENOSPC;
+       else if (available_mb <= d->opd_reserved_mb_high)
+               msfs->os_state |= old_state & OS_STATFS_ENOSPC;
+       /* else don't clear flags in new msfs->os_state sent from OST */
+
+       if (msfs->os_state & (OS_STATFS_ENOINO | OS_STATFS_ENOSPC)) {
+               d->opd_pre_status = -ENOSPC;
+               if (!(old_state & (OS_STATFS_ENOINO | OS_STATFS_ENOSPC)))
+                       CDEBUG(D_INFO, "%s: full: state=%x: rc = %x\n",
+                              d->opd_obd->obd_name, msfs->os_state,
+                              d->opd_pre_status);
+               CDEBUG(D_INFO, "uncommitted changes=%u in_progress=%u\n",
+                      atomic_read(&d->opd_sync_changes),
+                      atomic_read(&d->opd_sync_rpcs_in_progress));
+       } else if (old_state & (OS_STATFS_ENOINO | OS_STATFS_ENOSPC)) {
+               d->opd_pre_status = 0;
+               spin_lock(&d->opd_pre_lock);
+               d->opd_pre_create_slow = 0;
+               d->opd_pre_create_count = OST_MIN_PRECREATE;
+               spin_unlock(&d->opd_pre_lock);
+               wake_up(&d->opd_pre_waitq);
 
-       if (likely(msfs->os_type)) {
-               if (unlikely(d->opd_reserved_mb_high == 0 &&
-                            d->opd_reserved_mb_low == 0)) {
-                       /* Use ~0.1% by default to disable object allocation,
-                        * and ~0.2% to enable, size in MB, set both watermark
-                        */
-                       spin_lock(&d->opd_pre_lock);
-                       if (d->opd_reserved_mb_high == 0 &&
-                           d->opd_reserved_mb_low == 0) {
-                               d->opd_reserved_mb_low =
-                                       ((msfs->os_bsize >> 10) *
-                                       msfs->os_blocks) >> 20;
-                               if (d->opd_reserved_mb_low == 0)
-                                       d->opd_reserved_mb_low = 1;
-                               d->opd_reserved_mb_high =
-                                       (d->opd_reserved_mb_low << 1) + 1;
-                       }
-                       spin_unlock(&d->opd_pre_lock);
-               }
-               /* in MB */
-               available = (msfs->os_bavail * (msfs->os_bsize >> 10)) >> 10;
-               if (msfs->os_ffree < 32)
-                       msfs->os_state |= OS_STATE_ENOINO;
-               else if (msfs->os_ffree > 64)
-                       msfs->os_state &= ~OS_STATE_ENOINO;
-
-               if (available < d->opd_reserved_mb_low)
-                       msfs->os_state |= OS_STATE_ENOSPC;
-               else if (available > d->opd_reserved_mb_high)
-                       msfs->os_state &= ~OS_STATE_ENOSPC;
-               if (msfs->os_state & (OS_STATE_ENOINO | OS_STATE_ENOSPC)) {
-                       d->opd_pre_status = -ENOSPC;
-                       if (old != -ENOSPC)
-                               CDEBUG(D_INFO, "%s: status: %llu blocks, %llu "
-                                      "free, %llu avail, %llu MB avail, %u "
-                                      "hwm -> %d: rc = %d\n",
-                                      d->opd_obd->obd_name, msfs->os_blocks,
-                                      msfs->os_bfree, msfs->os_bavail,
-                                      available, d->opd_reserved_mb_high,
-                                      d->opd_pre_status, rc);
-                       CDEBUG(D_INFO,
-                              "non-committed changes: %u, in progress: %u\n",
-                              atomic_read(&d->opd_syn_changes),
-                              atomic_read(&d->opd_syn_rpc_in_progress));
-               } else if (unlikely(old == -ENOSPC)) {
-                       d->opd_pre_status = 0;
-                       spin_lock(&d->opd_pre_lock);
-                       d->opd_pre_create_slow = 0;
-                       d->opd_pre_create_count = OST_MIN_PRECREATE;
-                       spin_unlock(&d->opd_pre_lock);
-                       wake_up(&d->opd_pre_waitq);
-
-                       CDEBUG(D_INFO, "%s: space available: %llu blocks, %llu"
-                              " free, %llu avail, %lluMB avail, %u lwm"
-                              " -> %d: rc = %d\n", d->opd_obd->obd_name,
-                              msfs->os_blocks, msfs->os_bfree, msfs->os_bavail,
-                              available, d->opd_reserved_mb_low,
-                              d->opd_pre_status, rc);
-               }
+               CDEBUG(D_INFO,
+                      "%s: available: state=%x: rc = %d\n",
+                      d->opd_obd->obd_name, msfs->os_state,
+                      d->opd_pre_status);
+       } else {
+               /* we only get here if rc == 0 in the caller */
+               d->opd_pre_status = 0;
        }
-out:
-       wake_up(&d->opd_pre_user_waitq);
+
+       /* Object precreation skipped on OST if manually disabled */
+       if (d->opd_pre_max_create_count == 0)
+               msfs->os_state |= OS_STATFS_NOPRECREATE;
+       /* else don't clear flags in new msfs->os_state sent from OST */
+
+       /* copy only new statfs state to make it visible to MDS threads */
+       if (&d->opd_statfs != msfs)
+               d->opd_statfs = *msfs;
 }
 
 /**
@@ -1105,6 +1180,11 @@ out:
        RETURN(rc);
 }
 
+struct opt_args {
+       struct osp_device       *opta_dev;
+       struct lu_env           opta_env;
+       struct completion       *opta_started;
+};
 /**
  * The core of precreate functionality
  *
@@ -1123,101 +1203,91 @@ out:
  * \retval 0           on success
  * \retval negative    negated errno on error
  */
-static int osp_precreate_thread(void *_arg)
+static int osp_precreate_thread(void *_args)
 {
-       struct osp_device       *d = _arg;
-       struct ptlrpc_thread    *thread = &d->opd_pre_thread;
-       struct l_wait_info       lwi = { 0 };
-       struct l_wait_info       lwi2 = LWI_TIMEOUT(cfs_time_seconds(5),
-                                                   back_to_sleep, NULL);
-       struct lu_env            env;
+       struct opt_args         *args = _args;
+       struct osp_device       *d = args->opta_dev;
+       struct lu_env           *env = &args->opta_env;
        int                      rc;
 
        ENTRY;
 
-       rc = lu_env_init(&env, d->opd_dt_dev.dd_lu_dev.ld_type->ldt_ctx_tags);
-       if (rc) {
-               CERROR("%s: init env error: rc = %d\n", d->opd_obd->obd_name,
-                      rc);
-
-               spin_lock(&d->opd_pre_lock);
-               thread->t_flags = SVC_STOPPED;
-               spin_unlock(&d->opd_pre_lock);
-               wake_up(&thread->t_ctl_waitq);
-
-               RETURN(rc);
-       }
-
-       spin_lock(&d->opd_pre_lock);
-       thread->t_flags = SVC_RUNNING;
-       spin_unlock(&d->opd_pre_lock);
-       wake_up(&thread->t_ctl_waitq);
-
-       while (osp_precreate_running(d)) {
+       complete(args->opta_started);
+       while (!kthread_should_stop()) {
                /*
                 * need to be connected to OST
                 */
-               while (osp_precreate_running(d)) {
-                       if (d->opd_pre_recovering &&
+               while (!kthread_should_stop()) {
+                       if ((d->opd_pre == NULL || d->opd_pre_recovering) &&
                            d->opd_imp_connected &&
                            !d->opd_got_disconnected)
                                break;
-                       l_wait_event(d->opd_pre_waitq,
-                                    !osp_precreate_running(d) ||
-                                    d->opd_new_connection,
-                                    &lwi);
+                       wait_event_idle(d->opd_pre_waitq,
+                                       kthread_should_stop() ||
+                                       d->opd_new_connection);
 
                        if (!d->opd_new_connection)
                                continue;
 
+                       OBD_FAIL_TIMEOUT(OBD_FAIL_OSP_CON_EVENT_DELAY,
+                                        cfs_fail_val);
                        d->opd_new_connection = 0;
                        d->opd_got_disconnected = 0;
                        break;
                }
 
-               if (!osp_precreate_running(d))
+               if (kthread_should_stop())
                        break;
 
-               LASSERT(d->opd_obd->u.cli.cl_seq != NULL);
-               /* Sigh, fid client is not ready yet */
-               if (d->opd_obd->u.cli.cl_seq->lcs_exp == NULL)
-                       continue;
+               if (d->opd_pre) {
+                       LASSERT(d->opd_obd->u.cli.cl_seq != NULL);
+                       /* Sigh, fid client is not ready yet */
+                       if (d->opd_obd->u.cli.cl_seq->lcs_exp == NULL)
+                               continue;
 
-               /* Init fid for osp_precreate if necessary */
-               rc = osp_init_pre_fid(d);
-               if (rc != 0) {
-                       class_export_put(d->opd_exp);
-                       d->opd_obd->u.cli.cl_seq->lcs_exp = NULL;
-                       CERROR("%s: init pre fid error: rc = %d\n",
-                              d->opd_obd->obd_name, rc);
-                       continue;
+                       /* Init fid for osp_precreate if necessary */
+                       rc = osp_init_pre_fid(d);
+                       if (rc != 0) {
+                               class_export_put(d->opd_exp);
+                               d->opd_obd->u.cli.cl_seq->lcs_exp = NULL;
+                               CERROR("%s: init pre fid error: rc = %d\n",
+                                               d->opd_obd->obd_name, rc);
+                               continue;
+                       }
                }
 
-               if (osp_statfs_update(d)) {
-                       l_wait_event(d->opd_pre_waitq,
-                                    !osp_precreate_running(d), &lwi2);
+               if (osp_statfs_update(env, d)) {
+                       if (wait_event_idle_timeout(d->opd_pre_waitq,
+                                                   kthread_should_stop(),
+                                                   cfs_time_seconds(5)) == 0)
+                               l_wait_event_abortable(
+                                       d->opd_pre_waitq,
+                                       kthread_should_stop());
                        continue;
                }
 
-               /*
-                * Clean up orphans or recreate missing objects.
-                */
-               rc = osp_precreate_cleanup_orphans(&env, d);
-               if (rc != 0) {
-                       schedule_timeout_interruptible(cfs_time_seconds(1));
-                       continue;
+               if (d->opd_pre) {
+                       /*
+                        * Clean up orphans or recreate missing objects.
+                        */
+                       rc = osp_precreate_cleanup_orphans(env, d);
+                       if (rc != 0) {
+                               schedule_timeout_interruptible(cfs_time_seconds(1));
+                               continue;
+                       }
                }
+
                /*
                 * connected, can handle precreates now
                 */
-               while (osp_precreate_running(d)) {
-                       l_wait_event(d->opd_pre_waitq,
-                                    !osp_precreate_running(d) ||
-                                    osp_precreate_near_empty(&env, d) ||
-                                    osp_statfs_need_update(d) ||
-                                    d->opd_got_disconnected, &lwi);
-
-                       if (!osp_precreate_running(d))
+               while (!kthread_should_stop()) {
+                       wait_event_idle(d->opd_pre_waitq,
+                                       kthread_should_stop() ||
+                                       osp_precreate_near_empty(env, d) ||
+                                       osp_statfs_need_update(d) ||
+                                       d->opd_got_disconnected);
+
+                       if (kthread_should_stop())
                                break;
 
                        /* something happened to the connection
@@ -1226,29 +1296,32 @@ static int osp_precreate_thread(void *_arg)
                                break;
 
                        if (osp_statfs_need_update(d))
-                               if (osp_statfs_update(d))
+                               if (osp_statfs_update(env, d))
                                        break;
 
+                       if (d->opd_pre == NULL)
+                               continue;
+
                        /* To avoid handling different seq in precreate/orphan
                         * cleanup, it will hold precreate until current seq is
                         * used up. */
-                       if (unlikely(osp_precreate_end_seq(&env, d) &&
-                           !osp_create_end_seq(&env, d)))
+                       if (unlikely(osp_precreate_end_seq(env, d) &&
+                           !osp_create_end_seq(env, d)))
                                continue;
 
-                       if (unlikely(osp_precreate_end_seq(&env, d) &&
-                                    osp_create_end_seq(&env, d))) {
+                       if (unlikely(osp_precreate_end_seq(env, d) &&
+                                    osp_create_end_seq(env, d))) {
                                LCONSOLE_INFO("%s:%#llx is used up."
                                              " Update to new seq\n",
                                              d->opd_obd->obd_name,
                                         fid_seq(&d->opd_pre_last_created_fid));
-                               rc = osp_precreate_rollover_new_seq(&env, d);
+                               rc = osp_precreate_rollover_new_seq(env, d);
                                if (rc)
                                        continue;
                        }
 
-                       if (osp_precreate_near_empty(&env, d)) {
-                               rc = osp_precreate_send(&env, d);
+                       if (osp_precreate_near_empty(env, d)) {
+                               rc = osp_precreate_send(env, d);
                                /* osp_precreate_send() sets opd_pre_status
                                 * in case of error, that prevent the using of
                                 * failed device. */
@@ -1261,9 +1334,8 @@ static int osp_precreate_thread(void *_arg)
                }
        }
 
-       thread->t_flags = SVC_STOPPED;
-       lu_env_fini(&env);
-       wake_up(&thread->t_ctl_waitq);
+       lu_env_fini(env);
+       OBD_FREE_PTR(args);
 
        RETURN(0);
 }
@@ -1294,8 +1366,8 @@ static int osp_precreate_ready_condition(const struct lu_env *env,
                return 1;
 
        /* ready if OST reported no space and no destroys in progress */
-       if (atomic_read(&d->opd_syn_changes) +
-           atomic_read(&d->opd_syn_rpc_in_progress) == 0 &&
+       if (atomic_read(&d->opd_sync_changes) +
+           atomic_read(&d->opd_sync_rpcs_in_progress) == 0 &&
            d->opd_pre_status == -ENOSPC)
                return 1;
 
@@ -1315,22 +1387,6 @@ static int osp_precreate_ready_condition(const struct lu_env *env,
        return 0;
 }
 
-static int osp_precreate_timeout_condition(void *data)
-{
-       struct osp_device *d = data;
-
-       CDEBUG(D_HA, "%s: slow creates, last="DFID", next="DFID", "
-             "reserved=%llu, syn_changes=%u, "
-             "syn_rpc_in_progress=%d, status=%d\n",
-             d->opd_obd->obd_name, PFID(&d->opd_pre_last_created_fid),
-             PFID(&d->opd_pre_used_fid), d->opd_pre_reserved,
-             atomic_read(&d->opd_syn_changes),
-             atomic_read(&d->opd_syn_rpc_in_progress),
-             d->opd_pre_status);
-
-       return 1;
-}
-
 /**
  * Reserve object in precreate pool
  *
@@ -1355,11 +1411,11 @@ static int osp_precreate_timeout_condition(void *data)
  * \retval             -EAGAIN try later, slow precreation in progress
  * \retval             -EIO when no access to OST
  */
-int osp_precreate_reserve(const struct lu_env *env, struct osp_device *d)
+int osp_precreate_reserve(const struct lu_env *env, struct osp_device *d,
+                         bool can_block)
 {
-       struct l_wait_info       lwi;
-       cfs_time_t               expire = cfs_time_shift(obd_timeout);
-       int                      precreated, rc;
+       time64_t expire = ktime_get_seconds() + obd_timeout;
+       int precreated, rc, synced = 0;
 
        ENTRY;
 
@@ -1418,16 +1474,19 @@ int osp_precreate_reserve(const struct lu_env *env, struct osp_device *d)
                 * wait till that is done - some space might be released
                 */
                if (unlikely(rc == -ENOSPC)) {
-                       if (atomic_read(&d->opd_syn_changes)) {
+                       if (atomic_read(&d->opd_sync_changes) && synced == 0) {
                                /* force local commit to release space */
                                dt_commit_async(env, d->opd_storage);
+                               osp_sync_check_for_work(d);
+                               synced = 1;
                        }
-                       if (atomic_read(&d->opd_syn_rpc_in_progress)) {
-                               /* just wait till destroys are done */
-                               /* see l_wait_even() few lines below */
+                       if (atomic_read(&d->opd_sync_rpcs_in_progress)) {
+                               /* just wait till destroys are done
+                                * see wait_event_idle_timeout() below
+                                */
                        }
-                       if (atomic_read(&d->opd_syn_changes) +
-                           atomic_read(&d->opd_syn_rpc_in_progress) == 0) {
+                       if (atomic_read(&d->opd_sync_changes) +
+                           atomic_read(&d->opd_sync_rpcs_in_progress) == 0) {
                                /* no hope for free space */
                                break;
                        }
@@ -1436,15 +1495,32 @@ int osp_precreate_reserve(const struct lu_env *env, struct osp_device *d)
                /* XXX: don't wake up if precreation is in progress */
                wake_up(&d->opd_pre_waitq);
 
-               lwi = LWI_TIMEOUT(expire - cfs_time_current(),
-                               osp_precreate_timeout_condition, d);
-               if (cfs_time_aftereq(cfs_time_current(), expire)) {
+               if (ktime_get_seconds() >= expire) {
                        rc = -ETIMEDOUT;
                        break;
                }
 
-               l_wait_event(d->opd_pre_user_waitq,
-                            osp_precreate_ready_condition(env, d), &lwi);
+               if (!can_block) {
+                       LASSERT(d->opd_pre);
+                       rc = -ENOBUFS;
+                       break;
+               }
+
+               if (wait_event_idle_timeout(
+                           d->opd_pre_user_waitq,
+                           osp_precreate_ready_condition(env, d),
+                           cfs_time_seconds(obd_timeout)) == 0) {
+                       CDEBUG(D_HA,
+                              "%s: slow creates, last="DFID", next="DFID", "
+                              "reserved=%llu, sync_changes=%u, "
+                              "sync_rpcs_in_progress=%d, status=%d\n",
+                              d->opd_obd->obd_name,
+                              PFID(&d->opd_pre_last_created_fid),
+                              PFID(&d->opd_pre_used_fid), d->opd_pre_reserved,
+                              atomic_read(&d->opd_sync_changes),
+                              atomic_read(&d->opd_sync_rpcs_in_progress),
+                              d->opd_pre_status);
+               }
        }
 
        RETURN(rc);
@@ -1475,6 +1551,7 @@ int osp_precreate_reserve(const struct lu_env *env, struct osp_device *d)
 int osp_precreate_get_fid(const struct lu_env *env, struct osp_device *d,
                          struct lu_fid *fid)
 {
+       struct lu_fid *pre_used_fid = &d->opd_pre_used_fid;
        /* grab next id from the pool */
        spin_lock(&d->opd_pre_lock);
 
@@ -1484,6 +1561,20 @@ int osp_precreate_get_fid(const struct lu_env *env, struct osp_device *d,
                 PFID(&d->opd_pre_used_fid),
                 PFID(&d->opd_pre_last_created_fid));
 
+       /*
+        * When sequence is used up, new one should be allocated in
+        * osp_precreate_rollover_new_seq. So ASSERT here to avoid
+        * objid overflow.
+        */
+       LASSERTF(osp_fid_end_seq(env, pre_used_fid) == 0,
+                "next fid "DFID" last created fid "DFID"\n",
+                PFID(&d->opd_pre_used_fid),
+                PFID(&d->opd_pre_last_created_fid));
+       /* Non IDIF fids shoulnd't get here with oid == 0xFFFFFFFF. */
+       if (fid_is_idif(pre_used_fid) &&
+           unlikely(fid_oid(pre_used_fid) == LUSTRE_DATA_SEQ_MAX_WIDTH))
+               pre_used_fid->f_seq++;
+
        d->opd_pre_used_fid.f_oid++;
        memcpy(fid, &d->opd_pre_used_fid, sizeof(*fid));
        d->opd_pre_reserved--;
@@ -1499,7 +1590,8 @@ int osp_precreate_get_fid(const struct lu_env *env, struct osp_device *d,
         * all reservations are released, see comment in
         * osp_precreate_thread() just before orphan cleanup
         */
-       if (unlikely(d->opd_pre_reserved == 0 && d->opd_pre_status))
+       if (unlikely(d->opd_pre_reserved == 0 &&
+                    (d->opd_pre_recovering || d->opd_pre_status)))
                wake_up(&d->opd_pre_waitq);
 
        return 0;
@@ -1540,8 +1632,6 @@ int osp_object_truncate(const struct lu_env *env, struct dt_object *dt,
        if (req == NULL)
                RETURN(-ENOMEM);
 
-       /* XXX: capa support? */
-       /* osc_set_capa_size(req, &RMF_CAPA1, capa); */
        rc = ptlrpc_request_pack(req, LUSTRE_OST_VERSION, OST_PUNCH);
        if (rc) {
                ptlrpc_request_free(req);
@@ -1552,12 +1642,12 @@ int osp_object_truncate(const struct lu_env *env, struct dt_object *dt,
         * XXX: decide how do we do here with resend
         * if we don't resend, then client may see wrong file size
         * if we do resend, then MDS thread can get stuck for quite long
-        * and if we don't resend, then client will also get -EWOULDBLOCK !!
+        * and if we don't resend, then client will also get -EAGAIN !!
         * (see LU-7975 and sanity/test_27F use cases)
         * but let's decide not to resend/delay this truncate request to OST
         * and allow Client to decide to resend, in a less agressive way from
         * after_reply(), by returning -EINPROGRESS instead of
-        * -EAGAIN/-EWOULDBLOCK upon return from ptlrpc_queue_wait() at the
+        * -EAGAIN/-EAGAIN upon return from ptlrpc_queue_wait() at the
         * end of this routine
         */
        req->rq_no_resend = req->rq_no_delay = 1;
@@ -1587,14 +1677,14 @@ int osp_object_truncate(const struct lu_env *env, struct dt_object *dt,
 
        rc = ptlrpc_queue_wait(req);
        if (rc) {
-               /* -EWOULDBLOCK/-EAGAIN means OST is unreachable at the moment
+               /* -EAGAIN/-EWOULDBLOCK means OST is unreachable at the moment
                 * since we have decided not to resend/delay, but this could
                 * lead to wrong size to be seen at Client side and even process
                 * trying to open to exit/fail if not itself handling -EAGAIN.
                 * So it should be better to return -EINPROGRESS instead and
                 * leave the decision to resend at Client side in after_reply()
                 */
-               if (rc == -EWOULDBLOCK) {
+               if (rc == -EAGAIN) {
                        rc = -EINPROGRESS;
                        CDEBUG(D_HA, "returning -EINPROGRESS instead of "
                               "-EWOULDBLOCK/-EAGAIN to allow Client to "
@@ -1622,9 +1712,6 @@ out:
  */
 int osp_init_precreate(struct osp_device *d)
 {
-       struct l_wait_info       lwi = { 0 };
-       struct task_struct              *task;
-
        ENTRY;
 
        OBD_ALLOC_PTR(d->opd_pre);
@@ -1632,11 +1719,13 @@ int osp_init_precreate(struct osp_device *d)
                RETURN(-ENOMEM);
 
        /* initially precreation isn't ready */
+       init_waitqueue_head(&d->opd_pre_user_waitq);
        d->opd_pre_status = -EAGAIN;
        fid_zero(&d->opd_pre_used_fid);
        d->opd_pre_used_fid.f_oid = 1;
        fid_zero(&d->opd_pre_last_created_fid);
        d->opd_pre_last_created_fid.f_oid = 1;
+       d->opd_last_id = 0;
        d->opd_pre_reserved = 0;
        d->opd_got_disconnected = 1;
        d->opd_pre_create_slow = 0;
@@ -1646,71 +1735,100 @@ int osp_init_precreate(struct osp_device *d)
        d->opd_reserved_mb_high = 0;
        d->opd_reserved_mb_low = 0;
 
+       RETURN(0);
+}
+
+/**
+ * Finish precreate functionality of OSP
+ *
+ *
+ * Asks all the activity (the thread, update timer) to stop, then
+ * wait till that is done.
+ *
+ * \param[in] d                OSP device
+ */
+void osp_precreate_fini(struct osp_device *d)
+{
+       ENTRY;
+
+       if (d->opd_pre == NULL)
+               RETURN_EXIT;
+
+       OBD_FREE_PTR(d->opd_pre);
+       d->opd_pre = NULL;
+
+       EXIT;
+}
+
+int osp_init_statfs(struct osp_device *d)
+{
+       struct task_struct      *task;
+       struct opt_args         *args;
+       DECLARE_COMPLETION_ONSTACK(started);
+       int                     rc;
+
+       ENTRY;
+
        spin_lock_init(&d->opd_pre_lock);
        init_waitqueue_head(&d->opd_pre_waitq);
-       init_waitqueue_head(&d->opd_pre_user_waitq);
-       thread_set_flags(&d->opd_pre_thread, SVC_INIT);
-       init_waitqueue_head(&d->opd_pre_thread.t_ctl_waitq);
 
        /*
         * Initialize statfs-related things
         */
-       d->opd_statfs_maxage = 5; /* default update interval */
-       d->opd_statfs_fresh_till = cfs_time_shift(-1000);
-       CDEBUG(D_OTHER, "current %llu, fresh till %llu\n",
-              (unsigned long long)cfs_time_current(),
-              (unsigned long long)d->opd_statfs_fresh_till);
-       setup_timer(&d->opd_statfs_timer, osp_statfs_timer_cb,
-                   (unsigned long)d);
+       d->opd_statfs_maxage = 5; /* defaultupdate interval */
+       d->opd_statfs_fresh_till = ktime_sub_ns(ktime_get(),
+                                               1000 * NSEC_PER_SEC);
+       CDEBUG(D_OTHER, "current %lldns, fresh till %lldns\n",
+              ktime_get_ns(),
+              ktime_to_ns(d->opd_statfs_fresh_till));
+       cfs_timer_setup(&d->opd_statfs_timer, osp_statfs_timer_cb,
+                       (unsigned long)d, 0);
 
        if (d->opd_storage->dd_rdonly)
                RETURN(0);
 
+       OBD_ALLOC_PTR(args);
+       if (!args)
+               RETURN(0);
+       args->opta_dev = d;
+       args->opta_started = &started;
+       rc = lu_env_init(&args->opta_env,
+                        d->opd_dt_dev.dd_lu_dev.ld_type->ldt_ctx_tags);
+       if (rc) {
+               CERROR("%s: init env error: rc = %d\n", d->opd_obd->obd_name,
+                      rc);
+               OBD_FREE_PTR(args);
+               RETURN(0);
+       }
+
        /*
         * start thread handling precreation and statfs updates
         */
-       task = kthread_run(osp_precreate_thread, d,
-                          "osp-pre-%u-%u", d->opd_index, d->opd_group);
+       task = kthread_create(osp_precreate_thread, args,
+                             "osp-pre-%u-%u", d->opd_index, d->opd_group);
        if (IS_ERR(task)) {
                CERROR("can't start precreate thread %ld\n", PTR_ERR(task));
+               lu_env_fini(&args->opta_env);
+               OBD_FREE_PTR(args);
                RETURN(PTR_ERR(task));
        }
-
-       l_wait_event(d->opd_pre_thread.t_ctl_waitq,
-                    osp_precreate_running(d) || osp_precreate_stopped(d),
-                    &lwi);
+       d->opd_pre_task = task;
+       wake_up_process(task);
+       wait_for_completion(&started);
 
        RETURN(0);
 }
 
-/**
- * Finish precreate functionality of OSP
- *
- *
- * Asks all the activity (the thread, update timer) to stop, then
- * wait till that is done.
- *
- * \param[in] d                OSP device
- */
-void osp_precreate_fini(struct osp_device *d)
+void osp_statfs_fini(struct osp_device *d)
 {
-       struct ptlrpc_thread *thread = &d->opd_pre_thread;
+       struct task_struct *task = d->opd_pre_task;
        ENTRY;
 
        del_timer(&d->opd_statfs_timer);
 
-       if (d->opd_pre == NULL)
-               RETURN_EXIT;
-
-       if (!thread_is_init(thread) && !thread_is_stopped(thread)) {
-               thread->t_flags = SVC_STOPPING;
-               wake_up(&d->opd_pre_waitq);
-               wait_event(thread->t_ctl_waitq, thread_is_stopped(thread));
-       }
-
-       OBD_FREE_PTR(d->opd_pre);
-       d->opd_pre = NULL;
+       d->opd_pre_task = NULL;
+       if (task)
+               kthread_stop(task);
 
        EXIT;
 }
-