Whamcloud - gitweb
Branch HEAD
authorbobijam <bobijam>
Tue, 12 Aug 2008 02:24:12 +0000 (02:24 +0000)
committerbobijam <bobijam>
Tue, 12 Aug 2008 02:24:12 +0000 (02:24 +0000)
b=16573
o=adilger
i=johann, nathan

Description: Export bytes_read/bytes_write count on OSC/OST.

lustre/ChangeLog
lustre/include/lprocfs_status.h
lustre/include/lustre_net.h
lustre/osc/osc_request.c
lustre/ost/ost_handler.c
lustre/ptlrpc/lproc_ptlrpc.c
lustre/tests/sanity.sh

index f637574..5a23afc 100644 (file)
@@ -12,15 +12,19 @@ tbd  Sun Microsystems, Inc.
        * RHEL 4 and RHEL 5/SLES 10 clients behaves differently on 'cd' to a
         removed cwd "./" (refer to Bugzilla 14399).
 
+Severity   : enhancement
+Bugzilla   : 16573
+Description: Export bytes_read/bytes_write count on OSC/OST.
+
 Severity   : normal
-Bugzil   : 16237
+Bugzilla   : 16237
 Description: Early reply size mismatch, MGC loses connection
 Details    : Apply the MGS_CONNECT_SUPPORTED mask at reconnect time so
              the connect flags are properly negotiated.
 
 Severity   : normal
 Bugzilla   : 16006
-Description: Properly propagate oinfo flags from lov to osc for statfs   
+Description: Properly propagate oinfo flags from lov to osc for statfs
 Details    : restore missing copy oi_flags to lov requests.
 
 Severity   : enhancement
@@ -32,7 +36,7 @@ Details    : included man pages for llobdstat(8), llstat(8),
 
 Severity   : enhancement
 Bugzilla   : 16091
-Description: configure's --enable-quota should check the 
+Description: configure's --enable-quota should check the
            : kernel .config for CONFIG_QUOTA
 Details    : configure is terminated if --enable-quota is passed but
            : no quota support is in kernel
@@ -81,7 +85,7 @@ Severity   : normal
 Bugzilla   : 15825
 Description: Kernel BUG tries to release flock
 Details    : Lustre does not destroy flock lock before last reference goes
-            away. So always drop flock locks when client is evicted and 
+            away. So always drop flock locks when client is evicted and
             perform unlock regardless of successfulness of speaking to MDS.
 
 Severity   : normal
@@ -995,7 +999,7 @@ Details    : Console messages can now be disabled via lnet.printk.
 
 Severity   : normal
 Bugzilla   : 14614
-Description: User code with malformed file open parameter crashes client node 
+Description: User code with malformed file open parameter crashes client node
 Details    : Before packing join_file req, all the related reference should be
             checked carefully in case some malformed flags cause fake
             join_file req on client.
@@ -1126,7 +1130,7 @@ Severity   : normal
 Bugzilla   : 14257
 Description: LASSERT on MDS when client holding flock lock dies
 Details    : ldlm pool logic depends on number of granted locks equal to
-            number of released locks which is not true for flock locks, so 
+            number of released locks which is not true for flock locks, so
             just exclude such locks from consideration.
 
 Severity   : normal
@@ -1191,7 +1195,7 @@ Severity   : normal
 Bugzilla   : 15574
 Description: MDS LBUG: ASSERTION(!IS_ERR(dchild))
 Details    : Change LASSERTs to client eviction (i.e. abort client's recovery)
-            because LASSERT on both the data supplied by a client, and the data 
+            because LASSERT on both the data supplied by a client, and the data
             on disk is dangerous and incorrect.
 
 Severity   : enhancement
@@ -1331,7 +1335,7 @@ Severity   : minor
 Frequency  : at statup only
 Bugzilla   : 12860
 Description: mds_lov_synchronize race leads to various problems
-Details    : simultaneous MDT->OST connections at startup can cause the 
+Details    : simultaneous MDT->OST connections at startup can cause the
             sync to abort, leaving the OSC in a bad state.
 
 Severity   : enhancement
@@ -1500,7 +1504,7 @@ Details    : When osc reconnect ost, OST(filter) should check whether it
             to update the client grant space info.
 
 Severity   : normal
-Frequency  : when client reconnect to OST 
+Frequency  : when client reconnect to OST
 Bugzilla   : 11662
 Description: Grant Leak when osc do resend and replay bulk write
 Details           : When osc reconnect to OST, OST(filter)should clear grant info of
@@ -1509,33 +1513,33 @@ Details    : When osc reconnect to OST, OST(filter)should clear grant info of
             these of resend/replay write req.
 
 Severity   : normal
-Frequency  : rare 
+Frequency  : rare
 Bugzilla   : 11662
 Description: Grant space more than avaiable left space sometimes.
 Details           : When then OST is about to be full, if two bulk writing from
             different clients came to OST. Accord the avaliable space of the
             OST, the first req should be permitted, and the second one
-            should be denied by ENOSPC. But if the seconde arrived before 
+            should be denied by ENOSPC. But if the seconde arrived before
             the first one is commited. The OST might wrongly permit second
             writing, which will cause grant space > avaiable space.
 
 Severity   : normal
-Frequency  : when client is evicted 
+Frequency  : when client is evicted
 Bugzilla   : 12371
 Description: Grant might be wrongly erased when osc is evicted by OST
-Details    : when the import is evicted by server, it will fork another 
-            thread ptlrpc_invalidate_import_thread to invalidate the 
-            import, where the grant will be set to 0.  While the original 
-            thread will update the grant it got when connecting. So if 
-            the former happened latter, the grant will be wrongly errased 
+Details    : when the import is evicted by server, it will fork another
+            thread ptlrpc_invalidate_import_thread to invalidate the
+            import, where the grant will be set to 0.  While the original
+            thread will update the grant it got when connecting. So if
+            the former happened latter, the grant will be wrongly errased
             because of this race.
 
 Severity   : normal
-Frequency  : rare 
+Frequency  : rare
 Bugzilla   : 12401
-Description: Checking Stale with correct fid 
-Details    : ll_revalidate_it should uses de_inode instead of op_data.fid2 
-            to check whether it is stale, because sometimes, we want the 
+Description: Checking Stale with correct fid
+Details    : ll_revalidate_it should uses de_inode instead of op_data.fid2
+            to check whether it is stale, because sometimes, we want the
             enqueue happened anyway, and op_data.fid2 will not be initialized.
 
 Severity   : enhancement
@@ -1551,29 +1555,29 @@ Details    : size of struct ll_inode_info is to big for union inode.u and this
             can be cause of random memory corruption.
 
 Severity   : normal
-Frequency  : rare 
+Frequency  : rare
 Bugzilla   : 10818
 Description: Memory leak in recovery
 Details    : Lov_mds_md was not free in an error handler in mds_create_object.
-            It should also check obd_fail before fsfilt_start, otherwise if 
+            It should also check obd_fail before fsfilt_start, otherwise if
             fsfilt_start return -EROFS,(failover mds during mds recovery).
-            then the req will return with repmsg->transno = 0 and rc = EROFS. 
+            then the req will return with repmsg->transno = 0 and rc = EROFS.
             and we met hit the assert LASSERT(req->rq_reqmsg->transno ==
-            req->rq_repmsg->transno) in ptlrpc_replay_interpret.  Fcc should 
+            req->rq_repmsg->transno) in ptlrpc_replay_interpret.  Fcc should
             be freed no matter whether fsfilt_commit success or not.
 
 Severity   : minor
 Frequency  : only with huge count clients
 Bugzilla   : 11817
-Description: Prevents from taking the superblock lock in llap_from_page for 
+Description: Prevents from taking the superblock lock in llap_from_page for
             a soon died page.
-Details    : using LL_ORIGIN_REMOVEPAGE origin flag instead of LL_ORIGIN_UNKNOW 
-            for llap_from_page call in ll_removepage prevents from taking the 
+Details    : using LL_ORIGIN_REMOVEPAGE origin flag instead of LL_ORIGIN_UNKNOW
+            for llap_from_page call in ll_removepage prevents from taking the
             superblock lock for a soon died page.
 
 Severity   : normal
 Frequency  : rare
-Bugzilla   : 11935 
+Bugzilla   : 11935
 Description: Not check open intent error before release open handle
 Details    : in some rare cases, the open intent error is not checked before
             release open handle, which may cause
@@ -1582,9 +1586,9 @@ Details    : in some rare cases, the open intent error is not checked before
 
 Severity   : normal
 Frequency  : rare
-Bugzilla   : 12556 
-Description: Set cat log bitmap only after create log success. 
-Details    : in some rare cases, the cat log bitmap is set too early. and it 
+Bugzilla   : 12556
+Description: Set cat log bitmap only after create log success.
+Details    : in some rare cases, the cat log bitmap is set too early. and it
             should be set only after create log success.
 
 Severity   : major
@@ -1601,11 +1605,11 @@ Details    : Insert cond_resched to give other threads a chance to use some CPU
 
 Severity   : normal
 Frequency  : rare
-Bugzilla   : 12086 
-Description: the cat log was not initialized in recovery 
+Bugzilla   : 12086
+Description: the cat log was not initialized in recovery
 Details    : When mds(mgs) do recovery, the tgt_count might be zero, so the
             unlink log on mds will not be initialized until mds post
-            recovery. And also in mds post recovery, the unlink log will 
+            recovery. And also in mds post recovery, the unlink log will
             initialization will be done asynchronausly, so there will be race
             between add unlink log and unlink log initialization.
 
@@ -1626,7 +1630,7 @@ Details    : imp_lock should be held while iterating over imp_sending_list for
 Severity   : normal
 Bugzilla   : 12689
 Description: replay-single.sh test 52 fails
-Details    : A lock's skiplist need to be cleanup when it being unlinked 
+Details    : A lock's skiplist need to be cleanup when it being unlinked
             from its resource list.
 
 Severity   : normal
@@ -1655,7 +1659,7 @@ Details    : If a OST has no remain object, system will block on the creating
 Severity   : enhancement
 Bugzilla   : 12702
 Description: refine locking for avoid write wrong info into lov_objid file
