Whamcloud - gitweb
LU-6050 target: control OST-index in IDIF via ROCOMPAT flag
[fs/lustre-release.git] / lustre / target / tgt_lastrcvd.c
index 88dd030..2457993 100644 (file)
@@ -27,7 +27,7 @@
  * Copyright (c) 2009, 2010, Oracle and/or its affiliates. All rights reserved.
  * Use is subject to license terms.
  *
- * Copyright (c) 2011, 2012, Intel Corporation.
+ * Copyright (c) 2011, 2014, Intel Corporation.
  */
 /*
  * This file is part of Lustre, http://www.lustre.org/
@@ -127,7 +127,6 @@ int tgt_client_data_read(const struct lu_env *env, struct lu_target *tgt,
               lcd->lcd_last_close_result, rc);
        return rc;
 }
-EXPORT_SYMBOL(tgt_client_data_read);
 
 int tgt_client_data_write(const struct lu_env *env, struct lu_target *tgt,
                          struct lsd_client_data *lcd, loff_t *off,
@@ -143,12 +142,12 @@ int tgt_client_data_write(const struct lu_env *env, struct lu_target *tgt,
 
        return dt_record_write(env, tgt->lut_last_rcvd, &tti->tti_buf, off, th);
 }
-EXPORT_SYMBOL(tgt_client_data_write);
 
 /**
  * Update client data in last_rcvd
  */
-int tgt_client_data_update(const struct lu_env *env, struct obd_export *exp)
+static int tgt_client_data_update(const struct lu_env *env,
+                                 struct obd_export *exp)
 {
        struct tg_export_data   *ted = &exp->exp_target_data;
        struct lu_target        *tgt = class_exp2tgt(exp);
@@ -162,8 +161,9 @@ int tgt_client_data_update(const struct lu_env *env, struct obd_export *exp)
        if (IS_ERR(th))
                RETURN(PTR_ERR(th));
 
+       tti_buf_lcd(tti);
        rc = dt_declare_record_write(env, tgt->lut_last_rcvd,
-                                    sizeof(struct lsd_client_data),
+                                    &tti->tti_buf,
                                     ted->ted_lr_off, th);
        if (rc)
                GOTO(out, rc);
@@ -216,7 +216,6 @@ int tgt_server_data_read(const struct lu_env *env, struct lu_target *tgt)
               tgt->lut_lsd.lsd_uuid, tgt->lut_lsd.lsd_last_transno, rc);
         return rc;
 }
-EXPORT_SYMBOL(tgt_server_data_read);
 
 int tgt_server_data_write(const struct lu_env *env, struct lu_target *tgt,
                          struct thandle *th)
@@ -239,7 +238,6 @@ int tgt_server_data_write(const struct lu_env *env, struct lu_target *tgt,
 
        RETURN(rc);
 }
-EXPORT_SYMBOL(tgt_server_data_write);
 
 /**
  * Update server data in last_rcvd
@@ -269,9 +267,9 @@ int tgt_server_data_update(const struct lu_env *env, struct lu_target *tgt,
 
        th->th_sync = sync;
 
+       tti_buf_lsd(tti);
        rc = dt_declare_record_write(env, tgt->lut_last_rcvd,
-                                    sizeof(struct lr_server_data),
-                                    tti->tti_off, th);
+                                    &tti->tti_buf, tti->tti_off, th);
        if (rc)
                GOTO(out, rc);
 
@@ -325,9 +323,9 @@ cleanup:
 
        RETURN(rc);
 }
-EXPORT_SYMBOL(tgt_truncate_last_rcvd);
 
-void tgt_client_epoch_update(const struct lu_env *env, struct obd_export *exp)
+static void tgt_client_epoch_update(const struct lu_env *env,
+                                   struct obd_export *exp)
 {
        struct lsd_client_data  *lcd = exp->exp_target_data.ted_lcd;
        struct lu_target        *tgt = class_exp2tgt(exp);
@@ -348,7 +346,7 @@ void tgt_boot_epoch_update(struct lu_target *tgt)
        struct lu_env            env;
        struct ptlrpc_request   *req;
        __u32                    start_epoch;
-       cfs_list_t               client_list;
+       struct list_head         client_list;
        int                      rc;
 
        if (tgt->lut_obd->obd_stopping)
@@ -367,33 +365,32 @@ void tgt_boot_epoch_update(struct lu_target *tgt)
        tgt->lut_lsd.lsd_start_epoch = start_epoch;
        spin_unlock(&tgt->lut_translock);
 
-       CFS_INIT_LIST_HEAD(&client_list);
+       INIT_LIST_HEAD(&client_list);
        /**
         * The recovery is not yet finished and final queue can still be updated
         * with resend requests. Move final list to separate one for processing
         */
        spin_lock(&tgt->lut_obd->obd_recovery_task_lock);
