Whamcloud - gitweb
LU-1194 llog: fix for not sync llcd at thread stop
[fs/lustre-release.git] / lustre / obdfilter / filter.c
index 9dfef42..dfb7d2f 100644 (file)
@@ -1,6 +1,4 @@
-/* -*- mode: c; c-basic-offset: 8; indent-tabs-mode: nil; -*-
- * vim:expandtab:shiftwidth=8:tabstop=8:
- *
+/*
  * GPL HEADER START
  *
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
@@ -137,12 +135,12 @@ int filter_finish_transno(struct obd_export *exp, struct inode *inode,
         if (!exp->exp_obd->obd_replayable || oti == NULL)
                 RETURN(rc);
 
-        cfs_mutex_down(&ted->ted_lcd_lock);
+        cfs_mutex_lock(&ted->ted_lcd_lock);
         lcd = ted->ted_lcd;
         /* if the export has already been disconnected, we have no last_rcvd slot,
          * update server data with latest transno then */
         if (lcd == NULL) {
-                cfs_mutex_up(&ted->ted_lcd_lock);
+                cfs_mutex_unlock(&ted->ted_lcd_lock);
                 CWARN("commit transaction for disconnected client %s: rc %d\n",
                       exp->exp_client_uuid.uuid, rc);
                 err = filter_update_server_data(exp->exp_obd);
@@ -167,7 +165,7 @@ int filter_finish_transno(struct obd_export *exp, struct inode *inode,
                         exp->exp_vbr_failed = 1;
                         cfs_spin_unlock(&exp->exp_lock);
                         cfs_spin_unlock(&obt->obt_lut->lut_translock);
-                        cfs_mutex_up(&ted->ted_lcd_lock);
+                        cfs_mutex_unlock(&ted->ted_lcd_lock);
                         RETURN(-EOVERFLOW);
                 }
         }
@@ -209,7 +207,7 @@ int filter_finish_transno(struct obd_export *exp, struct inode *inode,
 
         CDEBUG(log_pri, "wrote trans "LPU64" for client %s at #%d: err = %d\n",
                last_rcvd, lcd->lcd_uuid, ted->ted_lr_idx, err);
-        cfs_mutex_up(&ted->ted_lcd_lock);
+        cfs_mutex_unlock(&ted->ted_lcd_lock);
         RETURN(rc);
 }
 