-Details    : fix possible races with add new target and write/update data in 
+Details    : fix possible races with add new target and write/update data in
             lov_objid file.
 
 --------------------------------------------------------------------------------
@@ -1689,7 +1693,7 @@ Details    : The __iget() symbol export is missing.  To avoid the need for
         special upgrade step is needed. Please read the
         user documentation before upgrading any part of a live system.
        * WIRE PROTOCOL CHANGE from previous 1.6 beta versions.  This
-         version will not interoperate with 1.6 betas before beta5 (1.5.95).  
+         version will not interoperate with 1.6 betas before beta5 (1.5.95).
        * WARNING: Lustre configuration and startup changes are required with
         this release.  See https://mail.clusterfs.com/wikis/lustre/MountConf
         for details.
@@ -1728,7 +1732,7 @@ Bugzilla   : 9862
 Description: optimized stripe assignment
 Details    : stripe assignments are now made based on ost space available,
             ost previous usage, and OSS previous usage, in order to try
-            to optimize storage space and networking resources. 
+            to optimize storage space and networking resources.
        
 Severity   : enhancement
 Bugzilla   : 4226
@@ -1765,16 +1769,16 @@ Severity   : enhancement
 Bugzilla   : 22484
 Description: client read/write statistics
 Details    : Add client read/write call usage stats for performance
-            analysis of user processes. 
+            analysis of user processes.
             /proc/fs/lustre/llite/*/offset_stats shows non-sequential
             file access. extents_stats shows chunk size distribution.
             extents_stats_per_process show chunk size distribution per
-            user process. 
+            user process.
 
 Severity   : enhancement
 Bugzilla   : 22486
 Description: mds statistics
-Details    : Add detailed mds operations statistics in  
+Details    : Add detailed mds operations statistics in
             /proc/fs/lustre/mds/*/stats.
 
 Severity   : minor
@@ -4014,7 +4018,7 @@ Severity   : Minor
 Frequency  : Rare
 Bugzilla   : 11248
 Description: merge and cleanup kernel patches.
-Details    : 
+Details    :
 
 -----------------------------------------------------------------------------
 
index ffcd711..32d3248 100644 (file)
@@ -290,6 +290,8 @@ enum {
         MDS_REINT_SETATTR,
         MDS_REINT_RENAME,
         MDS_REINT_UNLINK,
+        BRW_READ_BYTES,
+        BRW_WRITE_BYTES,
         EXTRA_LAST_OPC
 };
 
