Whamcloud - gitweb
LU-3534 osp: move RPC pack from declare to execution phase
[fs/lustre-release.git] / lustre / osp / osp_trans.c
index 896f2d0..740270d 100644 (file)
  * GPL HEADER END
  */
 /*
- * Copyright (c) 2013, Intel Corporation.
+ * Copyright (c) 2014, Intel Corporation.
  */
 /*
  * lustre/osp/osp_trans.c
  *
+ *
+ * 1. OSP (Object Storage Proxy) transaction methods
+ *
+ * Implement OSP layer transaction related interfaces for the dt_device API
+ * dt_device_operations.
+ *
+ *
+ * 2. Handle asynchronous idempotent operations
+ *
+ * The OSP uses OUT (Object Unified Target) RPC to talk with other server
+ * (MDT or OST) for kinds of operations, such as create, unlink, insert,
+ * delete, lookup, set_(x)attr, get_(x)attr, and etc. To reduce the number
+ * of RPCs, we allow multiple operations to be packaged together in single
+ * OUT RPC.
+ *
+ * For the asynchronous idempotent operations, such as get_(x)attr, related
+ * RPCs will be inserted into a osp_device based shared asynchronous request
+ * queue - osp_device::opd_async_requests. When the queue is full, all the
+ * requests in the queue will be packaged into a single OUT RPC and given to
+ * the ptlrpcd daemon (for sending), then the queue is purged and other new
+ * requests can be inserted into it.
+ *
+ * When the asynchronous idempotent operation inserts the request into the
+ * shared queue, it will register an interpreter. When the packaged OUT RPC
+ * is replied (or failed to be sent out), all the registered interpreters
+ * will be called one by one to handle each own result.
+ *
+ *
+ * There are three kinds of transactions
+ *
+ * 1. Local transaction, all of updates of the transaction are in the local MDT.
+ * 2. Remote transaction, all of updates of the transaction are in one remote
+ * MDT, which only happens in LFSCK now.
+ * 3. Distribute transaction, updates for the transaction are in mulitple MDTs.
+ *
  * Author: Di Wang <di.wang@intel.com>
  * Author: Fan, Yong <fan.yong@intel.com>
  */
 
 #include "osp_internal.h"
 
