Whamcloud - gitweb
LU-8753 osp: add rpc generation
[fs/lustre-release.git] / lustre / osp / osp_trans.c
index d0a8c39..ebecd8a 100644 (file)
@@ -20,7 +20,7 @@
  * GPL HEADER END
  */
 /*
- * Copyright (c) 2014, Intel Corporation.
+ * Copyright (c) 2014, 2016, Intel Corporation.
  */
 /*
  * lustre/osp/osp_trans.c
 
 #define DEBUG_SUBSYSTEM S_MDS
 
+#include <lustre_net.h>
 #include "osp_internal.h"
 
 /**
  * The argument for the interpreter callback of osp request.
  */
 struct osp_update_args {
-       struct dt_update_request *oaua_update;
+       struct osp_update_request *oaua_update;
        atomic_t                 *oaua_count;
        wait_queue_head_t        *oaua_waitq;
        bool                      oaua_flow_control;
+       const struct lu_env      *oaua_update_env;
 };
 
 /**
  * Call back for each update request.
  */
 struct osp_update_callback {
-       /* list in the dt_update_request::dur_cb_items */
+       /* list in the osp_update_request::our_cb_items */
        struct list_head                 ouc_list;
 
        /* The target of the async update request. */
@@ -109,63 +111,143 @@ static struct object_update_request *object_update_request_alloc(size_t size)
        return ourq;
 }
 
-static void object_update_request_free(struct object_update_request *ourq,
-                                      size_t ourq_size)
+/**
+ * Allocate new update request
+ *
+ * Allocate new update request and insert it to the req_update_list.
+ *
+ * \param [in] our     osp_udate_request where to create a new
+ *                      update request
+ *
+ * \retval     0 if creation succeeds.
+ * \retval     negative errno if creation fails.
+ */
+int osp_object_update_request_create(struct osp_update_request *our,
+                                    size_t size)
 {
-       if (ourq != NULL)
-               OBD_FREE_LARGE(ourq, ourq_size);
+       struct osp_update_request_sub *ours;
+
+       OBD_ALLOC_PTR(ours);
+       if (ours == NULL)
+               return -ENOMEM;
+
+       if (size < OUT_UPDATE_INIT_BUFFER_SIZE)
+               size = OUT_UPDATE_INIT_BUFFER_SIZE;
+
+       ours->ours_req = object_update_request_alloc(size);
+
+       if (IS_ERR(ours->ours_req)) {
+               OBD_FREE_PTR(ours);
+               return -ENOMEM;
+       }
+
+       ours->ours_req_size = size;
+       INIT_LIST_HEAD(&ours->ours_list);
+       list_add_tail(&ours->ours_list, &our->our_req_list);
+       our->our_req_nr++;
+
+       return 0;
 }
 
 /**
- * Allocate and initialize dt_update_request
+ * Get current update request
  *
- * dt_update_request is being used to track updates being executed on
+ * Get current object update request from our_req_list in
+ * osp_update_request, because we always insert the new update
+ * request in the last position, so the last update request
+ * in the list will be the current update req.
+ *
+ * \param[in] our      osp update request where to get the
+ *                      current object update.
+ *
+ * \retval             the current object update.
+ **/
+struct osp_update_request_sub *
+osp_current_object_update_request(struct osp_update_request *our)
+{
+       if (list_empty(&our->our_req_list))
+               return NULL;
+
+       return list_entry(our->our_req_list.prev, struct osp_update_request_sub,
+                         ours_list);
+}
+
+/**
+ * Allocate and initialize osp_update_request
+ *
+ * osp_update_request is being used to track updates being executed on
  * this dt_device(OSD or OSP). The update buffer will be 4k initially,
  * and increased if needed.
  *
  * \param [in] dt      dt device
  *
- * \retval             dt_update_request being allocated if succeed
+ * \retval             osp_update_request being allocated if succeed
  * \retval             ERR_PTR(errno) if failed
  */
-struct dt_update_request *dt_update_request_create(struct dt_device *dt)
+struct osp_update_request *osp_update_request_create(struct dt_device *dt)
 {
-       struct dt_update_request *dt_update;
-       struct object_update_request *ourq;
+       struct osp_update_request *our;
+       int rc;
 
-       OBD_ALLOC_PTR(dt_update);
-       if (dt_update == NULL)
+       OBD_ALLOC_PTR(our);
+       if (our == NULL)
                return ERR_PTR(-ENOMEM);
 
-       ourq = object_update_request_alloc(OUT_UPDATE_INIT_BUFFER_SIZE);
-       if (IS_ERR(ourq)) {
-               OBD_FREE_PTR(dt_update);
-               return ERR_CAST(ourq);
-       }
-
-       dt_update->dur_buf.ub_req = ourq;
-       dt_update->dur_buf.ub_req_size = OUT_UPDATE_INIT_BUFFER_SIZE;
-
-       dt_update->dur_dt = dt;
-       dt_update->dur_batchid = 0;
-       INIT_LIST_HEAD(&dt_update->dur_cb_items);
+       INIT_LIST_HEAD(&our->our_req_list);
+       INIT_LIST_HEAD(&our->our_cb_items);
+       INIT_LIST_HEAD(&our->our_list);
+       INIT_LIST_HEAD(&our->our_invalidate_cb_list);
+       spin_lock_init(&our->our_list_lock);
 
-       return dt_update;
+       rc = osp_object_update_request_create(our, OUT_UPDATE_INIT_BUFFER_SIZE);
+       if (rc != 0) {
+               OBD_FREE_PTR(our);
+               return ERR_PTR(rc);
+       }
+       return our;
 }
 
-/**
- * Destroy dt_update_request
- *
- * \param [in] dt_update       dt_update_request being destroyed
- */
-void dt_update_request_destroy(struct dt_update_request *dt_update)
+void osp_update_request_destroy(const struct lu_env *env,
+                               struct osp_update_request *our)
 {
-       if (dt_update == NULL)
+       struct osp_update_request_sub *ours;
+       struct osp_update_request_sub *tmp;
+
+       if (our == NULL)
                return;
 
-       object_update_request_free(dt_update->dur_buf.ub_req,
-                                  dt_update->dur_buf.ub_req_size);
-       OBD_FREE_PTR(dt_update);
+       list_for_each_entry_safe(ours, tmp, &our->our_req_list, ours_list) {
+               list_del(&ours->ours_list);
+               if (ours->ours_req != NULL)
+                       OBD_FREE_LARGE(ours->ours_req, ours->ours_req_size);
+               OBD_FREE_PTR(ours);
+       }
+
+       if (!list_empty(&our->our_invalidate_cb_list)) {
+               struct lu_env lenv;
+               struct osp_object *obj;
+               struct osp_object *next;
+
+               if (env == NULL) {
+                       lu_env_init(&lenv, LCT_MD_THREAD | LCT_DT_THREAD);
+                       env = &lenv;
+               }
+
+               list_for_each_entry_safe(obj, next,
+                                        &our->our_invalidate_cb_list,
+                                        opo_invalidate_cb_list) {
+                       spin_lock(&obj->opo_lock);
+                       list_del_init(&obj->opo_invalidate_cb_list);
+                       spin_unlock(&obj->opo_lock);
+
+                       lu_object_put(env, &obj->opo_obj.do_lu);
+               }
+
+               if (env == &lenv)
+                       lu_env_fini(&lenv);
+       }
+
+       OBD_FREE_PTR(our);
 }
 
 static void
