Whamcloud - gitweb
LU-6864 osp: manage number of modify RPCs in flight 75/14375/23
authorGregoire Pichon <gregoire.pichon@bull.net>
Thu, 5 May 2022 21:42:56 +0000 (17:42 -0400)
committerOleg Drokin <green@whamcloud.com>
Mon, 6 Jun 2022 06:27:59 +0000 (06:27 +0000)
Currently we use a rpc_lock to ensure concurrent in-flight
request are handled serially to prevent the execution status
from being over written. This patch changes the osp component
to send multiple modify RPCs in parallel to the MDT. This will
improve metadata performance of cross-MDT operations.

For testing replace mkdirmany with createmany -d which does the
same thing.

Signed-off-by: Gregoire Pichon <gregoire.pichon@bull.net>
Signed-off-by: James Simmons <jsimmons@infradead.org>
Change-Id: Icb601afabd6767463634a4c7943ec4206bc758ec
Reviewed-on: https://review.whamcloud.com/14375
Reviewed-by: Andreas Dilger <adilger@whamcloud.com>
Reviewed-by: Lai Siyao <lai.siyao@whamcloud.com>
Tested-by: jenkins <devops@whamcloud.com>
Tested-by: Maloo <maloo@whamcloud.com>
Reviewed-by: Oleg Drokin <green@whamcloud.com>
12 files changed:
lustre/include/obd.h
lustre/lod/lod_lov.c
lustre/osp/lproc_osp.c
lustre/osp/osp_dev.c
lustre/osp/osp_internal.h
lustre/osp/osp_trans.c
lustre/target/out_handler.c
lustre/tests/.gitignore
lustre/tests/Makefile.am
lustre/tests/createmany.c
lustre/tests/mkdirmany.c [deleted file]
lustre/tests/sanity.sh

index f3bf355..65cf50b 100644 (file)
@@ -175,7 +175,6 @@ enum obd_cl_sem_lock_class {
        OBD_CLI_SEM_MDCOSC,
 };
 
