Whamcloud - gitweb
LU-15146 mdt: mdt_lvb2reply crash fix
[fs/lustre-release.git] / lustre / mdt / mdt_io.c
index 3999e26..bf80be4 100644 (file)
 #define DEBUG_SUBSYSTEM S_FILTER
 
 #include <dt_object.h>
+#include <linux/falloc.h>
+#include <lustre_nodemap.h>
+
 #include "mdt_internal.h"
 
 /* functions below are stubs for now, they will be implemented with
  * grant support on MDT */
-static inline void mdt_io_counter_incr(struct obd_export *exp, int opcode,
-                                      char *jobid, long amount)
-{
-       return;
-}
-
 static inline void mdt_dom_read_lock(struct mdt_object *mo)
 {
        down_read(&mo->mot_dom_sem);
@@ -61,23 +58,6 @@ static inline void mdt_dom_write_unlock(struct mdt_object *mo)
        up_write(&mo->mot_dom_sem);
 }
 
-/**
- * Lock prolongation for Data-on-MDT.
- * This is similar to OFD code but for DOM ibits lock.
- */
-static inline time64_t prolong_timeout(struct ptlrpc_request *req)
-{
-       struct ptlrpc_service_part *svcpt = req->rq_rqbd->rqbd_svcpt;
-       time64_t req_timeout;
-
-       if (AT_OFF)
-               return obd_timeout / 2;
-
-       req_timeout = req->rq_deadline - req->rq_arrival_time.tv_sec;
-       return max_t(time64_t, at_est2timeout(at_get(&svcpt->scp_at_estimate)),
-                    req_timeout);
-}
-
 static void mdt_dom_resource_prolong(struct ldlm_prolong_args *arg)
 {
        struct ldlm_resource *res;
@@ -86,7 +66,7 @@ static void mdt_dom_resource_prolong(struct ldlm_prolong_args *arg)
        ENTRY;
 
        res = ldlm_resource_get(arg->lpa_export->exp_obd->obd_namespace, NULL,
-                               &arg->lpa_resid, LDLM_EXTENT, 0);
+                               &arg->lpa_resid, LDLM_IBITS, 0);
        if (IS_ERR(res)) {
                CDEBUG(D_DLMTRACE,
                       "Failed to get resource for resid %llu/%llu\n",
@@ -99,7 +79,11 @@ static void mdt_dom_resource_prolong(struct ldlm_prolong_args *arg)
                if (ldlm_has_dom(lock)) {
                        LDLM_DEBUG(lock, "DOM lock to prolong ");
                        ldlm_lock_prolong_one(lock, arg);
-                       break;
+                       /* only one PW or EX lock can be granted,
+                        * no need to continue search
+                        */
+                       if (lock->l_granted_mode & (LCK_PW | LCK_EX))
+                               break;
                }
        }
        unlock_res(res);
@@ -160,7 +144,7 @@ static int mdt_rw_hpreq_lock_match(struct ptlrpc_request *req,
                RETURN(0);
 
        /* a bulk write can only hold a reference on a PW extent lock. */
-       mode = LCK_PW;
+       mode = LCK_PW | LCK_GROUP;
        if (opc == OST_READ)
                /* whereas a bulk read can be protected by either a PR or PW
                 * extent lock */
@@ -200,18 +184,18 @@ static int mdt_rw_hpreq_check(struct ptlrpc_request *req)
        LASSERT(rnb != NULL);
        LASSERT(!(rnb->rnb_flags & OBD_BRW_SRVLOCK));
 
-       pa.lpa_mode = LCK_PW;
+       pa.lpa_mode = LCK_PW | LCK_GROUP;
        if (opc == OST_READ)
                pa.lpa_mode |= LCK_PR;
 
-       DEBUG_REQ(D_RPCTRACE, req, "%s %s: refresh rw locks: "DFID"\n",
+       DEBUG_REQ(D_RPCTRACE, req, "%s %s: refresh rw locks for " DFID,
                  tgt_name(tsi->tsi_tgt), current->comm, PFID(&tsi->tsi_fid));
 
        mdt_prolong_dom_lock(tsi, &pa);
 
        if (pa.lpa_blocks_cnt > 0) {
                CDEBUG(D_DLMTRACE,
-                      "%s: refreshed %u locks timeout for req %p.\n",
+                      "%s: refreshed %u locks timeout for req %p\n",
                       tgt_name(tsi->tsi_tgt), pa.lpa_blocks_cnt, req);
                RETURN(1);
        }
@@ -288,7 +272,7 @@ static int mdt_punch_hpreq_lock_match(struct ptlrpc_request *req,
        if (!fid_res_name_eq(&oa->o_oi.oi_fid, &lock->l_resource->lr_name))
                RETURN(0);
 
-       if (!(lock->l_granted_mode & LCK_PW))
+       if (!(lock->l_granted_mode & (LCK_PW | LCK_GROUP)))
                RETURN(0);
 
        RETURN(1);
@@ -326,14 +310,13 @@ static int mdt_punch_hpreq_check(struct ptlrpc_request *req)
        LASSERT(!(oa->o_valid & OBD_MD_FLFLAGS &&
                  oa->o_flags & OBD_FL_SRVLOCK));
 
-       pa.lpa_mode = LCK_PW;
+       pa.lpa_mode = LCK_PW | LCK_GROUP;
 
        CDEBUG(D_DLMTRACE, "%s: refresh DOM lock for "DFID"\n",
               tgt_name(tsi->tsi_tgt), PFID(&tsi->tsi_fid));
 
        mdt_prolong_dom_lock(tsi, &pa);
 
-
        if (pa.lpa_blocks_cnt > 0) {
                CDEBUG(D_DLMTRACE,
                       "%s: refreshed %u locks timeout for req %p.\n",
@@ -381,24 +364,54 @@ static int mdt_preprw_read(const struct lu_env *env, struct obd_export *exp,
                           struct niobuf_remote *rnb, int *nr_local,
                           struct niobuf_local *lnb, char *jobid)
 {
+       struct tgt_session_info *tsi = tgt_ses_info(env);
+       struct ptlrpc_request *req = tgt_ses_req(tsi);
        struct dt_object *dob;
        int i, j, rc, tot_bytes = 0;
+       int maxlnb = *nr_local;
+       int level;
 
        ENTRY;
 
        mdt_dom_read_lock(mo);
-       if (!mdt_object_exists(mo))
-               GOTO(unlock, rc = -ENOENT);
+       *nr_local = 0;
+       /* the only valid case when READ can find object is missing or stale
+        * when export is just evicted and open files are closed forcefully
+        * on server while client's READ can be in progress.
+        * This should not happen on healthy export, object can't be missing
+        * or dying because both states means it was finally destroyed.
+        */
+       level = exp->exp_failed ? D_INFO : D_ERROR;
+       if (!mdt_object_exists(mo)) {
+               CDEBUG_LIMIT(level,
+                            "%s: READ IO to missing obj "DFID": rc = %d\n",
+                            exp->exp_obd->obd_name, PFID(mdt_object_fid(mo)),
+                            -ENOENT);
+               /* return 0 and continue with empty commit to skip such READ
+                * without more BRW errors.
+                */
+               RETURN(0);
+       }
+       if (lu_object_is_dying(&mo->mot_header)) {
+               CDEBUG_LIMIT(level,
+                            "%s: READ IO to stale obj "DFID": rc = %d\n",
+                            exp->exp_obd->obd_name, PFID(mdt_object_fid(mo)),
+                            -ESTALE);
+               /* return 0 and continue with empty commit to skip such READ
+                * without more BRW errors.
+                */
+               RETURN(0);
+       }
 
        dob = mdt_obj2dt(mo);
        /* parse remote buffers to local buffers and prepare the latter */
-       *nr_local = 0;
        for (i = 0, j = 0; i < niocount; i++) {
-               rc = dt_bufs_get(env, dob, rnb + i, lnb + j, 0);
+               rc = dt_bufs_get(env, dob, rnb + i, lnb + j, maxlnb, 0);
                if (unlikely(rc < 0))
                        GOTO(buf_put, rc);
                /* correct index for local buffers to continue with */
                j += rc;
+               maxlnb -= rc;
                *nr_local += rc;
                tot_bytes += rnb[i].rnb_len;
        }
@@ -411,11 +424,10 @@ static int mdt_preprw_read(const struct lu_env *env, struct obd_export *exp,
        if (unlikely(rc))
                GOTO(buf_put, rc);
 
-       mdt_io_counter_incr(exp, LPROC_MDT_IO_READ, jobid, tot_bytes);
+       mdt_counter_incr(req, LPROC_MDT_IO_READ, tot_bytes);
        RETURN(0);
 buf_put:
        dt_bufs_put(env, dob, lnb, *nr_local);
-unlock:
        mdt_dom_read_unlock(mo);
        return rc;
 }
@@ -427,8 +439,11 @@ static int mdt_preprw_write(const struct lu_env *env, struct obd_export *exp,
                            struct niobuf_remote *rnb, int *nr_local,
                            struct niobuf_local *lnb, char *jobid)
 {
+       struct tgt_session_info *tsi = tgt_ses_info(env);
+       struct ptlrpc_request *req = tgt_ses_req(tsi);
        struct dt_object *dob;
        int i, j, k, rc = 0, tot_bytes = 0;
+       int maxlnb = *nr_local;
 
        ENTRY;
 
@@ -437,17 +452,36 @@ static int mdt_preprw_write(const struct lu_env *env, struct obd_export *exp,
        tgt_grant_prepare_write(env, exp, oa, rnb, obj->ioo_bufcnt);
 
        mdt_dom_read_lock(mo);
+       *nr_local = 0;
+       /* don't report error in cases with failed export */
        if (!mdt_object_exists(mo)) {
-               CDEBUG(D_ERROR, "%s: BRW to missing obj "DFID"\n",
-                      exp->exp_obd->obd_name, PFID(mdt_object_fid(mo)));
-               GOTO(unlock, rc = -ENOENT);
+               int level = exp->exp_failed ? D_INFO : D_ERROR;
+
+               rc = -ENOENT;
+               CDEBUG_LIMIT(level,
+                            "%s: WRITE IO to missing obj "DFID": rc = %d\n",
+                            exp->exp_obd->obd_name, PFID(mdt_object_fid(mo)),
+                            rc);
+               /* exit with no data written, note nr_local = 0 above */
+               GOTO(unlock, rc);
+       }
+       if (lu_object_is_dying(&mo->mot_header)) {
+               /* This is possible race between object destroy followed by
+                * discard BL AST and client cache flushing. Object is
+                * referenced until discard finish.
+                */
+               CDEBUG(D_INODE, "WRITE IO to stale object "DFID"\n",
+                      PFID(mdt_object_fid(mo)));
+               /* Note: continue with no error here to don't cause BRW errors
+                * but skip transaction in commitrw silently so no data is
+                * written.
+                */
        }
 
        dob = mdt_obj2dt(mo);
        /* parse remote buffers to local buffers and prepare the latter */
-       *nr_local = 0;
        for (i = 0, j = 0; i < obj->ioo_bufcnt; i++) {
-               rc = dt_bufs_get(env, dob, rnb + i, lnb + j, 1);
+               rc = dt_bufs_get(env, dob, rnb + i, lnb + j, maxlnb, 1);
                if (unlikely(rc < 0))
                        GOTO(err, rc);
                /* correct index for local buffers to continue with */
@@ -457,6 +491,7 @@ static int mdt_preprw_write(const struct lu_env *env, struct obd_export *exp,
                                lnb[j + k].lnb_rc = -ENOSPC;
                }
                j += rc;
+               maxlnb -= rc;
                *nr_local += rc;
                tot_bytes += rnb[i].rnb_len;
        }
@@ -465,7 +500,7 @@ static int mdt_preprw_write(const struct lu_env *env, struct obd_export *exp,
        if (likely(rc))
                GOTO(err, rc);
 
-       mdt_io_counter_incr(exp, LPROC_MDT_IO_WRITE, jobid, tot_bytes);
+       mdt_counter_incr(req, LPROC_MDT_IO_WRITE, tot_bytes);
        RETURN(0);
 err:
        dt_bufs_put(env, dob, lnb, *nr_local);
@@ -546,11 +581,10 @@ static int mdt_commitrw_read(const struct lu_env *env, struct mdt_device *mdt,
 
        ENTRY;
 
-       LASSERT(niocount > 0);
-
        dob = mdt_obj2dt(mo);
 
-       dt_bufs_put(env, dob, lnb, niocount);
+       if (niocount)
+               dt_bufs_put(env, dob, lnb, niocount);
 
        mdt_dom_read_unlock(mo);
        RETURN(rc);
@@ -558,16 +592,16 @@ static int mdt_commitrw_read(const struct lu_env *env, struct mdt_device *mdt,
 
 static int mdt_commitrw_write(const struct lu_env *env, struct obd_export *exp,
                              struct mdt_device *mdt, struct mdt_object *mo,
-                             struct lu_attr *la, int objcount, int niocount,
-                             struct niobuf_local *lnb, unsigned long granted,
-                             int old_rc)
+                             struct lu_attr *la, struct obdo *oa, int objcount,
+                             int niocount, struct niobuf_local *lnb,
+                             unsigned long granted, int old_rc)
 {
        struct dt_device *dt = mdt->mdt_bottom;
        struct dt_object *dob;
        struct thandle *th;
        int rc = 0;
        int retries = 0;
-       int i;
+       int i, restart = 0;
 
        ENTRY;
 
@@ -581,6 +615,16 @@ retry:
        if (!dt_object_exists(dob))
                GOTO(out, rc = -ENOENT);
 
+       if (niocount == 0) {
+               rc = -EPROTO;
+               DEBUG_REQ(D_WARNING, tgt_ses_req(tgt_ses_info(env)),
+                         "%s: commit with no pages for "DFID": rc = %d\n",
+                         exp->exp_obd->obd_name, PFID(mdt_object_fid(mo)), rc);
+               GOTO(out, rc);
+       }
+
+       CFS_FAIL_TIMEOUT(OBD_FAIL_MDS_COMMITRW_DELAY, cfs_fail_val);
+
        th = dt_trans_create(env, dt);
        if (IS_ERR(th))
                GOTO(out, rc = PTR_ERR(th));
@@ -612,9 +656,17 @@ retry:
                GOTO(out_stop, rc);
 
        dt_write_lock(env, dob, 0);
-       rc = dt_write_commit(env, dob, lnb, niocount, th);
-       if (rc)
+       if (lu_object_is_dying(&mo->mot_header)) {
+               /* Commit to stale object can be just skipped silently. */
+               CDEBUG(D_INODE, "skip commit to stale object "DFID"\n",
+                       PFID(mdt_object_fid(mo)));
+               GOTO(unlock, rc = 0);
+       }
+       rc = dt_write_commit(env, dob, lnb, niocount, th, oa->o_size);
+       if (rc) {
+               restart = th->th_restart_tran;
                GOTO(unlock, rc);
+       }
 
        if (la->la_valid) {
                rc = dt_attr_set(env, dob, la, th);
@@ -638,13 +690,23 @@ out_stop:
                        granted = 0;
        }
 
-       th->th_result = rc;
+       th->th_result = restart ? 0 : rc;
        dt_trans_stop(env, dt, th);
        if (rc == -ENOSPC && retries++ < 3) {
                CDEBUG(D_INODE, "retry after force commit, retries:%d\n",
                       retries);
                goto retry;
        }
+       if (restart) {
+               retries++;
+               restart = 0;
+               if (retries % 10000 == 0)
+                       CERROR("%s: restart IO write too many times: %d\n",
+                              exp->exp_obd->obd_name, retries);
+               CDEBUG(D_INODE, "retry transaction, retries:%d\n",
+                      retries);
+               goto retry;
+       }
 
 out:
        dt_bufs_put(env, dob, lnb, niocount);
@@ -655,7 +717,7 @@ out:
 }
 
 void mdt_dom_obj_lvb_update(const struct lu_env *env, struct mdt_object *mo,
-                           bool increase_only)
+                           struct obdo *oa, bool increase_only)
 {
        struct mdt_device *mdt = mdt_dev(mo->mot_obj.lo_dev);
        struct ldlm_res_id resid;
@@ -668,8 +730,23 @@ void mdt_dom_obj_lvb_update(const struct lu_env *env, struct mdt_object *mo,
                return;
 
        /* Update lvbo data if exists. */
-       if (mdt_dom_lvb_is_valid(res))
+       if (mdt_dom_lvb_is_valid(res)) {
                mdt_dom_disk_lvbo_update(env, mo, res, increase_only);
+               if (oa) {
+                       struct ost_lvb *res_lvb = res->lr_lvb_data;
+
+                       lock_res(res);
+                       oa->o_valid |= OBD_MD_FLBLOCKS | OBD_MD_FLSIZE |
+                                      OBD_MD_FLMTIME | OBD_MD_FLATIME |
+                                      OBD_MD_FLCTIME;
+                       oa->o_blocks = res_lvb->lvb_blocks;
+                       oa->o_size = res_lvb->lvb_size;
+                       oa->o_atime = res_lvb->lvb_atime;
+                       oa->o_mtime = res_lvb->lvb_mtime;
+                       oa->o_ctime = res_lvb->lvb_ctime;
+                       unlock_res(res);
+               }
+       }
        ldlm_resource_putref(res);
 }
 
@@ -684,16 +761,38 @@ int mdt_obd_commitrw(const struct lu_env *env, int cmd, struct obd_export *exp,
        struct lu_attr *la = &info->mti_attr.ma_attr;
        __u64 valid;
        int rc = 0;
-
-       if (npages == 0) {
-               CERROR("%s: no pages to commit\n",
-                      exp->exp_obd->obd_name);
-               rc = -EPROTO;
-       }
+       int root_squash = 0;
 
        LASSERT(mo);
 
        if (cmd == OBD_BRW_WRITE) {
+               struct lu_nodemap *nodemap;
+               __u32 mapped_uid, mapped_gid, mapped_projid;
+
+               nodemap = nodemap_get_from_exp(exp);
+               if (IS_ERR(nodemap))
+                       RETURN(PTR_ERR(nodemap));
+               mapped_uid = nodemap_map_id(nodemap, NODEMAP_UID,
+                                           NODEMAP_FS_TO_CLIENT,
+                                           oa->o_uid);
+               mapped_gid = nodemap_map_id(nodemap, NODEMAP_GID,
+                                           NODEMAP_FS_TO_CLIENT,
+                                           oa->o_gid);
+               mapped_projid = nodemap_map_id(nodemap, NODEMAP_PROJID,
+                                              NODEMAP_FS_TO_CLIENT,
+                                              oa->o_projid);
+               if (!IS_ERR_OR_NULL(nodemap)) {
+                       /* do not bypass quota enforcement if squashed uid */
+                       if (unlikely(mapped_uid == nodemap->nm_squash_uid)) {
+                               int idx;
+
+                               for (idx = 0; idx < npages; idx++)
+                                       lnb[idx].lnb_flags &=
+                                               ~OBD_BRW_SYS_RESOURCE;
+                               root_squash = 1;
+                       }
+                       nodemap_putref(nodemap);
+               }
                /* Don't update timestamps if this write is older than a
                 * setattr which modifies the timestamps. b=10150 */
 
@@ -709,14 +808,14 @@ int mdt_obd_commitrw(const struct lu_env *env, int cmd, struct obd_export *exp,
 
                la_from_obdo(la, oa, valid);
 
-               rc = mdt_commitrw_write(env, exp, mdt, mo, la, objcount,
+               rc = mdt_commitrw_write(env, exp, mdt, mo, la, oa, objcount,
                                        npages, lnb, oa->o_grant_used, old_rc);
                if (rc == 0)
                        obdo_from_la(oa, la, VALID_FLAGS | LA_GID | LA_UID);
                else
                        obdo_from_la(oa, la, LA_GID | LA_UID);
 
-               mdt_dom_obj_lvb_update(env, mo, false);
+               mdt_dom_obj_lvb_update(env, mo, NULL, false);
                /* don't report overquota flag if we failed before reaching
                 * commit */
                if (old_rc == 0 && (rc == 0 || rc == -EDQUOT)) {
@@ -742,26 +841,204 @@ int mdt_obd_commitrw(const struct lu_env *env, int cmd, struct obd_export *exp,
                                        oa->o_flags = OBD_FL_NO_PRJQUOTA;
                        }
 
+                       if (root_squash)
+                               oa->o_flags |= OBD_FL_ROOT_SQUASH;
+
                        oa->o_valid |= OBD_MD_FLFLAGS | OBD_MD_FLUSRQUOTA |
                                       OBD_MD_FLGRPQUOTA | OBD_MD_FLPRJQUOTA;
                }
+               /* Convert back to client IDs. LU-9671.
+                * nodemap_get_from_exp() may fail due to nodemap deactivated,
+                * server ID will be returned back to client in that case.
+                */
+               oa->o_uid = mapped_uid;
+               oa->o_gid = mapped_gid;
+               oa->o_projid = mapped_projid;
        } else if (cmd == OBD_BRW_READ) {
                /* If oa != NULL then mdt_preprw_read updated the inode
                 * atime and we should update the lvb so that other glimpses
                 * will also get the updated value. bug 5972 */
                if (oa)
-                       mdt_dom_obj_lvb_update(env, mo, true);
+                       mdt_dom_obj_lvb_update(env, mo, NULL, true);
                rc = mdt_commitrw_read(env, mdt, mo, objcount, npages, lnb);
                if (old_rc)
                        rc = old_rc;
        } else {
                rc = -EPROTO;
        }
-       /* this put is pair to object_get in ofd_preprw_write */
        mdt_thread_info_fini(info);
        RETURN(rc);
 }
 
+int mdt_object_fallocate(const struct lu_env *env, struct dt_device *dt,
+                        struct dt_object *dob, __u64 start, __u64 end,
+                        int mode, struct lu_attr *la)
+{
+       struct thandle *th;
+       int rc;
+
+       ENTRY;
+
+       if (!dt_object_exists(dob))
+               RETURN(-ENOENT);
+
+       th = dt_trans_create(env, dt);
+       if (IS_ERR(th))
+               RETURN(PTR_ERR(th));
+
+       rc = dt_declare_attr_set(env, dob, la, th);
+       if (rc)
+               GOTO(stop, rc);
+
+       rc = dt_declare_fallocate(env, dob, start, end, mode, th);
+       if (rc)
+               GOTO(stop, rc);
+
+       tgt_vbr_obj_set(env, dob);
+       rc = dt_trans_start(env, dt, th);
+       if (rc)
+               GOTO(stop, rc);
+
+       dt_write_lock(env, dob, 0);
+       rc = dt_falloc(env, dob, start, end, mode, th);
+       if (rc)
+               GOTO(unlock, rc);
+       rc = dt_attr_set(env, dob, la, th);
+       if (rc)
+               GOTO(unlock, rc);
+unlock:
+       dt_write_unlock(env, dob);
+stop:
+       th->th_result = rc;
+       dt_trans_stop(env, dt, th);
+       RETURN(rc);
+}
+
+/**
+ * MDT request handler for OST_FALLOCATE RPC.
+ *
+ * This is part of request processing. Validate request fields,
+ * preallocate the given MDT object and pack reply.
+ *
+ * \param[in] tsi      target session environment for this request
+ *
+ * \retval             0 if successful
+ * \retval             negative value on error
+ */
+int mdt_fallocate_hdl(struct tgt_session_info *tsi)
+{
+       struct obdo *oa = &tsi->tsi_ost_body->oa;
+       struct ptlrpc_request *req = tgt_ses_req(tsi);
+       struct ost_body *repbody;
+       struct mdt_thread_info *info;
+       struct ldlm_namespace *ns = tsi->tsi_tgt->lut_obd->obd_namespace;
+       struct obd_export *exp = tsi->tsi_exp;
+       struct mdt_device *mdt = mdt_dev(exp->exp_obd->obd_lu_dev);
+       struct mdt_object *mo;
+       struct dt_object *dob;
+       struct lu_attr *la;
+       __u64 flags = 0;
+       struct lustre_handle lh = { 0, };
+       int rc, mode;
+       __u64 start, end;
+       bool srvlock;
+       ktime_t kstart = ktime_get();
+
+       repbody = req_capsule_server_get(tsi->tsi_pill, &RMF_OST_BODY);
+       if (repbody == NULL)
+               RETURN(err_serious(-ENOMEM));
+
+       /*
+        * fallocate start and end are passed in o_size, o_blocks
+        * on the wire.
+        */
+       if ((oa->o_valid & (OBD_MD_FLSIZE | OBD_MD_FLBLOCKS)) !=
+           (OBD_MD_FLSIZE | OBD_MD_FLBLOCKS))
+               RETURN(err_serious(-EPROTO));
+
+       start = oa->o_size;
+       end = oa->o_blocks;
+       mode = oa->o_falloc_mode;
+
+       CDEBUG(D_INODE,
+              "fallocate: "DFID", mode = %#x, start = %lld, end = %lld\n",
+              PFID(&tsi->tsi_fid), mode, start, end);
+
+       /*
+        * mode == 0 (which is standard prealloc) and PUNCH is supported
+        * Rest of mode options are not supported yet.
+        */
+       if (mode & ~(FALLOC_FL_KEEP_SIZE | FALLOC_FL_PUNCH_HOLE))
+               RETURN(-EOPNOTSUPP);
+
+       if (mode & FALLOC_FL_PUNCH_HOLE && !(mode & FALLOC_FL_KEEP_SIZE)) {
+               CWARN("%s: PUNCH mode misses KEEP_SIZE flag, setting it\n",
+                     tsi->tsi_tgt->lut_obd->obd_name);
+               mode |= FALLOC_FL_KEEP_SIZE;
+       }
+
+       info = tsi2mdt_info(tsi);
+       la = &info->mti_attr.ma_attr;
+
+       repbody->oa.o_oi = oa->o_oi;
+       repbody->oa.o_valid = OBD_MD_FLID;
+
+       srvlock = oa->o_valid & OBD_MD_FLFLAGS &&
+                 oa->o_flags & OBD_FL_SRVLOCK;
+
+       if (srvlock) {
+               rc = tgt_mdt_data_lock(ns, &tsi->tsi_resid, &lh, LCK_PW,
+                                      &flags);
+               if (rc != 0)
+                       GOTO(out, rc);
+       }
+
+       mo = mdt_object_find(tsi->tsi_env, mdt, &tsi->tsi_fid);
+       if (IS_ERR(mo))
+               GOTO(out_unlock, rc = PTR_ERR(mo));
+
+       if (!mdt_object_exists(mo))
+               GOTO(out_put, rc = -ENOENT);
+
+       /* Shouldn't happen on dirs */
+       if (S_ISDIR(lu_object_attr(&mo->mot_obj))) {
+               rc = -EPERM;
+               CERROR("%s: fallocate on dir "DFID": rc = %d\n",
+                      exp->exp_obd->obd_name, PFID(&tsi->tsi_fid), rc);
+               GOTO(out_put, rc);
+       }
+
+       la_from_obdo(la, oa, OBD_MD_FLMTIME | OBD_MD_FLATIME | OBD_MD_FLCTIME);
+
+       mdt_dom_write_lock(mo);
+       dob = mdt_obj2dt(mo);
+
+       if (la->la_valid & (LA_ATIME | LA_MTIME | LA_CTIME))
+               tgt_fmd_update(tsi->tsi_exp, &tsi->tsi_fid,
+                              tgt_ses_req(tsi)->rq_xid);
+
+       rc = mdt_object_fallocate(tsi->tsi_env, mdt->mdt_bottom, dob, start,
+                                 end, mode, la);
+       mdt_dom_write_unlock(mo);
+       if (rc)
+               GOTO(out_put, rc);
+
+       mdt_dom_obj_lvb_update(tsi->tsi_env, mo, &repbody->oa, false);
+
+       mdt_counter_incr(req, LPROC_MDT_FALLOCATE,
+                        ktime_us_delta(ktime_get(), kstart));
+
+       EXIT;
+out_put:
+       lu_object_put(tsi->tsi_env, &mo->mot_obj);
+out_unlock:
+       if (srvlock)
+               tgt_data_unlock(&lh, LCK_PW);
+out:
+       mdt_thread_info_fini(info);
+       return rc;
+}
+
 int mdt_object_punch(const struct lu_env *env, struct dt_device *dt,
                     struct dt_object *dob, __u64 start, __u64 end,
                     struct lu_attr *la)
@@ -812,6 +1089,7 @@ stop:
 int mdt_punch_hdl(struct tgt_session_info *tsi)
 {
        const struct obdo *oa = &tsi->tsi_ost_body->oa;
+       struct ptlrpc_request *req = tgt_ses_req(tsi);
        struct ost_body *repbody;
        struct mdt_thread_info *info;
        struct lu_attr *la;
@@ -822,15 +1100,13 @@ int mdt_punch_hdl(struct tgt_session_info *tsi)
        struct dt_object *dob;
        __u64 flags = 0;
        struct lustre_handle lh = { 0, };
+       ktime_t kstart = ktime_get();
        __u64 start, end;
        int rc;
        bool srvlock;
 
        ENTRY;
 
-       /* check that we do support OBD_CONNECT_TRUNCLOCK. */
-       CLASSERT(OST_CONNECT_SUPPORTED & OBD_CONNECT_TRUNCLOCK);
-
        if ((oa->o_valid & (OBD_MD_FLSIZE | OBD_MD_FLBLOCKS)) !=
            (OBD_MD_FLSIZE | OBD_MD_FLBLOCKS))
                RETURN(err_serious(-EPROTO));
@@ -904,15 +1180,16 @@ int mdt_punch_hdl(struct tgt_session_info *tsi)
        if (rc)
                GOTO(out_put, rc);
 
-       mdt_dom_obj_lvb_update(tsi->tsi_env, mo, false);
-       mdt_io_counter_incr(tsi->tsi_exp, LPROC_MDT_IO_PUNCH,
-                           tsi->tsi_jobid, 1);
+       mdt_dom_obj_lvb_update(tsi->tsi_env, mo, &repbody->oa, false);
+
+       mdt_counter_incr(req, LPROC_MDT_IO_PUNCH,
+                        ktime_us_delta(ktime_get(), kstart));
        EXIT;
 out_put:
        lu_object_put(tsi->tsi_env, &mo->mot_obj);
 out_unlock:
        if (srvlock)
-               tgt_extent_unlock(&lh, LCK_PW);
+               tgt_data_unlock(&lh, LCK_PW);
 out:
        mdt_thread_info_fini(info);
        return rc;
@@ -933,7 +1210,7 @@ int mdt_do_glimpse(const struct lu_env *env, struct ldlm_namespace *ns,
        enum ldlm_mode mode;
        struct ldlm_lock *lock;
        struct ldlm_glimpse_work *gl_work;
-       struct list_head gl_list;
+       LIST_HEAD(gl_list);
        int rc;
 
        ENTRY;
@@ -942,7 +1219,7 @@ int mdt_do_glimpse(const struct lu_env *env, struct ldlm_namespace *ns,
        policy.l_inodebits.bits = MDS_INODELOCK_DOM;
        mode = ldlm_lock_match(ns, LDLM_FL_TEST_LOCK,
                               &res->lr_name, LDLM_IBITS, &policy,
-                              LCK_PW, &lockh, 0);
+                              LCK_PW, &lockh);
 
        /* There is no PW lock on this object; finished. */
        if (mode == 0)
@@ -974,7 +1251,6 @@ int mdt_do_glimpse(const struct lu_env *env, struct ldlm_namespace *ns,
        gl_work->gl_lock = LDLM_LOCK_GET(lock);
        /* The glimpse callback is sent to one single IO lock. As a result,
         * the gl_work list is just composed of one element */
-       INIT_LIST_HEAD(&gl_list);
        list_add_tail(&gl_work->gl_list, &gl_list);
        /* There is actually no need for a glimpse descriptor when glimpsing
         * IO locks */
@@ -997,22 +1273,28 @@ out:
        return rc;
 }
 
-static void mdt_lvb2body(struct ldlm_resource *res, struct mdt_body *mb)
+static void mdt_lvb2reply(struct ldlm_resource *res, struct mdt_body *mb,
+                         struct ost_lvb *lvb)
 {
        struct ost_lvb *res_lvb;
 
        lock_res(res);
        res_lvb = res->lr_lvb_data;
-       mb->mbo_dom_size = res_lvb->lvb_size;
-       mb->mbo_dom_blocks = res_lvb->lvb_blocks;
-       mb->mbo_mtime = res_lvb->lvb_mtime;
-       mb->mbo_ctime = res_lvb->lvb_ctime;
-       mb->mbo_atime = res_lvb->lvb_atime;
-
-       CDEBUG(D_DLMTRACE, "size %llu\n", res_lvb->lvb_size);
-
-       mb->mbo_valid |= OBD_MD_FLATIME | OBD_MD_FLCTIME | OBD_MD_FLMTIME |
-                        OBD_MD_DOM_SIZE;
+       if (res_lvb) {
+               if (lvb)
+                       *lvb = *res_lvb;
+
+               if (mb) {
+                       mb->mbo_dom_size = res_lvb->lvb_size;
+                       mb->mbo_dom_blocks = res_lvb->lvb_blocks;
+                       mb->mbo_mtime = res_lvb->lvb_mtime;
+                       mb->mbo_ctime = res_lvb->lvb_ctime;
+                       mb->mbo_atime = res_lvb->lvb_atime;
+                       mb->mbo_valid |= OBD_MD_FLATIME | OBD_MD_FLCTIME |
+                                        OBD_MD_FLMTIME | OBD_MD_DOM_SIZE;
+               }
+               CDEBUG(D_DLMTRACE, "size %llu\n", res_lvb->lvb_size);
+       }
        unlock_res(res);
 }
 
@@ -1041,9 +1323,9 @@ int mdt_dom_object_size(const struct lu_env *env, struct mdt_device *mdt,
 
        /* Update lvbo data if DoM lock returned or if LVB is not yet valid. */
        if (dom_lock || !mdt_dom_lvb_is_valid(res))
-               mdt_dom_lvbo_update(env, res, NULL, NULL, false);
+               mdt_dom_lvbo_update(res, NULL, NULL, false);
 
-       mdt_lvb2body(res, mb);
+       mdt_lvb2reply(res, mb, NULL);
        ldlm_resource_putref(res);
        RETURN(rc);
 }
@@ -1074,6 +1356,8 @@ int mdt_glimpse_enqueue(struct mdt_thread_info *mti, struct ldlm_namespace *ns,
        ldlm_processing_policy policy;
        struct ldlm_reply *rep;
        struct mdt_body *mbo;
+       struct ost_lvb *lvb;
+       bool old_client = !exp_connect_dom_lvb(mti->mti_exp);
        int rc;
 
        ENTRY;
@@ -1081,19 +1365,28 @@ int mdt_glimpse_enqueue(struct mdt_thread_info *mti, struct ldlm_namespace *ns,
        policy = ldlm_get_processing_policy(res);
        LASSERT(policy != NULL);
 
-       req_capsule_set_size(mti->mti_pill, &RMF_MDT_MD, RCL_SERVER, 0);
-       req_capsule_set_size(mti->mti_pill, &RMF_ACL, RCL_SERVER, 0);
+       if (unlikely(old_client)) {
+               req_capsule_set_size(mti->mti_pill, &RMF_MDT_MD, RCL_SERVER, 0);
+               req_capsule_set_size(mti->mti_pill, &RMF_ACL, RCL_SERVER, 0);
+       } else {
+               req_capsule_set_size(mti->mti_pill, &RMF_DLM_LVB, RCL_SERVER,
+                                    sizeof(*lvb));
+       }
        rc = req_capsule_server_pack(mti->mti_pill);
        if (rc)
                RETURN(err_serious(rc));
 
        rep = req_capsule_server_get(mti->mti_pill, &RMF_DLM_REP);
-       if (rep == NULL)
-               RETURN(-EPROTO);
 
-       mbo = req_capsule_server_get(mti->mti_pill, &RMF_MDT_BODY);
-       if (mbo == NULL)
-               RETURN(-EPROTO);
+       if (unlikely(old_client)) {
+               mbo = req_capsule_server_get(mti->mti_pill, &RMF_MDT_BODY);
+               LASSERT(mbo);
+               lvb = NULL;
+       } else {
+               lvb = req_capsule_server_get(mti->mti_pill, &RMF_DLM_LVB);
+               LASSERT(lvb);
+               mbo = NULL;
+       }
 
        lock_res(res);
        /* Check if this is a resend case (MSG_RESENT is set on RPC) and a
@@ -1124,14 +1417,12 @@ int mdt_glimpse_enqueue(struct mdt_thread_info *mti, struct ldlm_namespace *ns,
        if (rc == -ENOENT) {
                /* We are racing with unlink(); just return -ENOENT */
                rep->lock_policy_res2 = ptlrpc_status_hton(-ENOENT);
-               rc = 0;
        } else if (rc == -EINVAL) {
                /* this is possible is client lock has been cancelled but
                 * still exists on server. If that lock was found on server
                 * as only conflicting lock then the client has already
                 * size authority and glimpse is not needed. */
                CDEBUG(D_DLMTRACE, "Glimpse from the client owning lock\n");
-               rc = 0;
        } else if (rc < 0) {
                RETURN(rc);
        }
@@ -1139,8 +1430,9 @@ int mdt_glimpse_enqueue(struct mdt_thread_info *mti, struct ldlm_namespace *ns,
 fill_mbo:
        /* LVB can be without valid data in case of DOM */
        if (!mdt_dom_lvb_is_valid(res))
-               mdt_dom_lvbo_update(mti->mti_env, res, lock, NULL, false);
-       mdt_lvb2body(res, mbo);
+               mdt_dom_lvbo_update(res, lock, NULL, false);
+       mdt_lvb2reply(res, mbo, lvb);
+
        RETURN(rc);
 }
 
@@ -1159,8 +1451,6 @@ int mdt_brw_enqueue(struct mdt_thread_info *mti, struct ldlm_namespace *ns,
 
        ENTRY;
 
-       /* Get lock from request for possible resent case. */
-       mdt_intent_fixup_resent(mti, *lockp, lhc, flags);
        req_capsule_set_size(mti->mti_pill, &RMF_MDT_MD, RCL_SERVER, 0);
        req_capsule_set_size(mti->mti_pill, &RMF_ACL, RCL_SERVER, 0);
        rc = req_capsule_server_pack(mti->mti_pill);
@@ -1186,10 +1476,13 @@ int mdt_brw_enqueue(struct mdt_thread_info *mti, struct ldlm_namespace *ns,
        if (mdt_object_remote(mo))
                GOTO(out, rc = -EPROTO);
 
+       /* Get lock from request for possible resent case. */
+       mdt_intent_fixup_resent(mti, *lockp, lhc, flags);
        /* resent case */
        if (!lustre_handle_is_used(&lhc->mlh_reg_lh)) {
                mdt_lock_handle_init(lhc);
-               mdt_lock_reg_init(lhc, (*lockp)->l_req_mode);
+               mdt_lh_reg_init(lhc, *lockp);
+
                /* This will block MDT thread but it should be fine until
                 * client caches small amount of data for DoM, which should be
                 * smaller than one BRW RPC and should be able to be
@@ -1210,7 +1503,7 @@ int mdt_brw_enqueue(struct mdt_thread_info *mti, struct ldlm_namespace *ns,
                        GOTO(out_fail, rc);
                mdt_dom_disk_lvbo_update(mti->mti_env, mo, res, false);
        }
-       mdt_lvb2body(res, mbo);
+       mdt_lvb2reply(res, mbo, NULL);
 out_fail:
        rep->lock_policy_res2 = clear_serious(rc);
        if (rep->lock_policy_res2) {
@@ -1220,36 +1513,12 @@ out_fail:
 
        rc = mdt_intent_lock_replace(mti, lockp, lhc, flags, rc);
 out:
+       if (rc < 0)
+               lhc->mlh_reg_lh.cookie = 0ull;
        mdt_object_put(mti->mti_env, mo);
        RETURN(rc);
 }
 
-void mdt_dom_discard_data(struct mdt_thread_info *info,
-                         const struct lu_fid *fid)
-{
-       struct mdt_device *mdt = info->mti_mdt;
-       union ldlm_policy_data *policy = &info->mti_policy;
-       struct ldlm_res_id *res_id = &info->mti_res_id;
-       struct lustre_handle dom_lh;
-       __u64 flags = LDLM_FL_AST_DISCARD_DATA;
-       int rc = 0;
-
-       policy->l_inodebits.bits = MDS_INODELOCK_DOM;
-       policy->l_inodebits.try_bits = 0;
-       fid_build_reg_res_name(fid, res_id);
-
-       /* Tell the clients that the object is gone now and that they should
-        * throw away any cached pages. */
-       rc = ldlm_cli_enqueue_local(info->mti_env, mdt->mdt_namespace, res_id,
-                                   LDLM_IBITS, policy, LCK_PW, &flags,
-                                   ldlm_blocking_ast, ldlm_completion_ast,
-                                   NULL, NULL, 0, LVB_T_NONE, NULL, &dom_lh);
-
-       /* We only care about the side-effects, just drop the lock. */
-       if (rc == ELDLM_OK)
-               ldlm_lock_decref_and_cancel(&dom_lh, LCK_PW);
-}
-
 /* check if client has already DoM lock for given resource */
 bool mdt_dom_client_has_lock(struct mdt_thread_info *info,
                             const struct lu_fid *fid)
@@ -1257,17 +1526,21 @@ bool mdt_dom_client_has_lock(struct mdt_thread_info *info,
        struct mdt_device *mdt = info->mti_mdt;
        union ldlm_policy_data *policy = &info->mti_policy;
        struct ldlm_res_id *res_id = &info->mti_res_id;
+       __u64 open_flags = info->mti_spec.sp_cr_flags;
        struct lustre_handle lockh;
        enum ldlm_mode mode;
        struct ldlm_lock *lock;
+       enum ldlm_mode lm;
        bool rc;
 
        policy->l_inodebits.bits = MDS_INODELOCK_DOM;
        fid_build_reg_res_name(fid, res_id);
 
+
+       lm = (open_flags & MDS_FMODE_WRITE) ? LCK_PW : LCK_PR | LCK_PW;
        mode = ldlm_lock_match(mdt->mdt_namespace, LDLM_FL_BLOCK_GRANTED |
                               LDLM_FL_TEST_LOCK, res_id, LDLM_IBITS, policy,
-                              LCK_PW, &lockh, 0);
+                              lm, &lockh);
 
        /* There is no other PW lock on this object; finished. */
        if (mode == 0)
@@ -1373,7 +1646,7 @@ int mdt_data_version_get(struct tgt_session_info *tsi)
        rc = 0;
 out:
        if (srvlock)
-               tgt_mdt_data_unlock(&lh, lock_mode);
+               tgt_data_unlock(&lh, lock_mode);
 
        repbody->mbo_valid |= OBD_MD_FLFLAGS;
        repbody->mbo_flags = OBD_FL_FLUSH;
@@ -1396,9 +1669,9 @@ int mdt_dom_read_on_open(struct mdt_thread_info *mti, struct mdt_device *mdt,
        struct niobuf_remote *rnb = NULL;
        struct niobuf_local *lnb;
        int rc;
-       int max_reply_len;
        loff_t offset;
        unsigned int len, copied = 0;
+       __u64 real_dom_size;
        int lnbs, nr_local, i;
        bool dom_lock = false;
 
@@ -1412,6 +1685,11 @@ int mdt_dom_read_on_open(struct mdt_thread_info *mti, struct mdt_device *mdt,
        }
 
        mbo = req_capsule_server_get(pill, &RMF_MDT_BODY);
+       if (!(mbo->mbo_valid & OBD_MD_DOM_SIZE))
+               RETURN(0);
+
+       if (!mbo->mbo_dom_size)
+               RETURN(0);
 
        if (lustre_handle_is_used(lh)) {
                struct ldlm_lock *lock;
@@ -1427,26 +1705,23 @@ int mdt_dom_read_on_open(struct mdt_thread_info *mti, struct mdt_device *mdt,
        if (!dom_lock || !mdt->mdt_opts.mo_dom_read_open)
                RETURN(0);
 
-       if (!(mbo->mbo_valid & OBD_MD_DOM_SIZE))
-               RETURN(0);
-
-       if (mbo->mbo_dom_size == 0)
-               RETURN(0);
-
-       /* check the maximum size available in reply */
-       max_reply_len =
-               req->rq_rqbd->rqbd_svcpt->scp_service->srv_max_reply_size;
-
-       CDEBUG(D_INFO, "File size %llu, reply sizes %d/%d/%d\n",
-              mbo->mbo_dom_size, max_reply_len, req->rq_reqmsg->lm_repsize,
-              req->rq_replen);
+       /* if DoM object holds encrypted content, we need to make sure we
+        * send whole encryption units, or client will read corrupted content
+        */
+       if (mbo->mbo_valid & LA_FLAGS && mbo->mbo_flags & LUSTRE_ENCRYPT_FL &&
+           mbo->mbo_dom_size & ~LUSTRE_ENCRYPTION_MASK)
+               real_dom_size = (mbo->mbo_dom_size & LUSTRE_ENCRYPTION_MASK) +
+                               LUSTRE_ENCRYPTION_UNIT_SIZE;
+       else
+               real_dom_size = mbo->mbo_dom_size;
+
+       CDEBUG(D_INFO, "File size %llu, reply sizes %d/%d\n",
+              real_dom_size, req->rq_reqmsg->lm_repsize, req->rq_replen);
        len = req->rq_reqmsg->lm_repsize - req->rq_replen;
-       max_reply_len -= req->rq_replen;
 
        /* NB: at this moment we have the following sizes:
         * - req->rq_replen: used data in reply
         * - req->rq_reqmsg->lm_repsize: total allocated reply buffer at client
-        * - max_reply_len: maximum reply size allowed by protocol
         *
         * Ideal case when file size fits in allocated reply buffer,
         * that mean we can return whole data in reply. We can also fit more
@@ -1458,21 +1733,14 @@ int mdt_dom_read_on_open(struct mdt_thread_info *mti, struct mdt_device *mdt,
         *
         * At the moment the following strategy is used:
         * 1) try to fit into the buffer we have
-        * 2) respond with bigger buffer so client will re-allocate it and
-        *    resend (up to srv_max_reply_size value).
-        * 3) return just file tail otherwise.
+        * 2) return just file tail otherwise.
         */
-       if (mbo->mbo_dom_size <= len) {
+       if (real_dom_size <= len) {
                /* can fit whole data */
-               len = mbo->mbo_dom_size;
+               len = real_dom_size;
                offset = 0;
-       } else if (mbo->mbo_dom_size <= max_reply_len) {
-               /* It is worth to make this tunable ON/OFF because this will
-                * cause buffer re-allocation and resend
-                */
-               len = mbo->mbo_dom_size;
-               offset = 0;
-       } else {
+       } else if (real_dom_size <
+                  mdt_lmm_dom_stripesize(mti->mti_attr.ma_lmm)) {
                int tail, pgbits;
 
                /* File tail offset must be aligned with larger page size
@@ -1490,16 +1758,22 @@ int mdt_dom_read_on_open(struct mdt_thread_info *mti, struct mdt_device *mdt,
                }
                pgbits = max_t(int, PAGE_SHIFT,
                               req->rq_export->exp_target_data.ted_pagebits);
-               tail = mbo->mbo_dom_size % (1 << pgbits);
+               tail = real_dom_size % (1 << pgbits);
 
                /* no partial tail or tail can't fit in reply */
                if (tail == 0 || len < tail)
                        RETURN(0);
 
                len = tail;
-               offset = mbo->mbo_dom_size - len;
+               offset = real_dom_size - len;
+       } else {
+               /* DOM stripe is fully written, so don't expect its tail
+                * will be used by append.
+                */
+               RETURN(0);
        }
-       LASSERT((offset % PAGE_SIZE) == 0);
+
+       LASSERT((offset & ~PAGE_MASK) == 0);
        rc = req_capsule_server_grow(pill, &RMF_NIOBUF_INLINE,
                                     sizeof(*rnb) + len);
        if (rc != 0) {
@@ -1532,11 +1806,11 @@ int mdt_dom_read_on_open(struct mdt_thread_info *mti, struct mdt_device *mdt,
 
        /* parse remote buffers to local buffers and prepare the latter */
        lnbs = (len >> PAGE_SHIFT) + 1;
-       OBD_ALLOC(lnb, sizeof(*lnb) * lnbs);
+       OBD_ALLOC_PTR_ARRAY(lnb, lnbs);
        if (lnb == NULL)
                GOTO(unlock, rc = -ENOMEM);
 
-       rc = dt_bufs_get(env, mo, rnb, lnb, 0);
+       rc = dt_bufs_get(env, mo, rnb, lnb, lnbs, 0);
        if (unlikely(rc < 0))
                GOTO(free, rc);
        LASSERT(rc <= lnbs);
@@ -1573,7 +1847,7 @@ int mdt_dom_read_on_open(struct mdt_thread_info *mti, struct mdt_device *mdt,
 buf_put:
        dt_bufs_put(env, mo, lnb, nr_local);
 free:
-       OBD_FREE(lnb, sizeof(*lnb) * lnbs);
+       OBD_FREE_PTR_ARRAY(lnb, lnbs);
 unlock:
        dt_read_unlock(env, mo);
        lu_object_put(env, &mo->do_lu);
@@ -1589,3 +1863,105 @@ out:
        RETURN(0);
 }
 
+/**
+ * Completion AST for DOM discard locks:
+ *
+ * CP AST an DOM discard lock is called always right after enqueue or from
+ * reprocess if lock was blocked, in the latest case l_ast_data is set to
+ * the mdt_object which is kept while there are pending locks on it.
+ */
+int ldlm_dom_discard_cp_ast(struct ldlm_lock *lock, __u64 flags, void *data)
+{
+       struct mdt_object *mo;
+       struct lustre_handle dom_lh;
+       struct lu_env *env;
+
+       ENTRY;
+
+       /* l_ast_data is set when lock was not granted immediately
+        * in mdt_dom_discard_data() below but put into waiting list,
+        * so this CP callback means we are finished and corresponding
+        * MDT object should be released finally as well as lock itself.
+        */
+       lock_res_and_lock(lock);
+       if (!lock->l_ast_data) {
+               unlock_res_and_lock(lock);
+               RETURN(0);
+       }
+
+       mo = lock->l_ast_data;
+       lock->l_ast_data = NULL;
+       unlock_res_and_lock(lock);
+
+       ldlm_lock2handle(lock, &dom_lh);
+       ldlm_lock_decref(&dom_lh, LCK_PW);
+
+       env = lu_env_find();
+       LASSERT(env);
+       mdt_object_put(env, mo);
+
+       RETURN(0);
+}
+
+void mdt_dom_discard_data(struct mdt_thread_info *info,
+                         struct mdt_object *mo)
+{
+       struct ptlrpc_request *req = mdt_info_req(info);
+       struct mdt_device *mdt = mdt_dev(mo->mot_obj.lo_dev);
+       union ldlm_policy_data policy;
+       struct ldlm_res_id res_id;
+       struct lustre_handle dom_lh;
+       struct ldlm_lock *lock;
+       __u64 flags = LDLM_FL_AST_DISCARD_DATA;
+       int rc = 0;
+       bool old_client;
+
+       ENTRY;
+
+       if (req && req_is_replay(req))
+               RETURN_EXIT;
+
+       policy.l_inodebits.bits = MDS_INODELOCK_DOM;
+       policy.l_inodebits.try_bits = 0;
+       fid_build_reg_res_name(mdt_object_fid(mo), &res_id);
+
+       /* Keep blocking version of discard for an old client to avoid
+        * crashes on non-patched clients. LU-11359.
+        */
+       old_client = req && !(exp_connect_flags2(req->rq_export) &
+                             OBD_CONNECT2_ASYNC_DISCARD);
+
+       /* Tell the clients that the object is gone now and that they should
+        * throw away any cached pages. */
+       rc = ldlm_cli_enqueue_local(info->mti_env, mdt->mdt_namespace, &res_id,
+                                   LDLM_IBITS, &policy, LCK_PW, &flags,
+                                   ldlm_blocking_ast, old_client ?
+                                   ldlm_completion_ast :
+                                   ldlm_dom_discard_cp_ast,
+                                   NULL, NULL, 0, LVB_T_NONE, NULL, &dom_lh);
+       if (rc != ELDLM_OK) {
+               CDEBUG(D_DLMTRACE,
+                      "Failed to issue discard lock, rc = %d\n", rc);
+               RETURN_EXIT;
+       }
+
+       lock = ldlm_handle2lock(&dom_lh);
+       lock_res_and_lock(lock);
+       /* if lock is not granted then there are BL ASTs in progress and
+        * lock will be granted in result of reprocessing with CP callback
+        * notifying about that. The mdt object has to be kept until that and
+        * it is saved in l_ast_data of the lock. Lock reference is kept too
+        * until that to prevent it from canceling.
+        */
+       if (!is_granted_or_cancelled_nolock(lock)) {
+               mdt_object_get(info->mti_env, mo);
+               lock->l_ast_data = mo;
+               unlock_res_and_lock(lock);
+       } else {
+               unlock_res_and_lock(lock);
+               ldlm_lock_decref_and_cancel(&dom_lh, LCK_PW);
+       }
+       LDLM_LOCK_PUT(lock);
+
+       RETURN_EXIT;
+}