-struct osp_async_update_args {
+/**
+ * The argument for the interpreter callback of osp request.
+ */
+struct osp_update_args {
        struct dt_update_request *oaua_update;
-       unsigned int             oaua_fc:1;
+       atomic_t                 *oaua_count;
+       wait_queue_head_t        *oaua_waitq;
+       bool                      oaua_flow_control;
 };
 
-struct osp_async_update_item {
-       struct list_head                 oaui_list;
-       struct osp_object               *oaui_obj;
-       void                            *oaui_data;
-       osp_async_update_interpterer_t   oaui_interpterer;
+/**
+ * Call back for each update request.
+ */
+struct osp_update_callback {
+       /* list in the dt_update_request::dur_cb_items */
+       struct list_head                 ouc_list;
+
+       /* The target of the async update request. */
+       struct osp_object               *ouc_obj;
+
+       /* The data used by or_interpreter. */
+       void                            *ouc_data;
+
+       /* The interpreter function called after the async request handled. */
+       osp_update_interpreter_t        ouc_interpreter;
 };
 
-static struct osp_async_update_item *
-osp_async_update_item_init(struct osp_object *obj, void *data,
-                          osp_async_update_interpterer_t interpterer)
+static struct object_update_request *object_update_request_alloc(size_t size)
+{
+       struct object_update_request *ourq;
+
+       OBD_ALLOC_LARGE(ourq, size);
+       if (ourq == NULL)
+               return ERR_PTR(-ENOMEM);
+
+       ourq->ourq_magic = UPDATE_REQUEST_MAGIC;
+       ourq->ourq_count = 0;
+
+       return ourq;
+}
+
+static void object_update_request_free(struct object_update_request *ourq,
+                                      size_t ourq_size)
+{
+       if (ourq != NULL)
+               OBD_FREE_LARGE(ourq, ourq_size);
+}
+
+/**
+ * Allocate and initialize dt_update_request
+ *
+ * dt_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             ERR_PTR(errno) if failed
+ */
+struct dt_update_request *dt_update_request_create(struct dt_device *dt)
 {
-       struct osp_async_update_item *oaui;
+       struct dt_update_request *dt_update;
+       struct object_update_request *ourq;
+
+       OBD_ALLOC_PTR(dt_update);
+       if (dt_update == 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);
+
+       return dt_update;
+}
 
-       OBD_ALLOC_PTR(oaui);
-       if (oaui == NULL)
+/**
+ * Destroy dt_update_request
+ *
+ * \param [in] dt_update       dt_update_request being destroyed
+ */
+void dt_update_request_destroy(struct dt_update_request *dt_update)
+{
+       if (dt_update == NULL)
+               return;
+
+       object_update_request_free(dt_update->dur_buf.ub_req,
+                                  dt_update->dur_buf.ub_req_size);
+       OBD_FREE_PTR(dt_update);
+}
+
+/**
+ * Allocate an osp request and initialize it with the given parameters.
+ *
+ * \param[in] obj              pointer to the operation target
+ * \param[in] data             pointer to the data used by the interpreter
+ * \param[in] interpreter      pointer to the interpreter function
+ *
+ * \retval                     pointer to the asychronous request
+ * \retval                     NULL if the allocation failed
+ */
+static struct osp_update_callback *
+osp_update_callback_init(struct osp_object *obj, void *data,
+                        osp_update_interpreter_t interpreter)
+{
+       struct osp_update_callback *ouc;
+
+       OBD_ALLOC_PTR(ouc);
+       if (ouc == NULL)
                return NULL;
 
        lu_object_get(osp2lu_obj(obj));
-       INIT_LIST_HEAD(&oaui->oaui_list);
-       oaui->oaui_obj = obj;
-       oaui->oaui_data = data;
-       oaui->oaui_interpterer = interpterer;
+       INIT_LIST_HEAD(&ouc->ouc_list);
+       ouc->ouc_obj = obj;
+       ouc->ouc_data = data;
+       ouc->ouc_interpreter = interpreter;
 
-       return oaui;
+       return ouc;
 }
 
-static void osp_async_update_item_fini(const struct lu_env *env,
-                                      struct osp_async_update_item *oaui)
+/**
+ * Destroy the osp_update_callback.
+ *
+ * \param[in] env      pointer to the thread context
+ * \param[in] ouc      pointer to osp_update_callback
+ */
+static void osp_update_callback_fini(const struct lu_env *env,
+                                    struct osp_update_callback *ouc)
 {
-       LASSERT(list_empty(&oaui->oaui_list));
+       LASSERT(list_empty(&ouc->ouc_list));
 
-       lu_object_put(env, osp2lu_obj(oaui->oaui_obj));
-       OBD_FREE_PTR(oaui);
+       lu_object_put(env, osp2lu_obj(ouc->ouc_obj));
+       OBD_FREE_PTR(ouc);
 }
 
-static int osp_async_update_interpret(const struct lu_env *env,
-                                     struct ptlrpc_request *req,
-                                     void *arg, int rc)
+/**
+ * Interpret the packaged OUT RPC results.
+ *
+ * For every packaged sub-request, call its registered interpreter function.
+ * Then destroy the sub-request.
+ *
+ * \param[in] env      pointer to the thread context
+ * \param[in] req      pointer to the RPC
+ * \param[in] arg      pointer to data used by the interpreter
+ * \param[in] rc       the RPC return value
+ *
+ * \retval             0 for success
+ * \retval             negative error number on failure
+ */
+static int osp_update_interpret(const struct lu_env *env,
+                               struct ptlrpc_request *req, void *arg, int rc)
 {
        struct object_update_reply      *reply  = NULL;
-       struct osp_async_update_args    *oaua   = arg;
+       struct osp_update_args          *oaua   = arg;
        struct dt_update_request        *dt_update = oaua->oaua_update;
-       struct osp_async_update_item    *oaui;
-       struct osp_async_update_item    *next;
-       struct osp_device               *osp    = dt2osp_dev(dt_update->dur_dt);
+       struct osp_update_callback      *ouc;
+       struct osp_update_callback      *next;
        int                              count  = 0;
        int                              index  = 0;
        int                              rc1    = 0;
 
-       if (oaua->oaua_fc)
-               up(&osp->opd_async_fc_sem);
+       if (oaua->oaua_flow_control)
+               obd_put_request_slot(
+                               &dt2osp_dev(dt_update->dur_dt)->opd_obd->u.cli);
 
-       if (rc == 0 || req->rq_repmsg != NULL) {
+       /* Unpack the results from the reply message. */
+       if (req->rq_repmsg != NULL) {
                reply = req_capsule_server_sized_get(&req->rq_pill,
                                                     &RMF_OUT_UPDATE_REPLY,
                                                     OUT_UPDATE_REPLY_SIZE);
@@ -102,9 +255,13 @@ static int osp_async_update_interpret(const struct lu_env *env,
                rc1 = rc;
        }
 
-       list_for_each_entry_safe(oaui, next, &dt_update->dur_cb_items,
-                                oaui_list) {
-               list_del_init(&oaui->oaui_list);
+       list_for_each_entry_safe(ouc, next, &dt_update->dur_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) {
                        struct object_update_result *result;
 
@@ -119,53 +276,83 @@ static int osp_async_update_interpret(const struct lu_env *env,
                                rc1 = -EINVAL;
                }
 
-               oaui->oaui_interpterer(env, reply, oaui->oaui_obj,
-                                      oaui->oaui_data, index, rc1);
-               osp_async_update_item_fini(env, oaui);
+               if (ouc->ouc_interpreter != NULL)
+                       ouc->ouc_interpreter(env, reply, req, ouc->ouc_obj,
+                                            ouc->ouc_data, index, rc1);
+
+               osp_update_callback_fini(env, ouc);
                index++;
        }
 
-       out_destroy_update_req(dt_update);
+       if (oaua->oaua_count != NULL && atomic_dec_and_test(oaua->oaua_count))
+               wake_up_all(oaua->oaua_waitq);
+
+       dt_update_request_destroy(dt_update);
 
        return 0;
 }
 
-int osp_unplug_async_update(const struct lu_env *env,
-                           struct osp_device *osp,
-                           struct dt_update_request *update)
+/**
+ * Pack all the requests in the shared asynchronous idempotent request queue
+ * into a single OUT RPC that will be given to the background ptlrpcd daemon.
+ *
+ * \param[in] env      pointer to the thread context
+ * \param[in] osp      pointer to the OSP device
+ * \param[in] update   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_async_update_args    *args;
-       struct ptlrpc_request           *req = NULL;
-       int                              rc;
+       struct osp_update_args  *args;
+       struct ptlrpc_request   *req = NULL;
+       int                      rc;
 
-       rc = out_prep_update_req(env, osp->opd_obd->u.cli.cl_import,
-                                update->dur_req, &req);
+       rc = osp_prep_update_req(env, osp->opd_obd->u.cli.cl_import,
+                                update->dur_buf.ub_req, &req);
        if (rc != 0) {
-               struct osp_async_update_item *oaui;
-               struct osp_async_update_item *next;
-
-               list_for_each_entry_safe(oaui, next,
-                                        &update->dur_cb_items, oaui_list) {
-                       list_del_init(&oaui->oaui_list);
-                       oaui->oaui_interpterer(env, NULL, oaui->oaui_obj,
-                                              oaui->oaui_data, 0, rc);
-                       osp_async_update_item_fini(env, oaui);
+               struct osp_update_callback *ouc;
+               struct osp_update_callback *next;
+
+               list_for_each_entry_safe(ouc, next,
+                                        &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);
                }
-               out_destroy_update_req(update);
+               dt_update_request_destroy(update);
        } else {
-               LASSERT(list_empty(&update->dur_list));
-
                args = ptlrpc_req_async_args(req);
                args->oaua_update = update;
-               req->rq_interpret_reply = osp_async_update_interpret;
+               args->oaua_count = NULL;
+               args->oaua_waitq = NULL;
+               args->oaua_flow_control = false;
+               req->rq_interpret_reply = osp_update_interpret;
                ptlrpcd_add_req(req, PDL_POLICY_LOCAL, -1);
        }
 
        return rc;
 }
 
-/* with osp::opd_async_requests_mutex held */
-struct dt_update_request *
+/**
+ * Find or create (if NOT exist or purged) the shared asynchronous idempotent
+ * 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.
+ *
+ * \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 *
 osp_find_or_create_async_update_request(struct osp_device *osp)
 {
        struct dt_update_request *update = osp->opd_async_requests;
@@ -173,229 +360,494 @@ osp_find_or_create_async_update_request(struct osp_device *osp)
        if (update != NULL)
                return update;
 
-       update = out_create_update_req(&osp->opd_dt_dev);
+       update = dt_update_request_create(&osp->opd_dt_dev);
        if (!IS_ERR(update))
                osp->opd_async_requests = update;
 
        return update;
 }
 
-/* with osp::opd_async_requests_mutex held */
-int osp_insert_async_update(const struct lu_env *env,
-                           struct dt_update_request *update, int op,
-                           struct osp_object *obj, int count,
-                           int *lens, const char **bufs, void *data,
-                           osp_async_update_interpterer_t interpterer)
+/**
+ * Insert an osp_update_callback into the dt_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
+ * callbacks will be called in rq_interpret_reply.
+ *
+ * \param[in] env              pointer to the thread context
+ * \param[in] obj              pointer to the operation target object
+ * \param[in] data             pointer to the data used by the interpreter
+ * \param[in] interpreter      pointer to the interpreter function
+ *
+ * \retval                     0 for success
+ * \retval                     negative error number on failure
+ */
+int osp_insert_update_callback(const struct lu_env *env,
+                              struct dt_update_request *update,
+                              struct osp_object *obj, void *data,
+                              osp_update_interpreter_t interpreter)
+{
+       struct osp_update_callback  *ouc;
+
+       ouc = osp_update_callback_init(obj, data, interpreter);
+       if (ouc == NULL)
+               RETURN(-ENOMEM);
+
+       list_add_tail(&ouc->ouc_list, &update->dur_cb_items);
+
+       return 0;
+}
+
+/**
+ * Insert an asynchronous idempotent request to the shared request queue that
+ * is attached to the osp_device.
+ *
+ * This function generates a new osp_async_request with the given parameters,
+ * then tries to insert the request into the osp_device-based shared request
+ * queue. If the queue is full, then triggers the packaged OUT RPC to purge
+ * the shared queue firstly, and then re-tries.
+ *
+ * NOTE: must hold the osp::opd_async_requests_mutex to serialize concurrent
+ *      osp_insert_async_request call from others.
+ *
+ * \param[in] env              pointer to the thread context
+ * \param[in] op               operation type, see 'enum update_type'
+ * \param[in] obj              pointer to the operation target
+ * \param[in] count            array size of the subsequent \a lens and \a bufs
+ * \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] interpreter      pointer to the interpreter function
+ *
+ * \retval                     0 for success
+ * \retval                     negative error number on failure
+ */
+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,
+                            osp_update_interpreter_t interpreter)
 {
-       struct osp_async_update_item *oaui;
        struct osp_device            *osp = lu2osp_dev(osp2lu_obj(obj)->lo_dev);
+       struct dt_update_request     *update;
        int                           rc  = 0;
        ENTRY;
 
-       oaui = osp_async_update_item_init(obj, data, interpterer);
-       if (oaui == NULL)
-               RETURN(-ENOMEM);
+       update = osp_find_or_create_async_update_request(osp);
+       if (IS_ERR(update))
+               RETURN(PTR_ERR(update));
 
 again:
-       rc = out_insert_update(env, update, op, lu_object_fid(osp2lu_obj(obj)),
-                              count, lens, bufs);
+       /* The queue is full. */
+       rc = out_update_pack(env, &update->dur_buf, op,
+                            lu_object_fid(osp2lu_obj(obj)), count, lens, bufs,
+                            0);
        if (rc == -E2BIG) {
                osp->opd_async_requests = NULL;
                mutex_unlock(&osp->opd_async_requests_mutex);
 
-               rc = osp_unplug_async_update(env, osp, update);
+               rc = osp_unplug_async_request(env, osp, update);
                mutex_lock(&osp->opd_async_requests_mutex);
                if (rc != 0)
-                       GOTO(out, rc);
+                       RETURN(rc);
 
                update = osp_find_or_create_async_update_request(osp);
                if (IS_ERR(update))
-                       GOTO(out, rc = PTR_ERR(update));
+                       RETURN(PTR_ERR(update));
 
                goto again;
        }
 
-       if (rc == 0)
-               list_add_tail(&oaui->oaui_list, &update->dur_cb_items);
+       rc = osp_insert_update_callback(env, update, obj, data, interpreter);
 
-       GOTO(out, rc);
+       RETURN(rc);
+}
 
-out:
-       if (rc != 0)
-               osp_async_update_item_fini(env, oaui);
+int osp_trans_update_request_create(struct thandle *th)
+{
+       struct osp_thandle              *oth = thandle_to_osp_thandle(th);
+       struct dt_update_request        *update;
 
-       return rc;
-}
+       if (oth->ot_dur != 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);
+       }
 
+       oth->ot_dur = update;
+       return 0;
+}
 /**
- * If the transaction creation goes to OSP, it means the update
- * in this transaction only includes remote UPDATE. It is only
- * used by LFSCK right now.
- **/
+ * The OSP layer dt_device_operations::dt_trans_create() interface
+ * to create a transaction.
+ *
+ * There are two kinds of transactions that will involve OSP:
+ *
+ * 1) If the transaction only contains the updates on remote server
+ *    (MDT or OST), such as re-generating the lost OST-object for
+ *    LFSCK, then it is a remote transaction. For remote transaction,
+ *    the upper layer caller (such as the LFSCK engine) will call the
+ *    dt_trans_create() (with the OSP dt_device as the parameter),
+ *    then the call will be directed to the osp_trans_create() that
+ *    creates the transaction handler and returns it to the caller.
+ *
+ * 2) If the transcation contains both local and remote updates,
+ *    such as cross MDTs create under DNE mode, then the upper layer
+ *    caller will not trigger osp_trans_create(). Instead, it will
+ *    call dt_trans_create() on other dt_device, such as LOD that
+ *    will generate the transaction handler. Such handler will be
+ *    used by the whole transaction in subsequent sub-operations.
+ *
+ * \param[in] env      pointer to the thread context
+ * \param[in] d                pointer to the OSP dt_device
+ *
+ * \retval             pointer to the transaction handler
+ * \retval             negative error number on failure
+ */
 struct thandle *osp_trans_create(const struct lu_env *env, struct dt_device *d)
 {
-       struct thandle *th = NULL;
-       struct thandle_update *tu = NULL;
-       int rc = 0;
+       struct osp_thandle              *oth;
+       struct thandle                  *th = NULL;
+       ENTRY;
 
-       OBD_ALLOC_PTR(th);
-       if (unlikely(th == NULL))
-               GOTO(out, rc = -ENOMEM);
+       OBD_ALLOC_PTR(oth);
+       if (unlikely(oth == NULL))
+               RETURN(ERR_PTR(-ENOMEM));
 
+       th = &oth->ot_super;
        th->th_dev = d;
        th->th_tags = LCT_TX_HANDLE;
-       atomic_set(&th->th_refc, 1);
-       th->th_alloc_size = sizeof(*th);
 
-       OBD_ALLOC_PTR(tu);
-       if (tu == NULL)
-               GOTO(out, rc = -ENOMEM);
+       RETURN(th);
+}
 
