void *lut_cb_data;
};
-void lut_boot_epoch_update(struct lu_target *);
+void lut_boot_epoch_update(struct lu_target *lut);
int lut_last_commit_cb_add(struct thandle *th, struct lu_target *lut,
- struct obd_export *exp, __u64 transno);
+ struct obd_export *exp, __u64 transno);
int lut_new_client_cb_add(struct thandle *th, struct obd_export *exp);
-int lut_init(const struct lu_env *, struct lu_target *,
- struct obd_device *, struct dt_device *);
-void lut_fini(const struct lu_env *, struct lu_target *);
-int lut_client_alloc(struct obd_export *);
-void lut_client_free(struct obd_export *);
+int lut_init(const struct lu_env *env, struct lu_target *lut,
+ struct obd_device *obd, struct dt_device *dt);
+void lut_fini(const struct lu_env *env, struct lu_target *lut);
+int lut_client_alloc(struct obd_export *exp);
+void lut_client_free(struct obd_export *exp);
+int lut_client_del(const struct lu_env *env, struct obd_export *exp);
+int lut_client_add(const struct lu_env *env, struct obd_export *exp, int);
+int lut_client_new(const struct lu_env *env, struct obd_export *exp);
+int lut_client_data_read(const struct lu_env *env, struct lu_target *tg,
+ struct lsd_client_data *lcd, loff_t *off, int index);
+int lut_client_data_write(const struct lu_env *env, struct lu_target *tg,
+ struct lsd_client_data *lcd, loff_t *off, struct thandle *th);
+int lut_server_data_read(const struct lu_env *env, struct lu_target *tg);
+int lut_server_data_write(const struct lu_env *env, struct lu_target *tg,
+ struct thandle *th);
+int lut_server_data_update(const struct lu_env *env, struct lu_target *tg, int sync);
+int lut_truncate_last_rcvd(const struct lu_env *env, struct lu_target *tg, loff_t off);
+
#endif /* __LUSTRE_LU_TARGET_H */
int i, rc;
mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
- th = mdt_trans_create(env, mdt);
- if (IS_ERR(th))
- RETURN(PTR_ERR(th));
+ th = dt_trans_create(env, mdt->mdt_bottom);
+ if (IS_ERR(th))
+ RETURN(PTR_ERR(th));
- rc = dt_declare_record_write(env, mdt->mdt_ck_obj,
- sizeof(*tmp) * 3, 0, th);
- if (rc)
- goto stop;
+ rc = dt_declare_record_write(env, mdt->mdt_ck_obj,
+ sizeof(*tmp) * 3, 0, th);
+ if (rc)
+ goto stop;
- rc = mdt_trans_start(env, mdt, th);
+ rc = dt_trans_start_local(env, mdt->mdt_bottom, th);
if (rc)
goto stop;
}
stop:
- mdt_trans_stop(env, mdt, th);
+ dt_trans_stop(env, mdt->mdt_bottom, th);
CDEBUG(D_INFO, "write capability keys rc = %d:\n", rc);
return rc;
rc = mdt_connect_internal(lexp, mdt, data);
if (rc == 0) {
- struct mdt_thread_info *mti;
struct lsd_client_data *lcd = lexp->exp_target_data.ted_lcd;
+
LASSERT(lcd);
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
- LASSERT(mti != NULL);
- mti->mti_exp = lexp;
- memcpy(lcd->lcd_uuid, cluuid, sizeof lcd->lcd_uuid);
- rc = mdt_client_new(env, mdt);
+ info->mti_exp = lexp;
+ memcpy(lcd->lcd_uuid, cluuid, sizeof lcd->lcd_uuid);
+ rc = lut_client_new(env, lexp);
if (rc == 0)
mdt_export_stats_init(obd, lexp, localdata);
}
RETURN(rc);
}
+
static int mdt_export_cleanup(struct obd_export *exp)
{
struct mdt_export_data *med = &exp->exp_mdt_data;
/* cleanup client slot early */
/* Do not erase record for recoverable client. */
if (!(exp->exp_flags & OBD_OPT_FAILOVER) || exp->exp_failed)
- mdt_client_del(&env, mdt);
+ lut_client_del(&env, exp);
lu_env_fini(&env);
RETURN(rc);
.ldt_ctx_tags = LCT_MD_THREAD
};
-static struct lu_local_obj_desc mdt_last_recv = {
- .llod_name = LAST_RCVD,
- .llod_oid = MDT_LAST_RECV_OID,
- .llod_is_index = 0,
-};
-
static int __init mdt_mod_init(void)
{
struct lprocfs_static_vars lvars;
int rc;
- llo_local_obj_register(&mdt_last_recv);
-
if (mdt_num_threads > 0) {
if (mdt_num_threads > MDT_MAX_THREADS)
mdt_num_threads = MDT_MAX_THREADS;
static void __exit mdt_mod_exit(void)
{
- llo_local_obj_unregister(&mdt_last_recv);
class_unregister_type(LUSTRE_MDT_NAME);
}
struct mdt_ioepoch *mti_ioepoch;
__u64 mti_replayepoch;
- /* server and client data buffers */
- struct lr_server_data mti_lsd;
- struct lsd_client_data mti_lcd;
loff_t mti_off;
struct lu_buf mti_buf;
struct lustre_capa_key mti_capa_key;
struct lu_name mti_name;
};
-typedef void (*mdt_cb_t)(const struct mdt_device *mdt, __u64 transno,
- void *data, int err);
-struct mdt_commit_cb {
- mdt_cb_t mdt_cb_func;
- void *mdt_cb_data;
-};
-
-enum mdt_txn_op {
- MDT_TXN_CAPA_KEYS_WRITE_OP,
- MDT_TXN_LAST_RCVD_WRITE_OP,
-};
-
static inline const struct md_device_operations *
mdt_child_ops(struct mdt_device * m)
{
struct obd_device *, struct lustre_sb_info *lsi);
void mdt_fs_cleanup(const struct lu_env *, struct mdt_device *);
-int mdt_client_del(const struct lu_env *env,
- struct mdt_device *mdt);
-int mdt_client_add(const struct lu_env *env,
- struct mdt_device *mdt,
- int cl_idx);
-int mdt_client_new(const struct lu_env *env,
- struct mdt_device *mdt);
-
int mdt_export_stats_init(struct obd_device *obd,
struct obd_export *exp,
void *client_nid);
const struct md_attr *);
void mdt_reconstruct_open(struct mdt_thread_info *, struct mdt_lock_handle *);
-struct thandle *mdt_trans_create(const struct lu_env *env,
- struct mdt_device *mdt);
-int mdt_trans_start(const struct lu_env *env, struct mdt_device *mdt,
- struct thandle *th);
-void mdt_trans_stop(const struct lu_env *env,
- struct mdt_device *mdt, struct thandle *th);
-int mdt_record_write(const struct lu_env *env,
- struct dt_object *dt, const struct lu_buf *buf,
- loff_t *pos, struct thandle *th);
-int mdt_record_read(const struct lu_env *env,
- struct dt_object *dt, struct lu_buf *buf, loff_t *pos);
-
struct lu_buf *mdt_buf(const struct lu_env *env, void *area, ssize_t len);
const struct lu_buf *mdt_buf_const(const struct lu_env *env,
const void *area, ssize_t len);
#include "mdt_internal.h"
-static int mdt_server_data_update(const struct lu_env *env,
- struct mdt_device *mdt);
-
struct lu_buf *mdt_buf(const struct lu_env *env, void *area, ssize_t len)
{
struct lu_buf *buf;
return buf;
}
-struct thandle *mdt_trans_create(const struct lu_env *env,
- struct mdt_device *mdt)
-{
- return mdt->mdt_bottom->dd_ops->dt_trans_create(env, mdt->mdt_bottom);
-}
-
-int mdt_trans_start(const struct lu_env *env, struct mdt_device *mdt,
- struct thandle *th)
-{
- return dt_trans_start(env, mdt->mdt_bottom, th);
-}
-
void mdt_trans_stop(const struct lu_env *env,
struct mdt_device *mdt, struct thandle *th)
{
- struct mdt_thread_info *mti;
-
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
- /* export can require sync operations */
- if (mti->mti_exp != NULL)
- th->th_sync |= mti->mti_exp->exp_need_sync;
dt_trans_stop(env, mdt->mdt_bottom, th);
}
-static inline int mdt_last_rcvd_header_read(const struct lu_env *env,
- struct mdt_device *mdt)
-{
- struct mdt_thread_info *mti;
- int rc;
-
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
-
- mti->mti_off = 0;
- rc = dt_record_read(env, mdt->mdt_lut.lut_last_rcvd,
- mdt_buf(env, &mti->mti_lsd, sizeof(mti->mti_lsd)),
- &mti->mti_off);
- if (rc == 0)
- lsd_le_to_cpu(&mti->mti_lsd, &mdt->mdt_lut.lut_lsd);
-
- CDEBUG(D_INFO, "read last_rcvd header rc = %d, uuid = %s, "
- "last_transno = "LPU64"\n", rc, mdt->mdt_lut.lut_lsd.lsd_uuid,
- mdt->mdt_lut.lut_lsd.lsd_last_transno);
- return rc;
-}
-
-static int mdt_declare_last_rcvd_header_write(const struct lu_env *env,
- struct mdt_device *mdt,
- struct thandle *th)
-{
- struct mdt_thread_info *mti;
-
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
-
- return dt_declare_record_write(env, mdt->mdt_lut.lut_last_rcvd,
- sizeof(mti->mti_lsd), 0, th);
-}
-
-static int mdt_last_rcvd_header_write(const struct lu_env *env,
- struct mdt_device *mdt,
- struct thandle *th)
-{
- struct mdt_thread_info *mti;
- int rc;
- ENTRY;
-
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
-
- mti->mti_off = 0;
- lsd_cpu_to_le(&mdt->mdt_lut.lut_lsd, &mti->mti_lsd);
-
- rc = dt_record_write(env, mdt->mdt_lut.lut_last_rcvd,
- mdt_buf_const(env, &mti->mti_lsd,
- sizeof(mti->mti_lsd)),
- &mti->mti_off, th);
-
- CDEBUG(D_INFO, "write last_rcvd header rc = %d, uuid = %s, "
- "last_transno = "LPU64"\n", rc, mdt->mdt_lut.lut_lsd.lsd_uuid,
- mdt->mdt_lut.lut_lsd.lsd_last_transno);
-
- RETURN(rc);
-}
-
-static int mdt_last_rcvd_read(const struct lu_env *env, struct mdt_device *mdt,
- struct lsd_client_data *lcd, loff_t *off,
- int index)
-{
- struct mdt_thread_info *mti;
- struct lsd_client_data *tmp;
- int rc;
-
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
- tmp = &mti->mti_lcd;
- rc = dt_record_read(env, mdt->mdt_lut.lut_last_rcvd,
- mdt_buf(env, tmp, sizeof(*tmp)), off);
- if (rc == 0) {
- check_lcd(mdt2obd_dev(mdt)->obd_name, index, tmp);
- lcd_le_to_cpu(tmp, lcd);
- }
-
- CDEBUG(D_INFO, "read lcd @%d rc = %d, uuid = %s, last_transno = "LPU64
- ", last_xid = "LPU64", last_result = %u, last_data = %u, "
- "last_close_transno = "LPU64", last_close_xid = "LPU64", "
- "last_close_result = %u\n", (int)(*off - sizeof(*tmp)),
- rc, lcd->lcd_uuid, lcd->lcd_last_transno, lcd->lcd_last_xid,
- lcd->lcd_last_result, lcd->lcd_last_data,
- lcd->lcd_last_close_transno, lcd->lcd_last_close_xid,
- lcd->lcd_last_close_result);
- return rc;
-}
-
-static int mdt_declare_last_rcvd_write(const struct lu_env *env,
- struct mdt_device *mdt,
- loff_t off, struct thandle *th)
-{
- return dt_declare_record_write(env, mdt->mdt_lut.lut_last_rcvd,
- sizeof(struct lsd_client_data), off, th);
-}
-
-static int mdt_last_rcvd_write(const struct lu_env *env,
- struct mdt_device *mdt,
- struct lsd_client_data *lcd,
- loff_t *off, struct thandle *th)
-{
- struct mdt_thread_info *mti;
- struct lsd_client_data *tmp;
- int rc;
-
- LASSERT(th != NULL);
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
- tmp = &mti->mti_lcd;
-
- lcd_cpu_to_le(lcd, tmp);
-
- rc = dt_record_write(env, mdt->mdt_lut.lut_last_rcvd,
- mdt_buf_const(env, tmp, sizeof(*tmp)), off, th);
-
- CDEBUG(D_INFO, "write lcd @%d rc = %d, uuid = %s, last_transno = "LPU64
- ", last_xid = "LPU64", last_result = %u, last_data = %u, "
- "last_close_transno = "LPU64", last_close_xid = "LPU64" ,"
- "last_close_result = %u\n", (int)(*off - sizeof(*tmp)),
- rc, lcd->lcd_uuid, lcd->lcd_last_transno, lcd->lcd_last_xid,
- lcd->lcd_last_result, lcd->lcd_last_data,
- lcd->lcd_last_close_transno, lcd->lcd_last_close_xid,
- lcd->lcd_last_close_result);
- return rc;
-}
-
static int mdt_clients_data_init(const struct lu_env *env,
struct mdt_device *mdt,
unsigned long last_size)
off = lsd->lsd_client_start +
cl_idx * lsd->lsd_client_size;
- rc = mdt_last_rcvd_read(env, mdt, lcd, &off, cl_idx);
+ rc = lut_client_data_read(env, &mdt->mdt_lut, lcd, &off, cl_idx);
if (rc) {
CERROR("error reading MDS %s idx %d, off %llu: rc %d\n",
LAST_RCVD, cl_idx, off, rc);
mti->mti_exp = exp;
/* copy on-disk lcd to the export */
*exp->exp_target_data.ted_lcd = *lcd;
- rc = mdt_client_add(env, mdt, cl_idx);
+ rc = lut_client_add(env, exp, cl_idx);
/* can't fail existing */
LASSERTF(rc == 0, "rc = %d\n", rc);
/* VBR: set export last committed version */
struct mdt_thread_info *mti;
struct dt_object *obj;
struct lu_attr *la;
- struct lustre_disk_data *ldd;
unsigned long last_rcvd_size;
__u64 mount_count;
int rc;
la = &mti->mti_attr.ma_attr;
obj = mdt->mdt_lut.lut_last_rcvd;
- rc = obj->do_ops->do_attr_get(env, obj, la, BYPASS_CAPA);
+ rc = dt_attr_get(env, obj, la, BYPASS_CAPA);
if (rc)
RETURN(rc);
OBD_INCOMPAT_MULTI_OI;
} else {
LCONSOLE_WARN("%s: used disk, loading\n", obd->obd_name);
- rc = mdt_last_rcvd_header_read(env, mdt);
+ rc = lut_server_data_read(env, &mdt->mdt_lut);
if (rc) {
CERROR("error reading MDS %s: rc %d\n", LAST_RCVD, rc);
GOTO(out, rc);
}
mount_count = lsd->lsd_mount_count;
- ldd = lsi->lsi_ldd;
-
if (lsd->lsd_feature_incompat & ~MDT_INCOMPAT_SUPP) {
CERROR("%s: unsupported incompat filesystem feature(s) %x\n",
obd->obd_name,
LCONSOLE_WARN("Mounting %s at first time on 1.8 FS, "
"remove all clients for interop needs\n",
obd->obd_name);
- simple_truncate(lsi->lsi_srv_mnt->mnt_sb->s_root,
- lsi->lsi_srv_mnt, LAST_RCVD,
- lsd->lsd_client_start);
+ rc = lut_truncate_last_rcvd(env, &mdt->mdt_lut,
+ lsd->lsd_client_start);
+ if (rc)
+ GOTO(out, rc);
last_rcvd_size = lsd->lsd_client_start;
}
/** set 2.0 flag to upgrade/downgrade between 1.8 and 2.0 */
lsd->lsd_feature_compat |= OBD_COMPAT_20;
}
- if (ldd->ldd_flags & LDD_F_IAM_DIR)
+ if (lsi->lsi_ldd->ldd_flags & LDD_F_IAM_DIR)
lsd->lsd_feature_incompat |= OBD_INCOMPAT_IAM_DIR;
lsd->lsd_feature_incompat |= OBD_INCOMPAT_FID;
lsd->lsd_mount_count = obd->u.obt.obt_mount_count;
/* save it, so mount count and last_transno is current */
- rc = mdt_server_data_update(env, mdt);
+ rc = lut_server_data_update(env, &mdt->mdt_lut, 0);
if (rc)
GOTO(err_client, rc);
return rc;
}
-static int mdt_server_data_update(const struct lu_env *env,
- struct mdt_device *mdt)
-{
- struct mdt_thread_info *mti;
- struct thandle *th;
- int rc;
-
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
-
- th = mdt_trans_create(env, mdt);
- if (IS_ERR(th))
- RETURN(PTR_ERR(th));
-
- rc = mdt_declare_last_rcvd_header_write(env, mdt, th);
- if (rc)
- goto out;
-
- rc = mdt_trans_start(env, mdt, th);
- if (rc)
- goto out;
-
- CDEBUG(D_SUPER, "MDS mount_count is "LPU64", last_transno is "LPU64"\n",
- mdt->mdt_lut.lut_obd->u.obt.obt_mount_count,
- mdt->mdt_lut.lut_last_transno);
-
- cfs_spin_lock(&mdt->mdt_lut.lut_translock);
- mdt->mdt_lut.lut_lsd.lsd_last_transno = mdt->mdt_lut.lut_last_transno;
- cfs_spin_unlock(&mdt->mdt_lut.lut_translock);
-
- rc = mdt_last_rcvd_header_write(env, mdt, th);
-
-out:
- mdt_trans_stop(env, mdt, th);
- return rc;
-}
-
-
-int mdt_client_new(const struct lu_env *env, struct mdt_device *mdt)
-{
- unsigned long *bitmap = mdt->mdt_lut.lut_client_bitmap;
- struct mdt_thread_info *mti;
- struct tg_export_data *ted;
- struct lr_server_data *lsd = &mdt->mdt_lut.lut_lsd;
- struct obd_device *obd = mdt2obd_dev(mdt);
- struct thandle *th;
- loff_t off;
- int rc;
- int cl_idx;
- ENTRY;
-
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
- LASSERT(mti != NULL);
-
- ted = &mti->mti_exp->exp_target_data;
-
- LASSERT(bitmap != NULL);
- if (!strcmp(ted->ted_lcd->lcd_uuid, obd->obd_uuid.uuid))
- RETURN(0);
-
- /* the bitmap operations can handle cl_idx > sizeof(long) * 8, so
- * there's no need for extra complication here
- */
- cfs_spin_lock(&mdt->mdt_lut.lut_client_bitmap_lock);
- cl_idx = cfs_find_first_zero_bit(bitmap, LR_MAX_CLIENTS);
- if (cl_idx >= LR_MAX_CLIENTS ||
- OBD_FAIL_CHECK(OBD_FAIL_MDS_CLIENT_ADD)) {
- CERROR("no room for %u clients - fix LR_MAX_CLIENTS\n",
- cl_idx);
- cfs_spin_unlock(&mdt->mdt_lut.lut_client_bitmap_lock);
- RETURN(-EOVERFLOW);
- }
- cfs_set_bit(cl_idx, bitmap);
- cfs_spin_unlock(&mdt->mdt_lut.lut_client_bitmap_lock);
-
- CDEBUG(D_INFO, "client at idx %d with UUID '%s' added\n",
- cl_idx, ted->ted_lcd->lcd_uuid);
-
- ted->ted_lr_idx = cl_idx;
- ted->ted_lr_off = lsd->lsd_client_start +
- (cl_idx * lsd->lsd_client_size);
- cfs_mutex_init(&ted->ted_lcd_lock);
-
- LASSERTF(ted->ted_lr_off > 0, "ted_lr_off = %llu\n", ted->ted_lr_off);
-
- /* Write new client data. */
- off = ted->ted_lr_off;
-
- if (OBD_FAIL_CHECK(OBD_FAIL_TGT_CLIENT_ADD))
- RETURN(-ENOSPC);
-
- th = mdt_trans_create(env, mdt);
- if (IS_ERR(th))
- RETURN(PTR_ERR(th));
-
- rc = mdt_declare_last_rcvd_write(env, mdt, off, th);
- if (rc)
- GOTO(stop, rc);
-
- rc = mdt_trans_start(env, mdt, th);
- if (rc)
- GOTO(stop, rc);
-
- /*
- * Until this operations will be committed the sync is needed
- * for this export. This should be done _after_ starting the
- * transaction so that many connecting clients will not bring
- * server down with lots of sync writes.
- */
- rc = lut_new_client_cb_add(th, mti->mti_exp);
- if (rc) {
- /* can't add callback, do sync now */
- th->th_sync = 1;
- } else {
- cfs_spin_lock(&mti->mti_exp->exp_lock);
- mti->mti_exp->exp_need_sync = 1;
- cfs_spin_unlock(&mti->mti_exp->exp_lock);
- }
-
- rc = mdt_last_rcvd_write(env, mdt, ted->ted_lcd, &off, th);
- CDEBUG(D_INFO, "wrote client lcd at idx %u off %llu (len %u)\n",
- cl_idx, ted->ted_lr_off, (int)sizeof(*(ted->ted_lcd)));
-
-stop:
- mdt_trans_stop(env, mdt, th);
-
- RETURN(rc);
-}
-
-/* Add client data to the MDS. We use a bitmap to locate a free space
- * in the last_rcvd file if cl_off is -1 (i.e. a new client).
- * Otherwise, we just have to read the data from the last_rcvd file and
- * we know its offset.
- *
- * It should not be possible to fail adding an existing client - otherwise
- * mdt_init_server_data() callsite needs to be fixed.
- */
-int mdt_client_add(const struct lu_env *env,
- struct mdt_device *mdt, int cl_idx)
-{
- struct mdt_thread_info *mti;
- struct tg_export_data *ted;
- unsigned long *bitmap = mdt->mdt_lut.lut_client_bitmap;
- struct obd_device *obd = mdt2obd_dev(mdt);
- struct lr_server_data *lsd = &mdt->mdt_lut.lut_lsd;
- int rc = 0;
- ENTRY;
-
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
- LASSERT(mti != NULL);
-
- ted = &mti->mti_exp->exp_target_data;
-
- LASSERT(bitmap != NULL);
- LASSERTF(cl_idx >= 0, "%d\n", cl_idx);
-
- if (!strcmp(ted->ted_lcd->lcd_uuid, obd->obd_uuid.uuid))
- RETURN(0);
-
- cfs_spin_lock(&mdt->mdt_lut.lut_client_bitmap_lock);
- if (cfs_test_and_set_bit(cl_idx, bitmap)) {
- CERROR("MDS client %d: bit already set in bitmap!!\n",
- cl_idx);
- LBUG();
- }
- cfs_spin_unlock(&mdt->mdt_lut.lut_client_bitmap_lock);
-
- CDEBUG(D_INFO, "client at idx %d with UUID '%s' added\n",
- cl_idx, ted->ted_lcd->lcd_uuid);
-
- ted->ted_lr_idx = cl_idx;
- ted->ted_lr_off = lsd->lsd_client_start +
- (cl_idx * lsd->lsd_client_size);
- cfs_mutex_init(&ted->ted_lcd_lock);
-
- LASSERTF(ted->ted_lr_off > 0, "ted_lr_off = %llu\n", ted->ted_lr_off);
-
- RETURN(rc);
-}
-
-int mdt_client_del(const struct lu_env *env, struct mdt_device *mdt)
-{
- struct mdt_thread_info *mti;
- struct tg_export_data *ted;
- struct obd_device *obd = mdt2obd_dev(mdt);
- struct obd_export *exp;
- struct thandle *th;
- loff_t off;
- int rc = 0;
- ENTRY;
-
- mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
- LASSERT(mti != NULL);
-
- exp = mti->mti_exp;
- ted = &exp->exp_target_data;
- if (!ted->ted_lcd)
- RETURN(0);
-
- /* XXX: If lcd_uuid were a real obd_uuid, I could use obd_uuid_equals */
- if (!strcmp(ted->ted_lcd->lcd_uuid, obd->obd_uuid.uuid))
- GOTO(free, 0);
-
- CDEBUG(D_INFO, "freeing client at idx %u, offset %lld\n",
- ted->ted_lr_idx, ted->ted_lr_off);
-
- off = ted->ted_lr_off;
-
- /*
- * Don't clear ted_lr_idx here as it is likely also unset. At worst we
- * leak a client slot that will be cleaned on the next recovery.
- */
- if (off <= 0) {
- CERROR("client idx %d has offset %lld\n",
- ted->ted_lr_idx, off);
- GOTO(free, rc = -EINVAL);
- }
-
- /*
- * Clear the bit _after_ zeroing out the client so we don't race with
- * mdt_client_add and zero out new clients.
- */
- if (!cfs_test_bit(ted->ted_lr_idx, mdt->mdt_lut.lut_client_bitmap)) {
- CERROR("MDT client %u: bit already clear in bitmap!!\n",
- ted->ted_lr_idx);
- LBUG();
- }
-
- /* Make sure the server's last_transno is up to date.
- * This should be done before zeroing client slot so last_transno will
- * be in server data or in client data in case of failure */
- mdt_server_data_update(env, mdt);
-
- th = mdt_trans_create(env, mdt);
- if (IS_ERR(th))
- GOTO(free, rc = PTR_ERR(th));
-
- rc = mdt_declare_last_rcvd_write(env, mdt, off, th);
- if (rc)
- GOTO(stop, rc);
-
- rc = mdt_trans_start(env, mdt, th);
- if (rc)
- GOTO(stop, rc);
-
- cfs_mutex_lock(&ted->ted_lcd_lock);
- memset(ted->ted_lcd->lcd_uuid, 0, sizeof ted->ted_lcd->lcd_uuid);
- rc = mdt_last_rcvd_write(env, mdt, ted->ted_lcd, &off, th);
- cfs_mutex_unlock(&ted->ted_lcd_lock);
-
-stop:
- mdt_trans_stop(env, mdt, th);
-
- CDEBUG(rc == 0 ? D_INFO : D_ERROR, "Zeroing out client idx %u in "
- "%s, rc %d\n", ted->ted_lr_idx, LAST_RCVD, rc);
- RETURN(0);
-free:
- return 0;
-}
-
/*
* last_rcvd & last_committed update callbacks
*/
cfs_mutex_unlock(&ted->ted_lcd_lock);
CWARN("commit transaction for disconnected client %s: rc %d\n",
req->rq_export->exp_client_uuid.uuid, rc);
- err = mdt_last_rcvd_header_write(mti->mti_env, mdt, th);
+ err = lut_server_data_write(mti->mti_env, &mdt->mdt_lut, th);
RETURN(err);
}
CERROR("client idx %d has offset %lld\n", ted->ted_lr_idx, off);
err = -EINVAL;
} else {
- err = mdt_last_rcvd_write(mti->mti_env, mdt, lcd, &off, th);
+ err = lut_client_data_write(mti->mti_env, &mdt->mdt_lut, lcd,
+ &off, th);
}
cfs_mutex_unlock(&ted->ted_lcd_lock);
RETURN(err);
/* add credits for last_rcvd update */
static int mdt_txn_start_cb(const struct lu_env *env,
- struct thandle *th, void *cookie)
+ struct thandle *th, void *cookie)
{
- struct mdt_device *mdt = cookie;
+ struct mdt_device *mdt = cookie;
+ struct mdt_thread_info *mti;
+ int rc;
+ ENTRY;
+
+ mti = lu_context_key_get(&env->le_ctx, &mdt_thread_key);
+
+ LASSERT(mdt->mdt_lut.lut_last_rcvd);
+ if (mti->mti_exp == NULL)
+ RETURN(0);
+
+ rc = dt_declare_record_write(env, mdt->mdt_lut.lut_last_rcvd,
+ sizeof(struct lsd_client_data),
+ mti->mti_exp->exp_target_data.ted_lr_off,
+ th);
+ if (rc)
+ return rc;
+
+ rc = dt_declare_record_write(env, mdt->mdt_lut.lut_last_rcvd,
+ sizeof(struct lr_server_data), 0, th);
+ if (rc)
+ return rc;
+
+ if (mti->mti_mos != NULL)
+ rc = dt_declare_version_set(env, mdt_obj2dt(mti->mti_mos), th);
- /* XXX: later we'll be declaring this at specific offset */
- return dt_declare_record_write(env, mdt->mdt_lut.lut_last_rcvd,
- sizeof(struct lsd_client_data), 0, th);
+ return rc;
}
/* Update last_rcvd records with latests transaction data */
{ MDD_ORPHAN_OID, "" /* "PENDING" */ },
{ MDD_LOV_OBJ_OID, "" /* LOV_OBJID */ },
{ MDD_CAPA_KEYS_OID, "" /* CAPA_KEYS */ },
- { MDT_LAST_RECV_OID, "" /* LAST_RCVD */ },
+ { MDT_LAST_RECV_OID, LAST_RCVD },
{ OFD_LAST_RECV_OID, "" /* LAST_RCVD */ },
{ OFD_LAST_GROUP_OID, "" /* "LAST_GROUP" */ },
{ LLOG_CATALOGS_OID, "" /* "CATALOGS" */ },
return (rc == -ENOTCONN || rc == -ENODEV);
}
+#ifdef HAVE_SERVER_SUPPORT
+/* target.c */
+int lut_mod_init(void);
+void lut_mod_exit(void);
+#else
+static inline int lut_mod_init(void)
+{
+ return 0;
+}
+
+static inline void lut_mod_exit(void)
+{
+ return;
+}
+#endif
+
static inline void ptlrpc_reqset_put(struct ptlrpc_request_set *set)
{
if (cfs_atomic_dec_and_test(&set->set_refcount))
if (rc)
GOTO(cleanup, rc);
+#ifdef __KERNEL__
+ cleanup_phase = 7;
+ rc = lut_mod_init();
+ if (rc)
+ GOTO(cleanup, rc);
+#endif
RETURN(0);
cleanup:
switch(cleanup_phase) {
+#ifdef __KERNEL__
+ case 7:
+ llog_recov_fini();
+#endif
case 6:
sptlrpc_fini();
case 5:
#ifdef __KERNEL__
static void __exit ptlrpc_exit(void)
{
+ lut_mod_exit();
llog_recov_fini();
sptlrpc_fini();
ldlm_exit();
#include <obd.h>
#include <lustre_fsfilt.h>
#include <obd_class.h>
+#include <lustre_fid.h>
+
+/**
+ * Common data shared by tg-level handlers. This is allocated per-thread to
+ * reduce stack consumption.
+ */
+struct tg_thread_info {
+ /* server and client data buffers */
+ struct lr_server_data tti_lsd;
+ struct lsd_client_data tti_lcd;
+ struct lu_buf tti_buf;
+ loff_t tti_off;
+};
+
+static inline struct lu_buf *tti_buf_lsd(struct tg_thread_info *tti)
+{
+ tti->tti_buf.lb_buf = &tti->tti_lsd;
+ tti->tti_buf.lb_len = sizeof(tti->tti_lsd);
+ return &tti->tti_buf;
+}
+
+static inline struct lu_buf *tti_buf_lcd(struct tg_thread_info *tti)
+{
+ tti->tti_buf.lb_buf = &tti->tti_lcd;
+ tti->tti_buf.lb_len = sizeof(tti->tti_lcd);
+ return &tti->tti_buf;
+}
+
+extern struct lu_context_key tg_thread_key;
+
+static inline struct tg_thread_info *tg_th_info(const struct lu_env *env)
+{
+ struct tg_thread_info *tti;
+
+ tti = lu_context_key_get(&env->le_ctx, &tg_thread_key);
+ LASSERT(tti);
+ return tti;
+}
/**
* Update client data in last_rcvd file. An obd API
}
/**
- * write data in last_rcvd file.
- */
-static int lut_last_rcvd_write(const struct lu_env *env, struct lu_target *lut,
- const struct lu_buf *buf, loff_t *off, int sync)
-{
- struct thandle *th;
- int rc;
- ENTRY;
-
- th = dt_trans_create(env, lut->lut_bottom);
- if (IS_ERR(th))
- RETURN(PTR_ERR(th));
-
- rc = dt_declare_record_write(env, lut->lut_last_rcvd,
- buf->lb_len, *off, th);
- if (rc)
- goto stop;
-
- rc = dt_trans_start_local(env, lut->lut_bottom, th);
- if (rc)
- goto stop;
-
- rc = dt_record_write(env, lut->lut_last_rcvd, buf, off, th);
-
-stop:
- dt_trans_stop(env, lut->lut_bottom, th);
-
- CDEBUG(D_INFO, "write last_rcvd header rc = %d:\n"
- "uuid = %s\nlast_transno = "LPU64"\n",
- rc, lut->lut_lsd.lsd_uuid, lut->lut_lsd.lsd_last_transno);
-
- RETURN(rc);
-}
-
-/**
* Allocate in-memory data for client slot related to export.
*/
int lut_client_alloc(struct obd_export *exp)
if (ted->ted_lr_idx < 0)
return;
/* Clear bit when lcd is freed */
- cfs_spin_lock(&lut->lut_client_bitmap_lock);
+ LASSERT(lut->lut_client_bitmap);
if (!cfs_test_and_clear_bit(ted->ted_lr_idx, lut->lut_client_bitmap)) {
CERROR("%s: client %u bit already clear in bitmap\n",
exp->exp_obd->obd_name, ted->ted_lr_idx);
LBUG();
}
- cfs_spin_unlock(&lut->lut_client_bitmap_lock);
}
EXPORT_SYMBOL(lut_client_free);
+int lut_client_data_read(const struct lu_env *env, struct lu_target *tg,
+ struct lsd_client_data *lcd, loff_t *off, int index)
+{
+ struct tg_thread_info *tti = tg_th_info(env);
+ int rc;
+
+ tti_buf_lcd(tti);
+ rc = dt_record_read(env, tg->lut_last_rcvd, &tti->tti_buf, off);
+ if (rc == 0) {
+ check_lcd(tg->lut_obd->obd_name, index, &tti->tti_lcd);
+ lcd_le_to_cpu(&tti->tti_lcd, lcd);
+ }
+
+ CDEBUG(D_INFO, "read lcd @%lld rc = %d, uuid = %s, last_transno = "LPU64
+ ", last_xid = "LPU64", last_result = %u, last_data = %u, "
+ "last_close_transno = "LPU64", last_close_xid = "LPU64", "
+ "last_close_result = %u\n", *off,
+ rc, lcd->lcd_uuid, lcd->lcd_last_transno, lcd->lcd_last_xid,
+ lcd->lcd_last_result, lcd->lcd_last_data,
+ lcd->lcd_last_close_transno, lcd->lcd_last_close_xid,
+ lcd->lcd_last_close_result);
+ return rc;
+}
+EXPORT_SYMBOL(lut_client_data_read);
+
+int lut_client_data_write(const struct lu_env *env, struct lu_target *tg,
+ struct lsd_client_data *lcd, loff_t *off,
+ struct thandle *th)
+{
+ struct tg_thread_info *tti = tg_th_info(env);
+
+ lcd_cpu_to_le(lcd, &tti->tti_lcd);
+ tti_buf_lcd(tti);
+
+ return dt_record_write(env, tg->lut_last_rcvd, &tti->tti_buf, off, th);
+}
+EXPORT_SYMBOL(lut_client_data_write);
+
/**
* Update client data in last_rcvd
*/
int lut_client_data_update(const struct lu_env *env, struct obd_export *exp)
{
- struct tg_export_data *ted = &exp->exp_target_data;
- struct lu_target *lut = class_exp2tgt(exp);
- struct lsd_client_data tmp_lcd;
- loff_t tmp_off = ted->ted_lr_off;
- struct lu_buf tmp_buf = {
- .lb_buf = &tmp_lcd,
- .lb_len = sizeof(tmp_lcd)
- };
- int rc = 0;
-
- lcd_cpu_to_le(ted->ted_lcd, &tmp_lcd);
- LASSERT(lut->lut_last_rcvd);
- rc = lut_last_rcvd_write(env, lut, &tmp_buf, &tmp_off, 0);
+ struct tg_export_data *ted = &exp->exp_target_data;
+ struct lu_target *tg = class_exp2tgt(exp);
+ struct tg_thread_info *tti = tg_th_info(env);
+ struct thandle *th;
+ int rc = 0;
+
+ ENTRY;
+
+ th = dt_trans_create(env, tg->lut_bottom);
+ if (IS_ERR(th))
+ RETURN(PTR_ERR(th));
+
+ rc = dt_declare_record_write(env, tg->lut_last_rcvd,
+ sizeof(struct lsd_client_data),
+ ted->ted_lr_off, th);
+ if (rc)
+ GOTO(out, rc);
+
+ rc = dt_trans_start_local(env, tg->lut_bottom, th);
+ if (rc)
+ GOTO(out, rc);
+ /*
+ * Until this operations will be committed the sync is needed
+ * for this export. This should be done _after_ starting the
+ * transaction so that many connecting clients will not bring
+ * server down with lots of sync writes.
+ */
+ rc = lut_new_client_cb_add(th, exp);
+ if (rc) {
+ /* can't add callback, do sync now */
+ th->th_sync = 1;
+ } else {
+ cfs_spin_lock(&exp->exp_lock);
+ exp->exp_need_sync = 1;
+ cfs_spin_unlock(&exp->exp_lock);
+ }
+
+ tti->tti_off = ted->ted_lr_off;
+ rc = lut_client_data_write(env, tg, ted->ted_lcd, &tti->tti_off, th);
+ EXIT;
+out:
+ dt_trans_stop(env, tg->lut_bottom, th);
+ CDEBUG(D_INFO, "write last_rcvd, rc = %d:\n"
+ "uuid = %s\nlast_transno = "LPU64"\n",
+ rc, tg->lut_lsd.lsd_uuid, tg->lut_lsd.lsd_last_transno);
+
+ return rc;
+}
+int lut_server_data_read(const struct lu_env *env, struct lu_target *tg)
+{
+ struct tg_thread_info *tti = tg_th_info(env);
+ int rc;
+
+ tti->tti_off = 0;
+ tti_buf_lsd(tti);
+ rc = dt_record_read(env, tg->lut_last_rcvd, &tti->tti_buf, &tti->tti_off);
+ if (rc == 0)
+ lsd_le_to_cpu(&tti->tti_lsd, &tg->lut_lsd);
+
+ CDEBUG(D_INFO, "%s: read last_rcvd header, rc = %d, uuid = %s, "
+ "last_transno = "LPU64"\n", tg->lut_obd->obd_name, rc,
+ tg->lut_lsd.lsd_uuid, tg->lut_lsd.lsd_last_transno);
return rc;
}
+EXPORT_SYMBOL(lut_server_data_read);
+
+int lut_server_data_write(const struct lu_env *env, struct lu_target *tg,
+ struct thandle *th)
+{
+ struct tg_thread_info *tti = tg_th_info(env);
+ int rc;
+ ENTRY;
+
+ tti->tti_off = 0;
+ tti_buf_lsd(tti);
+ lsd_cpu_to_le(&tg->lut_lsd, &tti->tti_lsd);
+
+ rc = dt_record_write(env, tg->lut_last_rcvd, &tti->tti_buf,
+ &tti->tti_off, th);
+
+ CDEBUG(D_INFO, "write last_rcvd header rc = %d:\n"
+ "uuid = %s\nlast_transno = "LPU64"\n",
+ rc, tg->lut_lsd.lsd_uuid, tg->lut_lsd.lsd_last_transno);
+
+ RETURN(rc);
+}
+EXPORT_SYMBOL(lut_server_data_write);
/**
* Update server data in last_rcvd
*/
-int lut_server_data_update(const struct lu_env *env,
- struct lu_target *lut, int sync)
+int lut_server_data_update(const struct lu_env *env, struct lu_target *tg,
+ int sync)
{
- struct lr_server_data tmp_lsd;
- loff_t tmp_off = 0;
- struct lu_buf tmp_buf = {
- .lb_buf = &tmp_lsd,
- .lb_len = sizeof(tmp_lsd)
- };
- int rc = 0;
- ENTRY;
-
- CDEBUG(D_SUPER,
- "%s: mount_count is "LPU64", last_transno is "LPU64"\n",
- lut->lut_lsd.lsd_uuid, lut->lut_obd->u.obt.obt_mount_count,
- lut->lut_last_transno);
-
- cfs_spin_lock(&lut->lut_translock);
- lut->lut_lsd.lsd_last_transno = lut->lut_last_transno;
- cfs_spin_unlock(&lut->lut_translock);
+ struct tg_thread_info *tti = tg_th_info(env);
+ struct thandle *th;
+ int rc = 0;
+
+ ENTRY;
+
+ CDEBUG(D_SUPER,
+ "%s: mount_count is "LPU64", last_transno is "LPU64"\n",
+ tg->lut_lsd.lsd_uuid, tg->lut_obd->u.obt.obt_mount_count,
+ tg->lut_last_transno);
+
+ /* Always save latest transno to keep it fresh */
+ cfs_spin_lock(&tg->lut_translock);
+ tg->lut_lsd.lsd_last_transno = tg->lut_last_transno;
+ cfs_spin_unlock(&tg->lut_translock);
+
+ th = dt_trans_create(env, tg->lut_bottom);
+ if (IS_ERR(th))
+ RETURN(PTR_ERR(th));
+
+ th->th_sync = sync;
+
+ rc = dt_declare_record_write(env, tg->lut_last_rcvd,
+ sizeof(struct lr_server_data),
+ tti->tti_off, th);
+ if (rc)
+ GOTO(out, rc);
+
+ rc = dt_trans_start(env, tg->lut_bottom, th);
+ if (rc)
+ GOTO(out, rc);
+
+ rc = lut_server_data_write(env, tg, th);
+out:
+ dt_trans_stop(env, tg->lut_bottom, th);
+
+ CDEBUG(D_INFO, "write last_rcvd header, rc = %d:\n"
+ "uuid = %s\nlast_transno = "LPU64"\n",
+ rc, tg->lut_lsd.lsd_uuid, tg->lut_lsd.lsd_last_transno);
+ RETURN(rc);
+}
+EXPORT_SYMBOL(lut_server_data_update);
- lsd_cpu_to_le(&lut->lut_lsd, &tmp_lsd);
- if (lut->lut_last_rcvd != NULL)
- rc = lut_last_rcvd_write(env, lut, &tmp_buf, &tmp_off, sync);
- RETURN(rc);
+int lut_truncate_last_rcvd(const struct lu_env *env, struct lu_target *tg,
+ loff_t size)
+{
+ struct dt_object *dt = tg->lut_last_rcvd;
+ struct thandle *th;
+ struct lu_attr attr;
+ int rc;
+
+ ENTRY;
+
+ attr.la_size = size;
+ attr.la_valid = LA_SIZE;
+
+ th = dt_trans_create(env, tg->lut_bottom);
+ if (IS_ERR(th))
+ RETURN(PTR_ERR(th));
+ rc = dt_declare_punch(env, dt, size, OBD_OBJECT_EOF, th);
+ if (rc)
+ GOTO(cleanup, rc);
+ rc = dt_declare_attr_set(env, dt, &attr, th);
+ if (rc)
+ GOTO(cleanup, rc);
+ rc = dt_trans_start_local(env, tg->lut_bottom, th);
+ if (rc)
+ GOTO(cleanup, rc);
+
+ rc = dt_punch(env, dt, size, OBD_OBJECT_EOF, th, BYPASS_CAPA);
+ if (rc == 0)
+ rc = dt_attr_set(env, dt, &attr, th, BYPASS_CAPA);
+
+cleanup:
+ dt_trans_stop(env, tg->lut_bottom, th);
+
+ RETURN(rc);
}
+EXPORT_SYMBOL(lut_truncate_last_rcvd);
void lut_client_epoch_update(const struct lu_env *env, struct obd_export *exp)
{
if (lut->lut_bottom == NULL)
return obt_boot_epoch_update(lut);
- rc = lu_env_init(&env, LCT_LOCAL);
+ rc = lu_env_init(&env, LCT_DT_THREAD);
if (rc) {
CERROR("Can't initialize environment rc=%d\n", rc);
return;
}
return rc;
}
-EXPORT_SYMBOL(lut_new_client_cb_add);
+
+/**
+ * Add new client to the last_rcvd upon new connection.
+ *
+ * We use a bitmap to locate a free space in the last_rcvd file and initialize
+ * tg_export_data.
+ */
+int lut_client_new(const struct lu_env *env, struct obd_export *exp)
+{
+ struct tg_export_data *ted = &exp->exp_target_data;
+ struct lu_target *tg = class_exp2tgt(exp);
+ int rc = 0, idx;
+
+ ENTRY;
+
+ LASSERT(tg->lut_client_bitmap != NULL);
+ if (!strcmp(ted->ted_lcd->lcd_uuid, tg->lut_obd->obd_uuid.uuid))
+ RETURN(0);
+
+ /* the bitmap operations can handle cl_idx > sizeof(long) * 8, so
+ * there's no need for extra complication here
+ */
+ idx = cfs_find_first_zero_bit(tg->lut_client_bitmap, LR_MAX_CLIENTS);
+repeat:
+ if (idx >= LR_MAX_CLIENTS ||
+ OBD_FAIL_CHECK(OBD_FAIL_MDS_CLIENT_ADD)) {
+ CERROR("%s: no room for %u clients - fix LR_MAX_CLIENTS\n",
+ tg->lut_obd->obd_name, idx);
+ RETURN(-EOVERFLOW);
+ }
+ if (cfs_test_and_set_bit(idx, tg->lut_client_bitmap)) {
+ idx = cfs_find_next_zero_bit(tg->lut_client_bitmap,
+ LR_MAX_CLIENTS, idx);
+ goto repeat;
+ }
+
+ CDEBUG(D_INFO, "%s: client at idx %d with UUID '%s' added\n",
+ tg->lut_obd->obd_name, idx, ted->ted_lcd->lcd_uuid);
+
+ ted->ted_lr_idx = idx;
+ ted->ted_lr_off = tg->lut_lsd.lsd_client_start +
+ idx * tg->lut_lsd.lsd_client_size;
+
+ 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, "%s: new client at index %d (%llu) with UUID '%s'\n",
+ tg->lut_obd->obd_name, ted->ted_lr_idx, ted->ted_lr_off,
+ ted->ted_lcd->lcd_uuid);
+
+ if (OBD_FAIL_CHECK(OBD_FAIL_TGT_CLIENT_ADD))
+ RETURN(-ENOSPC);
+
+ rc = lut_client_data_update(env, exp);
+ if (rc)
+ CERROR("%s: Failed to write client lcd at idx %d, rc %d\n",
+ tg->lut_obd->obd_name, idx, rc);
+
+ RETURN(rc);
+}
+EXPORT_SYMBOL(lut_client_new);
+
+/* Add client data to the MDS. We use a bitmap to locate a free space
+ * in the last_rcvd file if cl_off is -1 (i.e. a new client).
+ * Otherwise, we just have to read the data from the last_rcvd file and
+ * we know its offset.
+ *
+ * It should not be possible to fail adding an existing client - otherwise
+ * mdt_init_server_data() callsite needs to be fixed.
+ */
+int lut_client_add(const struct lu_env *env, struct obd_export *exp, int idx)
+{
+ struct tg_export_data *ted = &exp->exp_target_data;
+ struct lu_target *tg = class_exp2tgt(exp);
+
+ ENTRY;
+
+ LASSERT(tg->lut_client_bitmap != NULL);
+ LASSERTF(idx >= 0, "%d\n", idx);
+
+ if (!strcmp(ted->ted_lcd->lcd_uuid, tg->lut_obd->obd_uuid.uuid))
+ RETURN(0);
+
+ if (cfs_test_and_set_bit(idx, tg->lut_client_bitmap)) {
+ CERROR("%s: client %d: bit already set in bitmap!!\n",
+ tg->lut_obd->obd_name, idx);
+ LBUG();
+ }
+
+ CDEBUG(D_INFO, "%s: client at idx %d with UUID '%s' added\n",
+ tg->lut_obd->obd_name, idx, ted->ted_lcd->lcd_uuid);
+
+ ted->ted_lr_idx = idx;
+ ted->ted_lr_off = tg->lut_lsd.lsd_client_start +
+ idx * tg->lut_lsd.lsd_client_size;
+
+ cfs_mutex_init(&ted->ted_lcd_lock);
+
+ LASSERTF(ted->ted_lr_off > 0, "ted_lr_off = %llu\n", ted->ted_lr_off);
+
+ RETURN(0);
+}
+EXPORT_SYMBOL(lut_client_add);
+
+int lut_client_del(const struct lu_env *env, struct obd_export *exp)
+{
+ struct tg_export_data *ted = &exp->exp_target_data;
+ struct lu_target *tg = class_exp2tgt(exp);
+ int rc;
+
+ ENTRY;
+
+ LASSERT(ted->ted_lcd);
+
+ /* XXX if lcd_uuid were a real obd_uuid, I could use obd_uuid_equals */
+ if (!strcmp((char *)ted->ted_lcd->lcd_uuid,
+ (char *)tg->lut_obd->obd_uuid.uuid))
+ RETURN(0);
+
+ CDEBUG(D_INFO, "%s: del client at idx %u, off %lld, UUID '%s'\n",
+ tg->lut_obd->obd_name, ted->ted_lr_idx, ted->ted_lr_off,
+ ted->ted_lcd->lcd_uuid);
+
+ /* Clear the bit _after_ zeroing out the client so we don't
+ race with filter_client_add and zero out new clients.*/
+ if (!cfs_test_bit(ted->ted_lr_idx, tg->lut_client_bitmap)) {
+ CERROR("%s: client %u: bit already clear in bitmap!!\n",
+ tg->lut_obd->obd_name, ted->ted_lr_idx);
+ LBUG();
+ }
+
+ /* Do not erase record for recoverable client. */
+ if (exp->exp_flags & OBD_OPT_FAILOVER)
+ RETURN(0);
+
+ /* Make sure the server's last_transno is up to date.
+ * This should be done before zeroing client slot so last_transno will
+ * be in server data or in client data in case of failure */
+ rc = lut_server_data_update(env, tg, 0);
+ if (rc != 0) {
+ CERROR("%s: failed to update server data, skip client %s "
+ "zeroing, rc %d\n", tg->lut_obd->obd_name,
+ ted->ted_lcd->lcd_uuid, rc);
+ RETURN(rc);
+ }
+
+ cfs_mutex_lock(&ted->ted_lcd_lock);
+ memset(ted->ted_lcd->lcd_uuid, 0, sizeof ted->ted_lcd->lcd_uuid);
+ rc = lut_client_data_update(env, exp);
+ cfs_mutex_unlock(&ted->ted_lcd_lock);
+
+ CDEBUG(rc == 0 ? D_INFO : D_ERROR,
+ "%s: zeroing out client %s at idx %u (%llu), rc %d\n",
+ tg->lut_obd->obd_name, ted->ted_lcd->lcd_uuid,
+ ted->ted_lr_idx, ted->ted_lr_off, rc);
+ RETURN(rc);
+}
+EXPORT_SYMBOL(lut_client_del);
int lut_init(const struct lu_env *env, struct lu_target *lut,
- struct obd_device *obd, struct dt_device *dt)
+ struct obd_device *obd, struct dt_device *dt)
{
- struct lu_fid fid;
- struct dt_object *o;
- int rc = 0;
+ struct dt_object_format dof;
+ struct lu_attr attr;
+ struct lu_fid fid;
+ struct dt_object *o;
+ int rc = 0;
ENTRY;
LASSERT(lut);
obd->u.obt.obt_lut = lut;
cfs_spin_lock_init(&lut->lut_translock);
- cfs_spin_lock_init(&lut->lut_client_bitmap_lock);
OBD_ALLOC(lut->lut_client_bitmap, LR_MAX_CLIENTS >> 3);
if (lut->lut_client_bitmap == NULL)
/** obdfilter has no lu_device stack yet */
if (dt == NULL)
RETURN(rc);
- o = dt_store_open(env, lut->lut_bottom, "", LAST_RCVD, &fid);
+
+ memset(&attr, 0, sizeof(attr));
+ attr.la_valid = LA_MODE;
+ attr.la_mode = S_IFREG | S_IRUGO | S_IWUSR;
+ dof.dof_type = dt_mode_to_dft(S_IFREG);
+
+ lu_local_obj_fid(&fid, MDT_LAST_RECV_OID);
+
+ o = dt_find_or_create(env, lut->lut_bottom, &fid, &dof, &attr);
if (!IS_ERR(o)) {
lut->lut_last_rcvd = o;
} else {
EXIT;
}
EXPORT_SYMBOL(lut_fini);
+
+/* context key constructor/destructor: tg_key_init, tg_key_fini */
+LU_KEY_INIT_FINI(tg, struct tg_thread_info);
+/* context key: tg_thread_key */
+LU_CONTEXT_KEY_DEFINE(tg, LCT_MD_THREAD|LCT_DT_THREAD);
+LU_KEY_INIT_GENERIC(tg);
+EXPORT_SYMBOL(tg_thread_key);
+
+int lut_mod_init(void)
+{
+ tg_key_init_generic(&tg_thread_key, NULL);
+ lu_context_key_register_many(&tg_thread_key, NULL);
+ return 0;
+}
+
+void lut_mod_exit(void)
+{
+ lu_context_key_degister_many(&tg_thread_key, NULL);
+}
+