@@ -362,10 +364,10 @@ extern void lprocfs_counter_sub(struct lprocfs_stats *stats, int idx,
 #define lprocfs_counter_decr(stats, idx) \
         lprocfs_counter_sub(stats, idx, 1)
 
-extern __s64 lprocfs_read_helper(struct lprocfs_counter *lc, 
+extern __s64 lprocfs_read_helper(struct lprocfs_counter *lc,
                                  enum lprocfs_fields_flags field);
-static inline __u64 lprocfs_stats_collector(struct lprocfs_stats *stats, 
-                                            int idx, 
+static inline __u64 lprocfs_stats_collector(struct lprocfs_stats *stats,
+                                            int idx,
                                             enum lprocfs_fields_flags field)
 {
         __u64 ret = 0;
@@ -382,7 +384,7 @@ extern struct lprocfs_stats *lprocfs_alloc_stats(unsigned int num,
                                                  enum lprocfs_stats_flags flags);
 extern void lprocfs_clear_stats(struct lprocfs_stats *stats);
 extern void lprocfs_free_stats(struct lprocfs_stats **stats);
-extern void lprocfs_init_ops_stats(int num_private_stats, 
+extern void lprocfs_init_ops_stats(int num_private_stats,
                                    struct lprocfs_stats *stats);
 extern int lprocfs_alloc_obd_stats(struct obd_device *obddev,
                                    unsigned int num_private_stats);
@@ -431,13 +433,13 @@ extern cfs_proc_dir_entry_t *lprocfs_srch(cfs_proc_dir_entry_t *root,
 extern int lprocfs_obd_setup(struct obd_device *obd, struct lprocfs_vars *list);
 extern int lprocfs_obd_cleanup(struct obd_device *obd);
 extern int lprocfs_add_simple(struct proc_dir_entry *root, char *name,
-                              cfs_read_proc_t *read_proc, 
+                              cfs_read_proc_t *read_proc,
                               cfs_write_proc_t *write_proc,
                               void *data);
 extern void lprocfs_free_per_client_stats(struct obd_device *obd);
 extern struct file_operations lprocfs_evict_client_fops;
 
-extern int lprocfs_seq_create(cfs_proc_dir_entry_t *parent, char *name, 
+extern int lprocfs_seq_create(cfs_proc_dir_entry_t *parent, char *name,
                               mode_t mode, struct file_operations *seq_fops,
                               void *data);
 extern int lprocfs_obd_seq_create(struct obd_device *dev, char *name,
@@ -504,7 +506,7 @@ extern int lprocfs_write_helper(const char *buffer, unsigned long count,
                                 int *val);
 extern int lprocfs_write_frac_helper(const char *buffer, unsigned long count,
                                      int *val, int mult);
-extern int lprocfs_read_frac_helper(char *buffer, unsigned long count, 
+extern int lprocfs_read_frac_helper(char *buffer, unsigned long count,
                                     long val, int mult);
 extern int lprocfs_write_u64_helper(const char *buffer, unsigned long count,
                                     __u64 *val);
@@ -550,7 +552,7 @@ extern struct rw_semaphore _lprocfs_lock;
         up_write(&_lprocfs_lock);       \
 } while(0)
 
-/* You must use these macros when you want to refer to 
+/* You must use these macros when you want to refer to
  * the import in a client obd_device for a lprocfs entry */
 #define LPROCFS_CLIMP_CHECK(obd) do {           \
         typecheck(struct obd_device *, obd);    \
@@ -564,8 +566,8 @@ extern struct rw_semaphore _lprocfs_lock;
         up_read(&(obd)->u.cli.cl_sem);
 
 
-/* write the name##_seq_show function, call LPROC_SEQ_FOPS_RO for read-only 
-  proc entries; otherwise, you will define name##_seq_write function also for 
+/* write the name##_seq_show function, call LPROC_SEQ_FOPS_RO for read-only
+  proc entries; otherwise, you will define name##_seq_write function also for
   a read-write proc entry, and then call LPROC_SEQ_SEQ instead. Finally,
   call lprocfs_obd_seq_create(obd, filename, 0444, &name#_fops, data); */
 #define __LPROC_SEQ_FOPS(name, custom_seq_write)                           \
@@ -616,8 +618,8 @@ static inline void lprocfs_counter_init(struct lprocfs_stats *stats,
                                         const char *name, const char *units)
 { return; }
 
-static inline __u64 lc_read_helper(struct lprocfs_counter *lc, 
-                                   enum lprocfs_fields_flags field) 
+static inline __u64 lc_read_helper(struct lprocfs_counter *lc,
+                                   enum lprocfs_fields_flags field)
 { return 0; }
 
 static inline struct lprocfs_stats* lprocfs_alloc_stats(unsigned int num,
@@ -631,7 +633,7 @@ static inline int lprocfs_register_stats(cfs_proc_dir_entry_t *root,
                                             const char *name,
                                             struct lprocfs_stats *stats)
 { return 0; }
-static inline void lprocfs_init_ops_stats(int num_private_stats, 
+static inline void lprocfs_init_ops_stats(int num_private_stats,
                                           struct lprocfs_stats *stats)
 { return; }
 static inline int lprocfs_alloc_obd_stats(struct obd_device *obddev,
@@ -759,7 +761,7 @@ int lprocfs_counter_write(struct file *file, const char *buffer,
                           unsigned long count, void *data) { return 0; }
 
 static inline
-__u64 lprocfs_stats_collector(struct lprocfs_stats *stats, int idx, 
+__u64 lprocfs_stats_collector(struct lprocfs_stats *stats, int idx,
                                enum lprocfs_fields_flags field)
 { return (__u64)0; }
 
index 641dc09..2a2d129 100644 (file)
@@ -483,7 +483,7 @@ static inline void lustre_set_rep_swabbed(struct ptlrpc_request *req, int index)
 }
 
 static inline int lustre_req_swabbed(struct ptlrpc_request *req, int index)
-{ 
+{
         LASSERT(index < sizeof(req->rq_req_swab_mask) * 8);
         return req->rq_req_swab_mask & (1 << index);
 }
@@ -602,7 +602,7 @@ struct ptlrpc_bulk_desc {
 
 struct ptlrpc_thread {
 
-        struct list_head t_link; /* active threads for service, from svc->srv_threads */
+        struct list_head t_link; /* active threads in svc->srv_threads */
 
         void *t_data;            /* thread-private data (preallocated memory) */
         __u32 t_flags;
@@ -741,7 +741,7 @@ struct ptlrpcd_ctl {
         char                        pc_name[16];
 #ifndef __KERNEL__
         /**
-         * Async rpcs flag to make sure that ptlrpcd_check() is called only 
+         * Async rpcs flag to make sure that ptlrpcd_check() is called only
          * once.
          */
         int                         pc_recurred;
@@ -771,7 +771,7 @@ enum ptlrpcd_ctl_flags {
          */
         LIOD_STOP        = 1 << 1,
         /**
-         * Ptlrpc thread stop force flag. This will cause also 
+         * Ptlrpc thread stop force flag. This will cause also
          * aborting any inflight rpcs handled by thread.
          */
         LIOD_STOP_FORCE  = 1 << 2
@@ -1144,12 +1144,11 @@ const char* ll_opcode2str(__u32 opcode);
 #ifdef LPROCFS
 void ptlrpc_lprocfs_register_obd(struct obd_device *obd);
 void ptlrpc_lprocfs_unregister_obd(struct obd_device *obd);
-void ptlrpc_lprocfs_brw(struct ptlrpc_request *req, int opc, int bytes);
+void ptlrpc_lprocfs_brw(struct ptlrpc_request *req, int bytes);
 #else
 static inline void ptlrpc_lprocfs_register_obd(struct obd_device *obd) {}
 static inline void ptlrpc_lprocfs_unregister_obd(struct obd_device *obd) {}
-static inline void ptlrpc_lprocfs_brw(struct ptlrpc_request *req, int opc,
-                                      int bytes) {}
+static inline void ptlrpc_lprocfs_brw(struct ptlrpc_request *req, int bytes) {}
 #endif
 
 /* ptlrpc/llog_server.c */
index dfb123b..22f3dcd 100644 (file)
@@ -281,7 +281,7 @@ static int osc_getattr(struct obd_export *exp, struct obd_info *oinfo)
         osc_pack_req_body(req, oinfo);
 
         ptlrpc_request_set_replen(req);
+
         rc = ptlrpc_queue_wait(req);
         if (rc)
                 GOTO(out, rc);
@@ -328,7 +328,6 @@ static int osc_setattr(struct obd_export *exp, struct obd_info *oinfo,
         osc_pack_req_body(req, oinfo);
 
         ptlrpc_request_set_replen(req);
 
         rc = ptlrpc_queue_wait(req);
         if (rc)
@@ -388,7 +387,7 @@ static int osc_setattr_async(struct obd_export *exp, struct obd_info *oinfo,
         osc_pack_req_body(req, oinfo);
 
         ptlrpc_request_set_replen(req);
+
         if (oinfo->oi_oa->o_valid & OBD_MD_FLCOOKIE) {
                 LASSERT(oti);
                 *obdo_logcookie(oinfo->oi_oa) = *oti->oti_logcookies;
@@ -702,7 +701,7 @@ static int osc_destroy(struct obd_export *exp, struct obdo *oa,
                 RETURN(-ENOMEM);
         }
 
-        rc = ldlm_prep_elc_req(exp, req, LUSTRE_OST_VERSION, OST_DESTROY, 
+        rc = ldlm_prep_elc_req(exp, req, LUSTRE_OST_VERSION, OST_DESTROY,
                                0, &cancels, count);
         if (rc) {
                 ptlrpc_request_free(req);
@@ -1023,7 +1022,7 @@ static obd_count osc_checksum_bulk(int nob, obd_count pg_count,
 
 static int osc_brw_prep_request(int cmd, struct client_obd *cli,struct obdo *oa,
                                 struct lov_stripe_md *lsm, obd_count page_count,
-                                struct brw_page **pga, 
+                                struct brw_page **pga,
                                 struct ptlrpc_request **reqp,
                                 struct obd_capa *ocapa)
 {
@@ -1045,7 +1044,7 @@ static int osc_brw_prep_request(int cmd, struct client_obd *cli,struct obdo *oa,
 
         if ((cmd & OBD_BRW_WRITE) != 0) {
                 opc = OST_WRITE;
-                req = ptlrpc_request_alloc_pool(cli->cl_import, 
+                req = ptlrpc_request_alloc_pool(cli->cl_import,
                                                 cli->cl_import->imp_rq_pool,
                                                 &RQF_OST_BRW);
         } else {
@@ -1137,8 +1136,8 @@ static int osc_brw_prep_request(int cmd, struct client_obd *cli,struct obdo *oa,
         LASSERTF((void *)(niobuf - niocount) ==
                 lustre_msg_buf(req->rq_reqmsg, REQ_REC_OFF + 2,
                                niocount * sizeof(*niobuf)),
-                "want %p - real %p\n", lustre_msg_buf(req->rq_reqmsg, 
-                REQ_REC_OFF + 2, niocount * sizeof(*niobuf)), 
+                "want %p - real %p\n", lustre_msg_buf(req->rq_reqmsg,
+                REQ_REC_OFF + 2, niocount * sizeof(*niobuf)),
                 (void *)(niobuf - niocount));
 
         osc_announce_cached(cli, &body->oa, opc == OST_WRITE ? requested_nob:0);
@@ -1244,7 +1243,7 @@ static int check_write_checksum(struct obdo *oa, const lnet_process_id_t *peer,
                            "["LPU64"-"LPU64"]\n",
                            msg, libcfs_nid2str(peer->nid),
                            oa->o_valid & OBD_MD_FLFID ? oa->o_fid : (__u64)0,
-                           oa->o_valid & OBD_MD_FLFID ? oa->o_generation : 
+                           oa->o_valid & OBD_MD_FLFID ? oa->o_generation :
                                                         (__u64)0,
                            oa->o_id,
                            oa->o_valid & OBD_MD_FLGROUP ? oa->o_gr : (__u64)0,
@@ -1253,7 +1252,7 @@ static int check_write_checksum(struct obdo *oa, const lnet_process_id_t *peer,
         CERROR("original client csum %x (type %x), server csum %x (type %x), "
                "client csum now %x\n", client_cksum, client_cksum_type,
                server_cksum, cksum_type, new_cksum);
-        return 1;        
+        return 1;
 }
 
 /* Note rc enters this function as number of bytes transferred */
@@ -1452,7 +1451,7 @@ restart_bulk:
 
                 goto restart_bulk;
         }
-        
+
         RETURN (rc);
 }
 
@@ -1482,7 +1481,7 @@ int osc_brw_redo_request(struct ptlrpc_request *request,
                                         OST_WRITE ? OBD_BRW_WRITE :OBD_BRW_READ,
                                   aa->aa_cli, aa->aa_oa,
                                   NULL /* lsm unused by osc currently */,
-                                  aa->aa_page_count, aa->aa_ppga, 
+                                  aa->aa_page_count, aa->aa_ppga,
                                   &new_req, NULL /* ocapa */);
         if (rc)
                 RETURN(rc);
@@ -1521,9 +1520,9 @@ int osc_brw_redo_request(struct ptlrpc_request *request,
                 }
         }
 
-        /* use ptlrpc_set_add_req is safe because interpret functions work 
-         * in check_set context. only one way exist with access to request 
-         * from different thread got -EINTR - this way protected with 
+        /* use ptlrpc_set_add_req is safe because interpret functions work
+         * in check_set context. only one way exist with access to request
+         * from different thread got -EINTR - this way protected with
          * cl_loi_list_lock */
         ptlrpc_set_add_req(set, new_req);
 
@@ -1562,13 +1561,12 @@ static int async_internal(int cmd, struct obd_export *exp, struct obdo *oa,
         if (cmd == OBD_BRW_READ) {
                 lprocfs_oh_tally_log2(&cli->cl_read_page_hist, page_count);
                 lprocfs_oh_tally(&cli->cl_read_rpc_hist, cli->cl_r_in_flight);
-                ptlrpc_lprocfs_brw(req, OST_READ, aa->aa_requested_nob);
         } else {
-                 lprocfs_oh_tally_log2(&cli->cl_write_page_hist, page_count);
+                lprocfs_oh_tally_log2(&cli->cl_write_page_hist, page_count);
                 lprocfs_oh_tally(&cli->cl_write_rpc_hist,
                                  cli->cl_w_in_flight);
-                ptlrpc_lprocfs_brw(req, OST_WRITE, aa->aa_requested_nob);
         }
+        ptlrpc_lprocfs_brw(req, aa->aa_requested_nob);
 
         LASSERT(list_empty(&aa->aa_oaps));
         if (rc == 0) {
@@ -2347,15 +2345,14 @@ static int osc_send_oap_rpc(struct client_obd *cli, struct lov_oinfo *loi,
                 lprocfs_oh_tally(&cli->cl_read_rpc_hist, cli->cl_r_in_flight);
                 lprocfs_oh_tally_log2(&cli->cl_read_offset_hist,
                                       (starting_offset >> CFS_PAGE_SHIFT) + 1);
-                ptlrpc_lprocfs_brw(req, OST_READ, aa->aa_requested_nob);
         } else {
                 lprocfs_oh_tally_log2(&cli->cl_write_page_hist, page_count);
                 lprocfs_oh_tally(&cli->cl_write_rpc_hist,
                                  cli->cl_w_in_flight);
                 lprocfs_oh_tally_log2(&cli->cl_write_offset_hist,
                                       (starting_offset >> CFS_PAGE_SHIFT) + 1);
-                ptlrpc_lprocfs_brw(req, OST_WRITE, aa->aa_requested_nob);
         }
+        ptlrpc_lprocfs_brw(req, aa->aa_requested_nob);
 
         client_obd_list_lock(&cli->cl_loi_list_lock);
 
@@ -2670,7 +2667,7 @@ int osc_prep_async_page(struct obd_export *exp, struct lov_stripe_md *lsm,
 
         spin_lock_init(&oap->oap_lock);
 
-        /* If the page was marked as notcacheable - don't add to any locks */ 
+        /* If the page was marked as notcacheable - don't add to any locks */
         if (!nocache) {
                 osc_build_res_name(loi->loi_id, loi->loi_gr, &oid);
                 /* This is the only place where we can call cache_add_extent
@@ -2980,12 +2977,12 @@ int osc_extent_blocking_cb(struct ldlm_lock *lock,
 {
         struct lustre_handle lockh = { 0 };
         int rc;
-        ENTRY;  
-                
+        ENTRY;
+
         if ((unsigned long)data > 0 && (unsigned long)data < 0x1000) {
                 LDLM_ERROR(lock, "cancelling lock with bad data %p", data);
-                LBUG(); 
-        }       
+                LBUG();
+        }
 
         switch (flag) {
         case LDLM_CB_BLOCKING:
@@ -3051,7 +3048,7 @@ static void osc_set_data_with_check(struct lustre_handle *lockh, void *data,
 static int osc_change_cbdata(struct obd_export *exp, struct lov_stripe_md *lsm,
                              ldlm_iterator_t replace, void *data)
 {
-        struct ldlm_res_id res_id; 
+        struct ldlm_res_id res_id;
         struct obd_device *obd = class_exp2obd(exp);
 
         osc_build_res_name(lsm->lsm_object_id, lsm->lsm_object_gr, &res_id);
@@ -3261,7 +3258,7 @@ static int osc_match(struct obd_export *exp, struct lov_stripe_md *lsm,
         ENTRY;
 
         osc_build_res_name(lsm->lsm_object_id, lsm->lsm_object_gr, &res_id);
-        
+
         if (OBD_FAIL_CHECK(OBD_FAIL_OSC_MATCH))
                 RETURN(-EIO);
 
@@ -3403,7 +3400,7 @@ static int osc_statfs(struct obd_device *obd, struct obd_statfs *osfs,
         int rc;
         ENTRY;
 
-        /*Since the request might also come from lprocfs, so we need 
+        /*Since the request might also come from lprocfs, so we need
          *sync this with client_disconnect_export Bug15684*/
         down_read(&obd->u.cli.cl_sem);
         if (obd->u.cli.cl_import)
@@ -3411,7 +3408,7 @@ static int osc_statfs(struct obd_device *obd, struct obd_statfs *osfs,
         up_read(&obd->u.cli.cl_sem);
         if (!imp)
                 RETURN(-ENODEV);
-        
+
         /* We could possibly pass max_age in the request (as an absolute
          * timestamp or a "seconds.usec ago") so the target can avoid doing
          * extra calls into the filesystem if that isn't necessary (e.g.
@@ -3419,9 +3416,9 @@ static int osc_statfs(struct obd_device *obd, struct obd_statfs *osfs,
          * is not so great if request processing is slow, while absolute
          * timestamps are not ideal because they need time synchronization. */
         req = ptlrpc_request_alloc(imp, &RQF_OST_STATFS);
-        
+
         class_import_put(imp);
-        
+
         if (req == NULL)
                 RETURN(-ENOMEM);
 
index 6f5c421..2b0e3a2 100644 (file)
@@ -114,7 +114,7 @@ static int ost_destroy(struct obd_export *exp, struct ptlrpc_request *req,
                         RETURN (-EFAULT);
                 ldlm_request_cancel(req, dlm, 0);
         }
-        
+
         rc = lustre_pack_reply(req, 2, size, NULL);
         if (rc)
                 RETURN(rc);
@@ -777,7 +777,7 @@ static int ost_brw_read(struct ptlrpc_request *req, struct obd_trans_info *oti)
         if (rc != 0)
                 GOTO(out_bulk, rc);
 
-        /* 
+        /*
          * If getting the lock took more time than
          * client was willing to wait, drop it. b=11330
          */
@@ -928,6 +928,7 @@ out:
         LASSERT(rc <= 0);
         if (rc == 0) {
                 req->rq_status = nob;
+                ptlrpc_lprocfs_brw(req, nob);
                 target_committed_to_req(req);
                 ptlrpc_reply(req);
         } else if (!no_reply) {
@@ -967,7 +968,7 @@ static int ost_brw_write(struct ptlrpc_request *req, struct obd_trans_info *oti)
         int rc, swab, i, j;
         obd_count                client_cksum = 0, server_cksum = 0;
         cksum_type_t             cksum_type = OBD_CKSUM_CRC32;
-        int                      no_reply = 0; 
+        int                      no_reply = 0;
         ENTRY;
 
         req->rq_bulk_write = 1;
@@ -1079,7 +1080,7 @@ static int ost_brw_write(struct ptlrpc_request *req, struct obd_trans_info *oti)
         if (rc != 0)
                 GOTO(out_bulk, rc);
 
-        /* 
+        /*
          * If getting the lock took more time than
          * client was willing to wait, drop it. b=11330
          */
@@ -1102,10 +1103,10 @@ static int ost_brw_write(struct ptlrpc_request *req, struct obd_trans_info *oti)
                 if (body->oa.o_valid & OBD_MD_FLFLAGS)
                         cksum_type = cksum_type_unpack(body->oa.o_flags);
         }
-        
+
         /* Because we already sync grant info with client when reconnect,
-         * grant info will be cleared for resent req, then fed_grant and 
-         * total_grant will not be modified in following preprw_write */ 
+         * grant info will be cleared for resent req, then fed_grant and
+         * total_grant will not be modified in following preprw_write */
         if (lustre_msg_get_flags(req->rq_reqmsg) & (MSG_RESENT | MSG_REPLAY)) {
                 DEBUG_REQ(D_CACHE, req, "clear resent/replay req grant info");
                 body->oa.o_valid &= ~OBD_MD_FLGRANT;
@@ -1220,7 +1221,7 @@ static int ost_brw_write(struct ptlrpc_request *req, struct obd_trans_info *oti)
                         via = " via ";
                         router = libcfs_nid2str(desc->bd_sender);
                 }
-                
+
                 LCONSOLE_ERROR_MSG(0x168, "%s: BAD WRITE CHECKSUM: %s from "
                                    "%s%s%s inum "LPU64"/"LPU64" object "
                                    LPU64"/"LPU64" extent ["LPU64"-"LPU64"]\n",
@@ -1245,21 +1246,25 @@ static int ost_brw_write(struct ptlrpc_request *req, struct obd_trans_info *oti)
         ost_nio_pages_put(req, local_nb, npages);
 
         if (rc == 0) {
+                int nob = 0;
+
                 /* set per-requested niobuf return codes */
                 for (i = j = 0; i < niocount; i++) {
-                        int nob = remote_nb[i].len;
+                        int len = remote_nb[i].len;
 
+                        nob += len;
                         rcs[i] = 0;
                         do {
                                 LASSERT(j < npages);
                                 if (local_nb[j].rc < 0)
                                         rcs[i] = local_nb[j].rc;
-                                nob -= pp_rnb[j].len;
+                                len -= pp_rnb[j].len;
                                 j++;
-                        } while (nob > 0);
-                        LASSERT(nob == 0);
+                        } while (len > 0);
+                        LASSERT(len == 0);
                 }
                 LASSERT(j == npages);
+                ptlrpc_lprocfs_brw(req, nob);
         }
 
 out_lock:
@@ -1604,7 +1609,7 @@ int ost_handle(struct ptlrpc_request *req)
         }
 
         oti_init(oti, req);
-        
+
         rc = ost_msg_check_version(req->rq_reqmsg);
         if (rc)
                 RETURN(rc);
@@ -1911,19 +1916,19 @@ static int ost_setup(struct obd_device *obd, struct lustre_cfg* lcfg)
 
         if (oss_num_threads) {
                 /* If oss_num_threads is set, it is the min and the max. */
-                if (oss_num_threads > OSS_THREADS_MAX) 
+                if (oss_num_threads > OSS_THREADS_MAX)
                         oss_num_threads = OSS_THREADS_MAX;
                 if (oss_num_threads < OSS_THREADS_MIN)
                         oss_num_threads = OSS_THREADS_MIN;
                 oss_max_threads = oss_min_threads = oss_num_threads;
         } else {
                 /* Base min threads on memory and cpus */
-                oss_min_threads = num_possible_cpus() * num_physpages >> 
+                oss_min_threads = num_possible_cpus() * num_physpages >>
                         (27 - CFS_PAGE_SHIFT);
                 if (oss_min_threads < OSS_THREADS_MIN)
                         oss_min_threads = OSS_THREADS_MIN;
                 /* Insure a 4x range for dynamic threads */
-                if (oss_min_threads > OSS_THREADS_MAX / 4) 
+                if (oss_min_threads > OSS_THREADS_MAX / 4)
                         oss_min_threads = OSS_THREADS_MAX / 4;
                 oss_max_threads = min(OSS_THREADS_MAX, oss_min_threads * 4);
         }
index 7c075db..8d61f3d 100644 (file)
@@ -137,7 +137,9 @@ struct ll_eopcode {
         { MDS_REINT_OPEN,       "mds_reint_open" },
         { MDS_REINT_SETATTR,    "mds_reint_setattr" },
         { MDS_REINT_RENAME,     "mds_reint_rename" },
-        { MDS_REINT_UNLINK,     "mds_reint_unlink" }
+        { MDS_REINT_UNLINK,     "mds_reint_unlink" },
+        { BRW_READ_BYTES,       "read_bytes" },
+        { BRW_WRITE_BYTES,      "write_bytes" },
 };
 
 const char *ll_opcode2str(__u32 opcode)
@@ -199,9 +201,20 @@ void ptlrpc_lprocfs_register(struct proc_dir_entry *root, char *dir,
         lprocfs_counter_init(svc_stats, PTLRPC_REQBUF_AVAIL_CNTR,
                              svc_counter_config, "reqbuf_avail", "bufs");
         for (i = 0; i < EXTRA_LAST_OPC; i++) {
+                char *units;
+
+                switch(i) {
+                case BRW_WRITE_BYTES:
+                case BRW_READ_BYTES:
+                        units = "bytes";
+                        break;
+                default:
+                        units = "reqs";
+                        break;
+                }
                 lprocfs_counter_init(svc_stats, PTLRPC_LAST_CNTR + i,
                                      svc_counter_config,
-                                     ll_eopcode2str(i), "reqs");
+                                     ll_eopcode2str(i), units);
         }
         for (i = 0; i < LUSTRE_MAX_OPCODES; i++) {
                 __u32 opcode = ll_rpc_opcode_table[i].opcode;
@@ -553,20 +566,36 @@ void ptlrpc_lprocfs_rpc_sent(struct ptlrpc_request *req)
         if (svc_stats == NULL || opc <= 0)
                 return;
         LASSERT(opc < LUSTRE_MAX_OPCODES);
-        /* These two use the ptlrpc_lprocfs_brw below */
-        if (!(opc == OST_WRITE || opc == OST_READ || op == LDLM_ENQUEUE
-              || op == MDS_REINT))
+        if (!(op == LDLM_ENQUEUE || op == MDS_REINT))
                 lprocfs_counter_add(svc_stats, opc + EXTRA_MAX_OPCODES, 0);
 }
 
-void ptlrpc_lprocfs_brw(struct ptlrpc_request *req, int opc, int bytes)
+void ptlrpc_lprocfs_brw(struct ptlrpc_request *req, int bytes)
 {
         struct lprocfs_stats *svc_stats;
+        int idx;
+
+        if (!req->rq_import)
+                return;
         svc_stats = req->rq_import->imp_obd->obd_svc_stats;
-        if (!svc_stats) 
+        if (!svc_stats)
                 return;
-        lprocfs_counter_add(svc_stats, opc + EXTRA_MAX_OPCODES, bytes);
+        idx = lustre_msg_get_opc(req->rq_reqmsg);
+        switch (idx) {
+        case OST_READ:
+                idx = BRW_READ_BYTES;
+                break;
+        case OST_WRITE:
+                idx = BRW_WRITE_BYTES;
+                break;
+        default:
+                LASSERTF(0, "unsupported opcode %u\n", idx);
+                break;
+        }
+
+        lprocfs_counter_add(svc_stats, idx, bytes);
 }
+
 EXPORT_SYMBOL(ptlrpc_lprocfs_brw);
 
 void ptlrpc_lprocfs_unregister_service(struct ptlrpc_service *svc)
@@ -595,16 +624,16 @@ int lprocfs_wr_evict_client(struct file *file, const char *buffer,
         struct obd_device *obd = data;
         char tmpbuf[sizeof(struct obd_uuid)];
 
-        /* Kludge code(deadlock situation): the lprocfs lock has been held 
+        /* Kludge code(deadlock situation): the lprocfs lock has been held
          * since the client is evicted by writting client's
-         * uuid/nid to procfs "evict_client" entry. However, 
+         * uuid/nid to procfs "evict_client" entry. However,
          * obd_export_evict_by_uuid() will call lprocfs_remove() to destroy
          * the proc entries under the being destroyed export{}, so I have
-         * to drop the lock at first here. 
+         * to drop the lock at first here.
          * - jay, jxiong@clusterfs.com */
         class_incref(obd);
         LPROCFS_EXIT();
+
         sscanf(buffer, "%40s", tmpbuf);
         if (strncmp(tmpbuf, "nid:", 4) == 0)
                 obd_export_evict_by_nid(obd, tmpbuf + 4);
index 0a1e4ac..502f911 100644 (file)
@@ -130,7 +130,7 @@ check_runas_id $RUNAS_ID $RUNAS
 
 build_test_filter
 
-if [ "${ONLY}" = "MOUNT" ] ; then 
+if [ "${ONLY}" = "MOUNT" ] ; then
        echo "Lustre is up, please go on"
        exit
 fi
@@ -571,7 +571,7 @@ test_24d() {
 run_test 24d "mkdir .../R4/{f,g}; rename .../R4/f .../R4/g ====="
 
 test_24e() {
-       echo '-- cross directory renames --' 
+       echo '-- cross directory renames --'
        mkdir $DIR/R5{a,b}
        touch $DIR/R5a/f
        mv $DIR/R5a/f $DIR/R5b/g
@@ -626,7 +626,7 @@ test_24j() {
        $CHECKSTAT -a $DIR/R10/f || error
        $CHECKSTAT -a $DIR/R10/g || error
 }
-run_test 24j "source does not exist ============================" 
+run_test 24j "source does not exist ============================"
 
 test_24k() {
        mkdir $DIR/R11a $DIR/R11a/d
@@ -1061,7 +1061,7 @@ test_27u() { # bug 4900
         lctl set_param fail_loc=0
 
         TLOG=$DIR/$tfile.getstripe
-        $GETSTRIPE $DIR/d27u > $TLOG 
+        $GETSTRIPE $DIR/d27u > $TLOG
         OBJS=`awk -vobj=0 '($1 == 0) { obj += 1 } END { print obj;}' $TLOG`
         unlinkmany $DIR/d27u/t- 1000
         [ $OBJS -gt 0 ] && \
@@ -1308,27 +1308,27 @@ run_test 31m "link to file: the same, non-existing, dir==============="
 test_32a() {
        echo "== more mountpoints and symlinks ================="
        [ -e $DIR/d32a ] && rm -fr $DIR/d32a
-       mkdir -p $DIR/d32a/ext2-mountpoint 
+       mkdir -p $DIR/d32a/ext2-mountpoint
        mount -t ext2 -o loop $EXT2_DEV $DIR/d32a/ext2-mountpoint || error
-       $CHECKSTAT -t dir $DIR/d32a/ext2-mountpoint/.. || error  
+       $CHECKSTAT -t dir $DIR/d32a/ext2-mountpoint/.. || error
        $UMOUNT $DIR/d32a/ext2-mountpoint || error
 }
 run_test 32a "stat d32a/ext2-mountpoint/.. ====================="
 
 test_32b() {
        [ -e $DIR/d32b ] && rm -fr $DIR/d32b
-       mkdir -p $DIR/d32b/ext2-mountpoint 
+       mkdir -p $DIR/d32b/ext2-mountpoint
        mount -t ext2 -o loop $EXT2_DEV $DIR/d32b/ext2-mountpoint || error
        ls -al $DIR/d32b/ext2-mountpoint/.. || error
        $UMOUNT $DIR/d32b/ext2-mountpoint || error
 }
 run_test 32b "open d32b/ext2-mountpoint/.. ====================="
+
 test_32c() {
        [ -e $DIR/d32c ] && rm -fr $DIR/d32c
-       mkdir -p $DIR/d32c/ext2-mountpoint 
+       mkdir -p $DIR/d32c/ext2-mountpoint
        mount -t ext2 -o loop $EXT2_DEV $DIR/d32c/ext2-mountpoint || error
-       mkdir -p $DIR/d32c/d2/test_dir    
+       mkdir -p $DIR/d32c/d2/test_dir
        $CHECKSTAT -t dir $DIR/d32c/ext2-mountpoint/../d2/test_dir || error
        $UMOUNT $DIR/d32c/ext2-mountpoint || error
 }
@@ -1336,9 +1336,9 @@ run_test 32c "stat d32c/ext2-mountpoint/../d2/test_dir ========="
 
 test_32d() {
        [ -e $DIR/d32d ] && rm -fr $DIR/d32d
-       mkdir -p $DIR/d32d/ext2-mountpoint 
+       mkdir -p $DIR/d32d/ext2-mountpoint
        mount -t ext2 -o loop $EXT2_DEV $DIR/d32d/ext2-mountpoint || error
-       mkdir -p $DIR/d32d/d2/test_dir    
+       mkdir -p $DIR/d32d/d2/test_dir
        ls -al $DIR/d32d/ext2-mountpoint/../d2/test_dir || error
        $UMOUNT $DIR/d32d/ext2-mountpoint || error
 }
@@ -1346,10 +1346,10 @@ run_test 32d "open d32d/ext2-mountpoint/../d2/test_dir ========="
 
 test_32e() {
        [ -e $DIR/d32e ] && rm -fr $DIR/d32e
-       mkdir -p $DIR/d32e/tmp    
-       TMP_DIR=$DIR/d32e/tmp       
-       ln -s $DIR/d32e $TMP_DIR/symlink11 
-       ln -s $TMP_DIR/symlink11 $TMP_DIR/../symlink01 
+       mkdir -p $DIR/d32e/tmp
+       TMP_DIR=$DIR/d32e/tmp
+       ln -s $DIR/d32e $TMP_DIR/symlink11
+       ln -s $TMP_DIR/symlink11 $TMP_DIR/../symlink01
        $CHECKSTAT -t link $DIR/d32e/tmp/symlink11 || error
        $CHECKSTAT -t link $DIR/d32e/symlink01 || error
 }
@@ -1357,20 +1357,20 @@ run_test 32e "stat d32e/symlink->tmp/symlink->lustre-subdir ===="
 
 test_32f() {
        [ -e $DIR/d32f ] && rm -fr $DIR/d32f
-       mkdir -p $DIR/d32f/tmp    
-       TMP_DIR=$DIR/d32f/tmp       
-       ln -s $DIR/d32f $TMP_DIR/symlink11 
-       ln -s $TMP_DIR/symlink11 $TMP_DIR/../symlink01 
+       mkdir -p $DIR/d32f/tmp
+       TMP_DIR=$DIR/d32f/tmp
+       ln -s $DIR/d32f $TMP_DIR/symlink11
+       ln -s $TMP_DIR/symlink11 $TMP_DIR/../symlink01
        ls $DIR/d32f/tmp/symlink11  || error
        ls $DIR/d32f/symlink01 || error
 }
 run_test 32f "open d32f/symlink->tmp/symlink->lustre-subdir ===="
 
 test_32g() {
-       TMP_DIR=$DIR/$tdir/tmp       
+       TMP_DIR=$DIR/$tdir/tmp
        mkdir -p $TMP_DIR $DIR/${tdir}2
-       ln -s $DIR/${tdir}2 $TMP_DIR/symlink12 
-       ln -s $TMP_DIR/symlink12 $TMP_DIR/../symlink02 
+       ln -s $DIR/${tdir}2 $TMP_DIR/symlink12
+       ln -s $TMP_DIR/symlink12 $TMP_DIR/../symlink02
        $CHECKSTAT -t link $TMP_DIR/symlink12 || error
        $CHECKSTAT -t link $DIR/$tdir/symlink02 || error
        $CHECKSTAT -t dir -f $TMP_DIR/symlink12 || error
@@ -1380,10 +1380,10 @@ run_test 32g "stat d32g/symlink->tmp/symlink->lustre-subdir/${tdir}2"
 
 test_32h() {
        rm -fr $DIR/$tdir $DIR/${tdir}2
-       TMP_DIR=$DIR/$tdir/tmp       
-       mkdir -p $TMP_DIR $DIR/${tdir}2 
-       ln -s $DIR/${tdir}2 $TMP_DIR/symlink12 
-       ln -s $TMP_DIR/symlink12 $TMP_DIR/../symlink02 
+       TMP_DIR=$DIR/$tdir/tmp
+       mkdir -p $TMP_DIR $DIR/${tdir}2
+       ln -s $DIR/${tdir}2 $TMP_DIR/symlink12
+       ln -s $TMP_DIR/symlink12 $TMP_DIR/../symlink02
        ls $TMP_DIR/symlink12 || error
        ls $DIR/$tdir/symlink02  || error
 }
@@ -1391,17 +1391,17 @@ run_test 32h "open d32h/symlink->tmp/symlink->lustre-subdir/${tdir}2"
 
 test_32i() {
        [ -e $DIR/d32i ] && rm -fr $DIR/d32i
-       mkdir -p $DIR/d32i/ext2-mountpoint 
+       mkdir -p $DIR/d32i/ext2-mountpoint
        mount -t ext2 -o loop $EXT2_DEV $DIR/d32i/ext2-mountpoint || error
        touch $DIR/d32i/test_file
-       $CHECKSTAT -t file $DIR/d32i/ext2-mountpoint/../test_file || error  
+       $CHECKSTAT -t file $DIR/d32i/ext2-mountpoint/../test_file || error
        $UMOUNT $DIR/d32i/ext2-mountpoint || error
 }
 run_test 32i "stat d32i/ext2-mountpoint/../test_file ==========="
 
 test_32j() {
        [ -e $DIR/d32j ] && rm -fr $DIR/d32j
-       mkdir -p $DIR/d32j/ext2-mountpoint 
+       mkdir -p $DIR/d32j/ext2-mountpoint
        mount -t ext2 -o loop $EXT2_DEV $DIR/d32j/ext2-mountpoint || error
        touch $DIR/d32j/test_file
        cat $DIR/d32j/ext2-mountpoint/../test_file || error
@@ -1411,8 +1411,8 @@ run_test 32j "open d32j/ext2-mountpoint/../test_file ==========="
 
 test_32k() {
        rm -fr $DIR/d32k
-       mkdir -p $DIR/d32k/ext2-mountpoint 
-       mount -t ext2 -o loop $EXT2_DEV $DIR/d32k/ext2-mountpoint  
+       mkdir -p $DIR/d32k/ext2-mountpoint
+       mount -t ext2 -o loop $EXT2_DEV $DIR/d32k/ext2-mountpoint
        mkdir -p $DIR/d32k/d2
        touch $DIR/d32k/d2/test_file || error
        $CHECKSTAT -t file $DIR/d32k/ext2-mountpoint/../d2/test_file || error
@@ -1422,7 +1422,7 @@ run_test 32k "stat d32k/ext2-mountpoint/../d2/test_file ========"
 
 test_32l() {
        rm -fr $DIR/d32l
-       mkdir -p $DIR/d32l/ext2-mountpoint 
+       mkdir -p $DIR/d32l/ext2-mountpoint
        mount -t ext2 -o loop $EXT2_DEV $DIR/d32l/ext2-mountpoint || error
        mkdir -p $DIR/d32l/d2
        touch $DIR/d32l/d2/test_file
@@ -1433,10 +1433,10 @@ run_test 32l "open d32l/ext2-mountpoint/../d2/test_file ========"
 
 test_32m() {
        rm -fr $DIR/d32m
-       mkdir -p $DIR/d32m/tmp    
-       TMP_DIR=$DIR/d32m/tmp       
-       ln -s $DIR $TMP_DIR/symlink11 
-       ln -s $TMP_DIR/symlink11 $TMP_DIR/../symlink01 
+       mkdir -p $DIR/d32m/tmp
+       TMP_DIR=$DIR/d32m/tmp
+       ln -s $DIR $TMP_DIR/symlink11
+       ln -s $TMP_DIR/symlink11 $TMP_DIR/../symlink01
        $CHECKSTAT -t link $DIR/d32m/tmp/symlink11 || error
        $CHECKSTAT -t link $DIR/d32m/symlink01 || error
 }
@@ -1444,10 +1444,10 @@ run_test 32m "stat d32m/symlink->tmp/symlink->lustre-root ======"
 
 test_32n() {
        rm -fr $DIR/d32n
-       mkdir -p $DIR/d32n/tmp    
-       TMP_DIR=$DIR/d32n/tmp       
-       ln -s $DIR $TMP_DIR/symlink11 
-       ln -s $TMP_DIR/symlink11 $TMP_DIR/../symlink01 
+       mkdir -p $DIR/d32n/tmp
+       TMP_DIR=$DIR/d32n/tmp
+       ln -s $DIR $TMP_DIR/symlink11
+       ln -s $TMP_DIR/symlink11 $TMP_DIR/../symlink01
        ls -l $DIR/d32n/tmp/symlink11  || error
        ls -l $DIR/d32n/symlink01 || error
 }
@@ -1455,11 +1455,11 @@ run_test 32n "open d32n/symlink->tmp/symlink->lustre-root ======"
 
 test_32o() {
        rm -fr $DIR/d32o $DIR/$tfile
-       touch $DIR/$tfile 
-       mkdir -p $DIR/d32o/tmp    
-       TMP_DIR=$DIR/d32o/tmp       
-       ln -s $DIR/$tfile $TMP_DIR/symlink12 
-       ln -s $TMP_DIR/symlink12 $TMP_DIR/../symlink02 
+       touch $DIR/$tfile
+       mkdir -p $DIR/d32o/tmp
+       TMP_DIR=$DIR/d32o/tmp
+       ln -s $DIR/$tfile $TMP_DIR/symlink12
+       ln -s $TMP_DIR/symlink12 $TMP_DIR/../symlink02
        $CHECKSTAT -t link $DIR/d32o/tmp/symlink12 || error
        $CHECKSTAT -t link $DIR/d32o/symlink02 || error
        $CHECKSTAT -t file -f $DIR/d32o/tmp/symlink12 || error
@@ -1473,15 +1473,15 @@ test_32p() {
     log 32p_2
        rm -f $DIR/$tfile
     log 32p_3
-       touch $DIR/$tfile 
+       touch $DIR/$tfile
     log 32p_4
-       mkdir -p $DIR/d32p/tmp    
+       mkdir -p $DIR/d32p/tmp
     log 32p_5
-       TMP_DIR=$DIR/d32p/tmp       
+       TMP_DIR=$DIR/d32p/tmp
     log 32p_6
-       ln -s $DIR/$tfile $TMP_DIR/symlink12 
+       ln -s $DIR/$tfile $TMP_DIR/symlink12
     log 32p_7
-       ln -s $TMP_DIR/symlink12 $TMP_DIR/../symlink02 
+       ln -s $TMP_DIR/symlink12 $TMP_DIR/../symlink02
     log 32p_8
        cat $DIR/d32p/tmp/symlink12 || error
     log 32p_9
@@ -1560,7 +1560,7 @@ test_34b() {
 run_test 34b "O_RDONLY opening file doesn't create objects ====="
 
 test_34c() {
-       [ ! -f $DIR/f34 ] && test_34a 
+       [ ! -f $DIR/f34 ] && test_34a
        $CHECKSTAT -s $TEST_34_SIZE $DIR/f34 || error
        $OPENFILE -f O_RDWR $DIR/f34
        $GETSTRIPE $DIR/f34 2>&1 | grep -q "no stripe info" && error
@@ -1569,7 +1569,7 @@ test_34c() {
 run_test 34c "O_RDWR opening file-with-size works =============="
 
 test_34d() {
-       [ ! -f $DIR/f34 ] && test_34a 
+       [ ! -f $DIR/f34 ] && test_34a
        dd if=/dev/zero of=$DIR/f34 conv=notrunc bs=4k count=1 || error
        $CHECKSTAT -s $TEST_34_SIZE $DIR/f34 || error
        rm $DIR/f34
@@ -2007,7 +2007,7 @@ page_size() {
 
 # in a 2 stripe file (lov.sh), page 1023 maps to page 511 in its object.  this
 # test tickles a bug where re-dirtying a page was failing to be mapped to the
-# objects offset and an assert hit when an rpc was built with 1023's mapped 
+# objects offset and an assert hit when an rpc was built with 1023's mapped
 # offset 511 and 511's raw 511 offset. it also found general redirtying bugs.
 test_46() {
        f="$DIR/f46"
@@ -2206,7 +2206,7 @@ test_51d() {
                error "OST $N has less objects vs OST $NLAST (${OBJS[$N]} < ${OBJS[$NLAST]}"
            [ ${OBJS[$N]} -gt $((${OBJS[$NLAST]} + 20)) ] && \
                error "OST $N has less objects vs OST $NLAST (${OBJS[$N]} < ${OBJS[$NLAST]}"
-           
+
            [ ${OBJS0[$N]} -lt $((${OBJS0[$NLAST]} - 20)) ] && \
                error "OST $N has less #0 objects vs OST $NLAST (${OBJS0[$N]} < ${OBJS0[$NLAST]}"
            [ ${OBJS0[$N]} -gt $((${OBJS0[$NLAST]} + 20)) ] && \
@@ -2257,7 +2257,7 @@ run_test 52b "immutable flag test (should return errors) ======="
 test_53() {
        remote_mds && skip "remote MDS" && return
 
-       # only test MDT0000 
+       # only test MDT0000
         for value in `lctl get_param osc.*-osc-MDT0000.prealloc_last_id` ; do
                 param=`echo ${value[0]} | cut -d "=" -f1`
                 ostname=`echo $param | cut -d "." -f2 | cut -d - -f 1-2`
@@ -2284,7 +2284,7 @@ test_54b() {
        f="$DIR/f54b"
        mknod $f c 1 3
        chmod 0666 $f
-       dd if=/dev/zero of=$f bs=`page_size` count=1 
+       dd if=/dev/zero of=$f bs=`page_size` count=1
 }
 run_test 54b "char device works in lustre ======================"
 
@@ -2306,7 +2306,7 @@ test_54c() {
        tdir="$DIR/d54c"
        loopdev="$DIR/loop54c"
 
-       find_loop_dev 
+       find_loop_dev
        [ -z "$LOOPNUM" ] && echo "couldn't find empty loop device" && return
        mknod $loopdev b 7 $LOOPNUM
        echo "make a loop file system with $tfile on $loopdev ($LOOPNUM)..."
@@ -2615,7 +2615,7 @@ test_57a() {
        remote_mds && skip "remote MDS" && return
        local MNTDEV="osd.*MDT*.mntdev"
        DEV=$(lctl get_param -n $MNTDEV)
-       [ -z "$DEV" ] && error "can't access $MNTDEV" 
+       [ -z "$DEV" ] && error "can't access $MNTDEV"
        for DEV in `lctl get_param -n $MNTDEV`; do
                dumpe2fs -h $DEV > $TMP/t57a.dump || error "can't access $DEV"
                DEVISIZE=`awk '/Inode size:/ { print $3 }' $TMP/t57a.dump`
@@ -2707,9 +2707,9 @@ test_60b() { # bug 6411
        [ $LLOG_COUNT -gt 50 ] && error "CDEBUG_LIMIT not limiting messages ($LLOG_COUNT)"|| true
 }
 run_test 60b "limit repeated messages from CERROR/CWARN ========"
+
 test_60c() {
-       echo "create 5000 files" 
+       echo "create 5000 files"
        createmany -o $DIR/f60c- 5000
 #define OBD_FAIL_MDS_LLOG_CREATE_FAILED  0x13c
        lctl set_param fail_loc=0x8000013c
@@ -2884,9 +2884,9 @@ test_65h() {
           "`$GETSTRIPE -v $DIR/d65/dd1 | grep "^count"`" ] || error "stripe info inherit failed"
 }
 run_test 65h "directory stripe info inherit ===================="
+
 test_65i() { # bug6367
-        $SETSTRIPE $MOUNT -s 65536 -c -1 
+        $SETSTRIPE $MOUNT -s 65536 -c -1
 }
 run_test 65i "set non-default striping on root directory (bug 6367)="
 
@@ -3044,7 +3044,7 @@ run_test 71 "Running dbench on lustre (don't segment fault) ===="
 test_72() { # bug 5695 - Test that on 2.6 remove_suid works properly
        check_kernel_version 43 || return 0
        [ "$RUNAS_ID" = "$UID" ] && skip "RUNAS_ID = UID = $UID -- skipping" && return
-        
+
         # Check that testing environment is properly set up. Skip if not
         FAIL_ON_ERROR=false check_runas_id_ret $RUNAS_ID $RUNAS || {
                 skip "User $RUNAS_ID does not exist - skipping"
@@ -3067,7 +3067,7 @@ run_test 72 "Test that remove suid works properly (bug5695) ===="
 
 # bug 3462 - multiple simultaneous MDC requests
 test_73() {
-       mkdir $DIR/d73-1 
+       mkdir $DIR/d73-1
        mkdir $DIR/d73-2
        multiop_bg_pause $DIR/d73-1/f73-1 O_c || return 1
        pid1=$!
@@ -3086,8 +3086,8 @@ test_73() {
        sleep 25
 
        $CHECKSTAT -t file $DIR/d73-1/f73-1 || return 4
-       $CHECKSTAT -t file $DIR/d73-1/f73-2 || return 5 
-       $CHECKSTAT -t file $DIR/d73-2/f73-3 || return 6 
+       $CHECKSTAT -t file $DIR/d73-1/f73-2 || return 5
+       $CHECKSTAT -t file $DIR/d73-2/f73-3 || return 6
 
        rm -rf $DIR/d73-*
 }
@@ -3129,25 +3129,25 @@ export T75_PREP=no
 test75_prep() {
         [ $T75_PREP = "yes" ] && return
         echo "using F75=$F75, F128k=$F128k, FHEAD=$FHEAD, FTAIL=$FTAIL"
+
         dd if=/dev/urandom of=${F75}_128k bs=128k count=1 || error "dd failed"
         log "finished dd"
         chmod 777 ${F128k}
         T75_PREP=yes
 }
+
 test_75a() {
 #      skipped temporarily: we do not have join file currently
 #      please remove this when ready - huanghua
        return
         test75_prep
+
         cp -p ${F128k} ${FHEAD}
         log "finished cp to $FHEAD"
         cp -p ${F128k} ${FTAIL}
         log "finished cp to $FTAIL"
         cat ${F128k} ${F128k} > ${F75}_sim_sim
+
         $JOIN ${FHEAD} ${FTAIL} || error "join ${FHEAD} ${FTAIL} error"
         log "finished join $FHEAD to ${F75}_sim_sim"
         cmp ${FHEAD} ${F75}_sim_sim || error "${FHEAD} ${F75}_sim_sim differ"
@@ -3155,13 +3155,13 @@ test_75a() {
         $CHECKSTAT -a ${FTAIL} || error "tail ${FTAIL} still exist after join"
 }
 run_test 75a "TEST join file ===================================="
+
 test_75b() {
 #      skipped temporarily: we do not have join file currently
 #      please remove this when ready - huanghua
        return
         test75_prep
+
         cp -p ${F128k} ${FTAIL}
         cat ${F75}_sim_sim >> ${F75}_join_sim
         cat ${F128k} >> ${F75}_join_sim
@@ -3171,13 +3171,13 @@ test_75b() {
         $CHECKSTAT -a ${FTAIL} || error "tail ${FTAIL} exist after join"
 }
 run_test 75b "TEST join file 2 =================================="
+
 test_75c() {
 #      skipped temporarily: we do not have join file currently
 #      please remove this when ready - huanghua
        return
         test75_prep
+
         cp -p ${F128k} ${FTAIL}
         cat ${F128k} >> ${F75}_sim_join
         cat ${F75}_join_sim >> ${F75}_sim_join
@@ -3187,13 +3187,13 @@ test_75c() {
         $CHECKSTAT -a ${FHEAD} || error "tail ${FHEAD} exist after join"
 }
 run_test 75c "TEST join file 3 =================================="
+
 test_75d() {
 #      skipped temporarily: we do not have join file currently
 #      please remove this when ready - huanghua
        return
         test75_prep
+
         cp -p ${F128k} ${FHEAD}
         cp -p ${F128k} ${FHEAD}_tmp
         cat ${F75}_sim_sim >> ${F75}_join_join
@@ -3204,23 +3204,23 @@ test_75d() {
         $CHECKSTAT -a ${FTAIL} || error "tail ${FTAIL} exist after join (2)"
 }
 run_test 75d "TEST join file 4 =================================="
+
 test_75e() {
 #      skipped temporarily: we do not have join file currently
 #      please remove this when ready - huanghua
        return
         test75_prep
+
         rm -rf ${FHEAD} || "delete join file error"
 }
 run_test 75e "TEST join file 5 (remove joined file) ============="
+
 test_75f() {
 #      skipped temporarily: we do not have join file currently
 #      please remove this when ready - huanghua
        return
         test75_prep
+
         cp -p ${F128k} ${F75}_join_10_compare
         cp -p ${F128k} ${F75}_join_10
         for ((i = 0; i < 10; i++)); do
@@ -3234,16 +3234,16 @@ test_75f() {
                 error "files ${F75}_join_10 ${F75}_join_10_compare differ"
 }
 run_test 75f "TEST join file 6 (join 10 files) =================="
+
 test_75g() {
 #      skipped temporarily: we do not have join file currently
 #      please remove this when ready - huanghua
        return
         [ ! -f ${F75}_join_10 ] && echo "${F75}_join_10 missing" && return
         $LFS getstripe ${F75}_join_10
+
         $OPENUNLINK ${F75}_join_10 ${F75}_join_10 || error "files unlink open"
+
         ls -l $F75*
 }
 run_test 75g "TEST join file 7 (open unlink) ===================="
@@ -3325,7 +3325,7 @@ test_77b() { # bug 10889
 run_test 77b "checksum error on client write ===================="
 
 test_77c() { # bug 10889
-       [ ! -f $DIR/f77b ] && skip "requires 77b - skipping" && return  
+       [ ! -f $DIR/f77b ] && skip "requires 77b - skipping" && return
        set_checksums 1
        for algo in $CKSUM_TYPES; do
                cancel_lru_locks osc
@@ -3352,7 +3352,7 @@ test_77d() { # bug 10889
 run_test 77d "checksum error on OST direct write ==============="
 
 test_77e() { # bug 10889
-       [ ! -f $DIR/f77 ] && skip "requires 77d - skipping" && return  
+       [ ! -f $DIR/f77 ] && skip "requires 77d - skipping" && return
        #define OBD_FAIL_OSC_CHECKSUM_RECEIVE    0x408
        lctl set_param fail_loc=0x80000408
        set_checksums 1
@@ -3397,7 +3397,7 @@ run_test 77g "checksum error on OST write ======================"
 test_77h() { # bug 10889
        [ $(lctl get_param -n devices | grep -c obdfilter) -eq 0 ] && \
                skip "remote OST" && return
-       [ ! -f $DIR/f77 ] && skip "requires 77g - skipping" && return  
+       [ ! -f $DIR/f77 ] && skip "requires 77g - skipping" && return
        cancel_lru_locks osc
        #define OBD_FAIL_OST_CHECKSUM_SEND          0x21b
        lctl set_param fail_loc=0x8000021b
@@ -3492,15 +3492,15 @@ test_79() { # bug 12743
 
         ALLOWANCE=$((64 * $OSTCOUNT))
 
-        if [ $DFTOTAL -lt $(($BKTOTAL - $ALLOWANCE)) ] ||  
+        if [ $DFTOTAL -lt $(($BKTOTAL - $ALLOWANCE)) ] ||
            [ $DFTOTAL -gt $(($BKTOTAL + $ALLOWANCE)) ] ; then
                 error "df total($DFTOTAL) mismatch OST total($BKTOTAL)"
         fi
-        if [ $DFFREE -lt $(($BKFREE - $ALLOWANCE)) ] || 
+        if [ $DFFREE -lt $(($BKFREE - $ALLOWANCE)) ] ||
            [ $DFFREE -gt $(($BKFREE + $ALLOWANCE)) ] ; then
                 error "df free($DFFREE) mismatch OST free($BKFREE)"
         fi
-        if [ $DFAVAIL -lt $(($BKAVAIL - $ALLOWANCE)) ] || 
+        if [ $DFAVAIL -lt $(($BKAVAIL - $ALLOWANCE)) ] ||
            [ $DFAVAIL -gt $(($BKAVAIL + $ALLOWANCE)) ] ; then
                 error "df avail($DFAVAIL) mismatch OST avail($BKAVAIL)"
         fi
@@ -3647,7 +3647,7 @@ test_101() {
        if [ $(($discard * 10)) -gt $nreads ] ;then
                lctl get_param osc.*-osc*.rpc_stats
                lctl get_param llite.*.read_ahead_stats
-               error "too many ($discard) discarded pages" 
+               error "too many ($discard) discarded pages"
        fi
        rm -f $DIR/$tfile || true
 }
@@ -3707,7 +3707,7 @@ test_101b() {
        local ITERATION=$((FILE_LENGTH/STRIDE_SIZE))
        # prepare the read-ahead file
        setup_test101
-       cancel_lru_locks osc 
+       cancel_lru_locks osc
        for BIDX in 2 4 8 16 32 64 128 256
        do
                local BSIZE=$((BIDX*4096))
@@ -3730,18 +3730,18 @@ setup_test102() {
        [ "$SETUP_TEST102" = "yes" ] && return
        mkdir -p $DIR/$tdir
        STRIPE_SIZE=65536
-       STRIPE_COUNT=4 
+       STRIPE_COUNT=4
        STRIPE_OFFSET=2
 
        trap cleanup_test102 EXIT
        cd $DIR
        $SETSTRIPE $tdir -s $STRIPE_SIZE -i $STRIPE_OFFSET -c $STRIPE_COUNT
-       cd $DIR/$tdir 
+       cd $DIR/$tdir
        for num in 1 2 3 4
        do
                for count in 1 2 3 4
                do
-                       for offset in 0 1 2 3 
+                       for offset in 0 1 2 3
                        do
                                local stripe_size=`expr $STRIPE_SIZE \* $num`
                                local file=file"$num-$offset-$count"
@@ -3751,7 +3751,7 @@ setup_test102() {
        done
 
        cd $DIR
-       star -c  f=$TMP/f102.tar $tdir 
+       star -c  f=$TMP/f102.tar $tdir
        SETUP_TEST102=yes
 }
 
@@ -3778,7 +3778,7 @@ test_102a() {
         setfattr -n trusted.name1 -v value1 $testfile || error
         [ "`getfattr -n trusted.name1 $testfile 2> /dev/null | \
         grep "trusted.name1"`" == "trusted.name1=\"value1\"" ] || error
+
         setfattr -n user.author1 -v author1 $testfile || error
         [ "`getfattr -n user.author1 $testfile 2> /dev/null | \
         grep "user.author1"`" == "user.author1=\"author1\"" ] || error
@@ -3789,7 +3789,7 @@ test_102a() {
         [ `getfattr -d -m "^trusted" $testfile 2> /dev/null | \
         grep "trusted.name" | wc -l` -eq 3 ] || error
 
+
         setfattr -n user.author2 -v author2 $testfile || error
         setfattr -n user.author3 -v author3 $testfile || error
         [ `getfattr -d -m "^user" $testfile 2> /dev/null | \
@@ -3878,12 +3878,12 @@ compare_stripe_info1() {
        do
                for count in 1 2 3 4
                do
-                       for offset in 0 1 2 3 
+                       for offset in 0 1 2 3
                        do
                                local size=`expr $STRIPE_SIZE \* $num`
                                local file=file"$num-$offset-$count"
                                local tmp_file=out
-                               $GETSTRIPE -v $file > $tmp_file 
+                               $GETSTRIPE -v $file > $tmp_file
                                get_stripe_info  $tmp_file
                                if test $stripe_size -ne $size
                                then
@@ -3908,7 +3908,7 @@ compare_stripe_info2() {
        do
                for count in 1 2 3 4
                do
-                       for offset in 0 1 2 3 
+                       for offset in 0 1 2 3
                        do
                                local size=`expr $STRIPE_SIZE \* $num`
                                local file=file"$num-$offset-$count"
@@ -3935,7 +3935,7 @@ compare_stripe_info2() {
 
 test_102d() {
        # b10930: star test for trusted.lov xattr
-       star --xhelp 2>&1 | grep -q nolustre  
+       star --xhelp 2>&1 | grep -q nolustre
        if [ $? -ne 0 ]
        then
                skip "being skipped because a lustre-aware star is not installed." && return
@@ -3952,7 +3952,7 @@ run_test 102d "star restore stripe info from tarfile,not keep osts ==========="
 
 test_102e() {
        # b10930: star test for trusted.lov xattr
-       star --xhelp 2>&1 | grep -q nolustre  
+       star --xhelp 2>&1 | grep -q nolustre
        [ $? -ne 0 ] && skip "lustre-aware star is not installed" && return
        [ "$OSTCOUNT" -lt "4" ] && skip "skipping 4-stripe test" && return
        setup_test102
@@ -3965,7 +3965,7 @@ run_test 102e "star restore stripe info from tarfile, keep osts ==========="
 
 test_102f() {
        # b10930: star test for trusted.lov xattr
-       star --xhelp 2>&1 | grep -q nolustre  
+       star --xhelp 2>&1 | grep -q nolustre
        [ $? -ne 0 ] && skip "lustre-aware star is not installed" && return
        [ "$OSTCOUNT" -lt "4" ] && skip "skipping 4-stripe test" && return
        setup_test102
@@ -3979,7 +3979,7 @@ run_test 102f "star copy files, not keep osts ==========="
 
 test_102g() {
        # b10930: star test for trusted.lov xattr
-       star --xhelp 2>&1 | grep -q nolustre  
+       star --xhelp 2>&1 | grep -q nolustre
        [ $? -ne 0 ] && skip "lustre-aware star is not installed" && return
        [ "$OSTCOUNT" -lt "4" ] && skip "skipping 4-stripe test" && return
        setup_test102
@@ -4225,8 +4225,8 @@ free_min_max () {
                MINV=${AVAIL[i]}; MINI=$i
            fi
        done
-       echo Min free space: OST $MINI: $MINV 
-       echo Max free space: OST $MAXI: $MAXV 
+       echo Min free space: OST $MINI: $MINV
+       echo Max free space: OST $MAXI: $MAXV
 }
 
 test_116() {
@@ -4291,14 +4291,14 @@ test_116() {
        free_min_max
        DIFF2=$(($MAXV - $MINV))
        echo "free space delta: orig $DIFF final $DIFF2"
-       [ $DIFF2 -gt $DIFF ] && echo "delta got worse!" 
+       [ $DIFF2 -gt $DIFF ] && echo "delta got worse!"
        DIFF=$(($MINV1 - ${AVAIL[$MINI1]}))
        echo "Wrote $DIFF to smaller OST $MINI1"
        DIFF2=$(($MAXV1 - ${AVAIL[$MAXI1]}))
        echo "Wrote $DIFF2 to larger OST $MAXI1"
        [ $DIFF -gt 0 ] && echo "Wrote $(($DIFF2 * 100 / $DIFF - 100))% more data to larger OST $MAXI1"
 
-       # Figure out which files were written where 
+       # Figure out which files were written where
        UUID=$(lctl get_param -n lov.${FSNAME}-clilov-*.target_obd |
                awk '/'$MINI1': / {print $2; exit}')
        echo $UUID
@@ -4494,7 +4494,7 @@ test_118f() {
        fi
        
         lctl set_param fail_loc=0x0
-        
+
         LOCKED=$(lctl get_param -n llite.*.dump_page_cache | grep -c locked)
         DIRTY=$(lctl get_param -n llite.*.dump_page_cache | grep -c dirty)
         WRITEBACK=$(lctl get_param -n llite.*.dump_page_cache |
@@ -5024,9 +5024,9 @@ test_124a() {
 
         # use touch to produce $NR new locks
         log "create $NR files at $DIR/$tdir"
-        createmany -o $DIR/$tdir/f $NR || 
-                error "failed to create $NR files in $DIR/$tdir" 
-        
+        createmany -o $DIR/$tdir/f $NR ||
+                error "failed to create $NR files in $DIR/$tdir"
+
         cancel_lru_locks mdc
         ls -l $DIR/$tdir > /dev/null
 
@@ -5089,7 +5089,7 @@ test_124b() {
        [ -z "`lctl get_param -n mdc.*.connect_flags | grep lru_resize`" ] && \
                skip "no lru resize on server" && return 0
 
-        # even for cmd no matter what metadata namespace to use for getting 
+        # even for cmd no matter what metadata namespace to use for getting
         # the limit, we use appropriate.
         LIMIT=`lctl get_param -n ldlm.namespaces.*mdc*.pool.limit`
 
@@ -5098,7 +5098,7 @@ test_124b() {
                 NR=$LIMIT
         fi
         lru_resize_disable mdc
-        mkdir -p $DIR/$tdir/disable_lru_resize || 
+        mkdir -p $DIR/$tdir/disable_lru_resize ||
                error "failed to create $DIR/$tdir/disable_lru_resize"
 
         createmany -o $DIR/$tdir/disable_lru_resize/f $NR
@@ -5122,7 +5122,7 @@ test_124b() {
         unlinkmany $DIR/$tdir/disable_lru_resize/f $NR
 
         lru_resize_enable mdc
-        mkdir -p $DIR/$tdir/enable_lru_resize || 
+        mkdir -p $DIR/$tdir/enable_lru_resize ||
                error "failed to create $DIR/$tdir/enable_lru_resize"
 
         createmany -o $DIR/$tdir/enable_lru_resize/f $NR
@@ -5189,15 +5189,15 @@ test_127() { # bug 15521
                 echo "got $COUNT $NAME"
 
                 case $NAME in
-                        ost_read|ost_write)
+                        read_bytes|write_bytes)
                         [ $MIN -lt 4096 ] && error "min is too small: $MIN"
                         [ $MIN -gt $FSIZE ] && error "min is too big: $MIN"
                         [ $MAX -lt 4096 ] && error "max is too small: $MAX"
                         [ $MAX -gt $FSIZE ] && error "max is too big: $MAX"
                         [ $SUM -ne $FSIZE ] && error "sum is wrong: $SUM"
-                        [ $SUMSQ -lt $(((FSIZE /4096) * (4096 * 4096))) ] && 
+                        [ $SUMSQ -lt $(((FSIZE /4096) * (4096 * 4096))) ] &&
                                 error "sumsquare is too small: $SUMSQ"
-                        [ $SUMSQ -gt $((FSIZE * FSIZE)) ] && 
+                        [ $SUMSQ -gt $((FSIZE * FSIZE)) ] &&
                                 error "sumsquare is too big: $SUMSQ"
                         ;;
                         *) ;;
@@ -5205,8 +5205,8 @@ test_127() { # bug 15521
         done < $DIR/${tfile}.tmp
 
         #check that we actually got some stats
-        [ "$ost_read" ] || error "no read done"
-        [ "$ost_write" ] || error "no write done"
+        [ "$read_bytes" ] || error "no read done"
+        [ "$write_bytes" ] || error "no write done"
 }
 run_test 127 "verify the client stats are sane"
 
@@ -5227,7 +5227,7 @@ test_129() {
         [ "$FSTYPE" != "ldiskfs" ] && skip "not needed for FSTYPE=$FSTYPE" && return 0
 
         DEV=$(basename $(do_facet mds lctl get_param -n osd.*MDT*.mntdev))
-        [ -z "$DEV" ] && error "can't access mds mntdev" 
+        [ -z "$DEV" ] && error "can't access mds mntdev"
         EFBIG=27
         LDPROC=/proc/fs/ldiskfs/$DEV/max_dir_size
         MAX=16384