@@ -181,12 +263,13 @@ object_update_request_dump(const struct object_update_request *ourq,
 
                update = object_update_request_get(ourq, i, &size);
                LASSERT(update != NULL);
-               CDEBUG(mask, "i = %u fid = "DFID" op = %s master = %u"
-                      "params = %d batchid = "LPU64" size = %zu\n",
+               CDEBUG(mask, "i = %u fid = "DFID" op = %s "
+                      "params = %d batchid = %llu size = %zu repsize %u\n",
                       i, PFID(&update->ou_fid),
                       update_op_str(update->ou_type),
-                      update->ou_master_index, update->ou_params_count,
-                      update->ou_batchid, size);
+                      update->ou_params_count,
+                      update->ou_batchid, size,
+                      (unsigned)update->ou_result_size);
 
                total_size += size;
        }
@@ -196,6 +279,278 @@ object_update_request_dump(const struct object_update_request *ourq,
 }
 
 /**
+ * Prepare inline update request
+ *
+ * Prepare OUT update ptlrpc inline request, and the request usually includes
+ * one update buffer, which does not need bulk transfer.
+ *
+ * \param[in] env      execution environment
+ * \param[in] req      ptlrpc request
+ * \param[in] ours     sub osp_update_request to be packed
+ *
+ * \retval             0 if packing succeeds
+ * \retval             negative errno if packing fails
+ */
+int osp_prep_inline_update_req(const struct lu_env *env,
+                              struct ptlrpc_request *req,
+                              struct osp_update_request *our,
+                              int repsize)
+{
+       struct osp_update_request_sub *ours;
+       struct out_update_header *ouh;
+       __u32 update_req_size;
+       int rc;
+
+       ours = list_entry(our->our_req_list.next,
+                         struct osp_update_request_sub, ours_list);
+       update_req_size = object_update_request_size(ours->ours_req);
+       req_capsule_set_size(&req->rq_pill, &RMF_OUT_UPDATE_HEADER, RCL_CLIENT,
+                            update_req_size + sizeof(*ouh));
+
+       rc = ptlrpc_request_pack(req, LUSTRE_MDS_VERSION, OUT_UPDATE);
+       if (rc != 0)
+               RETURN(rc);
+
+       ouh = req_capsule_client_get(&req->rq_pill, &RMF_OUT_UPDATE_HEADER);
+       ouh->ouh_magic = OUT_UPDATE_HEADER_MAGIC;
+       ouh->ouh_count = 1;
+       ouh->ouh_inline_length = update_req_size;
+       ouh->ouh_reply_size = repsize;
+
+       memcpy(ouh->ouh_inline_data, ours->ours_req, update_req_size);
+
+       req_capsule_set_size(&req->rq_pill, &RMF_OUT_UPDATE_REPLY,
+                            RCL_SERVER, repsize);
+
+       ptlrpc_request_set_replen(req);
+       req->rq_request_portal = OUT_PORTAL;
+       req->rq_reply_portal = OSC_REPLY_PORTAL;
+
+       RETURN(rc);
+}
+
+/**
+ * Prepare update request.
+ *
+ * Prepare OUT update ptlrpc request, and the request usually includes
+ * all of updates (stored in \param ureq) from one operation.
+ *
+ * \param[in] env      execution environment
+ * \param[in] imp      import on which ptlrpc request will be sent
+ * \param[in] ureq     hold all of updates which will be packed into the req
+ * \param[in] reqp     request to be created
+ *
+ * \retval             0 if preparation succeeds.
+ * \retval             negative errno if preparation fails.
+ */
+int osp_prep_update_req(const struct lu_env *env, struct obd_import *imp,
+                       struct osp_update_request *our,
+                       struct ptlrpc_request **reqp)
+{
+       struct ptlrpc_request           *req;
+       struct ptlrpc_bulk_desc         *desc;
+       struct osp_update_request_sub   *ours;
+       const struct object_update_request *ourq;
+       struct out_update_header        *ouh;
+       struct out_update_buffer        *oub;
+       __u32                           buf_count = 0;
+       int                             repsize = 0;
+       struct object_update_reply      *reply;
+       int                             rc, i;
+       int                             total = 0;
+       ENTRY;
+
+       list_for_each_entry(ours, &our->our_req_list, ours_list) {
+               object_update_request_dump(ours->ours_req, D_INFO);
+
+               ourq = ours->ours_req;
+               for (i = 0; i < ourq->ourq_count; i++) {
+                       struct object_update    *update;
+                       size_t                  size = 0;
+
+
+                       /* XXX: it's very inefficient to lookup update
+                        *      this way, iterating from the beginning
+                        *      each time */
+                       update = object_update_request_get(ourq, i, &size);
+                       LASSERT(update != NULL);
+
+                       repsize += sizeof(reply->ourp_lens[0]);
+                       repsize += sizeof(struct object_update_result);
+                       repsize += update->ou_result_size;
+               }
+
+               buf_count++;
+       }
+       repsize += sizeof(*reply);
+       repsize = (repsize + OUT_UPDATE_REPLY_SIZE - 1) &
+                       ~(OUT_UPDATE_REPLY_SIZE - 1);
+       LASSERT(buf_count > 0);
+
+       req = ptlrpc_request_alloc(imp, &RQF_OUT_UPDATE);
+       if (req == NULL)
+               RETURN(-ENOMEM);
+
+       if (buf_count == 1) {
+               ours = list_entry(our->our_req_list.next,
+                                 struct osp_update_request_sub, ours_list);
+
+               /* Let's check if it can be packed inline */
+               if (object_update_request_size(ours->ours_req) +
+                   sizeof(struct out_update_header) <
+                               OUT_UPDATE_MAX_INLINE_SIZE) {
+                       rc = osp_prep_inline_update_req(env, req, our, repsize);
+                       if (rc == 0)
+                               *reqp = req;
+                       GOTO(out_req, rc);
+               }
+       }
+
+       req_capsule_set_size(&req->rq_pill, &RMF_OUT_UPDATE_HEADER, RCL_CLIENT,
+                            sizeof(struct osp_update_request));
+
+       req_capsule_set_size(&req->rq_pill, &RMF_OUT_UPDATE_BUF, RCL_CLIENT,
+                            buf_count * sizeof(*oub));
+
+       rc = ptlrpc_request_pack(req, LUSTRE_MDS_VERSION, OUT_UPDATE);
+       if (rc != 0)
+               GOTO(out_req, rc);
+
+       ouh = req_capsule_client_get(&req->rq_pill, &RMF_OUT_UPDATE_HEADER);
+       ouh->ouh_magic = OUT_UPDATE_HEADER_MAGIC;
+       ouh->ouh_count = buf_count;
+       ouh->ouh_inline_length = 0;
+       ouh->ouh_reply_size = repsize;
+       oub = req_capsule_client_get(&req->rq_pill, &RMF_OUT_UPDATE_BUF);
+       list_for_each_entry(ours, &our->our_req_list, ours_list) {
+               oub->oub_size = ours->ours_req_size;
+               oub++;
+       }
+
+       req->rq_bulk_write = 1;
+       desc = ptlrpc_prep_bulk_imp(req, buf_count,
+               MD_MAX_BRW_SIZE >> LNET_MTU_BITS,
+               PTLRPC_BULK_GET_SOURCE | PTLRPC_BULK_BUF_KVEC,
+               MDS_BULK_PORTAL, &ptlrpc_bulk_kvec_ops);
+       if (desc == NULL)
+               GOTO(out_req, rc = -ENOMEM);
+
+       /* NB req now owns desc and will free it when it gets freed */
+       list_for_each_entry(ours, &our->our_req_list, ours_list) {
+               desc->bd_frag_ops->add_iov_frag(desc, ours->ours_req,
+                                               ours->ours_req_size);
+               total += ours->ours_req_size;
+       }
+       CDEBUG(D_OTHER, "total %d in %u\n", total, our->our_update_nr);
+
+       req_capsule_set_size(&req->rq_pill, &RMF_OUT_UPDATE_REPLY,
+                            RCL_SERVER, repsize);
+
+       ptlrpc_request_set_replen(req);
+       req->rq_request_portal = OUT_PORTAL;
+       req->rq_reply_portal = OSC_REPLY_PORTAL;
+       *reqp = req;
+
+out_req:
+       if (rc < 0)
+               ptlrpc_req_finished(req);
+
+       RETURN(rc);
+}
+
+/**
+ * Send update RPC.
+ *
+ * Send update request to the remote MDT synchronously.
+ *
+ * \param[in] env      execution environment
+ * \param[in] imp      import on which ptlrpc request will be sent
+ * \param[in] our      hold all of updates which will be packed into the req
+ * \param[in] reqp     request to be created
+ *
+ * \retval             0 if RPC succeeds.
+ * \retval             negative errno if RPC fails.
+ */
+int osp_remote_sync(const struct lu_env *env, struct osp_device *osp,
+                   struct osp_update_request *our,
+                   struct ptlrpc_request **reqp)
+{
+       struct obd_import       *imp = osp->opd_obd->u.cli.cl_import;
+       struct ptlrpc_request   *req = NULL;
+       int                     rc;
+       ENTRY;
+
+       rc = osp_prep_update_req(env, imp, our, &req);
+       if (rc != 0)
+               RETURN(rc);
+
+       osp_set_req_replay(osp, req);
+       req->rq_allow_intr = 1;
+
+       /* Note: some dt index api might return non-zero result here, like
+        * osd_index_ea_lookup, so we should only check rc < 0 here */
+       rc = ptlrpc_queue_wait(req);
+       our->our_rc = rc;
+       if (rc < 0 || reqp == NULL)
+               ptlrpc_req_finished(req);
+       else
+               *reqp = req;
+
+       RETURN(rc);
+}
+
+/**
+ * Invalidate all objects in the osp thandle
+ *
+ * invalidate all of objects in the update request, which will be called
+ * when the transaction is aborted.
+ *
+ * \param[in] oth      osp thandle.
+ */
+static void osp_thandle_invalidate_object(const struct lu_env *env,
+                                         struct osp_thandle *oth,
+                                         int result)
+{
+       struct osp_update_request *our = oth->ot_our;
+       struct osp_object *obj;
+       struct osp_object *next;
+
+       if (our == NULL)
+               return;
+
+       list_for_each_entry_safe(obj, next, &our->our_invalidate_cb_list,
+                                opo_invalidate_cb_list) {
+               if (result < 0)
+                       osp_invalidate(env, &obj->opo_obj);
+
+               spin_lock(&obj->opo_lock);
+               list_del_init(&obj->opo_invalidate_cb_list);
+               spin_unlock(&obj->opo_lock);
+
+               lu_object_put(env, &obj->opo_obj.do_lu);
+       }
+}
+
+static void osp_trans_stop_cb(const struct lu_env *env,
+                             struct osp_thandle *oth, int result)
+{
+       struct dt_txn_commit_cb *dcb;
+       struct dt_txn_commit_cb *tmp;
+
+       /* call per-transaction stop callbacks if any */
+       list_for_each_entry_safe(dcb, tmp, &oth->ot_stop_dcb_list,
+                                dcb_linkage) {
+               LASSERTF(dcb->dcb_magic == TRANS_COMMIT_CB_MAGIC,
+                        "commit callback entry: magic=%x name='%s'\n",
+                        dcb->dcb_magic, dcb->dcb_name);
+               list_del_init(&dcb->dcb_linkage);
+               dcb->dcb_func(NULL, &oth->ot_super, dcb, result);
+       }
+
+       osp_thandle_invalidate_object(env, oth, result);
+}
+
+/**
  * Allocate an osp request and initialize it with the given parameters.
  *
  * \param[in] obj              pointer to the operation target
@@ -258,49 +613,74 @@ static int osp_update_interpret(const struct lu_env *env,
 {
        struct object_update_reply      *reply  = NULL;
        struct osp_update_args          *oaua   = arg;
-       struct dt_update_request        *dt_update = oaua->oaua_update;
+       struct osp_update_request       *our = oaua->oaua_update;
+       struct osp_thandle              *oth;
        struct osp_update_callback      *ouc;
        struct osp_update_callback      *next;
        int                              count  = 0;
        int                              index  = 0;
        int                              rc1    = 0;
 
-       if (oaua->oaua_flow_control)
-               obd_put_request_slot(
-                               &dt2osp_dev(dt_update->dur_dt)->opd_obd->u.cli);
+       ENTRY;
+
+       if (our == NULL)
+               RETURN(0);
+
+       /* Sigh env might be NULL in some cases, see
+        * this calling path.
+        * osp_send_update_thread()
+        *  ptlrpc_set_wait() ----> null env.
+        *   ptlrpc_check_set()
+        *    osp_update_interpret()
+        * Let's use env in oaua for this case.
+        */
+       if (env == NULL)
+               env = oaua->oaua_update_env;
+
+       oaua->oaua_update = NULL;
+       oth = our->our_th;
+       if (oaua->oaua_flow_control) {
+               struct osp_device *osp;
+
+               LASSERT(oth != NULL);
+               osp = dt2osp_dev(oth->ot_super.th_dev);
+               obd_put_request_slot(&osp->opd_obd->u.cli);
+       }
 
        /* Unpack the results from the reply message. */