@@ -354,7 +352,7 @@ static int filter_client_add(struct obd_device *obd, struct obd_export *exp,
         ted->ted_lr_idx = cl_idx;
         ted->ted_lr_off = le32_to_cpu(lsd->lsd_client_start) +
                           cl_idx * le16_to_cpu(lsd->lsd_client_size);
-        cfs_init_mutex(&ted->ted_lcd_lock);
+        cfs_mutex_init(&ted->ted_lcd_lock);
         LASSERTF(ted->ted_lr_off > 0, "ted_lr_off = %llu\n", ted->ted_lr_off);
 
         CDEBUG(D_INFO, "client at index %d (%llu) with UUID '%s' added\n",
@@ -455,12 +453,12 @@ static int filter_client_del(struct obd_export *exp)
          * be in server data or in client data in case of failure */
         filter_update_server_data(exp->exp_obd);
 
-        cfs_mutex_down(&ted->ted_lcd_lock);
+        cfs_mutex_lock(&ted->ted_lcd_lock);
         memset(ted->ted_lcd->lcd_uuid, 0, sizeof ted->ted_lcd->lcd_uuid);
         rc = fsfilt_write_record(exp->exp_obd, obt->obt_rcvd_filp,
                                  ted->ted_lcd,
                                  sizeof(*ted->ted_lcd), &off, 0);
-        cfs_mutex_up(&ted->ted_lcd_lock);
+        cfs_mutex_unlock(&ted->ted_lcd_lock);
         pop_ctxt(&saved, &exp->exp_obd->obd_lvfs_ctxt, NULL);
 
         CDEBUG(rc == 0 ? D_INFO : D_ERROR,
@@ -1198,14 +1196,14 @@ static int filter_read_groups(struct obd_device *obd, int last_group,
         struct filter_obd *filter = &obd->u.filter;
         int old_count, group, rc = 0;
 
-        cfs_down(&filter->fo_init_lock);
+        cfs_mutex_lock(&filter->fo_init_lock);
         old_count = filter->fo_group_count;
         for (group = old_count; group <= last_group; group++) {
                 rc = filter_read_group_internal(obd, group, create);
                 if (rc != 0)
                         break;
         }
-        cfs_up(&filter->fo_init_lock);
+        cfs_mutex_unlock(&filter->fo_init_lock);
         return rc;
 }
 
@@ -1777,7 +1775,7 @@ static int filter_intent_policy(struct ldlm_namespace *ns,
          * therefore, that res->lr_lvb_data cannot increase beyond the
          * end of already granted lock. As a result, it is safe to
          * check against "stale" reply_lvb->lvb_size value without
-         * res->lr_lvb_sem.
+         * res->lr_lvb_mutex.
          */
         arg.size = reply_lvb->lvb_size;
         arg.victim = &l;
@@ -1923,7 +1921,7 @@ static int filter_iobuf_pool_init(struct filter_obd *filter)
 
 
         OBD_ALLOC_GFP(filter->fo_iobuf_pool, OSS_THREADS_MAX * sizeof(*pool),
-                      GFP_KERNEL);
+                     CFS_ALLOC_KERNEL);
         if (filter->fo_iobuf_pool == NULL)
                 RETURN(-ENOMEM);
 
@@ -1969,6 +1967,7 @@ int filter_common_setup(struct obd_device *obd, struct lustre_cfg* lcfg,
 {
         struct filter_obd *filter = &obd->u.filter;
         struct vfsmount *mnt;
+        struct file_system_type *type;
         struct lustre_mount_info *lmi;
         struct obd_uuid uuid;
         __u8 *uuid_ptr;
@@ -1993,9 +1992,14 @@ int filter_common_setup(struct obd_device *obd, struct lustre_cfg* lcfg,
         } else {
                 /* old path - used by lctl */
                 CERROR("Using old MDS mount method\n");
-                mnt = ll_kern_mount(lustre_cfg_string(lcfg, 2),
-                                    MS_NOATIME|MS_NODIRATIME,
-                                    lustre_cfg_string(lcfg, 1), option);
+                type = get_fs_type(lustre_cfg_string(lcfg, 2));
+                if (!type) {
+                        CERROR("get_fs_type failed\n");
+                        RETURN(-ENODEV);
+                }
+                mnt = vfs_kern_mount(type, MS_NOATIME|MS_NODIRATIME,
+                                     lustre_cfg_string(lcfg, 1), option);
+                cfs_module_put(type->owner);
                 if (IS_ERR(mnt)) {
                         rc = PTR_ERR(mnt);
                         LCONSOLE_ERROR_MSG(0x135, "Can't mount disk %s (%d)\n",
@@ -2038,7 +2042,9 @@ int filter_common_setup(struct obd_device *obd, struct lustre_cfg* lcfg,
         filter->fo_fstype = mnt->mnt_sb->s_type->name;
         CDEBUG(D_SUPER, "%s: mnt = %p\n", filter->fo_fstype, mnt);
 
-        fsfilt_setup(obd, obd->u.obt.obt_sb);
+        rc = fsfilt_setup(obd, obd->u.obt.obt_sb);
+        if (rc)
+                GOTO(err_ops, rc);
 
         OBD_SET_CTXT_MAGIC(&obd->obd_lvfs_ctxt);
         obd->obd_lvfs_ctxt.pwdmnt = mnt;
@@ -2046,15 +2052,15 @@ int filter_common_setup(struct obd_device *obd, struct lustre_cfg* lcfg,
         obd->obd_lvfs_ctxt.fs = get_ds();
         obd->obd_lvfs_ctxt.cb_ops = filter_lvfs_ops;
 
-        cfs_init_mutex(&filter->fo_init_lock);
+        cfs_mutex_init(&filter->fo_init_lock);
         filter->fo_committed_group = 0;
         filter->fo_destroys_in_progress = 0;
         for (i = 0; i < 32; i++)
-                cfs_sema_init(&filter->fo_create_locks[i], 1);
+                cfs_mutex_init(&filter->fo_create_locks[i]);
 
         cfs_spin_lock_init(&filter->fo_objidlock);
         CFS_INIT_LIST_HEAD(&filter->fo_export_list);
-        cfs_sema_init(&filter->fo_alloc_lock, 1);
+        cfs_mutex_init(&filter->fo_alloc_lock);
         init_brw_stats(&filter->fo_filter_stats);
         cfs_spin_lock_init(&filter->fo_flags_lock);
         filter->fo_read_cache = 1; /* enable read-only cache by default */
@@ -2218,10 +2224,15 @@ static int filter_setup(struct obd_device *obd, struct lustre_cfg* lcfg)
                 GOTO(free_obd_stats, rc);
         }
 
+       rc = lprocfs_job_stats_init(obd, LPROC_FILTER_STATS_LAST,
+                                   filter_stats_counter_init);
+       if (rc)
+               GOTO(remove_entry_clear, rc);
+
         /* 2.6.9 selinux wants a full option page for do_kern_mount (bug6471) */
         OBD_PAGE_ALLOC(page, CFS_ALLOC_STD);
         if (!page)
-                GOTO(remove_entry_clear, rc = -ENOMEM);
+               GOTO(job_stats_fini, rc = -ENOMEM);
         addr = (unsigned long)cfs_page_address(page);
         clear_page((void *)addr);
         memcpy((void *)addr, lustre_cfg_buf(lcfg, 4),
@@ -2231,11 +2242,13 @@ static int filter_setup(struct obd_device *obd, struct lustre_cfg* lcfg)
         if (rc) {
                 CERROR("%s: filter_common_setup failed: %d.\n",
                        obd->obd_name, rc);
-                GOTO(remove_entry_clear, rc);
+               GOTO(job_stats_fini, rc);
         }
 
         RETURN(0);
 
+job_stats_fini:
+       lprocfs_job_stats_fini(obd);
 remove_entry_clear:
         lprocfs_remove_proc_entry("clear", obd->obd_proc_exports_entry);
 free_obd_stats:
@@ -2395,28 +2408,28 @@ static int filter_llog_finish(struct obd_device *obd, int count)
                  * We actually do sync in disconnect time, but disconnect
                  * may not come being marked rq_no_resend = 1.
                  */
-                llog_sync(ctxt, NULL);
+               llog_sync(ctxt, NULL, OBD_LLOG_FL_EXIT);
 
                 /*
                  * Balance class_import_get() in llog_receptor_accept().
                  * This is safe to do, as llog is already synchronized
                  * and its import may go.
                  */
-                cfs_mutex_down(&ctxt->loc_sem);
+                cfs_mutex_lock(&ctxt->loc_mutex);
                 if (ctxt->loc_imp) {
                         class_import_put(ctxt->loc_imp);
                         ctxt->loc_imp = NULL;
                 }
-                cfs_mutex_up(&ctxt->loc_sem);
+
+               if (filter->fo_lcm) {
+                       llog_recov_thread_fini(filter->fo_lcm, obd->obd_force);
+                       filter->fo_lcm = NULL;
+               }
+
+                cfs_mutex_unlock(&ctxt->loc_mutex);
                 llog_ctxt_put(ctxt);
         }
 
-        if (filter->fo_lcm) {
-                cfs_mutex_down(&ctxt->loc_sem);
-                llog_recov_thread_fini(filter->fo_lcm, obd->obd_force);
-                filter->fo_lcm = NULL;
-                cfs_mutex_up(&ctxt->loc_sem);
-        }
         RETURN(filter_olg_fini(&obd->obd_olg));
 }
 
@@ -2542,9 +2555,9 @@ static int filter_llog_connect(struct obd_export *exp,
         LASSERTF(ctxt != NULL, "ctxt is not null, ctxt idx %d \n",
                  body->lgdc_ctxt_idx);
 
-        CWARN("%s: Recovery from log "LPX64"/"LPX64":%x\n",
-              obd->obd_name, body->lgdc_logid.lgl_oid,
-              body->lgdc_logid.lgl_oseq, body->lgdc_logid.lgl_ogen);
+        CDEBUG(D_HA, "%s: Recovery from log "LPX64"/"LPX64":%x\n",
+               obd->obd_name, body->lgdc_logid.lgl_oid,
+               body->lgdc_logid.lgl_oseq, body->lgdc_logid.lgl_ogen);
 
         cfs_spin_lock(&obd->u.filter.fo_flags_lock);
         obd->u.filter.fo_mds_ost_sync = 1;
@@ -2612,6 +2625,7 @@ static int filter_precleanup(struct obd_device *obd,
                 obd_zombie_barrier();
 
                 rc = filter_llog_preclean(obd);
+               lprocfs_job_stats_fini(obd);
                 lprocfs_remove_proc_entry("clear", obd->obd_proc_exports_entry);
                 lprocfs_free_per_client_stats(obd);
                 lprocfs_obd_cleanup(obd);
@@ -2687,8 +2701,8 @@ static int filter_connect_internal(struct obd_export *exp,
 
         /* Kindly make sure the SKIP_ORPHAN flag is from MDS. */
         if (data->ocd_connect_flags & OBD_CONNECT_MDS)
-                CWARN("%s: Received MDS connection for group %u\n",
-                      exp->exp_obd->obd_name, data->ocd_group);
+                CDEBUG(D_HA, "%s: Received MDS connection for group %u\n",
+                       exp->exp_obd->obd_name, data->ocd_group);
         else if (data->ocd_connect_flags & OBD_CONNECT_SKIP_ORPHAN)
                 RETURN(-EPROTO);
 
@@ -3052,7 +3066,7 @@ static void filter_sync_llogs(struct obd_device *obd, struct obd_export *dexp)
                         ctxt = llog_group_get_ctxt(olg_min,
                                                    LLOG_MDS_OST_REPL_CTXT);
                         if (ctxt) {
-                                err = llog_sync(ctxt, olg_min->olg_exp);
+                               err = llog_sync(ctxt, olg_min->olg_exp, 0);
                                 llog_ctxt_put(ctxt);
                                 if (err) {
                                         CERROR("error flushing logs to MDS: "
@@ -3108,7 +3122,7 @@ static void filter_revimp_update(struct obd_export *exp)
         EXIT;
 }
 
-static int filter_ping(struct obd_export *exp)
+static int filter_ping(const struct lu_env *env, struct obd_export *exp)
 {
         filter_fmd_expire(exp);
         return 0;
@@ -3154,10 +3168,12 @@ struct dentry *__filter_oa2dentry(struct obd_device *obd, struct ost_id *ostid,
         return dchild;
 }
 
-static int filter_getattr(struct obd_export *exp, struct obd_info *oinfo)
+static int filter_getattr(const struct lu_env *env, struct obd_export *exp,
+                          struct obd_info *oinfo)
 {
         struct dentry *dentry = NULL;
         struct obd_device *obd;
+        __u64 curr_version;
         int rc = 0;
         ENTRY;
 
@@ -3178,7 +3194,14 @@ static int filter_getattr(struct obd_export *exp, struct obd_info *oinfo)
 
         /* Limit the valid bits in the return data to what we actually use */
         oinfo->oi_oa->o_valid = OBD_MD_FLID;
-        obdo_from_inode(oinfo->oi_oa, dentry->d_inode, NULL, FILTER_VALID_FLAGS);
+        obdo_from_inode(oinfo->oi_oa, dentry->d_inode, FILTER_VALID_FLAGS);
+
+        /* Store inode version in reply */
+        curr_version = fsfilt_get_version(exp->exp_obd, dentry->d_inode);
+        if ((__s64)curr_version != -EOPNOTSUPP) {
+                oinfo->oi_oa->o_valid |= OBD_MD_FLDATAVERSION;
+                oinfo->oi_oa->o_data_version = curr_version;
+        }
 
         f_dput(dentry);
         RETURN(rc);
@@ -3403,8 +3426,8 @@ out_unlock:
 }
 
 /* this is called from filter_truncate() until we have filter_punch() */
-int filter_setattr(struct obd_export *exp, struct obd_info *oinfo,
-                   struct obd_trans_info *oti)
+int filter_setattr(const struct lu_env *env, struct obd_export *exp,
+                   struct obd_info *oinfo, struct obd_trans_info *oti)
 {
         struct obdo *oa = oinfo->oi_oa;
         struct lustre_capa *capa = oinfo_capa(oinfo);
@@ -3495,9 +3518,11 @@ int filter_setattr(struct obd_export *exp, struct obd_info *oinfo,
         oa->o_valid = OBD_MD_FLID;
 
         /* Quota release need uid/gid info */
-        obdo_from_inode(oa, dentry->d_inode, NULL,
+        obdo_from_inode(oa, dentry->d_inode,
                         FILTER_VALID_FLAGS | OBD_MD_FLUID | OBD_MD_FLGID);
 
+       filter_counter_incr(exp, LPROC_FILTER_STATS_SETATTR,
+                           oti ? oti->oti_jobid : NULL, 1);
         EXIT;
 out_unlock:
         f_dput(dentry);
@@ -3571,7 +3596,7 @@ static int filter_destroy_precreated(struct obd_export *exp, struct obdo *oa,
         int skip_orphan;
         ENTRY;
 
-        LASSERT(down_trylock(&filter->fo_create_locks[oa->o_seq]) != 0);
+        LASSERT_MUTEX_LOCKED(&filter->fo_create_locks[oa->o_seq]);
 
         memset(&doa, 0, sizeof(doa));
 
@@ -3595,7 +3620,7 @@ static int filter_destroy_precreated(struct obd_export *exp, struct obdo *oa,
 
         for (id = last; id > oa->o_id; id--) {
                 doa.o_id = id;
-                rc = filter_destroy(exp, &doa, NULL, NULL, NULL, NULL);
+                rc = filter_destroy(NULL, exp, &doa, NULL, NULL, NULL, NULL);
                 if (rc && rc != -ENOENT) /* this is pretty fatal... */
                         CEMERG("error destroying precreate objid "LPU64": %d\n",
                                id, rc);
@@ -3653,18 +3678,18 @@ static int filter_handle_precreate(struct obd_export *exp, struct obdo *oa,
                 }
                 /* This causes inflight precreates to abort and drop lock */
                 cfs_set_bit(group, &filter->fo_destroys_in_progress);
-                cfs_down(&filter->fo_create_locks[group]);
+                cfs_mutex_lock(&filter->fo_create_locks[group]);
                 if (!cfs_test_bit(group, &filter->fo_destroys_in_progress)) {
                         CERROR("%s:["LPU64"] destroys_in_progress already cleared\n",
                                exp->exp_obd->obd_name, group);
-                        cfs_up(&filter->fo_create_locks[group]);
+                        cfs_mutex_unlock(&filter->fo_create_locks[group]);
                         RETURN(0);
                 }
                 diff = oa->o_id - last;
                 CDEBUG(D_HA, "filter_last_id() = "LPU64" -> diff = %d\n",
                        last, diff);
 
-                if (-diff > OST_MAX_PRECREATE) {
+                if (-diff > (OST_MAX_PRECREATE * 3) / 2) {
                         CERROR("%s: ignoring bogus orphan destroy request: "
                                "obdid "LPU64" last_id "LPU64"\n", obd->obd_name,
                                oa->o_id, last);
@@ -3682,7 +3707,7 @@ static int filter_handle_precreate(struct obd_export *exp, struct obdo *oa,
                         cfs_clear_bit(group, &filter->fo_destroys_in_progress);
                 }
         } else {
-                cfs_down(&filter->fo_create_locks[group]);
+                cfs_mutex_lock(&filter->fo_create_locks[group]);
                 if (oti->oti_conn_cnt < exp->exp_conn_cnt) {
                         CERROR("%s: dropping old precreate request\n",
                                obd->obd_name);
@@ -3711,13 +3736,14 @@ static int filter_handle_precreate(struct obd_export *exp, struct obdo *oa,
         /* else diff == 0 */
         GOTO(out, rc = 0);
 out:
-        cfs_up(&filter->fo_create_locks[group]);
+        cfs_mutex_unlock(&filter->fo_create_locks[group]);
         return rc;
 }
 
-static int filter_statfs(struct obd_device *obd, struct obd_statfs *osfs,
-                         __u64 max_age, __u32 flags)
+static int filter_statfs(const struct lu_env *env, struct obd_export *exp,
+                         struct obd_statfs *osfs, __u64 max_age, __u32 flags)
 {
+        struct obd_device *obd = class_exp2obd(exp);
         struct filter_obd *filter = &obd->u.filter;
         int blockbits = obd->u.obt.obt_sb->s_blocksize_bits;
         struct lr_server_data *lsd = class_server_data(obd);
@@ -3830,7 +3856,7 @@ static int filter_precreate(struct obd_device *obd, struct obdo *oa,
 
         filter = &obd->u.filter;
 
-        LASSERT(down_trylock(&filter->fo_create_locks[group]) != 0);
+        LASSERT_MUTEX_LOCKED(&filter->fo_create_locks[group]);
 
         OBD_FAIL_TIMEOUT(OBD_FAIL_TGT_DELAY_PRECREATE, obd_timeout / 2);
 
@@ -3841,7 +3867,7 @@ static int filter_precreate(struct obd_device *obd, struct obdo *oa,
                 OBD_ALLOC(osfs, sizeof(*osfs));
                 if (osfs == NULL)
                         RETURN(-ENOMEM);
-                rc = filter_statfs(obd, osfs,
+                rc = filter_statfs(NULL, obd->obd_self_export, osfs,
                                    cfs_time_shift_64(-OBD_STATFS_CACHE_SECONDS),
                                    0);
                 if (rc == 0 && osfs->os_bavail < (osfs->os_blocks >> 10)) {
@@ -4043,8 +4069,9 @@ set_last_id:
         RETURN(rc);
 }
 
-int filter_create(struct obd_export *exp, struct obdo *oa,
-                  struct lov_stripe_md **ea, struct obd_trans_info *oti)
+int filter_create(const struct lu_env *env, struct obd_export *exp,
+                  struct obdo *oa, struct lov_stripe_md **ea,
+                  struct obd_trans_info *oti)
 {
         struct obd_device *obd = exp->exp_obd;
         struct filter_export_data *fed;
@@ -4100,9 +4127,9 @@ int filter_create(struct obd_export *exp, struct obdo *oa,
                         rc = -EINVAL;
                 } else {
                         diff = 1;
-                        cfs_down(&filter->fo_create_locks[oa->o_seq]);
+                        cfs_mutex_lock(&filter->fo_create_locks[oa->o_seq]);
                         rc = filter_precreate(obd, oa, oa->o_seq, &diff);
-                        cfs_up(&filter->fo_create_locks[oa->o_seq]);
+                        cfs_mutex_unlock(&filter->fo_create_locks[oa->o_seq]);
                 }
         } else {
                 rc = filter_handle_precreate(exp, oa, oa->o_seq, oti);
@@ -4123,9 +4150,10 @@ int filter_create(struct obd_export *exp, struct obdo *oa,
         RETURN(rc);
 }
 
-int filter_destroy(struct obd_export *exp, struct obdo *oa,
-                   struct lov_stripe_md *md, struct obd_trans_info *oti,
-                   struct obd_export *md_exp, void *capa)
+int filter_destroy(const struct lu_env *env, struct obd_export *exp,
+                   struct obdo *oa, struct lov_stripe_md *md,
+                   struct obd_trans_info *oti, struct obd_export *md_exp,
+                   void *capa)
 {
         unsigned int qcids[MAXQUOTAS] = {0, 0};
         struct obd_device *obd;
@@ -4258,7 +4286,7 @@ int filter_destroy(struct obd_export *exp, struct obdo *oa,
         cleanup_phase = 4; /* fsfilt_commit */
 
         /* Quota release need uid/gid of inode */
-        obdo_from_inode(oa, dchild->d_inode, NULL, OBD_MD_FLUID|OBD_MD_FLGID);
+        obdo_from_inode(oa, dchild->d_inode, OBD_MD_FLUID | OBD_MD_FLGID);
 
         filter_fmd_drop(exp, oa->o_id, oa->o_seq);
 
@@ -4319,8 +4347,8 @@ cleanup:
 }
 
 /* NB start and end are used for punch, but not truncate */
-static int filter_truncate(struct obd_export *exp, struct obd_info *oinfo,
-                           struct obd_trans_info *oti,
+static int filter_truncate(const struct lu_env *env, struct obd_export *exp,
+                           struct obd_info *oinfo, struct obd_trans_info *oti,
                            struct ptlrpc_request_set *rqset)
 {
         int rc;
@@ -4337,12 +4365,12 @@ static int filter_truncate(struct obd_export *exp, struct obd_info *oinfo,
                 oinfo->oi_policy.l_extent.start);
 
         oinfo->oi_oa->o_size = oinfo->oi_policy.l_extent.start;
-        rc = filter_setattr(exp, oinfo, oti);
+        rc = filter_setattr(env, exp, oinfo, oti);
         RETURN(rc);
 }
 
-static int filter_sync(struct obd_export *exp, struct obd_info *oinfo,
-                       obd_off start, obd_off end,
+static int filter_sync(const struct lu_env *env, struct obd_export *exp,
+                       struct obd_info *oinfo, obd_off start, obd_off end,
                        struct ptlrpc_request_set *set)
 {
         struct lvfs_run_ctxt saved;
@@ -4390,17 +4418,17 @@ static int filter_sync(struct obd_export *exp, struct obd_info *oinfo,
         UNLOCK_INODE_MUTEX(dentry->d_inode);
 
         oinfo->oi_oa->o_valid = OBD_MD_FLID;
-        obdo_from_inode(oinfo->oi_oa, dentry->d_inode, NULL,
-                        FILTER_VALID_FLAGS);
+        obdo_from_inode(oinfo->oi_oa, dentry->d_inode, FILTER_VALID_FLAGS);
 
         pop_ctxt(&saved, &exp->exp_obd->obd_lvfs_ctxt, NULL);
 
+       filter_counter_incr(exp, LPROC_FILTER_STATS_SYNC, oinfo->oi_jobid, 1);
         f_dput(dentry);
         RETURN(rc);
 }
 
-static int filter_get_info(struct obd_export *exp, __u32 keylen,
-                           void *key, __u32 *vallen, void *val,
+static int filter_get_info(const struct lu_env *env, struct obd_export *exp,
+                           __u32 keylen, void *key, __u32 *vallen, void *val,
                            struct lov_stripe_md *lsm)
 {
         struct obd_device *obd;
@@ -4556,7 +4584,8 @@ out:
         RETURN(rc);
 }
 
-static int filter_set_info_async(struct obd_export *exp, __u32 keylen,
+static int filter_set_info_async(const struct lu_env *env,
+                                 struct obd_export *exp, __u32 keylen,
                                  void *key, __u32 vallen, void *val,
                                  struct ptlrpc_request_set *set)
 {
@@ -4632,8 +4661,8 @@ int filter_iocontrol(unsigned int cmd, struct obd_export *exp,
                 CDEBUG(D_HA, "syncing ost %s\n", obd->obd_name);
                 rc = fsfilt_sync(obd, obd->u.obt.obt_sb);
 
-                lvfs_set_rdonly(obd, obd->u.obt.obt_sb);
-                RETURN(0);
+                rc = lvfs_set_rdonly(obd, obd->u.obt.obt_sb);
+                RETURN(rc);
         }
 
         case OBD_IOC_CATLOGLIST: {
@@ -4665,7 +4694,7 @@ int filter_iocontrol(unsigned int cmd, struct obd_export *exp,
         RETURN(0);
 }
 
-static int filter_health_check(struct obd_device *obd)
+static int filter_health_check(const struct lu_env *env, struct obd_device *obd)
 {
 #ifdef USE_HEALTH_CHECK_WRITE
         struct filter_obd *filter = &obd->u.filter;
@@ -4754,7 +4783,6 @@ static struct obd_ops filter_obd_ops = {
         .o_create         = filter_create,
         .o_setattr        = filter_setattr,
         .o_destroy        = filter_destroy,
-        .o_brw            = filter_brw,
         .o_punch          = filter_truncate,
         .o_sync           = filter_sync,
         .o_preprw         = filter_preprw,