-struct mdc_rpc_lock;
 struct obd_import;
 struct client_obd {
        struct rw_semaphore      cl_sem;
index b3f9cd8..d2636c0 100644 (file)
@@ -207,6 +207,7 @@ int lod_add_device(const struct lu_env *env, struct lod_device *lod,
                data->ocd_connect_flags |= OBD_CONNECT_ACL |
                                           OBD_CONNECT_IBITS |
                                           OBD_CONNECT_MDS_MDS |
+                                          OBD_CONNECT_MULTIMODRPCS |
                                           OBD_CONNECT_FID |
                                           OBD_CONNECT_AT |
                                           OBD_CONNECT_FULL20 |
index f0eb765..f1e0a64 100644 (file)
@@ -276,6 +276,44 @@ static ssize_t max_rpcs_in_flight_store(struct kobject *kobj,
 }
 LUSTRE_RW_ATTR(max_rpcs_in_flight);
 
+static ssize_t max_mod_rpcs_in_flight_show(struct kobject *kobj,
+                                          struct attribute *attr,
+                                          char *buf)
+{
+       struct dt_device *dt = container_of(kobj, struct dt_device,
+                                           dd_kobj);
+       struct lu_device *lu = dt2lu_dev(dt);
+       struct obd_device *obd = lu->ld_obd;
+       u16 max;
+
+       max = obd_get_max_mod_rpcs_in_flight(&obd->u.cli);
+       return scnprintf(buf, PAGE_SIZE, "%hu\n", max);
+}
+
+static ssize_t max_mod_rpcs_in_flight_store(struct kobject *kobj,
+                                           struct attribute *attr,
+                                           const char *buffer,
+                                           size_t count)
+{
+       struct dt_device *dt = container_of(kobj, struct dt_device,
+                                           dd_kobj);
+       struct lu_device *lu = dt2lu_dev(dt);
+       struct obd_device *obd = lu->ld_obd;
+       u16 val;
+       int rc;
+
+       rc = kstrtou16(buffer, 10, &val);
+       if (rc)
+               return rc;
+
+       rc = obd_set_max_mod_rpcs_in_flight(&obd->u.cli, val);
+       if (rc)
+               count = rc;
+
+       return count;
+}
+LUSTRE_RW_ATTR(max_mod_rpcs_in_flight);
+
 /**
  * Show maximum number of RPCs in processing allowed
  *
@@ -905,6 +943,27 @@ osp_reserved_mb_high_seq_write(struct file *file, const char __user *buffer,
 }
 LDEBUGFS_SEQ_FOPS(osp_reserved_mb_high);
 
+static int osp_rpc_stats_seq_show(struct seq_file *seq, void *v)
+{
+       struct obd_device *dev = seq->private;
+
+       return obd_mod_rpc_stats_seq_show(&dev->u.cli, seq);
+}
+
+static ssize_t osp_rpc_stats_seq_write(struct file *file,
+                                      const char __user *buf,
+                                      size_t len, loff_t *off)
+{
+       struct seq_file *seq = file->private_data;
+       struct obd_device *dev = seq->private;
+       struct client_obd *cli = &dev->u.cli;
+
+       lprocfs_oh_clear(&cli->cl_mod_rpcs_hist);
+
+       return len;
+}
+LDEBUGFS_SEQ_FOPS(osp_rpc_stats);
+
 /**
  * Show low watermark (in megabytes). If available free space at OST is less
  * than low watermark, object allocation for OST is disabled.
@@ -1020,6 +1079,8 @@ static struct ldebugfs_vars ldebugfs_osp_md_vars[] = {
          .fops =       &osp_import_fops                },
        { .name =       "state",
          .fops =       &osp_state_fops                 },
+       { .name =       "rpc_stats",
+         .fops =       &osp_rpc_stats_fops             },
        { NULL }
 };
 
@@ -1056,6 +1117,7 @@ static struct attribute *osp_md_attrs[] = {
        &lustre_attr_destroys_in_flight.attr,
        &lustre_attr_active.attr,
        &lustre_attr_max_rpcs_in_flight.attr,
+       &lustre_attr_max_mod_rpcs_in_flight.attr,
        &lustre_attr_max_rpcs_in_progress.attr,
        &lustre_attr_maxage.attr,
        &lustre_attr_mdt_conn_uuid.attr,
index c316b52..a8cd2e5 100644 (file)
@@ -1179,8 +1179,6 @@ static int osp_init0(const struct lu_env *env, struct osp_device *osp,
        strcat(osdname, "-osd");
        CDEBUG(D_HA, "%s: connect to %s (%s)\n", obd->obd_name, osdname, src);
 
-       osp_init_rpc_lock(osp);
-
        osp->opd_dt_dev.dd_lu_dev.ld_ops = &osp_lu_ops;
        osp->opd_dt_dev.dd_ops = &osp_dt_ops;
 
index 8930a2c..ae8eb90 100644 (file)
@@ -147,13 +147,6 @@ struct osp_updates {
        struct lu_env           ou_env;
 };
 
-struct osp_rpc_lock {
-       /** Lock protecting in-flight RPC concurrency. */
-       struct mutex            rpcl_mutex;
-       /** Used for MDS/RPC load testing purposes. */
-       unsigned int            rpcl_fakes;
-};
-
 struct osp_device {
        struct dt_device                 opd_dt_dev;
        /* corresponded OST index */
@@ -176,8 +169,6 @@ struct osp_device {
        u64                             opd_last_id;
        struct lu_fid                   opd_gap_start_fid;
        int                              opd_gap_count;
-       /* connection to OST */
-       struct osp_rpc_lock              opd_rpc_lock;
        struct obd_device               *opd_obd;
        struct obd_export               *opd_exp;
        struct obd_connect_data         *opd_connect_data;
@@ -509,81 +500,6 @@ static inline struct seq_server_site *osp_seq_site(struct osp_device *osp)
        return osp->opd_dt_dev.dd_lu_dev.ld_site->ld_seq_site;
 }
 
-/**
- * Serializes in-flight MDT-modifying RPC requests to preserve idempotency.
- *
- * This mutex is used to implement execute-once semantics on the MDT.
- * The MDT stores the last transaction ID and result for every client in
- * its last_rcvd file. If the client doesn't get a reply, it can safely
- * resend the request and the MDT will reconstruct the reply being aware
- * that the request has already been executed. Without this lock,
- * execution status of concurrent in-flight requests would be
- * overwritten.
- *
- * This imlpementation limits the extent to which we can keep a full pipeline
- * of in-flight requests from a single client.  This limitation can be
- * overcome by allowing multiple slots per client in the last_rcvd file,
- * see LU-6864.
- */
-#define OSP_FAKE_RPCL_IT ((void *)0x2c0012bfUL)
-
-static inline void osp_init_rpc_lock(struct osp_device *osp)
-{
-       struct osp_rpc_lock *lck = &osp->opd_rpc_lock;
-
-       mutex_init(&lck->rpcl_mutex);
-       lck->rpcl_fakes = 0;
-}
-
-static inline void osp_get_rpc_lock(struct osp_device *osp)
-{
-       struct osp_rpc_lock *lck = &osp->opd_rpc_lock;
-
-       /* This would normally block until the existing request finishes.
-        * If fail_loc is set it will block until the regular request is
-        * done, then increment rpcl_fakes.  Once that is non-zero it
-        * will only be cleared when all fake requests are finished.
-        * Only when all fake requests are finished can normal requests
-        * be sent, to ensure they are recoverable again.
-        */
- again:
-       mutex_lock(&lck->rpcl_mutex);
-
-       if (CFS_FAIL_CHECK_QUIET(OBD_FAIL_MDC_RPCS_SEM) ||
-           CFS_FAIL_CHECK_QUIET(OBD_FAIL_OSP_RPCS_SEM)) {
-               lck->rpcl_fakes++;
-               mutex_unlock(&lck->rpcl_mutex);
-
-               return;
-       }
-
-       /* This will only happen when the CFS_FAIL_CHECK() was just turned
-        * off but there are still requests in progress.  Wait until they
-        * finish.  It doesn't need to be efficient in this extremely rare
-        * case, just have low overhead in the common case when it isn't true.
-        */
-       if (unlikely(lck->rpcl_fakes)) {
-               mutex_unlock(&lck->rpcl_mutex);
-               schedule_timeout_uninterruptible(cfs_time_seconds(1) / 4);
-
-               goto again;
-       }
-}
-
-static inline void osp_put_rpc_lock(struct osp_device *osp)
-{
-       struct osp_rpc_lock *lck = &osp->opd_rpc_lock;
-
-       if (lck->rpcl_fakes) { /* OBD_FAIL_OSP_RPCS_SEM */
-               mutex_lock(&lck->rpcl_mutex);
-
-               if (lck->rpcl_fakes) /* check again under lock */
-                       lck->rpcl_fakes--;
-       }
-
-       mutex_unlock(&lck->rpcl_mutex);
-}
-
 static inline int osp_fid_diff(const struct lu_fid *fid1,
                               const struct lu_fid *fid2)
 {
index 95777ed..302d172 100644 (file)
@@ -1187,10 +1187,10 @@ static int osp_send_update_req(const struct lu_env *env,
                 * env */
                args->oaua_update_env = env;
                if (osp->opd_connect_mdt)
-                       osp_get_rpc_lock(osp);
+                       ptlrpc_get_mod_rpc_slot(req);
                rc = ptlrpc_queue_wait(req);
                if (osp->opd_connect_mdt)
-                       osp_put_rpc_lock(osp);
+                       ptlrpc_put_mod_rpc_slot(req);
 
                /* We use rq_queued_time to distinguish between local
                 * and remote -ENOMEM. */
index 57c0d91..d530f82 100644 (file)
@@ -64,31 +64,32 @@ typedef void (*out_reconstruct_t)(const struct lu_env *env,
                                  struct object_update_reply *reply,
                                  int index);
 
-static inline bool out_check_resent(struct ptlrpc_request *req)
+static inline bool out_check_resent(struct ptlrpc_request *req,
+                                   struct tg_reply_data *trd)
 {
+       struct lsd_reply_data *lrd;
+       bool reconstruct = false;
+
        if (likely(!(lustre_msg_get_flags(req->rq_reqmsg) & MSG_RESENT)))
                return false;
 
-       if (req_xid_is_last(req)) {
-               struct lsd_client_data *lcd;
-
-               /* XXX this does not support mulitple transactions yet, i.e.
-                * only 1 update RPC each time betwee MDTs */
-               lcd = req->rq_export->exp_target_data.ted_lcd;
+       if (req_can_reconstruct(req, trd)) {
+               lrd = &trd->trd_reply;
+               req->rq_transno = lrd->lrd_transno;
+               req->rq_status = lrd->lrd_result;
 
-               req->rq_transno = lcd->lcd_last_transno;
-               req->rq_status = lcd->lcd_last_result;
                if (req->rq_status != 0)
                        req->rq_transno = 0;
                lustre_msg_set_transno(req->rq_repmsg, req->rq_transno);
                lustre_msg_set_status(req->rq_repmsg, req->rq_status);
 
                DEBUG_REQ(D_HA, req, "reconstruct resent RPC");
-               return true;
+               reconstruct = true;
+       } else {
+               DEBUG_REQ(D_HA, req, "no reply for RESENT req");
        }
-       DEBUG_REQ(D_HA, req, "reprocess RESENT req, last_xid is %lld",
-                 req->rq_export->exp_target_data.ted_lcd->lcd_last_xid);
-       return false;
+
+       return reconstruct;
 }
 
 static int out_create(struct tgt_session_info *tsi)
@@ -956,6 +957,7 @@ int out_handle(struct tgt_session_info *tsi)
        struct object_update            *update;
        struct object_update_reply      *reply;
        struct ptlrpc_bulk_desc         *desc = NULL;
+       struct tg_reply_data *trd = NULL;
        void                            **update_bufs;
        int                             current_batchid = -1;
        __u32                           update_buf_count;
@@ -1109,7 +1111,11 @@ int out_handle(struct tgt_session_info *tsi)
        tti->tti_u.update.tti_update_reply = reply;
        tti->tti_mult_trans = !req_is_replay(tgt_ses_req(tsi));
 
-       need_reconstruct = out_check_resent(pill->rc_req);
+       OBD_ALLOC_PTR(trd);
+       if (!trd)
+               GOTO(out_free, rc = -ENOMEM);
+
+       need_reconstruct = out_check_resent(pill->rc_req, trd);
 
        /* Walk through updates in the request to execute them */
        for (i = 0; i < update_buf_count; i++) {
@@ -1241,6 +1247,9 @@ out_free:
                OBD_FREE_PTR_ARRAY(update_bufs, update_buf_count);
        }
 
+       if (trd)
+               OBD_FREE_PTR(trd);
+
        if (desc != NULL)
                ptlrpc_free_bulk(desc);
 
index ff01194..a6763a2 100644 (file)
@@ -49,7 +49,6 @@
 /mcreate
 /memhog
 /mirror_io
-/mkdirmany
 /mlink
 /mmap_cat
 /mmap_mknod_test
index 2f9418d..567aa36 100644 (file)
@@ -71,7 +71,7 @@ THETESTS += createmany chownmany statmany multifstat createtest
 THETESTS += opendirunlink opendevunlink unlinkmany checkstat
 THETESTS += statone runas openfile smalliomany
 THETESTS += small_write multiop ll_sparseness_verify
-THETESTS += ll_sparseness_write mrename ll_dirstripe_verify mkdirmany
+THETESTS += ll_sparseness_write mrename ll_dirstripe_verify
 THETESTS += openfilleddirunlink rename_many memhog euid_access
 THETESTS += rw_seq_cst_vs_drop_caches
 THETESTS += mmap_sanity writemany reads flocks_test flock_deadlock
@@ -131,6 +131,7 @@ mirror_io_LDADD = $(LIBLUSTREAPI)
 ll_dirstripe_verify_LDADD = $(LIBLUSTREAPI)
 lov_getstripe_old_LDADD = $(LIBLUSTREAPI)
 flocks_test_LDADD = $(LIBLUSTREAPI) $(PTHREAD_LIBS)
+createmany_LDADD=$(LIBLUSTREAPI)
 create_foreign_dir_LDADD = $(LIBLUSTREAPI)
 check_fallocate_LDADD = $(LIBLUSTREAPI)
 fsx_LDADD = $(LIBLUSTREAPI)
index 732ae48..7367d0c 100644 (file)
 #include <time.h>
 #include <unistd.h>
 
+#include <linux/lustre/lustre_user.h>
+#include <lustre/lustreapi.h>
+
 static void usage(const char *prog)
 {
        printf("usage: %s {-o [-k]|-m|-d|-l<tgt>} [-u[<unlinkfmt>]] "
-              "[-t seconds] filenamefmt [[start] count]\n", prog);
-       printf("\t-l\tlink files to existing <tgt> file\n"
+              "[-i mdt_index] [-t seconds] filenamefmt [[start] count]\n", prog);
+       printf("\t-i\tMDT to create the directories on\n"
+              "\t-l\tlink files to existing <tgt> file\n"
               "\t-m\tmknod regular files (don't create OST objects)\n"
               "\t-o\topen+create files with path and printf format\n"
               "\t-k\t    keep files open until all files are opened\n"
@@ -83,6 +87,8 @@ int main(int argc, char ** argv)
        bool do_open = false, do_keep = false, do_link = false;
        bool do_unlink = false, do_mknod = false, do_mkdir = false;
        bool do_rmdir = false;
+       int stripe_pattern = LMV_HASH_TYPE_FNV_1A_64;
+       int stripe_offset = -1, stripe_count = 1;
        char *filename, *progname;
        char *fmt = NULL, *fmt_unlink = NULL, *tgt = NULL;
        char *endp = NULL;
@@ -109,11 +115,19 @@ int main(int argc, char ** argv)
        else
                progname = argv[0];
 
-       while ((c = getopt(argc, argv, "dl:kmor::t:u::")) != -1) {
+       while ((c = getopt(argc, argv, "i:dl:kmor::t:u::")) != -1) {
                switch (c) {
                case 'd':
                        do_mkdir = true;
                        break;
+               case 'i':
+                       stripe_offset = strtoul(optarg, &endp, 0);
+                       if (*endp != '\0') {
+                               fprintf(stderr, "invalid MDT index '%s'\n",
+                                       optarg);
+                               return 1;
+                       }
+                       break;
                case 'k':
                        do_keep = true;
                        break;
@@ -198,12 +212,24 @@ int main(int argc, char ** argv)
                                break;
                        }
                } else if (do_mkdir) {
-                       rc = mkdir(filename, 0755);
-                       if (rc) {
-                               printf("mkdir(%s) error: %s\n",
-                                      filename, strerror(errno));
-                               rc = errno;
-                               break;
+                       if (stripe_offset != -1) {
+                               rc = llapi_dir_create_pool(filename, 0755,
+                                                          stripe_offset, stripe_count,
+                                                          stripe_pattern, NULL);
+                               if (rc) {
+                                       printf("llapi_dir_create_pool(%s) error: %s\n",
+                                              filename, strerror(-rc));
+                                       rc = errno;
+                                       break;
+                               }
+                       } else {
+                               rc = mkdir(filename, 0755);
+                               if (rc) {
+                                       printf("mkdir(%s) error: %s\n",
+                                              filename, strerror(errno));
+                                       rc = errno;
+                                       break;
+                               }
                        }
                } else if (do_mknod) {
                        rc = mknod(filename, S_IFREG | 0444, 0);
diff --git a/lustre/tests/mkdirmany.c b/lustre/tests/mkdirmany.c
deleted file mode 100644 (file)
index 8a57df7..0000000
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * GPL HEADER START
- *
- * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
- *
- * This program is free software; you can redistribute it and/or modify
- * it under the terms of the GNU General Public License version 2 only,
- * as published by the Free Software Foundation.
- *
- * This program is distributed in the hope that it will be useful, but
- * WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
- * General Public License version 2 for more details (a copy is included
- * in the LICENSE file that accompanied this code).
- *
- * You should have received a copy of the GNU General Public License
- * version 2 along with this program; If not, see
- * http://www.gnu.org/licenses/gpl-2.0.html
- *
- * GPL HEADER END
- */
-/*
- * Copyright (c) 2002, 2010, Oracle and/or its affiliates. All rights reserved.
- * Use is subject to license terms.
- */
-/*
- * This file is part of Lustre, http://www.lustre.org/
- */
-
-#include <stdio.h>
-#include <sys/types.h>
-#include <sys/stat.h>
-#include <time.h>
-#include <errno.h>
-#include <string.h>
-#include <fcntl.h>
-#include <unistd.h>
-#include <stdlib.h>
-
-int main(int argc, char ** argv)
-{
-       int i, rc = 0, count;
-       char dirname[4096];
-
-       if (argc < 3) {
-               printf("Usage %s dirnamebase count\n", argv[0]);
-               return 1;
-       }
-
-       if (strlen(argv[1]) > 4080) {
-               printf("name too long\n");
-               return 1;
-       }
-
-       count = strtoul(argv[2], NULL, 0);
-
-       for (i = 0; i < count; i++) {
-               sprintf(dirname, "%s-%d", argv[1], i);
-               rc = mkdir(dirname, 0444);
-               if (rc) {
-                       printf("mkdir(%s) error: %s\n",
-                              dirname, strerror(errno));
-                       break;
-               }
-               if ((i % 10000) == 0)
-                   printf(" - created %d (time %ld)\n", i, time(0));
-       }
-       return rc;
-}
index f768f58..9a8de64 100755 (executable)
@@ -17844,7 +17844,7 @@ test_181() { # bug 22177
 }
 run_test 181 "Test open-unlinked dir ========================"
 
-test_182() {
+test_182a() {
        local fcount=1000
        local tcount=10
 
@@ -17870,7 +17870,75 @@ test_182() {
 
        rm -rf $DIR/$tdir
 }
-run_test 182 "Test parallel modify metadata operations ================"
+run_test 182a "Test parallel modify metadata operations from mdc"
+
+test_182b() {
+       [ $PARALLEL == "yes" ] && skip "skip parallel run" && return
+       [ $MDSCOUNT -lt 2 ] && skip "needs >= 2 MDTs" && return
+       local dcount=1000
+       local tcount=10
+       local stime
+       local etime
+       local delta
+
+       do_facet mds1 $LCTL list_param \
+               osp.$FSNAME-MDT*-osp-MDT*.rpc_stats ||
+               skip "MDS lacks parallel RPC handling"
+
+       $LFS mkdir -i 0 $DIR/$tdir || error "creating dir $DIR/$tdir"
+
+       rpc_count=$(do_facet mds1 $LCTL get_param -n \
+                   osp.$FSNAME-MDT0001-osp-MDT0000.max_mod_rpcs_in_flight)
+
+       stime=$(date +%s)
+       createmany -i 0 -d $DIR/$tdir/t- $tcount
+
+       for (( i = 0; i < $tcount; i++ )) ; do
+               createmany -i 0 -d $DIR/$tdir/t-$i/d- 0 $dcount &
+       done
+       wait
+       etime=$(date +%s)
+       delta=$((etime - stime))
+       echo "Time for file creation $delta sec for $rpc_count parallel RPCs"
+
+       stime=$(date +%s)
+       for (( i = 0; i < $tcount; i++ )) ; do
+               unlinkmany -d $DIR/$tdir/$i/d- $dcount &
+       done
+       wait
+       etime=$(date +%s)
+       delta=$((etime - stime))
+       echo "Time for file removal $delta sec for $rpc_count parallel RPCs"
+
+       rm -rf $DIR/$tdir
+
+       $LFS mkdir -i 0 $DIR/$tdir || error "creating dir $DIR/$tdir"
+
+       do_facet mds1 $LCTL set_param osp.$FSNAME-MDT0001-osp-MDT0000.max_mod_rpcs_in_flight=1
+
+       stime=$(date +%s)
+       createmany -i 0 -d $DIR/$tdir/t- $tcount
+
+       for (( i = 0; i < $tcount; i++ )) ; do
+               createmany -i 0 -d $DIR/$tdir/t-$i/d- 0 $dcount &
+       done
+       wait
+       etime=$(date +%s)
+       delta=$((etime - stime))
+       echo "Time for file creation $delta sec for 1 RPC sent at a time"
+
+       stime=$(date +%s)
+       for (( i = 0; i < $tcount; i++ )) ; do
+               unlinkmany -d $DIR/$tdir/t-$i/d- $dcount &
+       done
+       wait
+       etime=$(date +%s)
+       delta=$((etime - stime))
+       echo "Time for file removal $delta sec for 1 RPC sent at a time"
+
+       do_facet mds1 $LCTL set_param osp.$FSNAME-MDT0001-osp-MDT0000.max_mod_rpcs_in_flight=$rpc_count
+}
+run_test 182b "Test parallel modify metadata operations from osp"
 
 test_183() { # LU-2275
        [ $PARALLEL == "yes" ] && skip "skip parallel run"
@@ -20886,7 +20954,7 @@ test_244b()
 }
 run_test 244b "multi-threaded write with group lock"
 
-test_245() {
+test_245a() {
        local flagname="multi_mod_rpcs"
        local connect_data_name="max_mod_rpcs"
        local out
@@ -20909,7 +20977,35 @@ test_245() {
        echo "$out" | grep -qw $connect_data_name ||
                error "import should have connect data $connect_data_name"
 }
-run_test 245 "check mdc connection flag/data: multiple modify RPCs"
+run_test 245a "check mdc connection flag/data: multiple modify RPCs"
+
+test_245b() {
+       local flagname="multi_mod_rpcs"
+       local connect_data_name="max_mod_rpcs"
+       local out
+
+       remote_mds_nodsh && skip "remote MDS with nodsh" && return
+       [[ $MDSCOUNT -ge 2 ]] || skip "needs >= 2 MDTs"
+
+       # check if multiple modify RPCs flag is set
+       out=$(do_facet mds1 \
+               $LCTL get_param osp.$FSNAME-MDT0001-osp-MDT0000.import |
+               grep "connect_flags:")
+       echo "$out"
+
+       if [[ "$out" =~ $flagname ]]; then
+               echo "connect flag $flagname is not set"
+               return 0
+       fi
+
+       # check if multiple modify RPCs data is set
+       out=$(do_facet mds1 \
+               $LCTL get_param osp.$FSNAME-MDT0001-osp-MDT0000.import)
+
+       [[ "$out" =~ $connect_data_name ]] ||
+               error "import should have connect data $connect_data_name"
+}
+run_test 245b "check osp connection flag/data: multiple modify RPCs"
 
 cleanup_247() {
        local submount=$1