-       INIT_LIST_HEAD(&tu->tu_remote_update_list);
-       tu->tu_only_remote_trans = 1;
-       th->th_update = tu;
+/**
+ * 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,
+                       const struct object_update_request *ureq,
+                       struct ptlrpc_request **reqp)
+{
+       struct ptlrpc_request           *req;
+       struct object_update_request    *tmp;
+       int                             ureq_len;
+       int                             rc;
+       ENTRY;
 
-out:
+       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);
+
+       rc = ptlrpc_request_pack(req, LUSTRE_MDS_VERSION, OUT_UPDATE);
        if (rc != 0) {
-               if (tu != NULL)
-                       OBD_FREE_PTR(tu);
-               if (th != NULL)
-                       OBD_FREE_PTR(th);
-               th = ERR_PTR(rc);
+               ptlrpc_req_finished(req);
+               RETURN(rc);
+       }
+
+       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);
+
+       ptlrpc_request_set_replen(req);
+       req->rq_request_portal = OUT_PORTAL;
+       req->rq_reply_portal = OSC_REPLY_PORTAL;
+       *reqp = 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] 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.
+ */
+int osp_remote_sync(const struct lu_env *env, struct osp_device *osp,
+                   struct dt_update_request *dt_update,
+                   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, dt_update->dur_buf.ub_req, &req);
+       if (rc != 0)
+               RETURN(rc);
+
+       /* 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);
+       }
+
+       if (reqp != NULL) {
+               *reqp = req;
+               RETURN(rc);
        }
 
-       return th;
+       dt_update->dur_rc = rc;
+
+       ptlrpc_req_finished(req);
+
+       RETURN(rc);
 }
 
+/**
+ * Trigger the request for remote updates.
+ *
+ * If th_sync is set, then the request will be sent synchronously,
+ * otherwise, the RPC will be sent asynchronously.
+ *
+ * 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
+ *
+ * \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, bool fc)
+                            struct thandle *th, int *sent)
 {
-       struct thandle_update   *tu = th->th_update;
-       int                     rc = 0;
-
-       LASSERT(tu != NULL);
-
-       /* If the transaction only includes remote update, it should
-        * still be asynchronous */
-       if (is_only_remote_trans(th)) {
-               struct osp_async_update_args    *args;
-               struct ptlrpc_request           *req;
-
-               list_del_init(&dt_update->dur_list);
-               rc = out_prep_update_req(env, osp->opd_obd->u.cli.cl_import,
-                                        dt_update->dur_req, &req);
-               if (rc == 0) {
-                       args = ptlrpc_req_async_args(req);
-                       args->oaua_update = dt_update;
-                       args->oaua_fc = !!fc;
-                       req->rq_interpret_reply =
-                               osp_async_update_interpret;
-                       ptlrpcd_add_req(req, PDL_POLICY_LOCAL, -1);
-               } else {
-                       out_destroy_update_req(dt_update);
+       struct osp_update_args  *args;
+       struct ptlrpc_request   *req;
+       int     rc = 0;
+       ENTRY;
+
+       rc = osp_prep_update_req(env, osp->opd_obd->u.cli.cl_import,
+                                dt_update->dur_buf.ub_req, &req);
+       if (rc != 0)
+               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_flow_control = true;
+
+               if (!osp->opd_connect_mdt) {
+                       down_read(&osp->opd_async_updates_rwsem);
+                       args->oaua_count = &osp->opd_async_updates_count;
+                       args->oaua_waitq = &osp->opd_syn_barrier_waitq;
+                       up_read(&osp->opd_async_updates_rwsem);
+                       atomic_inc(args->oaua_count);
                }
+
+               ptlrpcd_add_req(req, PDL_POLICY_LOCAL, -1);
        } else {
-               /* Before we support async update, the cross MDT transaction
-                * has to been synchronized */
-               th->th_sync = 1;
-               rc = out_remote_sync(env, osp->opd_obd->u.cli.cl_import,
-                                    dt_update, NULL);
+               osp_get_rpc_lock(osp);
+               args->oaua_flow_control = false;
+               rc = ptlrpc_queue_wait(req);
+               osp_put_rpc_lock(osp);
+               ptlrpc_req_finished(req);
        }
 
-       return rc;
+       RETURN(rc);
 }
 