-       if (req->rq_repmsg != NULL) {
+       if (req->rq_repmsg != NULL && req->rq_replied) {
                reply = req_capsule_server_sized_get(&req->rq_pill,
                                                     &RMF_OUT_UPDATE_REPLY,
                                                     OUT_UPDATE_REPLY_SIZE);
-               if (reply == NULL || reply->ourp_magic != UPDATE_REPLY_MAGIC)
-                       rc1 = -EPROTO;
-               else
+               if (reply == NULL || reply->ourp_magic != UPDATE_REPLY_MAGIC) {
+                       if (rc == 0)
+                               rc = -EPROTO;
+               } else {
                        count = reply->ourp_count;
-       } else {
-               rc1 = rc;
+               }
        }
 
-       list_for_each_entry_safe(ouc, next, &dt_update->dur_cb_items,
-                                ouc_list) {
+       list_for_each_entry_safe(ouc, next, &our->our_cb_items, ouc_list) {
                list_del_init(&ouc->ouc_list);
 
                /* The peer may only have handled some requests (indicated
                 * by the 'count') in the packaged OUT RPC, we can only get
                 * results for the handled part. */
-               if (index < count && reply->ourp_lens[index] > 0) {
+               if (index < count && reply->ourp_lens[index] > 0 && rc >= 0) {
                        struct object_update_result *result;
 
                        result = object_update_result_get(reply, index, NULL);
                        if (result == NULL)
-                               rc1 = -EPROTO;
+                               rc1 = rc = -EPROTO;
                        else
-                               rc1 = result->our_rc;
-               } else {
-                       rc1 = rc;
-                       if (unlikely(rc1 == 0))
+                               rc1 = rc = result->our_rc;
+               } else if (rc1 >= 0) {
+                       /* The peer did not handle these request, let's return
+                        * -EINVAL to update interpret for now */
+                       if (rc >= 0)
                                rc1 = -EINVAL;
+                       else
+                               rc1 = rc;
                }
 
                if (ouc->ouc_interpreter != NULL)
@@ -314,9 +694,16 @@ static int osp_update_interpret(const struct lu_env *env,
        if (oaua->oaua_count != NULL && atomic_dec_and_test(oaua->oaua_count))
                wake_up_all(oaua->oaua_waitq);
 
-       dt_update_request_destroy(dt_update);
+       if (oth != NULL) {
+               /* oth and osp_update_requests will be destoryed in
+                * osp_thandle_put */
+               osp_trans_stop_cb(env, oth, rc);
+               osp_thandle_put(env, oth);
+       } else {
+               osp_update_request_destroy(env, our);
+       }
 
-       return 0;
+       RETURN(rc);
 }
 
 /**
@@ -325,27 +712,27 @@ static int osp_update_interpret(const struct lu_env *env,
  *
  * \param[in] env      pointer to the thread context
  * \param[in] osp      pointer to the OSP device
- * \param[in] update   pointer to the shared queue
+ * \param[in] our      pointer to the shared queue
  *
  * \retval             0 for success
  * \retval             negative error number on failure
  */
 int osp_unplug_async_request(const struct lu_env *env,
                             struct osp_device *osp,
-                            struct dt_update_request *update)
+                            struct osp_update_request *our)
 {
        struct osp_update_args  *args;
        struct ptlrpc_request   *req = NULL;
        int                      rc;
 
        rc = osp_prep_update_req(env, osp->opd_obd->u.cli.cl_import,
-                                update->dur_buf.ub_req, &req);
+                                our, &req);
        if (rc != 0) {
                struct osp_update_callback *ouc;
                struct osp_update_callback *next;
 
                list_for_each_entry_safe(ouc, next,
-                                        &update->dur_cb_items, ouc_list) {
+                                        &our->our_cb_items, ouc_list) {
                        list_del_init(&ouc->ouc_list);
                        if (ouc->ouc_interpreter != NULL)
                                ouc->ouc_interpreter(env, NULL, NULL,
@@ -353,15 +740,19 @@ int osp_unplug_async_request(const struct lu_env *env,
                                                     ouc->ouc_data, 0, rc);
                        osp_update_callback_fini(env, ouc);
                }
-               dt_update_request_destroy(update);
+               osp_update_request_destroy(env, our);
        } else {
                args = ptlrpc_req_async_args(req);
-               args->oaua_update = update;
+               args->oaua_update = our;
                args->oaua_count = NULL;
                args->oaua_waitq = NULL;
+               /* Note: this is asynchronous call for the request, so the
+                * interrupte cb and current function will be different
+                * thread, so we need use different env */
+               args->oaua_update_env = NULL;
                args->oaua_flow_control = false;
                req->rq_interpret_reply = osp_update_interpret;
-               ptlrpcd_add_req(req, PDL_POLICY_LOCAL, -1);
+               ptlrpcd_add_req(req);
        }
 
        return rc;
@@ -372,33 +763,35 @@ int osp_unplug_async_request(const struct lu_env *env,
  * request queue - osp_device::opd_async_requests.
  *
  * If the osp_device::opd_async_requests is not NULL, then return it directly;
- * otherwise create new dt_update_request and attach it to opd_async_requests.
+ * otherwise create new osp_update_request and attach it to opd_async_requests.
  *
  * \param[in] osp      pointer to the OSP device
  *
  * \retval             pointer to the shared queue
  * \retval             negative error number on failure
  */
-static struct dt_update_request *
+static struct osp_update_request *
 osp_find_or_create_async_update_request(struct osp_device *osp)
 {
-       struct dt_update_request *update = osp->opd_async_requests;
+       struct osp_update_request *our = osp->opd_async_requests;
+
+       if (our != NULL)
+               return our;
 
-       if (update != NULL)
-               return update;
+       our = osp_update_request_create(&osp->opd_dt_dev);
+       if (IS_ERR(our))
+               return our;
 
-       update = dt_update_request_create(&osp->opd_dt_dev);
-       if (!IS_ERR(update))
-               osp->opd_async_requests = update;
+       osp->opd_async_requests = our;
 
-       return update;
+       return our;
 }
 
 /**
- * Insert an osp_update_callback into the dt_update_request.
+ * Insert an osp_update_callback into the osp_update_request.
  *
- * Insert an osp_update_callback to the dt_update_request. Usually each update
- * in the dt_update_request will have one correspondent callback, and these
+ * Insert an osp_update_callback to the osp_update_request. Usually each update
+ * in the osp_update_request will have one correspondent callback, and these
  * callbacks will be called in rq_interpret_reply.
  *
  * \param[in] env              pointer to the thread context
@@ -410,7 +803,7 @@ osp_find_or_create_async_update_request(struct osp_device *osp)
  * \retval                     negative error number on failure
  */
 int osp_insert_update_callback(const struct lu_env *env,
-                              struct dt_update_request *update,
+                              struct osp_update_request *our,
                               struct osp_object *obj, void *data,
                               osp_update_interpreter_t interpreter)
 {
@@ -420,7 +813,7 @@ int osp_insert_update_callback(const struct lu_env *env,
        if (ouc == NULL)
                RETURN(-ENOMEM);
 
-       list_add_tail(&ouc->ouc_list, &update->dur_cb_items);
+       list_add_tail(&ouc->ouc_list, &our->our_cb_items);
 
        return 0;
 }
@@ -444,6 +837,7 @@ int osp_insert_update_callback(const struct lu_env *env,
  * \param[in] lens             buffer length array for the subsequent \a bufs
  * \param[in] bufs             the buffers to compose the request
  * \param[in] data             pointer to the data used by the interpreter
+ * \param[in] repsize          how many bytes the caller allocated for \a data
  * \param[in] interpreter      pointer to the interpreter function
  *
  * \retval                     0 for success
@@ -451,42 +845,48 @@ int osp_insert_update_callback(const struct lu_env *env,
  */
 int osp_insert_async_request(const struct lu_env *env, enum update_type op,
                             struct osp_object *obj, int count,
-                            __u16 *lens, const void **bufs, void *data,
+                            __u16 *lens, const void **bufs,
+                            void *data, __u32 repsize,
                             osp_update_interpreter_t interpreter)
 {
-       struct osp_device            *osp = lu2osp_dev(osp2lu_obj(obj)->lo_dev);
-       struct dt_update_request        *update;
+       struct osp_device               *osp;
+       struct osp_update_request       *our;
        struct object_update            *object_update;
        size_t                          max_update_size;
        struct object_update_request    *ureq;
+       struct osp_update_request_sub   *ours;
        int                             rc = 0;
        ENTRY;
 
-       update = osp_find_or_create_async_update_request(osp);
-       if (IS_ERR(update))
-               RETURN(PTR_ERR(update));
+       osp = lu2osp_dev(osp2lu_obj(obj)->lo_dev);
+       our = osp_find_or_create_async_update_request(osp);
+       if (IS_ERR(our))
+               RETURN(PTR_ERR(our));
 
 again:
-       ureq = update->dur_buf.ub_req;
-       max_update_size = update->dur_buf.ub_req_size -
-                           object_update_request_size(ureq);
+       ours = osp_current_object_update_request(our);
+
+       ureq = ours->ours_req;
+       max_update_size = ours->ours_req_size -
+                         object_update_request_size(ureq);
 
        object_update = update_buffer_get_update(ureq, ureq->ourq_count);
-       rc = out_update_pack(env, object_update, max_update_size, op,
-                            lu_object_fid(osp2lu_obj(obj)), count, lens, bufs);
+       rc = out_update_pack(env, object_update, &max_update_size, op,
+                            lu_object_fid(osp2lu_obj(obj)), count, lens, bufs,
+                            repsize);
        /* The queue is full. */
        if (rc == -E2BIG) {
                osp->opd_async_requests = NULL;
                mutex_unlock(&osp->opd_async_requests_mutex);
 
-               rc = osp_unplug_async_request(env, osp, update);
+               rc = osp_unplug_async_request(env, osp, our);
                mutex_lock(&osp->opd_async_requests_mutex);
                if (rc != 0)
                        RETURN(rc);
 
-               update = osp_find_or_create_async_update_request(osp);
-               if (IS_ERR(update))
-                       RETURN(PTR_ERR(update));
+               our = osp_find_or_create_async_update_request(osp);
+               if (IS_ERR(our))
+                       RETURN(PTR_ERR(our));
 
                goto again;
        } else {
@@ -494,9 +894,10 @@ again:
                        RETURN(rc);
 
                ureq->ourq_count++;
+               our->our_update_nr++;
        }
 
-       rc = osp_insert_update_callback(env, update, obj, data, interpreter);
+       rc = osp_insert_update_callback(env, our, obj, data, interpreter);
 
        RETURN(rc);
 }
@@ -504,24 +905,34 @@ again:
 int osp_trans_update_request_create(struct thandle *th)
 {
        struct osp_thandle              *oth = thandle_to_osp_thandle(th);
-       struct dt_update_request        *update;
+       struct osp_update_request       *our;
 
-       if (oth->ot_dur != NULL)
+       if (oth->ot_our != NULL)
                return 0;
 
-       update = dt_update_request_create(th->th_dev);
-       if (IS_ERR(update)) {
-               th->th_result = PTR_ERR(update);
-               return PTR_ERR(update);
+       our = osp_update_request_create(th->th_dev);
+       if (IS_ERR(our)) {
+               th->th_result = PTR_ERR(our);
+               return PTR_ERR(our);
        }
 
-       if (dt2osp_dev(th->th_dev)->opd_connect_mdt)
-               update->dur_flags = UPDATE_FL_SYNC;
+       oth->ot_our = our;
+       our->our_th = oth;
 
-       oth->ot_dur = update;
        return 0;
 }
 
+void osp_thandle_destroy(const struct lu_env *env,
+                        struct osp_thandle *oth)
+{
+       LASSERT(oth->ot_magic == OSP_THANDLE_MAGIC);
+       LASSERT(list_empty(&oth->ot_commit_dcb_list));
+       LASSERT(list_empty(&oth->ot_stop_dcb_list));
+       if (oth->ot_our != NULL)
+               osp_update_request_destroy(env, oth->ot_our);
+       OBD_FREE_PTR(oth);
+}
+
 /**
  * The OSP layer dt_device_operations::dt_trans_create() interface
  * to create a transaction.
@@ -559,154 +970,182 @@ struct thandle *osp_trans_create(const struct lu_env *env, struct dt_device *d)
        if (unlikely(oth == NULL))
                RETURN(ERR_PTR(-ENOMEM));
 
+       oth->ot_magic = OSP_THANDLE_MAGIC;
        th = &oth->ot_super;
        th->th_dev = d;
        th->th_tags = LCT_TX_HANDLE;
 
+       atomic_set(&oth->ot_refcount, 1);
+       INIT_LIST_HEAD(&oth->ot_commit_dcb_list);
+       INIT_LIST_HEAD(&oth->ot_stop_dcb_list);
+
        RETURN(th);
 }
 
 /**
- * Prepare update request.
+ * Add commit callback to transaction.
  *
- * Prepare OUT update ptlrpc request, and the request usually includes
- * all of updates (stored in \param ureq) from one operation.
+ * Add commit callback to the osp thandle, which will be called
+ * when the thandle is committed remotely.
  *
- * \param[in] env      execution environment
- * \param[in] imp      import on which ptlrpc request will be sent
- * \param[in] ureq     hold all of updates which will be packed into the req
- * \param[in] reqp     request to be created
+ * \param[in] th       the thandle
+ * \param[in] dcb      commit callback structure
  *
- * \retval             0 if preparation succeeds.
- * \retval             negative errno if preparation fails.
+ * \retval             only return 0 for now.
  */
-int osp_prep_update_req(const struct lu_env *env, struct obd_import *imp,
-                       const struct object_update_request *ureq,
-                       struct ptlrpc_request **reqp)
+int osp_trans_cb_add(struct thandle *th, struct dt_txn_commit_cb *dcb)
 {
-       struct ptlrpc_request           *req;
-       struct object_update_request    *tmp;
-       int                             ureq_len;
-       int                             rc;
-       ENTRY;
-
-       object_update_request_dump(ureq, D_INFO);
-       req = ptlrpc_request_alloc(imp, &RQF_OUT_UPDATE);
-       if (req == NULL)
-               RETURN(-ENOMEM);
-
-       ureq_len = object_update_request_size(ureq);
-       req_capsule_set_size(&req->rq_pill, &RMF_OUT_UPDATE, RCL_CLIENT,
-                            ureq_len);
+       struct osp_thandle *oth = thandle_to_osp_thandle(th);
+
+       LASSERT(dcb->dcb_magic == TRANS_COMMIT_CB_MAGIC);
+       LASSERT(&dcb->dcb_func != NULL);
+       if (dcb->dcb_flags & DCB_TRANS_STOP)
+               list_add(&dcb->dcb_linkage, &oth->ot_stop_dcb_list);
+       else
+               list_add(&dcb->dcb_linkage, &oth->ot_commit_dcb_list);
+       return 0;
+}
 
-       rc = ptlrpc_request_pack(req, LUSTRE_MDS_VERSION, OUT_UPDATE);
-       if (rc != 0) {
-               ptlrpc_req_finished(req);
-               RETURN(rc);
+static void osp_trans_commit_cb(struct osp_thandle *oth, int result)
+{
+       struct dt_txn_commit_cb *dcb;
+       struct dt_txn_commit_cb *tmp;
+
+       LASSERT(atomic_read(&oth->ot_refcount) > 0);
+       /* call per-transaction callbacks if any */
+       list_for_each_entry_safe(dcb, tmp, &oth->ot_commit_dcb_list,
+                                dcb_linkage) {
+               LASSERTF(dcb->dcb_magic == TRANS_COMMIT_CB_MAGIC,
+                        "commit callback entry: magic=%x name='%s'\n",
+                        dcb->dcb_magic, dcb->dcb_name);
+               list_del_init(&dcb->dcb_linkage);
+               dcb->dcb_func(NULL, &oth->ot_super, dcb, result);
        }
+}
 
-       req_capsule_set_size(&req->rq_pill, &RMF_OUT_UPDATE_REPLY,
-                            RCL_SERVER, OUT_UPDATE_REPLY_SIZE);
-
-       tmp = req_capsule_client_get(&req->rq_pill, &RMF_OUT_UPDATE);
-       memcpy(tmp, ureq, ureq_len);
+static void osp_request_commit_cb(struct ptlrpc_request *req)
+{
+       struct thandle          *th = req->rq_cb_data;
+       struct osp_thandle      *oth;
+       __u64                   last_committed_transno = 0;
+       int                     result = req->rq_status;
+       ENTRY;
 
-       ptlrpc_request_set_replen(req);
-       req->rq_request_portal = OUT_PORTAL;
-       req->rq_reply_portal = OSC_REPLY_PORTAL;
-       *reqp = req;
+       if (th == NULL)
+               RETURN_EXIT;
 
-       RETURN(rc);
+       oth = thandle_to_osp_thandle(th);
+       if (req->rq_repmsg != NULL &&
+           lustre_msg_get_last_committed(req->rq_repmsg))
+               last_committed_transno =
+                       lustre_msg_get_last_committed(req->rq_repmsg);
+
+       if (last_committed_transno <
+               req->rq_import->imp_peer_committed_transno)
+               last_committed_transno =
+                       req->rq_import->imp_peer_committed_transno;
+
+       CDEBUG(D_HA, "trans no %llu committed transno %llu\n",
+              req->rq_transno, last_committed_transno);
+
+       /* If the transaction is not really committed, mark result = 1 */
+       if (req->rq_transno != 0 &&
+           (req->rq_transno > last_committed_transno) && result == 0)
+               result = 1;
+
+       osp_trans_commit_cb(oth, result);
+       req->rq_committed = 1;
+       osp_thandle_put(NULL, oth);
+       EXIT;
 }
 
 /**
- * Send update RPC.
+ * callback of osp transaction
  *
- * Send update request to the remote MDT synchronously.
+ * Call all of callbacks for this osp thandle. This will only be
+ * called in error handler path. In the normal processing path,
+ * these callback will be called in osp_request_commit_cb() and
+ * osp_update_interpret().
  *
- * \param[in] env      execution environment
- * \param[in] imp      import on which ptlrpc request will be sent
- * \param[in] dt_update        hold all of updates which will be packed into the req
- * \param[in] reqp     request to be created
- *
- * \retval             0 if RPC succeeds.
- * \retval             negative errno if RPC fails.
+ * \param [in] env     execution environment
+ * \param [in] oth     osp thandle
+ * \param [in] rc      result of the osp thandle
  */
-int osp_remote_sync(const struct lu_env *env, struct osp_device *osp,
-                   struct dt_update_request *dt_update,
-                   struct ptlrpc_request **reqp)
+void osp_trans_callback(const struct lu_env *env,
+                       struct osp_thandle *oth, int rc)
 {
-       struct obd_import       *imp = osp->opd_obd->u.cli.cl_import;
-       struct ptlrpc_request   *req = NULL;
-       int                     rc;
-       ENTRY;
-
-       rc = osp_prep_update_req(env, imp, dt_update->dur_buf.ub_req, &req);
-       if (rc != 0)
-               RETURN(rc);
-
-       /* This will only be called with read-only update, and these updates
-        * might be used to retrieve update log during recovery process, so
-        * it will be allowed to send during recovery process */
-       req->rq_allow_replay = 1;
-
-       /* Note: some dt index api might return non-zero result here, like
-        * osd_index_ea_lookup, so we should only check rc < 0 here */
-       rc = ptlrpc_queue_wait(req);
-       if (rc < 0) {
-               ptlrpc_req_finished(req);
-               dt_update->dur_rc = rc;
-               RETURN(rc);
-       }
+       struct osp_update_callback *ouc;
+       struct osp_update_callback *next;
 
-       if (reqp != NULL) {
-               *reqp = req;
-               RETURN(rc);
+       if (oth->ot_our != NULL) {
+               list_for_each_entry_safe(ouc, next,
+                                        &oth->ot_our->our_cb_items, ouc_list) {
+                       list_del_init(&ouc->ouc_list);
+                       if (ouc->ouc_interpreter != NULL)
+                               ouc->ouc_interpreter(env, NULL, NULL,
+                                                    ouc->ouc_obj,
+                                                    ouc->ouc_data, 0, rc);
+                       osp_update_callback_fini(env, ouc);
+               }
        }
-
-       dt_update->dur_rc = rc;
-
-       ptlrpc_req_finished(req);
-
-       RETURN(rc);
+       osp_trans_stop_cb(env, oth, rc);
+       osp_trans_commit_cb(oth, rc);
 }
 
 /**
- * Trigger the request for remote updates.
+ * Send the request for remote updates.
  *
- * If th_sync is set, then the request will be sent synchronously,
- * otherwise, the RPC will be sent asynchronously.
+ * Send updates to the remote MDT. Prepare the request by osp_update_req
+ * and send them to remote MDT, for sync request, it will wait
+ * until the reply return, otherwise hand it to ptlrpcd.
  *
  * Please refer to osp_trans_create() for transaction type.
  *
  * \param[in] env              pointer to the thread context
  * \param[in] osp              pointer to the OSP device
- * \param[in] dt_update                pointer to the dt_update_request
- * \param[in] th               pointer to the transaction handler
- * \param[out] sent            whether the RPC has been sent
+ * \param[in] our              pointer to the osp_update_request
  *
  * \retval                     0 for success
  * \retval                     negative error number on failure
  */
-static int osp_trans_trigger(const struct lu_env *env, struct osp_device *osp,
-                            struct dt_update_request *dt_update,
-                            struct thandle *th, int *sent)
+static int osp_send_update_req(const struct lu_env *env,
+                              struct osp_device *osp,
+                              struct osp_update_request *our)
 {
        struct osp_update_args  *args;
        struct ptlrpc_request   *req;
+       struct osp_thandle      *oth = our->our_th;
        int     rc = 0;
        ENTRY;
 
+       LASSERT(oth != NULL);
        rc = osp_prep_update_req(env, osp->opd_obd->u.cli.cl_import,
-                                dt_update->dur_buf.ub_req, &req);
-       if (rc != 0)
+                                our, &req);
+       if (rc != 0) {
+               osp_trans_callback(env, oth, rc);
                RETURN(rc);
+       }
 
-       *sent = 1;
-       req->rq_interpret_reply = osp_update_interpret;
        args = ptlrpc_req_async_args(req);
-       args->oaua_update = dt_update;
-       if (is_only_remote_trans(th) && !th->th_sync) {
+       args->oaua_update = our;
+       /* set env to NULL, in case the interrupt cb and current function
+        * are in different thread */
+       args->oaua_update_env = NULL;
+       osp_thandle_get(oth); /* hold for update interpret */
+       req->rq_interpret_reply = osp_update_interpret;
+       if (!oth->ot_super.th_wait_submit && !oth->ot_super.th_sync) {
+               if (!osp->opd_imp_active || !osp->opd_imp_connected) {
+                       osp_trans_callback(env, oth, rc);
+                       osp_thandle_put(env, oth);
+                       GOTO(out, rc = -ENOTCONN);
+               }
+
+               rc = obd_get_request_slot(&osp->opd_obd->u.cli);
+               if (rc != 0) {
+                       osp_trans_callback(env, oth, rc);
+                       osp_thandle_put(env, oth);
+                       GOTO(out, rc = -ENOTCONN);
+               }
                args->oaua_flow_control = true;
 
                if (!osp->opd_connect_mdt) {
@@ -717,14 +1156,50 @@ static int osp_trans_trigger(const struct lu_env *env, struct osp_device *osp,
                        atomic_inc(args->oaua_count);
                }
 
-               ptlrpcd_add_req(req, PDL_POLICY_LOCAL, -1);
+               ptlrpcd_add_req(req);
+               req = NULL;
        } else {
-               osp_get_rpc_lock(osp);
+               osp_thandle_get(oth); /* hold for commit callback */
+               req->rq_commit_cb = osp_request_commit_cb;
+               req->rq_cb_data = &oth->ot_super;
                args->oaua_flow_control = false;
+
+               /* If the transaction is created during MDT recoverying
+                * process, it means this is an recovery update, we need
+                * to let OSP send it anyway without checking recoverying
+                * status, in case the other target is being recoveried
+                * at the same time, and if we wait here for the import
+                * to be recoveryed, it might cause deadlock */
+               osp_set_req_replay(osp, req);
+
+               /* Because this req will be synchronus, i.e. it will be called
+                * in the same thread, so it will be safe to use current
+                * env */
+               args->oaua_update_env = env;
+               if (osp->opd_connect_mdt)
+                       osp_get_rpc_lock(osp);
                rc = ptlrpc_queue_wait(req);
-               osp_put_rpc_lock(osp);
-               ptlrpc_req_finished(req);
+               if (osp->opd_connect_mdt)
+                       osp_put_rpc_lock(osp);
+               if ((rc == -ENOMEM && req->rq_set == NULL) ||
+                   (req->rq_transno == 0 && !req->rq_committed)) {
+                       if (args->oaua_update != NULL) {
+                               /* If osp_update_interpret is not being called,
+                                * release the osp_thandle */
+                               args->oaua_update = NULL;
+                               osp_thandle_put(env, oth);
+                       }
+
+                       req->rq_cb_data = NULL;
+                       rc = rc == 0 ? req->rq_status : rc;
+                       osp_trans_callback(env, oth, rc);
+                       osp_thandle_put(env, oth);
+                       GOTO(out, rc);
+               }
        }
+out:
+       if (req != NULL)
+               ptlrpc_req_finished(req);
 
        RETURN(rc);
 }
@@ -779,6 +1254,258 @@ struct thandle *osp_get_storage_thandle(const struct lu_env *env,
 }
 
 /**
+ * Set version for the transaction
+ *
+ * Set the version for the transaction and add the request to
+ * the sending list, then after transaction stop, the request
+ * will be sent in the order of version by the sending thread.
+ *
+ * \param [in] oth     osp thandle to be set version.
+ *
+ * \retval             0 if set version succeeds
+ *                      negative errno if set version fails.
+ */
+int osp_check_and_set_rpc_version(struct osp_thandle *oth,
+                                 struct osp_object *obj)
+{
+       struct osp_device *osp = dt2osp_dev(oth->ot_super.th_dev);
+       struct osp_updates *ou = osp->opd_update;
+
+       if (ou == NULL)
+               return -EIO;
+
+       if (oth->ot_our->our_version != 0)
+               return 0;
+
+       spin_lock(&ou->ou_lock);
+       spin_lock(&oth->ot_our->our_list_lock);
+       if (obj->opo_stale) {
+               spin_unlock(&oth->ot_our->our_list_lock);
+               spin_unlock(&ou->ou_lock);
+               return -ESTALE;
+       }
+
+       /* Assign the version and add it to the sending list */
+       osp_thandle_get(oth);
+       oth->ot_our->our_version = ou->ou_version++;
+       oth->ot_our->our_generation = ou->ou_generation;
+       list_add_tail(&oth->ot_our->our_list,
+                     &osp->opd_update->ou_list);
+       oth->ot_our->our_req_ready = 0;
+       spin_unlock(&oth->ot_our->our_list_lock);
+       spin_unlock(&ou->ou_lock);
+
+       LASSERT(oth->ot_super.th_wait_submit == 1);
+       CDEBUG(D_INFO, "%s: version %llu gen %llu oth:version %p:%llu\n",
+              osp->opd_obd->obd_name, ou->ou_version, ou->ou_generation, oth,
+              oth->ot_our->our_version);
+
+       return 0;
+}
+
+/**
+ * Get next OSP update request in the sending list
+ * Get next OSP update request in the sending list by version number, next
+ * request will be
+ * 1. transaction which does not have a version number.
+ * 2. transaction whose version == opd_rpc_version.
+ *
+ * \param [in] ou      osp update structure.
+ * \param [out] ourp   the pointer holding the next update request.
+ *
+ * \retval             true if getting the next transaction.
+ * \retval             false if not getting the next transaction.
+ */
+static bool
+osp_get_next_request(struct osp_updates *ou, struct osp_update_request **ourp)
+{
+       struct osp_update_request *our;
+       struct osp_update_request *tmp;
+       bool                    got_req = false;
+
+       spin_lock(&ou->ou_lock);
+       list_for_each_entry_safe(our, tmp, &ou->ou_list, our_list) {
+               LASSERT(our->our_th != NULL);
+               CDEBUG(D_HA, "ou %p version %llu rpc_version %llu\n",
+                      ou, our->our_version, ou->ou_rpc_version);
+               spin_lock(&our->our_list_lock);
+               /* Find next osp_update_request in the list */
+               if (our->our_version == ou->ou_rpc_version &&
+                   our->our_req_ready) {
+                       list_del_init(&our->our_list);
+                       spin_unlock(&our->our_list_lock);
+                       *ourp = our;
+                       got_req = true;
+                       break;
+               }
+               spin_unlock(&our->our_list_lock);
+       }
+       spin_unlock(&ou->ou_lock);
+
+       return got_req;
+}
+
+/**
+ * Invalidate update request
+ *
+ * Invalidate update request in the OSP sending list, so all of
+ * requests in the sending list will return error, which happens
+ * when it finds one update (with writing llog) requests fails or
+ * the OSP is evicted by remote target. see osp_send_update_thread().
+ *
+ * \param[in] osp      OSP device whose update requests will be
+ *                      invalidated.
+ **/
+void osp_invalidate_request(struct osp_device *osp)
+{
+       struct lu_env env;
+       struct osp_updates *ou = osp->opd_update;
+       struct osp_update_request *our;
+       struct osp_update_request *tmp;
+       LIST_HEAD(list);
+       int                     rc;
+       ENTRY;
+
+       if (ou == NULL)
+               return;
+
+       rc = lu_env_init(&env, osp->opd_dt_dev.dd_lu_dev.ld_type->ldt_ctx_tags);
+       if (rc < 0) {
+               CERROR("%s: init env error: rc = %d\n", osp->opd_obd->obd_name,
+                      rc);
+
+               spin_lock(&ou->ou_lock);
+               ou->ou_generation++;
+               spin_unlock(&ou->ou_lock);
+
+               return;
+       }
+
+       INIT_LIST_HEAD(&list);
+
+       spin_lock(&ou->ou_lock);
+       /* invalidate all of request in the sending list */
+       list_for_each_entry_safe(our, tmp, &ou->ou_list, our_list) {
+               spin_lock(&our->our_list_lock);
+               if (our->our_req_ready)
+                       list_move(&our->our_list, &list);
+               else
+                       list_del_init(&our->our_list);
+
+               if (our->our_th->ot_super.th_result == 0)
+                       our->our_th->ot_super.th_result = -EIO;
+
+               if (our->our_version >= ou->ou_rpc_version)
+                       ou->ou_rpc_version = our->our_version + 1;
+               spin_unlock(&our->our_list_lock);
+
+               CDEBUG(D_HA, "%s invalidate our %p\n", osp->opd_obd->obd_name,
+                      our);
+       }
+
+       /* Increase the generation, then the update request with old generation
+        * will fail with -EIO. */
+       ou->ou_generation++;
+       spin_unlock(&ou->ou_lock);
+
+       /* invalidate all of request in the sending list */
+       list_for_each_entry_safe(our, tmp, &list, our_list) {
+               spin_lock(&our->our_list_lock);
+               list_del_init(&our->our_list);
+               spin_unlock(&our->our_list_lock);
+               osp_trans_callback(&env, our->our_th,
+                                  our->our_th->ot_super.th_result);
+               osp_thandle_put(&env, our->our_th);
+       }
+       lu_env_fini(&env);
+}
+
+/**
+ * Sending update thread
+ *
+ * Create thread to send update request to other MDTs, this thread will pull
+ * out update request from the list in OSP by version number, i.e. it will
+ * make sure the update request with lower version number will be sent first.
+ *
+ * \param[in] arg      hold the OSP device.
+ *
+ * \retval             0 if the thread is created successfully.
+ * \retal              negative error if the thread is not created
+ *                      successfully.
+ */
+int osp_send_update_thread(void *arg)
+{
+       struct lu_env           env;
+       struct osp_device       *osp = arg;
+       struct l_wait_info       lwi = { 0 };
+       struct osp_updates      *ou = osp->opd_update;
+       struct ptlrpc_thread    *thread = &osp->opd_update_thread;
+       struct osp_update_request *our = NULL;
+       int                     rc;
+       ENTRY;
+
+       LASSERT(ou != NULL);
+       rc = lu_env_init(&env, osp->opd_dt_dev.dd_lu_dev.ld_type->ldt_ctx_tags);
+       if (rc < 0) {
+               CERROR("%s: init env error: rc = %d\n", osp->opd_obd->obd_name,
+                      rc);
+               RETURN(rc);
+       }
+
+       thread->t_flags = SVC_RUNNING;
+       wake_up(&thread->t_ctl_waitq);
+       while (1) {
+               our = NULL;
+               l_wait_event(ou->ou_waitq,
+                            !osp_send_update_thread_running(osp) ||
+                            osp_get_next_request(ou, &our), &lwi);
+
+               if (!osp_send_update_thread_running(osp)) {
+                       if (our != NULL) {
+                               osp_trans_callback(&env, our->our_th, -EINTR);
+                               osp_thandle_put(&env, our->our_th);
+                       }
+                       break;
+               }
+
+               LASSERT(our->our_th != NULL);
+               if (our->our_th->ot_super.th_result != 0) {
+                       osp_trans_callback(&env, our->our_th,
+                               our->our_th->ot_super.th_result);
+                       rc = our->our_th->ot_super.th_result;
+               } else if (ou->ou_generation != our->our_generation ||
+                          OBD_FAIL_CHECK(OBD_FAIL_INVALIDATE_UPDATE)) {
+                       rc = -EIO;
+                       osp_trans_callback(&env, our->our_th, rc);
+               } else {
+                       rc = osp_send_update_req(&env, osp, our);
+               }
+
+               /* Update the rpc version */
+               spin_lock(&ou->ou_lock);
+               if (our->our_version == ou->ou_rpc_version)
+                       ou->ou_rpc_version++;
+               spin_unlock(&ou->ou_lock);
+
+               /* If one update request fails, let's fail all of the requests
+                * in the sending list, because the request in the sending
+                * list are dependent on either other, continue sending these
+                * request might cause llog or filesystem corruption */
+               if (rc < 0)
+                       osp_invalidate_request(osp);
+
+               /* Balanced for thandle_get in osp_check_and_set_rpc_version */
+               osp_thandle_put(&env, our->our_th);
+       }
+
+       thread->t_flags = SVC_STOPPED;
+       lu_env_fini(&env);
+       wake_up(&thread->t_ctl_waitq);
+
+       RETURN(0);
+}
+
+/**
  * The OSP layer dt_device_operations::dt_trans_start() interface
  * to start the transaction.
  *
@@ -798,6 +1525,8 @@ int osp_trans_start(const struct lu_env *env, struct dt_device *dt,
 {
        struct osp_thandle      *oth = thandle_to_osp_thandle(th);
 
+       if (oth->ot_super.th_sync)
+               oth->ot_our->our_flags |= UPDATE_FL_SYNC;
        /* For remote thandle, if there are local thandle, start it here*/
        if (is_only_remote_trans(th) && oth->ot_storage_th != NULL)
                return dt_trans_start(env, oth->ot_storage_th->th_dev,
@@ -810,7 +1539,7 @@ int osp_trans_start(const struct lu_env *env, struct dt_device *dt,
  * to stop the transaction.
  *
  * If the transaction is a remote transaction, related remote
- * updates will be triggered here via osp_trans_trigger().
+ * updates will be triggered at the end of this function.
  *
  * For synchronous mode update or any failed update, the request
  * will be destroyed explicitly when the osp_trans_stop().
@@ -828,9 +1557,9 @@ int osp_trans_stop(const struct lu_env *env, struct dt_device *dt,
                   struct thandle *th)
 {
        struct osp_thandle       *oth = thandle_to_osp_thandle(th);
-       struct dt_update_request *dt_update;
+       struct osp_update_request *our = oth->ot_our;
+       struct osp_device        *osp = dt2osp_dev(dt);
        int                      rc = 0;
-       int                      sent = 0;
        ENTRY;
 
        /* For remote transaction, if there is local storage thandle,
@@ -841,62 +1570,44 @@ int osp_trans_stop(const struct lu_env *env, struct dt_device *dt,
                oth->ot_storage_th = NULL;
        }
 
-       dt_update = oth->ot_dur;
-       if (dt_update == NULL)
-               GOTO(out, rc);
-
-       LASSERT(dt_update != LP_POISON);
+       if (our == NULL || list_empty(&our->our_req_list)) {
+               osp_trans_callback(env, oth, th->th_result);
+               GOTO(out, rc = th->th_result);
+       }
 
-       /* If there are no updates, destroy dt_update and thandle */
-       if (dt_update->dur_buf.ub_req == NULL ||
-           dt_update->dur_buf.ub_req->ourq_count == 0) {
-               dt_update_request_destroy(dt_update);
+       if (!osp->opd_connect_mdt) {
+               osp_trans_callback(env, oth, th->th_result);
+               rc = osp_send_update_req(env, osp, oth->ot_our);
                GOTO(out, rc);
        }
 
-       if (is_only_remote_trans(th) && !th->th_sync) {
-               struct osp_device *osp = dt2osp_dev(th->th_dev);
-               struct client_obd *cli = &osp->opd_obd->u.cli;
-
-               rc = obd_get_request_slot(cli);
-               if (rc != 0)
-                       GOTO(out, rc);
-
-               if (!osp->opd_imp_active || !osp->opd_imp_connected) {
-                       obd_put_request_slot(cli);
-                       GOTO(out, rc = -ENOTCONN);
-               }
+       if (osp->opd_update == NULL ||
+           !osp_send_update_thread_running(osp)) {
+               osp_trans_callback(env, oth, -EIO);
+               GOTO(out, rc = -EIO);
+       }
 
-               rc = osp_trans_trigger(env, dt2osp_dev(dt),
-                                      dt_update, th, &sent);
-               if (rc != 0)
-                       obd_put_request_slot(cli);
+       CDEBUG(D_HA, "%s: add oth %p with version %llu\n",
+              osp->opd_obd->obd_name, oth, our->our_version);
+
+       LASSERT(our->our_req_ready == 0);
+       spin_lock(&our->our_list_lock);
+       if (likely(!list_empty(&our->our_list))) {
+               /* notify sending thread */
+               our->our_req_ready = 1;
+               wake_up(&osp->opd_update->ou_waitq);
+               spin_unlock(&our->our_list_lock);
+       } else if (th->th_result == 0) {
+               /* if the request does not needs to be serialized,
+                * read-only request etc, let's send it right away */
+               spin_unlock(&our->our_list_lock);
+               rc = osp_send_update_req(env, osp, our);
        } else {
-               rc = osp_trans_trigger(env, dt2osp_dev(dt), dt_update,
-                                      th, &sent);
+               spin_unlock(&our->our_list_lock);
+               osp_trans_callback(env, oth, th->th_result);
        }
-
 out:
-       /* If RPC is triggered successfully, dt_update will be freed in
-        * osp_update_interpreter() */
-       if (rc != 0 && dt_update != NULL && sent == 0) {
-               struct osp_update_callback *ouc;
-               struct osp_update_callback *next;
-
-               list_for_each_entry_safe(ouc, next, &dt_update->dur_cb_items,
-                                ouc_list) {
-                       list_del_init(&ouc->ouc_list);
-                       if (ouc->ouc_interpreter != NULL)
-                               ouc->ouc_interpreter(env, NULL, NULL,
-                                                    ouc->ouc_obj,
-                                                    ouc->ouc_data, 0, rc);
-                       osp_update_callback_fini(env, ouc);
-               }
-
-               dt_update_request_destroy(dt_update);
-       }
-
-       OBD_FREE_PTR(oth);
+       osp_thandle_put(env, oth);
 
        RETURN(rc);
 }