-       cfs_list_splice_init(&tgt->lut_obd->obd_final_req_queue, &client_list);
+       list_splice_init(&tgt->lut_obd->obd_final_req_queue, &client_list);
        spin_unlock(&tgt->lut_obd->obd_recovery_task_lock);
 
        /**
         * go through list of exports participated in recovery and
         * set new epoch for them
         */
-       cfs_list_for_each_entry(req, &client_list, rq_list) {
+       list_for_each_entry(req, &client_list, rq_list) {
                LASSERT(!req->rq_export->exp_delayed);
                if (!req->rq_export->exp_vbr_failed)
                        tgt_client_epoch_update(&env, req->rq_export);
        }
        /** return list back at once */
        spin_lock(&tgt->lut_obd->obd_recovery_task_lock);
-       cfs_list_splice_init(&client_list, &tgt->lut_obd->obd_final_req_queue);
+       list_splice_init(&client_list, &tgt->lut_obd->obd_final_req_queue);
        spin_unlock(&tgt->lut_obd->obd_recovery_task_lock);
        /** update server epoch */
        tgt_server_data_update(&env, tgt, 1);
        lu_env_fini(&env);
 }
-EXPORT_SYMBOL(tgt_boot_epoch_update);
 
 /**
  * commit callback, need to update last_commited value
@@ -405,8 +402,8 @@ struct tgt_last_committed_callback {
        __u64                    llcc_transno;
 };
 
-void tgt_cb_last_committed(struct lu_env *env, struct thandle *th,
-                          struct dt_txn_commit_cb *cb, int err)
+static void tgt_cb_last_committed(struct lu_env *env, struct thandle *th,
+                                 struct dt_txn_commit_cb *cb, int err)
 {
        struct tgt_last_committed_callback *ccb;
 
@@ -451,9 +448,8 @@ int tgt_last_commit_cb_add(struct thandle *th, struct lu_target *tgt,
 
        dcb = &ccb->llcc_cb;
        dcb->dcb_func = tgt_cb_last_committed;
-       CFS_INIT_LIST_HEAD(&dcb->dcb_linkage);
-       strncpy(dcb->dcb_name, "tgt_cb_last_committed", MAX_COMMIT_CB_STR_LEN);
-       dcb->dcb_name[MAX_COMMIT_CB_STR_LEN - 1] = '\0';
+       INIT_LIST_HEAD(&dcb->dcb_linkage);
+       strlcpy(dcb->dcb_name, "tgt_cb_last_committed", sizeof(dcb->dcb_name));
 
        rc = dt_trans_cb_add(th, dcb);
        if (rc) {
@@ -467,15 +463,14 @@ int tgt_last_commit_cb_add(struct thandle *th, struct lu_target *tgt,
 
        return rc;
 }
-EXPORT_SYMBOL(tgt_last_commit_cb_add);
 
 struct tgt_new_client_callback {
        struct dt_txn_commit_cb  lncc_cb;
        struct obd_export       *lncc_exp;
 };
 
-void tgt_cb_new_client(struct lu_env *env, struct thandle *th,
-                      struct dt_txn_commit_cb *cb, int err)
+static void tgt_cb_new_client(struct lu_env *env, struct thandle *th,
+                             struct dt_txn_commit_cb *cb, int err)
 {
        struct tgt_new_client_callback *ccb;
 
@@ -488,7 +483,12 @@ void tgt_cb_new_client(struct lu_env *env, struct thandle *th,
               ccb->lncc_exp->exp_client_uuid.uuid);
 
        spin_lock(&ccb->lncc_exp->exp_lock);
-       ccb->lncc_exp->exp_need_sync = 0;
+       /* XXX: Currently, we use per-export based sync/async policy for
+        *      the update via OUT RPC, it is coarse-grained policy, and
+        *      will be changed as per-request based by DNE II patches. */
+       if (!ccb->lncc_exp->exp_keep_sync)
+               ccb->lncc_exp->exp_need_sync = 0;
+
        spin_unlock(&ccb->lncc_exp->exp_lock);
        class_export_cb_put(ccb->lncc_exp);
 