-int osp_trans_start(const struct lu_env *env, struct dt_device *dt,
-                   struct thandle *th)
+/**
+ * Get local thandle for osp_thandle
+ *
+ * Get the local OSD thandle from the OSP thandle. Currently, there
+ * are a few OSP API (osp_object_create() and osp_sync_add()) needs
+ * to update the object on local OSD device.
+ *
+ * If the osp_thandle comes from normal stack (MDD->LOD->OSP), then
+ * we will get local thandle by thandle_get_sub_by_dt.
+ *
+ * If the osp_thandle is remote thandle (th_top == NULL, only used
+ * by LFSCK), then it will create a local thandle, and stop it in
+ * osp_trans_stop(). And this only happens on OSP for OST.
+ *
+ * These are temporary solution, once OSP accessing OSD object is
+ * being fixed properly, this function should be removed. XXX
+ *
+ * \param[in] env              pointer to the thread context
+ * \param[in] th               pointer to the transaction handler
+ * \param[in] dt               pointer to the OSP device
+ *
+ * \retval                     pointer to the local thandle
+ * \retval                     ERR_PTR(errno) if it fails.
+ **/
+struct thandle *osp_get_storage_thandle(const struct lu_env *env,
+                                       struct thandle *th,
+                                       struct osp_device *osp)
 {
-       struct thandle_update *tu = th->th_update;
-       struct dt_update_request *dt_update;
-       int rc = 0;
+       struct osp_thandle      *oth;
+       struct thandle          *local_th;
 
-       if (tu == NULL)
-               return rc;
+       if (th->th_top != NULL)
+               return thandle_get_sub_by_dt(env, th->th_top,
+                                            osp->opd_storage);
 
-       /* Check whether there are updates related with this OSP */
-       dt_update = out_find_update(tu, dt);
-       if (dt_update == NULL)
-               return rc;
-
-       /* Note: some updates needs to send before local transaction,
-        * some needs to send after local transaction.
-        *
-        * If the transaction only includes remote updates, it will
-        * send updates to remote MDT in osp_trans_stop.
-        *
-        * If it is remote create, it will send the remote req after
-        * local transaction. i.e. create the object locally first,
-        * then insert the name entry.
-        *
-        * If it is remote unlink, it will send the remote req before
-        * the local transaction, i.e. delete the name entry remote
-        * first, then destroy the local object. */
-       if (!is_only_remote_trans(th) && !tu->tu_sent_after_local_trans)
-               rc = osp_trans_trigger(env, dt2osp_dev(dt), dt_update, th,
-                                      false);
+       LASSERT(!osp->opd_connect_mdt);
+       oth = thandle_to_osp_thandle(th);
+       if (oth->ot_storage_th != NULL)
+               return oth->ot_storage_th;
 
-       return rc;
+       local_th = dt_trans_create(env, osp->opd_storage);
+       if (IS_ERR(local_th))
+               return local_th;
+
+       oth->ot_storage_th = local_th;
+
+       return local_th;
 }
 
+/**
+ * The OSP layer dt_device_operations::dt_trans_start() interface
+ * to start the transaction.
+ *
+ * If the transaction is a remote transaction, then related remote
+ * updates will be triggered in the osp_trans_stop().
+ * Please refer to osp_trans_create() for transaction type.
+ *
+ * \param[in] env              pointer to the thread context
+ * \param[in] dt               pointer to the OSP dt_device
+ * \param[in] th               pointer to the transaction handler
+ *
+ * \retval                     0 for success
+ * \retval                     negative error number on failure
+ */
+int osp_trans_start(const struct lu_env *env, struct dt_device *dt,
+                   struct thandle *th)
+{
+       struct osp_thandle      *oth = thandle_to_osp_thandle(th);
+
+       /* 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,
+                                     oth->ot_storage_th);
+       return 0;
+}
+
+/**
+ * The OSP layer dt_device_operations::dt_trans_stop() interface
+ * to stop the transaction.
+ *
+ * If the transaction is a remote transaction, related remote
+ * updates will be triggered here via osp_trans_trigger().
+ *
+ * For synchronous mode update or any failed update, the request
+ * will be destroyed explicitly when the osp_trans_stop().
+ *
+ * Please refer to osp_trans_create() for transaction type.
+ *
+ * \param[in] env              pointer to the thread context
+ * \param[in] dt               pointer to the OSP dt_device
+ * \param[in] th               pointer to the transaction handler
+ *
+ * \retval                     0 for success
+ * \retval                     negative error number on failure
+ */
 int osp_trans_stop(const struct lu_env *env, struct dt_device *dt,
                   struct thandle *th)
 {
-       struct thandle_update           *tu = th->th_update;
-       struct dt_update_request        *dt_update;
-       int rc = 0;
-
-       LASSERT(tu != NULL);
-       /* Check whether there are updates related with this OSP */
-       dt_update = out_find_update(tu, dt);
-       if (dt_update == NULL) {
-               if (!is_only_remote_trans(th))
-                       return rc;
-               goto put;
+       struct osp_thandle       *oth = thandle_to_osp_thandle(th);
+       struct dt_update_request *dt_update;
+       int                      rc = 0;
+       int                      sent = 0;
+       ENTRY;
+
+       /* For remote transaction, if there is local storage thandle,
+        * stop it first */
+       if (oth->ot_storage_th != NULL && th->th_top == NULL) {
+               dt_trans_stop(env, oth->ot_storage_th->th_dev,
+                             oth->ot_storage_th);
+               oth->ot_storage_th = NULL;
        }
 
-       if (dt_update->dur_req->ourq_count == 0) {
-               out_destroy_update_req(dt_update);
-               goto put;
+       dt_update = oth->ot_dur;
+       if (dt_update == NULL)
+               GOTO(out, rc);
+
+       LASSERT(dt_update != LP_POISON);
+
+       /* 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);
+               GOTO(out, rc);
        }
 
-       if (is_only_remote_trans(th)) {
-               if (th->th_result == 0) {
-                       struct osp_device *osp = dt2osp_dev(th->th_dev);
-
-                       do {
-                               if (!osp->opd_imp_active ||
-                                   osp->opd_got_disconnected) {
-                                       out_destroy_update_req(dt_update);
-                                       GOTO(put, rc = -ENOTCONN);
-                               }
-
-                               /* Get the semaphore to guarantee it has
-                                * free slot, which will be released via
-                                * osp_async_update_interpret(). */
-                               rc = down_timeout(&osp->opd_async_fc_sem, HZ);
-                       } while (rc != 0);
-
-                       rc = osp_trans_trigger(env, dt2osp_dev(dt),
-                                              dt_update, th, true);
-                       if (rc != 0)
-                               up(&osp->opd_async_fc_sem);
-               } else {
-                       rc = th->th_result;
-                       out_destroy_update_req(dt_update);
+       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);
                }
+
+               rc = osp_trans_trigger(env, dt2osp_dev(dt),
+                                      dt_update, th, &sent);
+               if (rc != 0)
+                       obd_put_request_slot(cli);
        } else {
-               if (tu->tu_sent_after_local_trans)
-                       rc = osp_trans_trigger(env, dt2osp_dev(dt),
-                                              dt_update, th, false);
-               rc = dt_update->dur_rc;
-               out_destroy_update_req(dt_update);
+               rc = osp_trans_trigger(env, dt2osp_dev(dt), dt_update,
+                                      th, &sent);
        }
 
-put:
-       thandle_put(th);
-       return rc;
+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);
+
+       RETURN(rc);
 }