@@ -509,9 +509,8 @@ int tgt_new_client_cb_add(struct thandle *th, struct obd_export *exp)
 
        dcb = &ccb->lncc_cb;
        dcb->dcb_func = tgt_cb_new_client;
-       CFS_INIT_LIST_HEAD(&dcb->dcb_linkage);
-       strncpy(dcb->dcb_name, "tgt_cb_new_client", MAX_COMMIT_CB_STR_LEN);
-       dcb->dcb_name[MAX_COMMIT_CB_STR_LEN - 1] = '\0';
+       INIT_LIST_HEAD(&dcb->dcb_linkage);
+       strlcpy(dcb->dcb_name, "tgt_cb_new_client", sizeof(dcb->dcb_name));
 
        rc = dt_trans_cb_add(th, dcb);
        if (rc) {
@@ -627,7 +626,6 @@ int tgt_client_add(const struct lu_env *env,  struct obd_export *exp, int idx)
 
        RETURN(0);
 }
-EXPORT_SYMBOL(tgt_client_add);
 
 int tgt_client_del(const struct lu_env *env, struct obd_export *exp)
 {
@@ -688,12 +686,11 @@ EXPORT_SYMBOL(tgt_client_del);
 /*
  * last_rcvd & last_committed update callbacks
  */
-int tgt_last_rcvd_update(const struct lu_env *env, struct lu_target *tgt,
-                        struct dt_object *obj, __u64 opdata,
-                        struct thandle *th, struct ptlrpc_request *req)
+static int tgt_last_rcvd_update(const struct lu_env *env, struct lu_target *tgt,
+                               struct dt_object *obj, __u64 opdata,
+                               struct thandle *th, struct ptlrpc_request *req)
 {
        struct tgt_thread_info  *tti = tgt_th_info(env);
-       struct tgt_session_info *tsi = tgt_ses_info(env);
        struct tg_export_data   *ted;
        __u64                   *transno_p;
        int                      rc = 0;
@@ -701,27 +698,16 @@ int tgt_last_rcvd_update(const struct lu_env *env, struct lu_target *tgt,
 
        ENTRY;
 
-       if (tsi->tsi_has_trans) {
-               /* XXX: currently there are allowed cases, but the wrong cases
-                * are also possible, so better check is needed here */
-               CDEBUG(D_INFO, "More than one transaction "LPU64"\n",
-                      tti->tti_transno);
-               return 0;
-       }
-
-       tsi->tsi_has_trans = 1;
-       /* that can be OUT target and we need tgt_session_info */
-       if (req == NULL) {
-               req = tgt_ses_req(tsi);
-               if (req == NULL) /* echo client case */
-                       RETURN(0);
-       }
-
        ted = &req->rq_export->exp_target_data;
 
        lw_client = exp_connect_flags(req->rq_export) & OBD_CONNECT_LIGHTWEIGHT;
+       if (ted->ted_lr_idx < 0 && !lw_client)
+               /* ofd connect may cause transaction before export has
+                * last_rcvd slot */
+               RETURN(0);
 
        tti->tti_transno = lustre_msg_get_transno(req->rq_reqmsg);
+
        spin_lock(&tgt->lut_translock);
        if (th->th_result != 0) {
                if (tti->tti_transno != 0) {
@@ -765,7 +751,7 @@ int tgt_last_rcvd_update(const struct lu_env *env, struct lu_target *tgt,
                 * last_rcvd, we still want to maintain the in-memory
                 * lsd_client_data structure in order to properly handle reply
                 * reconstruction. */
-       } else if (ted->ted_lr_off <= 0) {
+       } else if (ted->ted_lr_off == 0) {
                CERROR("%s: client idx %d has offset %lld\n",
                       tgt_name(tgt), ted->ted_lr_idx, ted->ted_lr_off);
                RETURN(-EINVAL);
@@ -788,7 +774,7 @@ int tgt_last_rcvd_update(const struct lu_env *env, struct lu_target *tgt,
                ted->ted_lcd->lcd_last_close_result = th->th_result;
        } else {
                /* VBR: save versions in last_rcvd for reconstruct. */
-               __u64 *pre_versions = lustre_msg_get_versions(req->rq_reqmsg);
+               __u64 *pre_versions = lustre_msg_get_versions(req->rq_repmsg);
 
                if (pre_versions) {
                        ted->ted_lcd->lcd_pre_versions[0] = pre_versions[0];
@@ -806,7 +792,8 @@ int tgt_last_rcvd_update(const struct lu_env *env, struct lu_target *tgt,
 
        /* Update transno in slot only if non-zero number, i.e. no errors */
        if (likely(tti->tti_transno != 0)) {
-               if (*transno_p > tti->tti_transno) {
+               if (*transno_p > tti->tti_transno &&
+                   !tgt->lut_no_reconstruct) {
                        CERROR("%s: trying to overwrite bigger transno:"
                               "on-disk: "LPU64", new: "LPU64" replay: %d. "
                               "see LU-617.\n", tgt_name(tgt), *transno_p,
@@ -837,5 +824,426 @@ srv_update:
                rc = tgt_server_data_write(env, tgt, th);
        return rc;
 }
-EXPORT_SYMBOL(tgt_last_rcvd_update);
 
+/*
+ * last_rcvd update for echo client simulation.
+ * It updates last_rcvd client slot and version of object in
+ * simple way but with all locks to simulate all drawbacks
+ */
+static int tgt_last_rcvd_update_echo(const struct lu_env *env,
+                                    struct lu_target *tgt,
+                                    struct dt_object *obj,
+                                    struct thandle *th,
+                                    struct obd_export *exp)
+{
+       struct tgt_thread_info  *tti = tgt_th_info(env);
+       struct tg_export_data   *ted = &exp->exp_target_data;
+       int                      rc = 0;
+
+       ENTRY;
+
+       tti->tti_transno = 0;
+
+       spin_lock(&tgt->lut_translock);
+       if (th->th_result == 0)
+               tti->tti_transno = ++tgt->lut_last_transno;
+       spin_unlock(&tgt->lut_translock);
+
+       /** VBR: set new versions */
+       if (th->th_result == 0 && obj != NULL)
+               dt_version_set(env, obj, tti->tti_transno, th);
+
+       /* if can't add callback, do sync write */
+       th->th_sync |= !!tgt_last_commit_cb_add(th, tgt, exp,
+                                               tti->tti_transno);
+
+       LASSERT(ted->ted_lr_off > 0);
+
+       mutex_lock(&ted->ted_lcd_lock);
+       LASSERT(ergo(tti->tti_transno == 0, th->th_result != 0));
+       ted->ted_lcd->lcd_last_transno = tti->tti_transno;
+       ted->ted_lcd->lcd_last_result = th->th_result;
+
+       tti->tti_off = ted->ted_lr_off;
+       rc = tgt_client_data_write(env, tgt, ted->ted_lcd, &tti->tti_off, th);
+       mutex_unlock(&ted->ted_lcd_lock);
+       RETURN(rc);
+}
+
+static int tgt_clients_data_init(const struct lu_env *env,
+                                struct lu_target *tgt,
+                                unsigned long last_size)
+{
+       struct obd_device       *obd = tgt->lut_obd;
+       struct lr_server_data   *lsd = &tgt->lut_lsd;
+       struct lsd_client_data  *lcd = NULL;
+       struct tg_export_data   *ted;
+       int                      cl_idx;
+       int                      rc = 0;
+       loff_t                   off = lsd->lsd_client_start;
+
+       ENTRY;
+
+       CLASSERT(offsetof(struct lsd_client_data, lcd_padding) +
+                sizeof(lcd->lcd_padding) == LR_CLIENT_SIZE);
+
+       OBD_ALLOC_PTR(lcd);
+       if (lcd == NULL)
+               RETURN(-ENOMEM);
+
+       for (cl_idx = 0; off < last_size; cl_idx++) {
+               struct obd_export       *exp;
+               __u64                    last_transno;
+
+               /* Don't assume off is incremented properly by
+                * read_record(), in case sizeof(*lcd)
+                * isn't the same as fsd->lsd_client_size.  */
+               off = lsd->lsd_client_start + cl_idx * lsd->lsd_client_size;
+               rc = tgt_client_data_read(env, tgt, lcd, &off, cl_idx);
+               if (rc) {
+                       CERROR("%s: error reading last_rcvd %s idx %d off "
+                              "%llu: rc = %d\n", tgt_name(tgt), LAST_RCVD,
+                              cl_idx, off, rc);
+                       rc = 0;
+                       break; /* read error shouldn't cause startup to fail */
+               }
+
+               if (lcd->lcd_uuid[0] == '\0') {
+                       CDEBUG(D_INFO, "skipping zeroed client at offset %d\n",
+                              cl_idx);
+                       continue;
+               }
+
+               last_transno = lcd_last_transno(lcd);
+
+               /* These exports are cleaned up by disconnect, so they
+                * need to be set up like real exports as connect does.
+                */
+               CDEBUG(D_HA, "RCVRNG CLIENT uuid: %s idx: %d lr: "LPU64
+                      " srv lr: "LPU64" lx: "LPU64"\n", lcd->lcd_uuid, cl_idx,
+                      last_transno, lsd->lsd_last_transno, lcd_last_xid(lcd));
+
+               exp = class_new_export(obd, (struct obd_uuid *)lcd->lcd_uuid);
+               if (IS_ERR(exp)) {
+                       if (PTR_ERR(exp) == -EALREADY) {
+                               /* export already exists, zero out this one */
+                               CERROR("%s: Duplicate export %s!\n",
+                                      tgt_name(tgt), lcd->lcd_uuid);
+                               continue;
+                       }
+                       GOTO(err_out, rc = PTR_ERR(exp));
+               }
+
+               ted = &exp->exp_target_data;
+               *ted->ted_lcd = *lcd;
+
+               rc = tgt_client_add(env, exp, cl_idx);
+               LASSERTF(rc == 0, "rc = %d\n", rc); /* can't fail existing */
+               /* VBR: set export last committed version */
+               exp->exp_last_committed = last_transno;
+               spin_lock(&exp->exp_lock);
+               exp->exp_connecting = 0;
+               exp->exp_in_recovery = 0;
+               spin_unlock(&exp->exp_lock);
+               obd->obd_max_recoverable_clients++;
+               class_export_put(exp);
+
+               /* Need to check last_rcvd even for duplicated exports. */
+               CDEBUG(D_OTHER, "client at idx %d has last_transno = "LPU64"\n",
+                      cl_idx, last_transno);
+
+               spin_lock(&tgt->lut_translock);
+               tgt->lut_last_transno = max(last_transno,
+                                           tgt->lut_last_transno);
+               spin_unlock(&tgt->lut_translock);
+       }
+
+err_out:
+       OBD_FREE_PTR(lcd);
+       RETURN(rc);
+}
+
+struct server_compat_data {
+       __u32 rocompat;
+       __u32 incompat;
+       __u32 rocinit;
+       __u32 incinit;
+};
+
+static struct server_compat_data tgt_scd[] = {
+       [LDD_F_SV_TYPE_MDT] = {
+               .rocompat = OBD_ROCOMPAT_LOVOBJID,
+               .incompat = OBD_INCOMPAT_MDT | OBD_INCOMPAT_COMMON_LR |
+                           OBD_INCOMPAT_FID | OBD_INCOMPAT_IAM_DIR |
+                           OBD_INCOMPAT_LMM_VER | OBD_INCOMPAT_MULTI_OI,
+               .rocinit = OBD_ROCOMPAT_LOVOBJID,
+               .incinit = OBD_INCOMPAT_MDT | OBD_INCOMPAT_COMMON_LR |
+                          OBD_INCOMPAT_MULTI_OI,
+       },
+       [LDD_F_SV_TYPE_OST] = {
+               .rocompat = OBD_ROCOMPAT_IDX_IN_IDIF,
+               .incompat = OBD_INCOMPAT_OST | OBD_INCOMPAT_COMMON_LR |
+                           OBD_INCOMPAT_FID,
+               .rocinit = OBD_ROCOMPAT_IDX_IN_IDIF,
+               .incinit = OBD_INCOMPAT_OST | OBD_INCOMPAT_COMMON_LR,
+       }
+};
+
+int tgt_server_data_init(const struct lu_env *env, struct lu_target *tgt)
+{
+       struct tgt_thread_info          *tti = tgt_th_info(env);
+       struct lr_server_data           *lsd = &tgt->lut_lsd;
+       unsigned long                    last_rcvd_size;
+       __u32                            index;
+       int                              rc, type;
+
+       rc = dt_attr_get(env, tgt->lut_last_rcvd, &tti->tti_attr, BYPASS_CAPA);
+       if (rc)
+               RETURN(rc);
+
+       last_rcvd_size = (unsigned long)tti->tti_attr.la_size;
+
+       /* ensure padding in the struct is the correct size */
+       CLASSERT(offsetof(struct lr_server_data, lsd_padding) +
+                sizeof(lsd->lsd_padding) == LR_SERVER_SIZE);
+
+       rc = server_name2index(tgt_name(tgt), &index, NULL);
+       if (rc < 0) {
+               CERROR("%s: Can not get index from name: rc = %d\n",
+                      tgt_name(tgt), rc);
+               RETURN(rc);
+       }
+       /* server_name2index() returns type */
+       type = rc;
+       if (type != LDD_F_SV_TYPE_MDT && type != LDD_F_SV_TYPE_OST) {
+               CERROR("%s: unknown target type %x\n", tgt_name(tgt), type);
+               RETURN(-EINVAL);
+       }
+
+       /* last_rcvd on OST doesn't provide reconstruct support because there
+        * may be up to 8 in-flight write requests per single slot in
+        * last_rcvd client data
+        */
+       tgt->lut_no_reconstruct = (type == LDD_F_SV_TYPE_OST);
+
+       if (last_rcvd_size == 0) {
+               LCONSOLE_WARN("%s: new disk, initializing\n", tgt_name(tgt));
+
+               memcpy(lsd->lsd_uuid, tgt->lut_obd->obd_uuid.uuid,
+                      sizeof(lsd->lsd_uuid));
+               lsd->lsd_last_transno = 0;
+               lsd->lsd_mount_count = 0;
+               lsd->lsd_server_size = LR_SERVER_SIZE;
+               lsd->lsd_client_start = LR_CLIENT_START;
+               lsd->lsd_client_size = LR_CLIENT_SIZE;
+               lsd->lsd_subdir_count = OBJ_SUBDIR_COUNT;
+               lsd->lsd_osd_index = index;
+               lsd->lsd_feature_rocompat = tgt_scd[type].rocinit;
+               lsd->lsd_feature_incompat = tgt_scd[type].incinit;
+       } else {
+               rc = tgt_server_data_read(env, tgt);
+               if (rc) {
+                       CERROR("%s: error reading LAST_RCVD: rc= %d\n",
+                              tgt_name(tgt), rc);
+                       RETURN(rc);
+               }
+               if (strcmp(lsd->lsd_uuid, tgt->lut_obd->obd_uuid.uuid)) {
+                       LCONSOLE_ERROR_MSG(0x157, "Trying to start OBD %s "
+                                          "using the wrong disk %s. Were the"
+                                          " /dev/ assignments rearranged?\n",
+                                          tgt->lut_obd->obd_uuid.uuid,
+                                          lsd->lsd_uuid);
+                       RETURN(-EINVAL);
+               }
+
+               if (lsd->lsd_osd_index != index) {
+                       LCONSOLE_ERROR_MSG(0x157, "%s: index %d in last rcvd "
+                                          "is different with the index %d in"
+                                          "config log, It might be disk"
+                                          "corruption!\n", tgt_name(tgt),
+                                          lsd->lsd_osd_index, index);
+                       RETURN(-EINVAL);
+               }
+       }
+
+       if (lsd->lsd_feature_incompat & ~tgt_scd[type].incompat) {
+               CERROR("%s: unsupported incompat filesystem feature(s) %x\n",
+                      tgt_name(tgt),
+                      lsd->lsd_feature_incompat & ~tgt_scd[type].incompat);
+               RETURN(-EINVAL);
+       }
+
+       if (type == LDD_F_SV_TYPE_MDT)
+               lsd->lsd_feature_incompat |= OBD_INCOMPAT_FID;
+
+       if (lsd->lsd_feature_rocompat & ~tgt_scd[type].rocompat) {
+               CERROR("%s: unsupported read-only filesystem feature(s) %x\n",
+                      tgt_name(tgt),
+                      lsd->lsd_feature_rocompat & ~tgt_scd[type].rocompat);
+               RETURN(-EINVAL);
+       }
+       /** Interop: evict all clients at first boot with 1.8 last_rcvd */
+       if (type == LDD_F_SV_TYPE_MDT &&
+           !(lsd->lsd_feature_compat & OBD_COMPAT_20)) {
+               if (last_rcvd_size > lsd->lsd_client_start) {
+                       LCONSOLE_WARN("%s: mounting at first time on 1.8 FS, "
+                                     "remove all clients for interop needs\n",
+                                     tgt_name(tgt));
+                       rc = tgt_truncate_last_rcvd(env, tgt,
+                                                   lsd->lsd_client_start);
+                       if (rc)
+                               RETURN(rc);
+                       last_rcvd_size = lsd->lsd_client_start;
+               }
+               /** set 2.0 flag to upgrade/downgrade between 1.8 and 2.0 */
+               lsd->lsd_feature_compat |= OBD_COMPAT_20;
+       }
+
+       spin_lock(&tgt->lut_translock);
+       tgt->lut_last_transno = lsd->lsd_last_transno;
+       spin_unlock(&tgt->lut_translock);
+
+       lsd->lsd_mount_count++;
+
+       CDEBUG(D_INODE, "=======,=BEGIN DUMPING LAST_RCVD========\n");
+       CDEBUG(D_INODE, "%s: server last_transno: "LPU64"\n",
+              tgt_name(tgt), tgt->lut_last_transno);
+       CDEBUG(D_INODE, "%s: server mount_count: "LPU64"\n",
+              tgt_name(tgt), lsd->lsd_mount_count);
+       CDEBUG(D_INODE, "%s: server data size: %u\n",
+              tgt_name(tgt), lsd->lsd_server_size);
+       CDEBUG(D_INODE, "%s: per-client data start: %u\n",
+              tgt_name(tgt), lsd->lsd_client_start);
+       CDEBUG(D_INODE, "%s: per-client data size: %u\n",
+              tgt_name(tgt), lsd->lsd_client_size);
+       CDEBUG(D_INODE, "%s: last_rcvd size: %lu\n",
+              tgt_name(tgt), last_rcvd_size);
+       CDEBUG(D_INODE, "%s: server subdir_count: %u\n",
+              tgt_name(tgt), lsd->lsd_subdir_count);
+       CDEBUG(D_INODE, "%s: last_rcvd clients: %lu\n", tgt_name(tgt),
+              last_rcvd_size <= lsd->lsd_client_start ? 0 :
+              (last_rcvd_size - lsd->lsd_client_start) /
+               lsd->lsd_client_size);
+       CDEBUG(D_INODE, "========END DUMPING LAST_RCVD========\n");
+
+       if (lsd->lsd_server_size == 0 || lsd->lsd_client_start == 0 ||
+           lsd->lsd_client_size == 0) {
+               CERROR("%s: bad last_rcvd contents!\n", tgt_name(tgt));
+               RETURN(-EINVAL);
+       }
+
+       if (!tgt->lut_obd->obd_replayable)
+               CWARN("%s: recovery support OFF\n", tgt_name(tgt));
+
+       rc = tgt_clients_data_init(env, tgt, last_rcvd_size);
+       if (rc < 0)
+               GOTO(err_client, rc);
+
+       spin_lock(&tgt->lut_translock);
+       /* obd_last_committed is used for compatibility
+        * with other lustre recovery code */
+       tgt->lut_obd->obd_last_committed = tgt->lut_last_transno;
+       spin_unlock(&tgt->lut_translock);
+
+       tgt->lut_obd->u.obt.obt_mount_count = lsd->lsd_mount_count;
+       tgt->lut_obd->u.obt.obt_instance = (__u32)lsd->lsd_mount_count;
+
+       /* save it, so mount count and last_transno is current */
+       rc = tgt_server_data_update(env, tgt, 0);
+       if (rc < 0)
+               GOTO(err_client, rc);
+
+       RETURN(0);
+
+err_client:
+       class_disconnect_exports(tgt->lut_obd);
+       return rc;
+}
+
+/* add credits for last_rcvd update */
+int tgt_txn_start_cb(const struct lu_env *env, struct thandle *th,
+                    void *cookie)
+{
+       struct lu_target        *tgt = cookie;
+       struct tgt_session_info *tsi;
+       struct tgt_thread_info  *tti = tgt_th_info(env);
+       int                      rc;
+
+       /* if there is no session, then this transaction is not result of
+        * request processing but some local operation */
+       if (env->le_ses == NULL)
+               return 0;
+
+       LASSERT(tgt->lut_last_rcvd);
+       tsi = tgt_ses_info(env);
+       /* OFD may start transaction without export assigned */
+       if (tsi->tsi_exp == NULL)
+               return 0;
+
+       tti_buf_lcd(tti);
+       rc = dt_declare_record_write(env, tgt->lut_last_rcvd,
+                                    &tti->tti_buf,
+                                    tsi->tsi_exp->exp_target_data.ted_lr_off,
+                                    th);
+       if (rc)
+               return rc;
+
+       tti_buf_lsd(tti);
+       rc = dt_declare_record_write(env, tgt->lut_last_rcvd,
+                                    &tti->tti_buf, 0, th);
+       if (rc)
+               return rc;
+
+       if (tsi->tsi_vbr_obj != NULL &&
+           !lu_object_remote(&tsi->tsi_vbr_obj->do_lu))
+               rc = dt_declare_version_set(env, tsi->tsi_vbr_obj, th);
+
+       return rc;
+}
+
+/* Update last_rcvd records with latests transaction data */
+int tgt_txn_stop_cb(const struct lu_env *env, struct thandle *th,
+                   void *cookie)
+{
+       struct lu_target        *tgt = cookie;
+       struct tgt_session_info *tsi;
+       struct tgt_thread_info  *tti = tgt_th_info(env);
+       struct dt_object        *obj = NULL;
+       int                      rc;
+       bool                     echo_client;
+
+       if (env->le_ses == NULL)
+               return 0;
+
+       tsi = tgt_ses_info(env);
+       /* OFD may start transaction without export assigned */
+       if (tsi->tsi_exp == NULL)
+               return 0;
+
+       echo_client = (tgt_ses_req(tsi) == NULL);
+
+       if (tti->tti_has_trans && !echo_client) {
+               if (tti->tti_mult_trans == 0) {
+                       CDEBUG(D_HA, "More than one transaction "LPU64"\n",
+                              tti->tti_transno);
+                       RETURN(0);
+               }
+               /* we need another transno to be assigned */
+               tti->tti_transno = 0;
+       } else if (th->th_result == 0) {
+               tti->tti_has_trans = 1;
+       }
+
+       if (tsi->tsi_vbr_obj != NULL &&
+           !lu_object_remote(&tsi->tsi_vbr_obj->do_lu)) {
+               obj = tsi->tsi_vbr_obj;
+       }
+
+       if (unlikely(echo_client)) /* echo client special case */
+               rc = tgt_last_rcvd_update_echo(env, tgt, obj, th,
+                                              tsi->tsi_exp);
+       else
+               rc = tgt_last_rcvd_update(env, tgt, obj, tsi->tsi_opdata, th,
+                                         tgt_ses_req(tsi));
+       return rc;
+}