-TBD
- * version TBD
+2002-11-18 Phil Schwan <phil@clusterfs.com>
+ * version v0_5_17
* bug fixes
- fix null d_it dereference (346)
- fix full OST/dbench hang (333)
- free locks on clients when inodes are removed due to memory
pressure (201)
- fix inode pointer in lock data (285)
- - partial support for multiple MDS on a single host (623279, 241)
+ - partial support for multiple MDS on a single host (241)
- data locks weren't cancelled at clear_inode time (290, 311)
- intent locks could lead to unbounded lock growth (205)
- added a maximum lock count, an LRU list, and a flusher
- fix multiple rename (365)
+ - properly abstracted the echo client
+ - OSC locked 1 byte too many; fixed
+ - rewrote brw callback code:
+ - fixed recovery bugs related to LOVs (306)
+ - fixed too-many-pages-in-one-write crash (191)
+ - fixed (again) crash in sync_io_timeout (214)
+ - probably fixed callback-related race (385)
* protocol change
- Add capability to MDS protocol
- LDLM cancellations and callbacks on different portals
struct lov_mds_md {
__u32 lmm_magic;
- __u32 lmm_easize; /* packed size of extended */
+ __u32 lmm_unused; /* was packed size of extended attribute */
__u64 lmm_object_id; /* lov object id */
__u32 lmm_stripe_offset; /* starting stripe offset in lmd_objects */
__u32 lmm_stripe_count; /* number of stipes in use for this object */
#define OBD_MD_FLEASIZE (0x00020000) /* extended attribute data */
#define OBD_MD_LINKNAME (0x00040000) /* symbolic link target */
#define OBD_MD_FLHANDLE (0x00080000) /* file handle */
-#define OBD_MD_FLNOTOBD (~(OBD_MD_FLOBDFLG | OBD_MD_FLBLOCKS | OBD_MD_LINKNAME))
+#define OBD_MD_FLNOTOBD (~(OBD_MD_FLOBDFLG | OBD_MD_FLBLOCKS | OBD_MD_LINKNAME|\
+ OBD_MD_FLEASIZE | OBD_MD_FLHANDLE))
struct obd_statfs {
__u64 os_type;
__u32 generation;
};
+/* This is probably redundant with OBD_MD_FLEASIZE, but we need an audit */
#define MDS_OPEN_HAS_EA 1 /* this open has an EA, for a delayed create*/
/* MDS update records */
void l_lock(struct lustre_lock *);
void l_unlock(struct lustre_lock *);
-
-/* page.c */
#define CB_PHASE_START 12
#define CB_PHASE_FINISH 13
-/*
- * io_cb_data: io callback data merged into one struct to simplify
- * memory managment. This may be turn out to be too simple.
- */
-struct brw_cb_data;
-typedef int (*brw_cb_t)(struct brw_cb_data *, int err, int phase);
-
-struct brw_cb_data {
+/* This list head doesn't need to be locked, because it's only manipulated by
+ * one thread at a time. */
+struct obd_brw_set {
+ struct list_head brw_desc_head; /* list of ptlrpc_bulk_desc */
wait_queue_head_t brw_waitq;
atomic_t brw_refcount;
- int brw_complete;
- int brw_err;
- struct ptlrpc_bulk_desc *brw_desc;
- brw_cb_t brw_cb;
- void *brw_data;
-};
+ int brw_flags;
-int ll_sync_brw_cb(struct brw_cb_data *brw_cbd, int err, int phase);
-struct brw_cb_data *ll_init_brw_cb_data(void);
+ int (*brw_callback)(struct obd_brw_set *, int phase);
+};
/* simple.c */
struct obd_run_ctxt;
__u32 fd_flags;
};
-struct lustre_intent_data {
- __u64 it_lock_handle[2];
- __u32 it_disposition;
- __u32 it_status;
- __u32 it_lock_mode;
+struct lustre_intent_data {
+ __u64 it_lock_handle[2];
+ __u32 it_disposition;
+ __u32 it_status;
+ __u32 it_lock_mode;
};
struct ll_dentry_data {
static inline struct ll_inode_info *ll_i2info(struct inode *inode)
{
#if (LINUX_VERSION_CODE >= KERNEL_VERSION(2,5,0))
- return container_of(inode, struct ll_inode_info, lli_vfs_inode);
+ return container_of(inode, struct ll_inode_info, lli_vfs_inode);
#else
return (struct ll_inode_info *)&(inode->u.generic_ip);
#endif
inode->i_mode & S_IFMT);
}
-static inline int ll_mds_easize(struct super_block *sb)
+static inline int ll_mds_max_easize(struct super_block *sb)
{
return sbi2mdc(ll_s2sbi(sb))->cl_max_mds_easize;
}
-static inline int ll_ost_easize(struct super_block *sb)
-{
- return sbi2mdc(ll_s2sbi(sb))->cl_max_ost_easize;
-}
-
/* namei.c */
int ll_lock(struct inode *dir, struct dentry *dentry,
struct lookup_intent *it, struct lustre_handle *lockh);
#define LL_IOC_LOV_SETSTRIPE _IOW ('f', 154, long)
#define LL_IOC_LOV_GETSTRIPE _IOW ('f', 155, long)
-struct lov_user_oinfo {
- __u64 luo_id; /* object ID on the target OBD */
- __u32 luo_idx; /* OBD stripe index in lmd_objects array */
- __u32 luo_pad;
-};
-
-struct lov_user_md {
- __u64 lum_stripe_size;
- __u32 lum_stripe_pattern;
- __u32 lum_stripe_offset;
- __u32 lum_stripe_count;
- struct lov_user_oinfo lum_luoinfo[0];
-};
-
#define O_LOV_DELAY_CREATE 0100000000 /* hopefully this does not conflict */
#define LL_FILE_IGNORE_LOCK 0x00000001
#include <linux/lustre_idl.h>
struct ldlm_lock_desc;
-struct lov_stripe_md;
struct mds_obd;
struct ptlrpc_connection;
struct ptlrpc_client;
struct inode *inode, const char *name, int namelen);
void mds_setattr_pack(struct ptlrpc_request *, int offset, struct inode *,
struct iattr *, const char *name, int namelen);
-void mds_create_pack(struct ptlrpc_request *, int offset, struct inode *,
- __u32 mode, __u64 id, __u32 uid, __u32 gid, __u64 time,
- const char *name, int namelen, const char *tgt,
- int tgtlen);
+void mds_create_pack(struct ptlrpc_request *, int offset, struct inode *dir,
+ __u32 mode, __u64 rdev, __u32 uid, __u32 gid, __u64 time,
+ const char *name, int namelen, const void *data,
+ int datalen);
void mds_unlink_pack(struct ptlrpc_request *, int offset, struct inode *inode,
struct inode *child, __u32 mode, const char *name,
int namelen);
struct lustre_handle *lockh);
struct dentry *mds_fid2dentry(struct mds_obd *mds, struct ll_fid *fid,
struct vfsmount **mnt);
-int mds_reint(int offset, struct ptlrpc_request *req);
+int mds_reint(struct ptlrpc_request *req, int offset);
+int mds_pack_md(struct mds_obd *mds, struct ptlrpc_request *req,
+ int offset, struct mds_body *body, struct inode *inode);
/* mdc/mdc_request.c */
int mdc_enqueue(struct lustre_handle *conn, int lock_type,
int mdc_setattr(struct lustre_handle *conn,
struct inode *, struct iattr *iattr, struct ptlrpc_request **);
int mdc_open(struct lustre_handle *conn, obd_id ino, int type, int flags,
- struct lov_stripe_md *, struct lustre_handle *fh,
+ struct lov_mds_md *lmm, int lmm_size, struct lustre_handle *fh,
struct ptlrpc_request **);
int mdc_close(struct lustre_handle *conn, obd_id ino, int type,
struct lustre_handle *fh, struct ptlrpc_request **req);
int type, __u64 offset, char *addr, struct ptlrpc_request **);
int mdc_create(struct lustre_handle *conn,
struct inode *dir, const char *name, int namelen,
- const char *tgt, int tgtlen, int mode, __u32 uid, __u32 gid,
- __u64 time, __u64 rdev, struct lov_stripe_md *md,
- struct ptlrpc_request **);
+ const void *data, int datalen, int mode, __u32 uid, __u32 gid,
+ __u64 time, __u64 rdev, struct ptlrpc_request **);
int mdc_unlink(struct lustre_handle *, struct inode *dir, struct inode *child,
__u32 mode, const char *name, int namelen,
struct ptlrpc_request **);
int (* fs_setattr)(struct dentry *dentry, void *handle,
struct iattr *iattr);
int (* fs_set_md)(struct inode *inode, void *handle,
- struct lov_mds_md *md);
- int (* fs_get_md)(struct inode *inode,
- struct lov_mds_md *md);
+ struct lov_mds_md *md, int size);
+ int (* fs_get_md)(struct inode *inode, struct lov_mds_md *md,
+ int size);
ssize_t (* fs_readpage)(struct file *file, char *buf, size_t count,
loff_t *offset);
void (* fs_delete_inode)(struct inode *inode);
*
* ?_NEVENTS # event queue entries
*
- * ?_NBUFS # request buffers
- * ?_BUFSIZE # bytes in a single request buffer
+ * ?_NBUFS # request buffers
+ * ?_BUFSIZE # bytes in a single request buffer
* total memory = ?_NBUFS * ?_BUFSIZE
*
* ?_MAXREQSIZE # maximum request service will receive
*/
#define LDLM_NUM_THREADS 4
-#define LDLM_NEVENTS 1024
-#define LDLM_NBUFS 10
-#define LDLM_BUFSIZE (64 * 1024)
-#define LDLM_MAXREQSIZE 1024
+#define LDLM_NEVENTS 1024
+#define LDLM_NBUFS 10
+#define LDLM_BUFSIZE (64 * 1024)
+#define LDLM_MAXREQSIZE 1024
#define MDT_NUM_THREADS 8
-#define MDS_NEVENTS 1024
-#define MDS_NBUFS 10
-#define MDS_BUFSIZE (64 * 1024)
-#define MDS_MAXREQSIZE 1024
+#define MDS_NEVENTS 1024
+#define MDS_NBUFS 10
+#define MDS_BUFSIZE (64 * 1024)
+#define MDS_MAXREQSIZE 1024
#define OST_NUM_THREADS 6
-#define OST_NEVENTS min(num_physpages / 16, 32768UL)
-#define OST_NBUFS min(OST_NEVENTS / 128, 256UL)
-#define OST_BUFSIZE ((OST_NEVENTS > 4096UL ? 128 : 64) * 1024)
-#define OST_MAXREQSIZE (8 * 1024)
+#define OST_NEVENTS min(num_physpages / 16, 32768UL)
+#define OST_NBUFS min(OST_NEVENTS / 128, 256UL)
+#define OST_BUFSIZE ((OST_NEVENTS > 4096UL ? 128 : 64) * 1024)
+#define OST_MAXREQSIZE (8 * 1024)
#define CONN_INVALID 1
#define PTL_RPC_FL_REPLAY (1 << 11) /* replay upon recovery */
#define PTL_RPC_FL_ALLOCREP (1 << 12) /* reply buffer allocated */
-struct ptlrpc_request {
+struct ptlrpc_request {
int rq_type; /* one of PTL_RPC_MSG_* */
struct list_head rq_list;
struct obd_device *rq_obd;
int rq_status;
- int rq_flags;
+ int rq_flags;
atomic_t rq_refcount;
int rq_request_portal; /* XXX FIXME bug 625069 */
#define DEBUG_REQ(level, req, fmt, args...) \
do { \
CDEBUG(level, \
- "@@@ " fmt " req x"LPD64"/t"LPD64" o%d->%s:%d lens %d/%d fl %x\n" , ## args, \
- req->rq_xid, req->rq_transno, \
+ "@@@ " fmt " req x"LPD64"/t"LPD64" o%d->%s:%d lens %d/%d fl " \
+ "%x\n" , ## args, req->rq_xid, req->rq_transno, \
req->rq_reqmsg ? req->rq_reqmsg->opc : -1, \
req->rq_connection ? (char *)req->rq_connection->c_remote_uuid : "<?>", \
(req->rq_import && req->rq_import->imp_client) ? \
struct ptlrpc_bulk_desc {
+ struct list_head bd_set_chain; /* entry in obd_brw_set */
+ struct obd_brw_set *bd_brw_set;
int bd_flags;
struct ptlrpc_connection *bd_connection;
struct ptlrpc_client *bd_client;
__u32 bd_portal;
struct lustre_handle bd_conn;
- void (*bd_ptl_ev_hdlr)(struct ptlrpc_bulk_desc *, void *);
- void *bd_ptl_ev_data;
+ void (*bd_ptl_ev_hdlr)(struct ptlrpc_bulk_desc *);
wait_queue_head_t bd_waitq;
struct list_head bd_page_list;
ptl_handle_md_t bd_md_h;
ptl_handle_me_t bd_me_h;
- atomic_t bd_source_callback_count;
+ atomic_t bd_source_callback_count;
struct iovec bd_iov[16]; /* self-sized pre-allocated iov */
};
struct ptlrpc_thread {
struct list_head t_link;
- __u32 t_flags;
+ __u32 t_flags;
wait_queue_head_t t_ctl_waitq;
};
/* incoming request buffers */
/* FIXME: perhaps a list of EQs, if multiple NIs are used? */
- __u32 srv_max_req_size; /* biggest request to receive */
- __u32 srv_buf_size; /* # bytes in a request buffer */
- struct list_head srv_rqbds; /* all the request buffer descriptors */
- __u32 srv_nrqbds; /* # request buffers */
- atomic_t srv_nrqbds_receiving; /* # request buffers posted for input */
+ __u32 srv_max_req_size; /* biggest request to receive */
+ __u32 srv_buf_size; /* # bytes in a request buffer */
+ struct list_head srv_rqbds; /* all the request buffer descriptors */
+ __u32 srv_nrqbds; /* # request buffers */
+ atomic_t srv_nrqbds_receiving; /* # request buffers posted for input */
__u32 srv_req_portal;
__u32 srv_rep_portal;
char *srv_name; /* only statically allocated strings here; we don't clean them */
};
-static inline void ptlrpc_hdl2req(struct ptlrpc_request *req, struct lustre_handle *h)
+static inline void ptlrpc_hdl2req(struct ptlrpc_request *req,
+ struct lustre_handle *h)
{
req->rq_reqmsg->addr = h->addr;
req->rq_reqmsg->cookie = h->cookie;
typedef int (*svc_handler_t)(struct ptlrpc_request *req);
/* rpc/connection.c */
-void ptlrpc_readdress_connection(struct ptlrpc_connection *conn, obd_uuid_t uuid);
+void ptlrpc_readdress_connection(struct ptlrpc_connection *, obd_uuid_t uuid);
struct ptlrpc_connection *ptlrpc_get_connection(struct lustre_peer *peer,
obd_uuid_t uuid);
int ptlrpc_put_connection(struct ptlrpc_connection *c);
int ptlrpc_send_bulk(struct ptlrpc_bulk_desc *);
int ptlrpc_register_bulk(struct ptlrpc_bulk_desc *);
int ptlrpc_abort_bulk(struct ptlrpc_bulk_desc *bulk);
+struct obd_brw_set *obd_brw_set_new(void);
+void obd_brw_set_add(struct obd_brw_set *, struct ptlrpc_bulk_desc *);
+void obd_brw_set_free(struct obd_brw_set *);
+
int ptlrpc_reply(struct ptlrpc_service *svc, struct ptlrpc_request *req);
int ptlrpc_error(struct ptlrpc_service *svc, struct ptlrpc_request *req);
void ptlrpc_resend_req(struct ptlrpc_request *request);
__u8 *ptlrpc_req_to_uuid(struct ptlrpc_request *req);
struct ptlrpc_connection *ptlrpc_uuid_to_connection(obd_uuid_t uuid);
+int ll_brw_sync_wait(struct obd_brw_set *, int phase);
+
int ptlrpc_queue_wait(struct ptlrpc_request *req);
void ptlrpc_continue_req(struct ptlrpc_request *req);
int ptlrpc_replay_req(struct ptlrpc_request *req);
/* rpc/service.c */
struct ptlrpc_service *
-ptlrpc_init_svc(__u32 nevents, __u32 nbufs, __u32 bufsize, __u32 max_req_size,
+ptlrpc_init_svc(__u32 nevents, __u32 nbufs, __u32 bufsize, __u32 max_req_size,
int req_portal, int rep_portal,
obd_uuid_t uuid, svc_handler_t, char *name);
void ptlrpc_stop_all_threads(struct ptlrpc_service *svc);
char *name);
int ptlrpc_unregister_service(struct ptlrpc_service *service);
-struct ptlrpc_svc_data {
+struct ptlrpc_svc_data {
char *name;
- struct ptlrpc_service *svc;
+ struct ptlrpc_service *svc;
struct ptlrpc_thread *thread;
struct obd_device *dev;
-};
+};
/* rpc/pack_generic.c */
int lustre_pack_msg(int count, int *lens, char **bufs, int *len,
struct lov_oinfo { /* per-child structure */
__u64 loi_id; /* object ID on the target OST */
- __u64 loi_size; /* size of this object on the target OST */
struct lustre_handle *loi_handle; /* handle for object on OST */
int loi_ost_idx; /* OST stripe index in lmd_objects array */
};
struct lov_stripe_md {
__u32 lsm_magic;
- __u32 lsm_mds_easize; /* packed size for MDS of ea - KILL ME*/
__u64 lsm_object_id; /* lov object id */
__u64 lsm_stripe_size; /* size of the stripe */
__u32 lsm_stripe_pattern; /* per-lov object stripe pattern */
int lsm_stripe_offset; /* offset of first stripe in lmd_objects */
int lsm_stripe_count; /* how many objects are being striped on */
- int lsm_ost_count; /* how many OSTs are in this LOV - KILL ME */
struct lov_oinfo lsm_oinfo[0];
};
struct semaphore cl_sem;
int cl_conn_count;
obd_uuid_t cl_target_uuid; /* XXX -> lustre_name */
+ /* max_mds_easize is purely a performance thing so we don't have to
+ * call obd_size_wiremd() all the time. */
int cl_max_mds_easize;
- int cl_max_ost_easize;
struct obd_device *cl_containing_lov;
};
struct lustre_handle ost_conn; /* the local connection to the OBD */
};
+struct echo_client_obd {
+ struct lustre_handle conn; /* the local connection to osc/lov */
+};
struct lov_tgt_desc {
obd_uuid_t uuid;
struct mds_obd mds;
struct client_obd cli;
struct ost_obd ost;
+ struct echo_client_obd echo_client;;
// struct osc_obd osc;
struct ldlm_obd ldlm;
struct echo_obd echo;
void* counters;
};
-struct brw_cb_data;
-
struct obd_ops {
int (*o_iocontrol)(long cmd, struct lustre_handle *, int len,
void *karg, void *uarg);
int (*o_statfs)(struct lustre_handle *conn, struct obd_statfs *osfs);
+ int (*o_packmd)(struct lustre_handle *, struct lov_mds_md **wire_tgt,
+ struct lov_stripe_md *mem_src);
+ int (*o_unpackmd)(struct lustre_handle *,
+ struct lov_stripe_md **mem_tgt,
+ struct lov_mds_md *wire_src);
int (*o_preallocate)(struct lustre_handle *, obd_count *req,
obd_id *ids);
int (*o_create)(struct lustre_handle *conn, struct obdo *oa,
struct lov_stripe_md *ea);
int (*o_brw)(int rw, struct lustre_handle *conn,
struct lov_stripe_md *ea, obd_count oa_bufs,
- struct brw_page *pgarr, brw_cb_t callback,
- struct brw_cb_data *data);
+ struct brw_page *pgarr, struct obd_brw_set *);
int (*o_punch)(struct lustre_handle *conn, struct obdo *tgt,
struct lov_stripe_md *ea, obd_size count,
obd_off offset);
}
static inline int mds_fs_set_md(struct mds_obd *mds, struct inode *inode,
- void *handle, struct lov_mds_md *md)
+ void *handle, struct lov_mds_md *md,
+ int size)
{
- return mds->mds_fsops->fs_set_md(inode, handle, md);
+ return mds->mds_fsops->fs_set_md(inode, handle, md, size);
}
static inline int mds_fs_get_md(struct mds_obd *mds, struct inode *inode,
- struct lov_mds_md *md)
+ struct lov_mds_md *md, int size)
{
- return mds->mds_fsops->fs_get_md(inode, md);
+ return mds->mds_fsops->fs_get_md(inode, md, size);
}
static inline ssize_t mds_fs_readpage(struct mds_obd *mds, struct file *file,
#endif
-/*
- * ======== OBD Device Declarations ===========
- */
+/* OBD Device Declarations */
#define MAX_OBD_DEVICES 128
extern struct obd_device obd_dev[MAX_OBD_DEVICES];
#define OBD_ATTACHED 0x1
#define OBD_SET_UP 0x2
-extern struct proc_dir_entry *
-proc_lustre_register_obd_device(struct obd_device *obd);
-extern void proc_lustre_release_obd_device(struct obd_device *obd);
-extern void proc_lustre_remove_obd_entry(const char* name,
- struct obd_device *obd);
-
-/*
- * ======== OBD Operations Declarations ===========
- */
+/* OBD Operations Declarations */
#ifdef __KERNEL__
static inline int obd_check_conn(struct lustre_handle *conn)
RETURN(rc);
}
+/* Pack an in-memory MD struct for sending to the MDS and/or disk.
+ * Returns +ve size of packed MD (0 for free), or -ve error.
+ *
+ * If @wire_tgt == NULL, MD size is returned (max size if @mem_src == NULL).
+ * If @*wire_tgt != NULL and @mem_src == NULL, @*wire_tgt will be freed.
+ * If @*wire_tgt == NULL, it will be allocated
+ */
+static inline int obd_packmd(struct lustre_handle *conn,
+ struct lov_mds_md **wire_tgt,
+ struct lov_stripe_md *mem_src)
+{
+ struct obd_export *exp;
+
+ OBD_CHECK_SETUP(conn, exp);
+ OBD_CHECK_OP(exp->exp_obd, packmd);
+
+ RETURN(OBP(exp->exp_obd, packmd)(conn, wire_tgt, mem_src));
+}
+
+static inline int obd_size_wiremd(struct lustre_handle *conn,
+ struct lov_stripe_md *mem_src)
+{
+ return obd_packmd(conn, NULL, mem_src);
+}
+
+/* helper functions */
+static inline int obd_alloc_wiremd(struct lustre_handle *conn,
+ struct lov_mds_md **wire_tgt)
+{
+ LASSERT(wire_tgt);
+ LASSERT(*wire_tgt == NULL);
+ return obd_packmd(conn, wire_tgt, NULL);
+}
+
+static inline int obd_free_wiremd(struct lustre_handle *conn,
+ struct lov_mds_md **wire_tgt)
+{
+ LASSERT(wire_tgt);
+ LASSERT(*wire_tgt);
+ return obd_packmd(conn, wire_tgt, NULL);
+}
+
+/* Unpack an MD struct from the MDS and/or disk to in-memory format.
+ * Returns +ve size of unpacked MD (0 for free), or -ve error.
+ *
+ * If @mem_tgt == NULL, MD size is returned (max size if @wire_src == NULL).
+ * If @*mem_tgt != NULL and @wire_src == NULL, @*mem_tgt will be freed.
+ * If @*mem_tgt == NULL, it will be allocated
+ */
+static inline int obd_unpackmd(struct lustre_handle *conn,
+ struct lov_stripe_md **mem_tgt,
+ struct lov_mds_md *wire_src)
+{
+ struct obd_export *exp;
+
+ OBD_CHECK_SETUP(conn, exp);
+ OBD_CHECK_OP(exp->exp_obd, unpackmd);
+
+ RETURN(OBP(exp->exp_obd, unpackmd)(conn, mem_tgt, wire_src));
+}
+
+static inline int obd_size_memmd(struct lustre_handle *conn,
+ struct lov_mds_md *wire_src)
+{
+ return obd_unpackmd(conn, NULL, wire_src);
+}
+
+/* helper functions */
+static inline int obd_alloc_memmd(struct lustre_handle *conn,
+ struct lov_stripe_md **mem_tgt)
+{
+ LASSERT(mem_tgt);
+ LASSERT(*mem_tgt == NULL);
+ return obd_unpackmd(conn, mem_tgt, NULL);
+}
+
+static inline int obd_free_memmd(struct lustre_handle *conn,
+ struct lov_stripe_md **mem_tgt)
+{
+ LASSERT(mem_tgt);
+ LASSERT(*mem_tgt);
+ return obd_unpackmd(conn, mem_tgt, NULL);
+}
+
static inline int obd_create(struct lustre_handle *conn, struct obdo *obdo,
struct lov_stripe_md **ea)
{
static inline int obd_brw(int cmd, struct lustre_handle *conn,
struct lov_stripe_md *ea, obd_count oa_bufs,
- struct brw_page *pg,
- brw_cb_t callback, struct brw_cb_data *data)
+ struct brw_page *pg, struct obd_brw_set *set)
{
struct obd_export *exp;
int rc;
LBUG();
}
- rc = OBP(exp->exp_obd, brw)(cmd, conn, ea, oa_bufs, pg, callback, data);
+ rc = OBP(exp->exp_obd, brw)(cmd, conn, ea, oa_bufs, pg, set);
RETURN(rc);
}
#endif
-/*
- * ======== OBD Metadata Support ===========
- */
+/* OBD Metadata Support */
extern int obd_init_caches(void);
extern void obd_cleanup_caches(void);
* See the file COPYING in this distribution
*/
-
-#ifndef OBD_ECHO_DEVICENAME
#define OBD_ECHO_DEVICENAME "obdecho"
-#endif
-
-extern struct obd_ops echo_obd_ops;
+#define OBD_ECHO_CLIENT_DEVICENAME "echo_client"
#endif
-
void lov_unpackdesc(struct lov_desc *ld);
void lov_packdesc(struct lov_desc *ld);
-void lov_packmd(struct lov_mds_md *mdsmd, struct lov_stripe_md *md);
-void lov_unpackmd(struct lov_stripe_md *md, struct lov_mds_md *mdsmd);
static inline int lov_stripe_md_size(int stripes)
{
return sizeof(struct lov_stripe_md) + stripes*sizeof(struct lov_oinfo);
}
+#endif
static inline int lov_mds_md_size(int stripes)
{
return sizeof(struct lov_mds_md) + stripes*sizeof(struct lov_object_id);
}
-#endif
-
#define IOC_LOV_TYPE 'g'
#define IOC_LOV_MIN_NR 50
#define IOC_LOV_SET_OSC_ACTIVE _IOWR('g', 50, long)
sizeof(lock->l_remote_handle));
*flags = reply->lock_flags;
- CDEBUG(D_INFO, "remote handle: %p, flags: %d\n",
+ CDEBUG(D_INFO, "local: %p, remote: %p, flags: %d\n", lock,
(void *)(unsigned long)reply->lock_handle.addr, *flags);
if (type == LDLM_EXTENT) {
CDEBUG(D_INFO, "requested extent: "LPU64" -> "LPU64", got "
}
}
- if (!req_passed_in)
- ptlrpc_req_finished(req);
-
if (!is_replay) {
rc = ldlm_lock_enqueue(lock, cookie, cookielen, flags,
completion, blocking);
lock->l_completion_ast(lock, *flags);
}
+ if (!req_passed_in)
+ ptlrpc_req_finished(req);
+
LDLM_DEBUG(lock, "client-side enqueue END");
EXIT;
out:
EXTRA_DIST = mds_updates.c obd_pack.c ll_pack.c simple.c
-EXTRA_DIST += client.c target.c lov_pack.c
+EXTRA_DIST += client.c target.c
include $(top_srcdir)/Rules
cli->cl_import.imp_obd = obddev;
cli->cl_max_mds_easize = sizeof(struct lov_mds_md);
- cli->cl_max_ost_easize = sizeof(struct lov_stripe_md);
MOD_INC_USE_COUNT;
RETURN(0);
/* packing of MDS records */
-void mds_create_pack(struct ptlrpc_request *req, int offset,
- struct inode *dir, __u32 mode, __u64 rdev, __u32 uid,
- __u32 gid, __u64 time, const char *name, int namelen,
- const char *tgt, int tgtlen)
+void mds_create_pack(struct ptlrpc_request *req, int offset, struct inode *dir,
+ __u32 mode, __u64 rdev, __u32 uid, __u32 gid, __u64 time,
+ const char *name, int namelen,
+ const void *data, int datalen)
{
struct mds_rec_create *rec;
char *tmp;
rec->cr_fsgid = HTON__u32(current->fsgid);
rec->cr_cap = HTON__u32(current->cap_effective);
ll_inode2fid(&rec->cr_fid, dir);
- memset(&rec->cr_replayfid, 0, sizeof rec->cr_replayfid);
+ memset(&rec->cr_replayfid, 0, sizeof(rec->cr_replayfid));
rec->cr_mode = HTON__u32(mode);
rec->cr_rdev = HTON__u64(rdev);
rec->cr_uid = HTON__u32(uid);
tmp = lustre_msg_buf(req->rq_reqmsg, offset + 1);
LOGL0(name, namelen, tmp);
- if (tgt) {
+ if (data) {
tmp = lustre_msg_buf(req->rq_reqmsg, offset + 2);
- LOGL0(tgt, tgtlen, tmp);
+ LOGL0(data, datalen, tmp);
}
}
r->ur_name = lustre_msg_buf(req->rq_reqmsg, offset + 1);
r->ur_namelen = req->rq_reqmsg->buflens[offset + 1];
- if (req->rq_reqmsg->bufcount == offset + 3) {
+ if (req->rq_reqmsg->bufcount == offset + 3) {
r->ur_tgt = lustre_msg_buf(req->rq_reqmsg, offset + 2);
r->ur_tgtlen = req->rq_reqmsg->buflens[offset + 2];
- } else {
+ } else {
r->ur_tgt = NULL;
r->ur_tgtlen = 0;
}
llite_SOURCES = dcache.c commit_callback.c super.c rw.c super25.c
llite_SOURCES += file.c dir.c sysctl.c symlink.c $(LINX)
-llite_SOURCES += lov_pack.c recover.c namei.c lproc_llite.c
+llite_SOURCES += recover.c namei.c lproc_llite.c
-lov_pack.c:
- test -e lov_pack.c || ln -sf $(top_srcdir)/lib/lov_pack.c .
ll_pack.c:
test -e ll_pack.c || ln -sf $(top_srcdir)/lib/ll_pack.c .
prepare_write: ll_dir_prepare_write
};
-#if (LINUX_VERSION_CODE < KERNEL_VERSION(2,5,0))
+#if (LINUX_VERSION_CODE < KERNEL_VERSION(2,4,3))
int waitfor_one_page(struct page *page)
{
int error = 0;
} while ((bh = bh->b_this_page) != head);
return error;
}
-#else
+#elif (LINUX_VERSION_CODE > KERNEL_VERSION(2,5,0))
int waitfor_one_page(struct page *page)
{
wait_on_page_locked(page);
#include <linux/lustre_dlm.h>
#include <linux/lustre_lite.h>
+#include <linux/obd_lov.h> /* for lov_mds_md_size() in lov_setstripe() */
#include <linux/random.h>
int ll_inode_setattr(struct inode *inode, struct iattr *attr, int do_trunc);
extern int ll_setattr(struct dentry *de, struct iattr *attr);
int ll_create_objects(struct super_block *sb, obd_id id, uid_t uid, gid_t gid,
- struct lov_stripe_md **lsmp)
+ struct lov_stripe_md **lsmp)
{
struct obdo *oa;
int rc;
RETURN(-ENOMEM);
oa->o_mode = S_IFREG | 0600;
- oa->o_easize = ll_mds_easize(sb);
oa->o_id = id;
oa->o_uid = uid;
oa->o_gid = gid;
oa->o_valid = OBD_MD_FLID | OBD_MD_FLTYPE | OBD_MD_FLMODE |
- OBD_MD_FLEASIZE | OBD_MD_FLUID | OBD_MD_FLGID;
+ OBD_MD_FLUID | OBD_MD_FLGID;
rc = obd_create(ll_s2obdconn(sb), oa, lsmp);
obdo_free(oa);
static int ll_file_open(struct inode *inode, struct file *file)
{
+ struct ll_sb_info *sbi = ll_i2sbi(inode);
+ struct ll_inode_info *lli = ll_i2info(inode);
+ struct lustre_handle *conn = ll_i2obdconn(inode);
struct ptlrpc_request *req = NULL;
struct ll_file_data *fd;
struct obdo *oa;
- struct lov_stripe_md *lsm = NULL;
- struct ll_sb_info *sbi = ll_i2sbi(inode);
- struct ll_inode_info *lli = ll_i2info(inode);
+ struct lov_stripe_md *lsm;
+ struct lov_mds_md *lmm = NULL;
+ int lmm_size = 0;
int rc = 0;
ENTRY;
lsm = lli->lli_smd;
}
+ /* XXX We should only send this to MDS if we just created these
+ * objects, except we also need to handle the user-stripe case.
+ */
+ rc = obd_packmd(conn, &lmm, lli->lli_smd);
+ if (rc < 0)
+ GOTO(out, rc);
+
+ lmm_size = rc;
+
fd = kmem_cache_alloc(ll_file_data_slab, SLAB_KERNEL);
- if (!fd)
+ if (!fd) {
+ if (lmm)
+ obd_free_wiremd(conn, &lmm);
GOTO(out, rc = -ENOMEM);
+ }
memset(fd, 0, sizeof(*fd));
fd->fd_mdshandle.addr = (__u64)(unsigned long)file;
get_random_bytes(&fd->fd_mdshandle.cookie,
sizeof(fd->fd_mdshandle.cookie));
rc = mdc_open(&sbi->ll_mdc_conn, inode->i_ino, S_IFREG | inode->i_mode,
- file->f_flags, lsm, &fd->fd_mdshandle, &req);
+ file->f_flags, lmm, lmm_size, &fd->fd_mdshandle, &req);
+ if (lmm)
+ obd_free_wiremd(conn, &lmm);
fd->fd_req = req;
/* This is the "reply" refcount. */
fd = (struct ll_file_data *)file->private_data;
if (!fd) {
- LBUG();
- GOTO(out, rc = -EINVAL);
+ LASSERT(file->f_flags & O_LOV_DELAY_CREATE);
+ GOTO(out, rc = 0);
}
memset(&oa, 0, sizeof(oa));
return retval;
}
+/* Retrieve object striping information.
+ *
+ * @arg is a pointer to a user struct with one or more of the fields set to
+ * indicate the application preference: lmm_stripe_count, lmm_stripe_size,
+ * lmm_stripe_offset, and lmm_stripe_pattern. lmm_magic must be LOV_MAGIC.
+ */
static int ll_lov_setstripe(struct inode *inode, struct file *file,
- struct lov_user_md *lum)
+ unsigned long arg)
{
struct ll_inode_info *lli = ll_i2info(inode);
- struct lov_stripe_md *lsm;
- int size = ll_mds_easize(inode->i_sb);
+ struct lov_mds_md *lmm = NULL, *lmmu = (void *)arg;
+ struct lustre_handle *conn = ll_i2obdconn(inode);
int rc;
- rc = verify_area(VERIFY_READ, lum, sizeof(*lum));
- if (rc)
+ rc = obd_alloc_wiremd(conn, &lmm);
+ if (rc < 0)
RETURN(rc);
+ rc = copy_from_user(lmm, lmmu, sizeof(*lmm));
+ if (rc)
+ GOTO(out_free, rc = -EFAULT);
+
+ if (lmm->lmm_magic != LOV_MAGIC) {
+ CERROR("bad LOV magic %X\n", lmm->lmm_magic);
+ GOTO(out_free, rc = -EINVAL);
+ }
+
down(&lli->lli_open_sem);
if (lli->lli_smd) {
CERROR("striping data already set for %d\n", inode->i_ino);
GOTO(out_lov_up, rc = -EPERM);
}
+ rc = obd_unpackmd(conn, &lli->lli_smd, lmm);
+ if (rc < 0) {
+ CERROR("error setting LOV striping on %d: rc = %d\n",
+ inode->i_ino, rc);
+ GOTO(out_lov_up, rc);
+ }
- OBD_ALLOC(lli->lli_smd, size);
- if (!lli->lli_smd)
- GOTO(out_lov_up, rc = -ENOMEM);
-
- lsm = lli->lli_smd;
- lsm->lsm_magic = LOV_MAGIC;
- lsm->lsm_stripe_size = lum->lum_stripe_size;
- lsm->lsm_stripe_pattern = lum->lum_stripe_pattern;
- lsm->lsm_stripe_offset = lum->lum_stripe_offset;
- lsm->lsm_stripe_count = lum->lum_stripe_count;
- lsm->lsm_mds_easize = size;
-
- file->f_flags &= ~O_LOV_DELAY_CREATE;
- rc = ll_create_objects(inode->i_sb, inode->i_ino, 0, 0, &lsm);
- if (rc)
- OBD_FREE(lli->lli_smd, size);
- else
+ rc = ll_create_objects(inode->i_sb, inode->i_ino, 0, 0, &lli->lli_smd);
+ if (rc) {
+ obd_free_memmd(conn, &lli->lli_smd);
+ } else {
+ file->f_flags &= ~O_LOV_DELAY_CREATE;
rc = ll_file_open(inode, file);
+ }
out_lov_up:
up(&lli->lli_open_sem);
+out_free:
+ obd_free_wiremd(conn, &lmm);
return rc;
}
+/* Retrieve object striping information.
+ *
+ * @arg is a pointer to a user struct with lmm_ost_count indicating
+ * the maximum number of OST indices which will fit in the user buffer.
+ * lmm_magic must be LOV_MAGIC.
+ */
static int ll_lov_getstripe(struct inode *inode, unsigned long arg)
{
- struct lov_user_md lum;
- struct lov_user_md *lump;
- struct ll_inode_info *lli = ll_i2info(inode);
- struct lov_stripe_md *lsm = lli->lli_smd;
- struct lov_user_oinfo *luoip;
- struct lov_oinfo *loip;
- int count, len, i, rc;
+ struct lov_mds_md lmm, *lmmu = (void *)arg, *lmmk = NULL;
+ struct lov_stripe_md *lsm = ll_i2info(inode)->lli_smd;
+ struct lustre_handle *conn = ll_i2obdconn(inode);
+ int ost_count, rc, lmm_size;
- rc = copy_from_user(&lum, (void *)arg, sizeof(lum));
- if (rc)
- RETURN(rc);
+ if (!lsm)
+ RETURN(-ENODATA);
- if ((count = lsm->lsm_stripe_count) == 0)
- count = 1;
+ rc = copy_from_user(&lmm, lmmu, sizeof(lmm));
+ if (rc)
+ RETURN(-EFAULT);
- if (lum.lum_stripe_count < count)
+ if (lmm.lmm_magic != LOV_MAGIC)
RETURN(-EINVAL);
- len = sizeof(*lump) + count * sizeof(*luoip);
+ if (lsm->lsm_stripe_count == 0)
+ ost_count = 1;
+ else {
+ struct obd_device *obd = class_conn2obd(conn);
+ struct lov_obd *lov = &obd->u.lov;
+ ost_count = lov->desc.ld_tgt_count;
+ }
- rc = verify_area(VERIFY_WRITE, (void *)arg, len);
- if (rc)
+ /* XXX we _could_ check if indices > user lmm_ost_count are zero */
+ if (lmm.lmm_ost_count < ost_count)
+ RETURN(-EOVERFLOW);
+
+ rc = obd_packmd(conn, &lmmk, lsm);
+ if (rc < 0)
RETURN(rc);
- lump = (struct lov_user_md *)arg;
- lump->lum_stripe_count = count;
- luoip = lump->lum_luoinfo;
+ lmm_size = rc;
+ /* LOV STACKING layering violation to make LOV/OSC return same data */
if (lsm->lsm_stripe_count == 0) {
- lump->lum_stripe_size = 0;
- lump->lum_stripe_pattern = 0;
- lump->lum_stripe_offset = 0;
- luoip->luo_idx = 0;
- luoip->luo_id = lsm->lsm_object_id;
- } else {
- lump->lum_stripe_size = lsm->lsm_stripe_size;
- lump->lum_stripe_pattern = lsm->lsm_stripe_pattern;
- lump->lum_stripe_offset = lsm->lsm_stripe_offset;
-
- loip = lsm->lsm_oinfo;
- for (i = 0; i < count; i++, luoip++, loip++) {
- luoip->luo_idx = loip->loi_ost_idx;
- luoip->luo_id = loip->loi_id;
+ struct lov_object_id *loi;
+
+ loi = (void *)lmmu + offsetof(typeof(*lmmu), lmm_objects);
+ rc = copy_to_user(loi, &lsm->lsm_object_id, sizeof(*loi));
+ if (rc) {
+ lmm_size = 0;
+ rc = -EFAULT;
+ } else {
+ lmmk->lmm_magic = LOV_MAGIC;
+ lmmk->lmm_ost_count = lmmk->lmm_stripe_count = 1;
}
}
- RETURN(0);
+ if (lmm_size && copy_to_user(lmmu, lmmk, lmm_size))
+ rc = -EFAULT;
+
+ obd_free_wiremd(conn, &lmmk);
+
+ RETURN(rc);
}
int ll_file_ioctl(struct inode *inode, struct file *file, unsigned int cmd,
fd->fd_flags &= ~flags;
return 0;
case LL_IOC_LOV_SETSTRIPE:
- return ll_lov_setstripe(inode, file, (struct lov_user_md *)arg);
+ return ll_lov_setstripe(inode, file, arg);
case LL_IOC_LOV_GETSTRIPE:
return ll_lov_getstripe(inode, arg);
#include <linux/obd_support.h>
#include <linux/lustre_lite.h>
#include <linux/lustre_dlm.h>
-#include <linux/obd_lov.h>
extern struct address_space_operations ll_aops;
{
struct inode *inode;
+ LASSERT(hash != 0);
inode = iget5_locked(sb, hash, ll_test_inode, ll_read_inode2, lic);
if (!inode)
struct ll_read_inode2_cookie *lic)
{
struct inode *inode;
+ LASSERT(hash != 0);
inode = iget4(sb, hash, ll_find_inode, lic);
return inode;
}
struct lustre_handle lockh;
struct lookup_intent lookup_it = { .it_op = IT_LOOKUP };
struct ptlrpc_request *request = NULL;
- char *tgt = NULL;
- int rc, lock_mode, tgtlen = 0, offset, flag = LL_LOOKUP_POSITIVE;
+ char *data = NULL;
+ int rc, lock_mode, datalen = 0, offset, flag = LL_LOOKUP_POSITIVE;
obd_id ino = 0;
ENTRY;
lock_mode = ll_intent_to_lock_mode(it);
if (it->it_op & IT_SYMLINK) {
- tgt = it->it_data;
- tgtlen = strlen(tgt);
+ data = it->it_data;
+ datalen = strlen(data) + 1;
it->it_data = NULL;
}
rc = mdc_enqueue(&sbi->ll_mdc_conn, LDLM_PLAIN, it, lock_mode, parent,
- dentry, &lockh, tgt, tgtlen, parent, sizeof(*parent));
+ dentry, &lockh, data, datalen, parent,sizeof(*parent));
if (rc < 0)
RETURN(rc);
memcpy(it->it_lock_handle, &lockh, sizeof(lockh));
* intent on our behalf. */
if (it->it_disposition) {
struct mds_body *mds_body;
- int mode, symlen = 0;
+ int mode;
obd_flag valid;
/* This long block is all about fixing up the local
* immediately with the contents of the reply (in the
* intent_finish callback). In the create case,
* however, we need to wait until ll_create_node to do
- * the iget() or the VFS will abort with -EEXISTS.
+ * the iget() or the VFS will abort with -EEXISTS.
*/
offset = 1;
}
/* Do a getattr now that we have the lock */
- valid = OBD_MD_FLNOTOBD | OBD_MD_FLEASIZE;
+ valid = OBD_MD_FLNOTOBD;
if (it->it_op == IT_READLINK) {
+ datalen = mds_body->size;
valid |= OBD_MD_LINKNAME;
- symlen = mds_body->size;
+ } else if (S_ISREG(mode)) {
+ datalen = obd_size_wiremd(&sbi->ll_osc_conn, NULL);
+ valid |= OBD_MD_FLEASIZE;
}
ptlrpc_req_finished(request);
request = NULL;
rc = mdc_getattr(&sbi->ll_mdc_conn, ino, mode,
- valid, symlen, &request);
+ valid, datalen, &request);
if (rc) {
CERROR("failure %d inode "LPX64"\n", rc, ino);
GOTO(drop_req, rc = -abs(rc));
offset = 0;
} else {
struct ll_inode_info *lli = ll_i2info(parent);
+ obd_flag valid;
int mode;
LBUG(); /* For the moment, no non-intent locks */
GOTO(drop_lock, rc = -ENOENT);
}
- rc = mdc_getattr(&sbi->ll_mdc_conn, ino, mode,
- OBD_MD_FLNOTOBD|OBD_MD_FLEASIZE, 0, &request);
+ valid = OBD_MD_FLNOTOBD;
+
+ if (S_ISREG(mode)) {
+ datalen = obd_size_wiremd(&sbi->ll_osc_conn, NULL),
+ valid |= OBD_MD_FLEASIZE;
+ }
+
+ rc = mdc_getattr(&sbi->ll_mdc_conn, ino, mode, valid,
+ datalen, &request);
if (rc) {
CERROR("failure %d inode "LPX64"\n", rc, ino);
GOTO(drop_req, rc = -abs(rc));
{
struct dentry *dentry = *de, *saved = *de;
struct inode *inode = NULL;
- struct ll_read_inode2_cookie lic;
+ struct ll_read_inode2_cookie lic = {.lic_body = NULL, .lic_lmm = NULL};
if (flag == LL_LOOKUP_POSITIVE) {
ENTRY;
}
/* No rpc's happen during iget4, -ENOMEM's are possible */
- LASSERT(ino != 0);
inode = ll_iget(dentry->d_sb, ino, &lic);
if (!inode) {
/* XXX make sure that request is freed in this case;
}
static struct inode *ll_create_node(struct inode *dir, const char *name,
- int namelen, const char *tgt, int tgtlen,
+ int namelen, const void *data, int datalen,
int mode, __u64 extra,
- struct lookup_intent *it,
- struct lov_stripe_md *lsm)
+ struct lookup_intent *it)
{
struct inode *inode;
struct ptlrpc_request *request = NULL;
struct mds_body *body;
time_t time = CURRENT_TIME;
struct ll_sb_info *sbi = ll_i2sbi(dir);
- struct ll_read_inode2_cookie lic;
- struct lov_mds_md *lmm = NULL;
+ struct ll_read_inode2_cookie lic = { .lic_lmm = NULL, };
ENTRY;
if (it && it->it_disposition) {
ll_invalidate_inode_pages(dir);
request = it->it_data;
body = lustre_msg_buf(request->rq_repmsg, 1);
- lic.lic_lmm = NULL;
} else {
int gid = current->fsgid;
int rc;
- if (lsm) {
- OBD_ALLOC(lmm, lsm->lsm_mds_easize);
- if (!lmm)
- RETURN(ERR_PTR(-ENOMEM));
- lov_packmd(lmm, lsm);
- lic.lic_lmm = lmm;
- } else
- lic.lic_lmm = NULL;
-
if (dir->i_mode & S_ISGID) {
gid = dir->i_gid;
if (S_ISDIR(mode))
mode |= S_ISGID;
}
- rc = mdc_create(&sbi->ll_mdc_conn, dir, name, namelen, tgt,
- tgtlen, mode, current->fsuid, gid,
- time, extra, lsm, &request);
+ rc = mdc_create(&sbi->ll_mdc_conn, dir, name, namelen,
+ data, datalen, mode, current->fsuid, gid,
+ time, extra, &request);
if (rc) {
inode = ERR_PTR(rc);
GOTO(out, rc);
lic.lic_body = body;
- LASSERT(body->ino != 0);
inode = ll_iget(dir->i_sb, body->ino, &lic);
if (IS_ERR(inode)) {
int rc = PTR_ERR(inode);
EXIT;
out:
- if (lsm && lmm)
- OBD_FREE(lmm, lsm->lsm_mds_easize);
ptlrpc_req_finished(request);
return inode;
}
/*
* By the time this is called, we already have created the directory cache
* entry for the new file, but it is so far negative - it has no inode.
+ *
* We defer creating the OBD object(s) until open, to keep the intent and
* non-intent code paths similar, and also because we do not have the MDS
* inode number before calling ll_create_node() (which is needed for LOV),
* so we would need to do yet another RPC to the MDS to store the LOV EA
- * data on the MDS.
+ * data on the MDS. If needed, we would pass the PACKED lmm as data and
+ * lmm_size in datalen (the MDS still has code which will handle that).
*
* If the create succeeds, we fill in the inode information
* with d_instantiate().
LL_GET_INTENT(dentry, it);
inode = ll_create_node(dir, dentry->d_name.name, dentry->d_name.len,
- NULL, 0, mode, 0, it, NULL);
+ NULL, 0, mode, 0, it);
if (IS_ERR(inode))
RETURN(PTR_ERR(inode));
LL_GET_INTENT(dentry, it);
inode = ll_create_node(dir, dentry->d_name.name, dentry->d_name.len,
- NULL, 0, mode, rdev, it, NULL);
+ NULL, 0, mode, rdev, it);
if (IS_ERR(inode))
RETURN(PTR_ERR(inode));
const char *symname)
{
struct lookup_intent *it;
- unsigned l = strlen(symname);
+ unsigned l = strlen(symname) + 1;
struct inode *inode;
struct ll_inode_info *lli;
int err = 0;
LL_GET_INTENT(dentry, it);
inode = ll_create_node(dir, dentry->d_name.name, dentry->d_name.len,
- symname, l, S_IFLNK | S_IRWXUGO, 0, it, NULL);
+ symname, l, S_IFLNK | S_IRWXUGO, 0, it);
if (IS_ERR(inode))
RETURN(PTR_ERR(inode));
lli = ll_i2info(inode);
- OBD_ALLOC(lli->lli_symlink_name, l + 1);
+ OBD_ALLOC(lli->lli_symlink_name, l);
/* this _could_ be a non-fatal error, since the symlink is already
* stored on the MDS by this point, and we can re-get it in readlink.
*/
if (!lli->lli_symlink_name)
RETURN(-ENOMEM);
- memcpy(lli->lli_symlink_name, symname, l + 1);
- inode->i_size = l;
+ memcpy(lli->lli_symlink_name, symname, l);
+ inode->i_size = l - 1;
/* no directory data updates when intents rule */
if (it && it->it_disposition)
ext2_inc_count(dir);
inode = ll_create_node(dir, dentry->d_name.name, dentry->d_name.len,
- NULL, 0, S_IFDIR | mode, 0, it, NULL);
+ NULL, 0, S_IFDIR | mode, 0, it);
err = PTR_ERR(inode);
if (IS_ERR(inode))
goto out_dir;
*
* Lustre Lite I/O Page Cache
*
- * Copyright (C) 2002 Cluster File Systems, Inc.
+ * Copyright (c) 2001, 2002 Cluster File Systems, Inc.
+ *
+ * This file is part of Lustre, http://www.lustre.org.
+ *
+ * Lustre is free software; you can redistribute it and/or
+ * modify it under the terms of version 2 of the GNU General Public
+ * License as published by the Free Software Foundation.
+ *
+ * Lustre is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Lustre; if not, write to the Free Software
+ * Foundation, Inc., 675 Mass Ave, Cambridge, MA 02139, USA.
*/
#include <linux/config.h>
{
struct ll_inode_info *lli = ll_i2info(inode);
struct lov_stripe_md *lsm = lli->lli_smd;
- struct brw_cb_data *brw_cbd = ll_init_brw_cb_data();
+ struct obd_brw_set *set;
struct brw_page pg;
- int err;
+ int rc;
ENTRY;
- if (!brw_cbd)
+ set = obd_brw_set_new();
+ if (set == NULL)
RETURN(-ENOMEM);
pg.pg = page;
pg.off = ((obd_off)page->index) << PAGE_SHIFT;
if (cmd == OBD_BRW_WRITE && (pg.off + PAGE_SIZE > inode->i_size))
- pg.count = inode->i_size % PAGE_SIZE;
+ pg.count = inode->i_size % PAGE_SIZE;
else
pg.count = PAGE_SIZE;
pg.flag = create ? OBD_BRW_CREATE : 0;
- err = obd_brw(cmd, ll_i2obdconn(inode),lsm, 1, &pg, ll_sync_brw_cb,
- brw_cbd);
+ set->brw_callback = ll_brw_sync_wait;
+ rc = obd_brw(cmd, ll_i2obdconn(inode), lsm, 1, &pg, set);
+ if (rc)
+ CERROR("error from obd_brw: rc = %d\n", rc);
+ else {
+ rc = ll_brw_sync_wait(set, CB_PHASE_START);
+ if (rc)
+ CERROR("error from callback: rc = %d\n", rc);
+ }
+ obd_brw_set_free(set);
- RETURN(err);
-} /* ll_brw */
+ RETURN(rc);
+}
/* returns the page unlocked, but with a reference */
static int ll_readpage(struct file *file, struct page *page)
static int ll_commit_write(struct file *file, struct page *page,
unsigned from, unsigned to)
{
- int create = 1;
struct inode *inode = page->mapping->host;
struct ll_inode_info *lli = ll_i2info(inode);
struct lov_stripe_md *md = lli->lli_smd;
struct brw_page pg;
- int err;
+ struct obd_brw_set *set;
+ int rc, create = 1;
loff_t size;
- struct brw_cb_data *cbd = ll_init_brw_cb_data();
ENTRY;
pg.pg = page;
pg.off = (((obd_off)page->index) << PAGE_SHIFT);
pg.flag = create ? OBD_BRW_CREATE : 0;
- if (!cbd)
+ set = obd_brw_set_new();
+ if (set == NULL)
RETURN(-ENOMEM);
SetPageUptodate(page);
CDEBUG(D_INODE, "commit_page writing (off "LPD64"), count "LPD64"\n",
pg.off, pg.count);
- err = obd_brw(OBD_BRW_WRITE, ll_i2obdconn(inode), md,
- 1, &pg, ll_sync_brw_cb, cbd);
+ set->brw_callback = ll_brw_sync_wait;
+ rc = obd_brw(OBD_BRW_WRITE, ll_i2obdconn(inode), md, 1, &pg, set);
+ if (rc)
+ CERROR("error from obd_brw: rc = %d\n", rc);
+ else {
+ rc = ll_brw_sync_wait(set, CB_PHASE_START);
+ if (rc)
+ CERROR("error from callback: rc = %d\n", rc);
+ }
+ obd_brw_set_free(set);
kunmap(page);
size = pg.off + pg.count;
if (size > inode->i_size)
inode->i_size = size;
- RETURN(err);
+ RETURN(rc);
} /* ll_commit_write */
struct ll_inode_info *lli = ll_i2info(inode);
struct lov_stripe_md *lsm = lli->lli_smd;
struct brw_page *pga;
+ struct obd_brw_set *set;
int i, rc = 0;
- struct brw_cb_data *cbd;
-
ENTRY;
+
if (!lsm || !lsm->lsm_object_id)
RETURN(-ENOMEM);
RETURN(-EINVAL);
}
- cbd = ll_init_brw_cb_data();
- if (!cbd)
+ set = obd_brw_set_new();
+ if (set == NULL)
RETURN(-ENOMEM);
OBD_ALLOC(pga, sizeof(*pga) * bufs_per_obdo);
if (!pga) {
- OBD_FREE(cbd, sizeof(*cbd));
+ obd_brw_set_free(set);
RETURN(-ENOMEM);
}
pga[i].flag = OBD_BRW_CREATE;
}
+ set->brw_callback = ll_brw_sync_wait;
rc = obd_brw(rw == WRITE ? OBD_BRW_WRITE : OBD_BRW_READ,
- ll_i2obdconn(inode), lsm, bufs_per_obdo, pga,
- ll_sync_brw_cb, cbd);
+ ll_i2obdconn(inode), lsm, bufs_per_obdo, pga, set);
+ if (rc)
+ CERROR("error from obd_brw: rc = %d\n", rc);
+ else {
+ rc = ll_brw_sync_wait(set, CB_PHASE_START);
+ if (rc)
+ CERROR("error from callback: rc = %d\n", rc);
+ }
+ obd_brw_set_free(set);
if (rc == 0)
rc = bufs_per_obdo * PAGE_SIZE;
#include <linux/version.h>
#include <linux/lustre_lite.h>
#include <linux/lustre_ha.h>
-#include <linux/obd_lov.h>
#include <linux/lustre_dlm.h>
#include <linux/init.h>
#include <linux/fs.h>
struct ll_read_inode2_cookie lic;
class_uuid_t uuid;
-
-
ENTRY;
MOD_INC_USE_COUNT;
lprocfs_dereg_mnt(sbi->ll_proc_root);
sbi->ll_proc_root = NULL;
-
+
obd_disconnect(&sbi->ll_mdc_conn);
spin_lock(&dcache_lock);
}
if (atomic_read(&inode->i_count) == 0) {
- struct lov_stripe_md *lsm = lli->lli_smd;
char *symlink_name = lli->lli_symlink_name;
- if (lsm) {
- OBD_FREE(lsm, ll_ost_easize(inode->i_sb));
- lli->lli_smd = NULL;
- }
+ if (lli->lli_smd)
+ obd_free_memmd(&sbi->ll_osc_conn, &lli->lli_smd);
+
if (symlink_name) {
OBD_FREE(symlink_name, strlen(symlink_name) + 1);
lli->lli_symlink_name = NULL;
GOTO(out, -ENOMEM);
oa->o_id = lsm->lsm_object_id;
- oa->o_easize = ll_mds_easize(inode->i_sb);
oa->o_mode = inode->i_mode;
oa->o_valid = OBD_MD_FLID | OBD_MD_FLEASIZE | OBD_MD_FLTYPE;
ll_update_inode(inode, body);
//if (body->valid & OBD_MD_FLEASIZE)
- if (lic && lic->lic_lmm) {
- struct lov_mds_md *lmm = lic->lic_lmm;
- int size;
-
- /* XXX This should probably not be an error in the future,
- * when we allow LOV OSTs to be added.
- */
- if (lmm->lmm_easize != ll_mds_easize(inode->i_sb)) {
- CERROR("Striping metadata size error %ld\n",
- inode->i_ino);
- LBUG();
- }
- size = ll_ost_easize(inode->i_sb);
- OBD_ALLOC(lli->lli_smd, size);
- if (!lli->lli_smd) {
- CERROR("No memory for %d\n", size);
- LBUG();
- }
- lov_unpackmd(lli->lli_smd, lmm);
- } else {
- lli->lli_smd = NULL;
- }
+ LASSERT(!lli->lli_smd);
+ if (lic && lic->lic_lmm)
+ obd_unpackmd(ll_i2obdconn(inode), &lli->lli_smd, lic->lic_lmm);
/* Get the authoritative file size */
if (lli->lli_smd && (inode->i_mode & S_IFREG)) {
CERROR("ll_file_size: %d\n", rc);
/* FIXME: need to somehow prevent inode creation */
LBUG();
+ make_bad_inode(inode);
}
}
#include <linux/version.h>
#include <linux/lustre_lite.h>
#include <linux/lustre_ha.h>
-#include <linux/obd_lov.h>
#include <linux/lustre_dlm.h>
#include <linux/init.h>
#include <linux/fs.h>
CDEBUG(D_SUPER, "this_char %s\n", this_char);
if ( (!*ost && (*ost = ll_read_opt("osc", this_char)))||
(!*mds && (*mds = ll_read_opt("mdc", this_char)))||
- (!(*flags & LL_SBI_NOLCK) && ((*flags) = (*flags) |
+ (!(*flags & LL_SBI_NOLCK) && ((*flags) = (*flags) |
ll_set_opt("nolock", this_char, LL_SBI_NOLCK))) )
continue;
}
goto out_dev;
} /* ll_fill_super */
-struct super_block * ll_get_sb(struct file_system_type *fs_type,
- int flags, char *devname, void * data)
+struct super_block * ll_get_sb(struct file_system_type *fs_type,
+ int flags, char *devname, void * data)
{
return get_sb_nodev(fs_type, flags, data, ll_fill_super);
}
if (atomic_read(&inode->i_count) == 0) {
struct ll_inode_info *lli = ll_i2info(inode);
- struct lov_stripe_md *lsm = lli->lli_smd;
char *symlink_name = lli->lli_symlink_name;
- if (lsm) {
- OBD_FREE(lsm, ll_ost_easize(inode->i_sb));
- lli->lli_smd = NULL;
- }
+ if (lli->lli_smd)
+ obd_free_memmd(&sbi->ll_osc_conn, &lli->lli_smd);
if (symlink_name) {
OBD_FREE(symlink_name, strlen(symlink_name) + 1);
lli->lli_symlink_name = NULL;
GOTO(out, -ENOMEM);
oa->o_id = lsm->lsm_object_id;
- oa->o_easize = ll_mds_easize(inode->i_sb);
oa->o_mode = inode->i_mode;
oa->o_valid = OBD_MD_FLID | OBD_MD_FLEASIZE | OBD_MD_FLTYPE;
struct ll_read_inode2_cookie *lic = opaque;
struct mds_body *body = lic->lic_body;
struct ll_inode_info *lli = ll_i2info(inode);
+ int rc = 0;
ENTRY;
sema_init(&lli->lli_open_sem, 1);
ll_update_inode(inode, body);
//if (body->valid & OBD_MD_FLEASIZE)
- if (lic && lic->lic_lmm) {
- struct lov_mds_md *lmm = lic->lic_lmm;
- int size;
-
- /* XXX This should probably not be an error in the future,
- * when we allow LOV OSTs to be added.
- */
- if (lmm->lmm_easize != ll_mds_easize(inode->i_sb)) {
- CERROR("Striping metadata size error %ld\n",
- inode->i_ino);
- LBUG();
- }
- size = ll_ost_easize(inode->i_sb);
- OBD_ALLOC(lli->lli_smd, size);
- if (!lli->lli_smd) {
- CERROR("No memory for %d\n", size);
- LBUG();
- }
- lov_unpackmd(lli->lli_smd, lmm);
- } else {
- lli->lli_smd = NULL;
- }
+ LASSERT(!lli->lli_smd);
+ if (lic && lic->lic_lmm)
+ obd_unpackmd(ll_i2obdconn(inode), &lli->lli_smd, lic->lic_lmm);
/* Get the authoritative file size */
- if (lli->lli_smd && (inode->i_mode & S_IFREG)) {
- int rc;
-
+ if (lli->lli_smd && S_ISREG(inode->i_mode)) {
rc = ll_file_size(inode, lli->lli_smd);
if (rc) {
CERROR("ll_file_size: %d\n", rc);
/* FIXME: need to somehow prevent inode creation */
LBUG();
+ make_bad_inode(inode);
}
}
kdev_t_to_nr(inode->i_rdev));
EXIT;
}
- return 0;
+
+ return rc;
}
static inline void invalidate_request_list(struct list_head *req_list)
{
struct list_head *tmp, *n;
list_for_each_safe(tmp, n, req_list) {
- struct ptlrpc_request *req =
+ struct ptlrpc_request *req =
list_entry(tmp, struct ptlrpc_request, rq_list);
CERROR("invalidating req xid %d op %d to %s:%d\n",
(unsigned long long)req->rq_xid, req->rq_reqmsg->opc,
struct list_head *ctmp;
ENTRY;
-
+
list_for_each(ctmp, &sbi->ll_conn_chain) {
struct ptlrpc_connection *conn;
conn = list_entry(ctmp, struct ptlrpc_connection, c_sb_chain);
struct file_system_type lustre_lite_fs_type = {
.owner = THIS_MODULE,
- .name = "lustre_lite",
+ .name = "lustre_lite",
.get_sb = ll_get_sb,
.kill_sb = kill_litter_super,
};
int rc;
printk(KERN_INFO "Lustre Lite 0.5.14, info@clusterfs.com\n");
rc = ll_init_inodecache();
- if (rc)
+ if (rc)
return -ENOMEM;
ll_file_data_slab = kmem_cache_create("ll_file_data",
sizeof(struct ll_file_data), 0,
SLAB_HWCACHE_ALIGN, NULL, NULL);
- if (ll_file_data_slab == NULL) {
+ if (ll_file_data_slab == NULL) {
ll_destroy_inodecache();
return -ENOMEM;
}
{
struct ll_inode_info *lli = ll_i2info(inode);
struct ll_sb_info *sbi = ll_i2sbi(inode);
- int rc, len = inode->i_size + 1;
+ int rc, symlen = inode->i_size + 1;
ENTRY;
*request = NULL;
}
rc = mdc_getattr(&sbi->ll_mdc_conn, inode->i_ino, S_IFLNK,
- OBD_MD_LINKNAME, len, request);
+ OBD_MD_LINKNAME, symlen, request);
if (rc) {
CERROR("inode "LPD64": rc = %d\n", inode->i_ino, rc);
RETURN(rc);
*symname = lustre_msg_buf((*request)->rq_repmsg, 1);
- OBD_ALLOC(lli->lli_symlink_name, len);
+ OBD_ALLOC(lli->lli_symlink_name, symlen);
/* do not return an error if we cannot cache the symlink locally */
if (lli->lli_symlink_name)
- memcpy(lli->lli_symlink_name, *symname, len);
+ memcpy(lli->lli_symlink_name, *symname, symlen);
RETURN(0);
}
MODULE = lov
modulefs_DATA = lov.o
EXTRA_PROGRAMS = lov
-LINX= lov_pack.c
+LINX=
-lov_SOURCES = lov_obd.c lproc_lov.c $(LINX)
-
-lov_pack.c:
- test -e lov_pack.c || ln -sf $(top_srcdir)/lib/lov_pack.c
+lov_SOURCES = lov_obd.c lov_pack.c lproc_lov.c $(LINX)
include $(top_srcdir)/Rules
struct lustre_handle *lfh_handles;
};
+extern int lov_packmd(struct lustre_handle *conn, struct lov_mds_md **lmm,
+ struct lov_stripe_md *lsm);
+extern int lov_unpackmd(struct lustre_handle *conn, struct lov_stripe_md **lsm,
+ struct lov_mds_md *lmm);
+
/* obd methods */
+int lov_attach(struct obd_device *dev, obd_count len, void *data)
+{
+ return lprocfs_reg_obd(dev, status_var_nm_1, dev);
+}
+
+int lov_detach(struct obd_device *dev)
+{
+ return lprocfs_dereg_obd(dev);
+}
+
static int lov_connect(struct lustre_handle *conn, struct obd_device *obd,
obd_uuid_t cluuid, struct recovd_obd *recovd,
ptlrpc_recovery_cb_t recover)
GOTO(out_conn, rc = -EINVAL);
}
- mdc->cl_max_mds_easize = lov_mds_md_size(desc->ld_tgt_count);
- mdc->cl_max_ost_easize = lov_stripe_md_size(desc->ld_tgt_count);
-
if (memcmp(obd->obd_uuid, desc->ld_uuid, sizeof(desc->ld_uuid))) {
CERROR("LOV desc: uuid %s not on mds device (%s)\n",
obd->obd_uuid, desc->ld_uuid);
lov->tgts[i].active = 1;
}
+ mdc->cl_max_mds_easize = obd_size_wiremd(conn, NULL);
+
out:
ptlrpc_req_finished(req);
RETURN(rc);
static int lov_setup(struct obd_device *obd, obd_count len, void *buf)
{
- struct obd_ioctl_data* data = buf;
+ struct obd_ioctl_data *data = buf;
struct lov_obd *lov = &obd->u.lov;
int rc = 0;
ENTRY;
struct lov_stripe_md *lsm;
struct lov_oinfo *loi;
struct obdo *tmp;
- int ost_count, ost_idx = 1, i, rc = 0;
+ int ost_count, ost_idx = 1;
+ int rc = 0, i;
ENTRY;
LASSERT(ea);
lov = &export->exp_obd->u.lov;
+ if (!lov->desc.ld_active_tgt_count)
+ RETURN(-EIO);
+
spin_lock(&lov->lov_lock);
ost_count = lov->desc.ld_tgt_count;
- oa->o_easize = lov_stripe_md_size(ost_count);
lsm = *ea;
- if (!lsm) {
- OBD_ALLOC(lsm, oa->o_easize);
- if (!lsm) {
+
+ /* Free the user lsm if it needs to be changed, to avoid memory leaks */
+ if (!lsm || (lsm &&
+ lsm->lsm_stripe_count > lov->desc.ld_active_tgt_count)) {
+ struct lov_stripe_md *lsm_new = NULL;
+ rc = obd_alloc_memmd(conn, &lsm_new);
+ if (rc < 0) {
spin_unlock(&lov->lov_lock);
- GOTO(out_tmp, rc = -ENOMEM);
+ if (lsm)
+ obd_free_memmd(conn, &lsm);
+ GOTO(out_tmp, rc);
}
- lsm->lsm_magic = LOV_MAGIC;
- lsm->lsm_mds_easize = lov_mds_md_size(ost_count);
+ if (lsm) {
+ LASSERT(lsm->lsm_magic == LOV_MAGIC);
+ CERROR("replace user LOV MD: stripes %u > %u active\n",
+ lsm->lsm_stripe_count,
+ lov->desc.ld_active_tgt_count);
+ lsm_new->lsm_stripe_offset = lsm->lsm_stripe_offset;
+ lsm_new->lsm_stripe_size = lsm->lsm_stripe_size;
+ lsm_new->lsm_stripe_pattern = lsm->lsm_stripe_pattern;
+ obd_free_memmd(conn, &lsm);
+ }
+ lsm = lsm_new;
ost_idx = 0; /* if lsm->lsm_stripe_offset is set yet */
+ lsm->lsm_magic = LOV_MAGIC;
}
LASSERT(oa->o_valid & OBD_MD_FLID);
lsm->lsm_object_id = oa->o_id;
- if (!lsm->lsm_stripe_count)
- lsm->lsm_stripe_count = lov->desc.ld_default_stripe_count;
- if (!lsm->lsm_stripe_count)
- lsm->lsm_stripe_count = lov->desc.ld_active_tgt_count;
- else if (lsm->lsm_stripe_count > lov->desc.ld_active_tgt_count)
- lsm->lsm_stripe_count = lov->desc.ld_active_tgt_count;
-
if (!lsm->lsm_stripe_size)
lsm->lsm_stripe_size = lov->desc.ld_default_stripe_size;
GOTO(out_free, rc = -EINVAL);
}
- lsm->lsm_ost_count = ost_count;
if (!ost_idx || lsm->lsm_stripe_offset >= ost_count) {
int mult = lsm->lsm_object_id * lsm->lsm_stripe_count;
int stripe_offset = mult % ost_count;
lsm->lsm_stripe_offset = stripe_offset + sub_offset;
}
+ /* Start with lsm_stripe_offset on an active OSC to avoid confusion */
while (!lov->tgts[lsm->lsm_stripe_offset].active)
lsm->lsm_stripe_offset = (lsm->lsm_stripe_offset+1) % ost_count;
/* create data objects with "parent" OA */
memcpy(tmp, oa, sizeof(*tmp));
- tmp->o_easize = sizeof(struct lov_stripe_md);
+ /* XXX: LOV STACKING: use real "obj_mdp" sub-data */
rc = obd_create(&lov->tgts[ost_idx].conn, tmp, &obj_mdp);
if (rc) {
CERROR("error creating objid "LPX64" sub-object on "
GOTO(out_cleanup, rc);
}
loi->loi_id = tmp->o_id;
- loi->loi_size = tmp->o_size;
CDEBUG(D_INODE, "objid "LPX64" has subobj "LPX64" at idx %d\n",
lsm->lsm_object_id, loi->loi_id, ost_idx);
}
err);
}
out_free:
- OBD_FREE(lsm, oa->o_easize);
+ if (!*ea)
+ obd_free_memmd(conn, &lsm);
goto out_tmp;
}
RETURN(rc);
}
-static int lov_osc_brw_cb(struct brw_cb_data *brw_cbd, int err, int phase)
-{
- int ret = 0;
- ENTRY;
-
- if (phase == CB_PHASE_START)
- RETURN(0);
-
- if (phase == CB_PHASE_FINISH) {
- if (err)
- brw_cbd->brw_err = err;
- if (atomic_dec_and_test(&brw_cbd->brw_refcount))
- ret = brw_cbd->brw_cb(brw_cbd->brw_data, brw_cbd->brw_err, phase);
- RETURN(ret);
- }
-
- LBUG();
- return 0;
-}
-
static inline int lov_brw(int cmd, struct lustre_handle *conn,
struct lov_stripe_md *lsm, obd_count oa_bufs,
- struct brw_page *pga,
- brw_cb_t brw_cb, struct brw_cb_data *brw_cbd)
+ struct brw_page *pga, struct obd_brw_set *set)
{
- int stripe_count = lsm->lsm_stripe_count;
- struct obd_export *export = class_conn2export(conn);
- struct lov_obd *lov;
struct {
int bufct;
int index;
struct lov_stripe_md lsm;
int ost_idx;
} *stripeinfo, *si, *si_last;
+ struct obd_export *export = class_conn2export(conn);
+ struct lov_obd *lov;
struct brw_page *ioarr;
- int rc, i;
- struct brw_cb_data *osc_brw_cbd;
struct lov_oinfo *loi;
- int *where;
+ int rc = 0, i, *where, stripe_count = lsm->lsm_stripe_count;
ENTRY;
if (!lsm) {
lov = &export->exp_obd->u.lov;
- osc_brw_cbd = ll_init_brw_cb_data();
- if (!osc_brw_cbd)
- RETURN(-ENOMEM);
-
OBD_ALLOC(stripeinfo, stripe_count * sizeof(*stripeinfo));
if (!stripeinfo)
GOTO(out_cbdata, rc = -ENOMEM);
if (!ioarr)
GOTO(out_where, rc = -ENOMEM);
- /* This is the only race-free way I can think of to get the refcount
- * correct. -phil */
- atomic_set(&osc_brw_cbd->brw_refcount, 0);
- osc_brw_cbd->brw_cb = brw_cb;
- osc_brw_cbd->brw_data = brw_cbd;
-
for (i = 0; i < oa_bufs; i++) {
where[i] = lov_stripe_number(lsm, pga[i].off);
- if (stripeinfo[where[i]].bufct++ == 0)
- atomic_inc(&osc_brw_cbd->brw_refcount);
+ stripeinfo[where[i]].bufct++;
}
for (i = 0, loi = lsm->lsm_oinfo, si_last = si = stripeinfo;
LASSERT(shift < oa_bufs);
/* XXX handle error returns here */
obd_brw(cmd, &lov->tgts[si->ost_idx].conn,
- &si->lsm, si->bufct, &ioarr[shift],
- lov_osc_brw_cb, osc_brw_cbd);
+ &si->lsm, si->bufct, &ioarr[shift], set);
}
}
- rc = brw_cb(brw_cbd, 0, CB_PHASE_START);
-
OBD_FREE(ioarr, sizeof(*ioarr) * oa_bufs);
out_where:
OBD_FREE(where, sizeof(*where) * oa_bufs);
out_sinfo:
OBD_FREE(stripeinfo, stripe_count * sizeof(*stripeinfo));
out_cbdata:
- OBD_FREE(osc_brw_cbd, sizeof(*osc_brw_cbd));
RETURN(rc);
}
continue;
submd.lsm_object_id = loi->loi_id;
- /* XXX submd lsm_mds_easize should be that from the subobj,
- * and the subobj should get it opaquely from the LOV.
+ /* XXX submd should be that from the subobj, it should come
+ * opaquely from the LOV.
*/
- submd.lsm_mds_easize = lov_mds_md_size(lsm->lsm_ost_count);
submd.lsm_stripe_count = 0;
/* XXX submd is not fully initialized here */
rc = obd_enqueue(&(lov->tgts[loi->loi_ost_idx].conn), &submd,
continue;
submd.lsm_object_id = loi->loi_id;
- submd.lsm_mds_easize = lov_mds_md_size(lsm->lsm_ost_count);
submd.lsm_stripe_count = 0;
rc = obd_cancel(&lov->tgts[loi->loi_ost_idx].conn, &submd,
mode, &lockhs[i]);
struct lov_stripe_md submd;
submd.lsm_object_id = loi->loi_id;
- submd.lsm_mds_easize = lov_mds_md_size(lsm->lsm_ost_count);
submd.lsm_stripe_count = 0;
rc = obd_cancel_unused(&lov->tgts[loi->loi_ost_idx].conn,
&submd, flags);
void *karg, void *uarg)
{
struct obd_device *obddev = class_conn2obd(conn);
- struct obd_ioctl_data *data = karg;
struct lov_obd *lov = &obddev->u.lov;
- struct lov_desc *desc;
- struct lov_tgt_desc *tgtdesc;
- obd_uuid_t *uuidp;
- char *buf;
- int rc, i, count;
+ struct obd_ioctl_data *data = karg;
+ int i, count = lov->desc.ld_tgt_count;
+ int rc;
+
ENTRY;
switch (cmd) {
- case IOC_LOV_SET_OSC_ACTIVE:
+ case IOC_LOV_SET_OSC_ACTIVE: {
rc = lov_set_osc_active(lov,data->ioc_inlbuf1,data->ioc_offset);
break;
- case OBD_IOC_LOV_GET_CONFIG:
+ }
+ case OBD_IOC_LOV_GET_CONFIG: {
+ struct lov_tgt_desc *tgtdesc;
+ struct lov_desc *desc;
+ obd_uuid_t *uuidp;
+ char *buf = NULL;
+
buf = NULL;
len = 0;
if (obd_ioctl_getdata(&buf, &len, (void *)uarg))
RETURN(-EINVAL);
}
- count = lov->desc.ld_tgt_count;
-
if (sizeof(*uuidp) * count > data->ioc_inllen2) {
OBD_FREE(buf, len);
RETURN(-EINVAL);
memcpy(uuidp, tgtdesc->uuid, sizeof(*uuidp));
rc = copy_to_user((void *)uarg, buf, len);
+ if (rc)
+ rc = -EFAULT;
OBD_FREE(buf, len);
break;
+ }
default:
- if (lov->desc.ld_tgt_count == 0)
+ if (count == 0)
RETURN(-ENOTTY);
rc = 0;
- for (i = 0; i < lov->desc.ld_tgt_count; i++) {
+ for (i = 0; i < count; i++) {
int err = obd_iocontrol(cmd, &lov->tgts[i].conn,
- len, data, NULL);
+ len, karg, uarg);
if (err && !rc)
rc = err;
}
RETURN(rc);
}
-int lov_attach(struct obd_device *dev, obd_count len, void *data)
-{
- return lprocfs_reg_obd(dev, status_var_nm_1, dev);
-}
-
-int lov_detach(struct obd_device *dev)
-{
- return lprocfs_dereg_obd(dev);
-}
-
struct obd_ops lov_obd_ops = {
o_attach: lov_attach,
o_detach: lov_detach,
o_setup: lov_setup,
o_connect: lov_connect,
o_disconnect: lov_disconnect,
+ o_statfs: lov_statfs,
+ o_packmd: lov_packmd,
+ o_unpackmd: lov_unpackmd,
o_create: lov_create,
o_destroy: lov_destroy,
o_getattr: lov_getattr,
o_setattr: lov_setattr,
- o_statfs: lov_statfs,
o_open: lov_open,
o_close: lov_close,
o_brw: lov_brw,
0, 0, NULL, NULL);
if (!lov_file_cache)
RETURN(-ENOMEM);
+
rc = class_register_type(&lov_obd_ops, status_class_var,
OBD_LOV_DEVICENAME);
RETURN(rc);
--- /dev/null
+/* -*- mode: c; c-basic-offset: 8; indent-tabs-mode: nil; -*-
+ * vim:expandtab:shiftwidth=8:tabstop=8:
+ *
+ * Copyright (C) 2002 Cluster File Systems, Inc. <adilger@clusterfs.com>
+ *
+ * This file is part of Lustre, http://www.lustre.org.
+ *
+ * Lustre is free software; you can redistribute it and/or
+ * modify it under the terms of version 2 of the GNU General Public
+ * License as published by the Free Software Foundation.
+ *
+ * Lustre is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Lustre; if not, write to the Free Software
+ * Foundation, Inc., 675 Mass Ave, Cambridge, MA 02139, USA.
+ *
+ * (Un)packing of OST/MDS requests
+ *
+ */
+
+#define DEBUG_SUBSYSTEM S_LLITE
+
+#include <linux/lustre_net.h>
+#include <linux/obd.h>
+#include <linux/obd_lov.h>
+#include <linux/obd_support.h>
+
+/* lov_packdesc() is in mds/mds_lov.c */
+
+void lov_unpackdesc(struct lov_desc *ld)
+{
+ ld->ld_tgt_count = NTOH__u32(ld->ld_tgt_count);
+ ld->ld_default_stripe_count = HTON__u32(ld->ld_default_stripe_count);
+ ld->ld_default_stripe_size = HTON__u32(ld->ld_default_stripe_size);
+ ld->ld_pattern = HTON__u32(ld->ld_pattern);
+}
+
+/* Pack LOV object metadata for shipment to the MDS.
+ *
+ * XXX In the future, this will be enhanced to get the EA size from the
+ * underlying OSC device(s) to get their EA sizes so we can stack
+ * LOVs properly. For now lov_mds_md_size() just assumes one obd_id
+ * per stripe.
+ */
+int lov_packmd(struct lustre_handle *conn, struct lov_mds_md **lmmp,
+ struct lov_stripe_md *lsm)
+{
+ struct obd_device *obd = class_conn2obd(conn);
+ struct lov_obd *lov = &obd->u.lov;
+ struct lov_oinfo *loi;
+ struct lov_mds_md *lmm;
+ int ost_count = lov->desc.ld_tgt_count;
+ int stripe_count = ost_count;
+ int lmm_size;
+ int i;
+ ENTRY;
+
+ if (lsm)
+ stripe_count = lsm->lsm_stripe_count;
+
+ /* XXX LOV STACKING call into osc for sizes */
+ lmm_size = lov_mds_md_size(ost_count);
+
+ if (!lmmp)
+ RETURN(lmm_size);
+
+ if (*lmmp && !lsm) {
+ /* endianness */
+ ost_count = ((*lmmp)->lmm_ost_count);
+ OBD_FREE(*lmmp, lov_mds_md_size(ost_count));
+ *lmmp = NULL;
+ RETURN(0);
+ }
+
+ if (!*lmmp) {
+ OBD_ALLOC(*lmmp, lmm_size);
+ if (!*lmmp)
+ RETURN(-ENOMEM);
+ }
+
+ lmm = *lmmp;
+
+ lmm->lmm_stripe_count = (stripe_count);
+ if (!lsm)
+ RETURN(lmm_size);
+ /* XXX endianness */
+ lmm->lmm_magic = (lsm->lsm_magic);
+ lmm->lmm_object_id = (lsm->lsm_object_id);
+ lmm->lmm_stripe_size = (lsm->lsm_stripe_size);
+ lmm->lmm_stripe_pattern = (lsm->lsm_stripe_pattern);
+ lmm->lmm_stripe_offset = (lsm->lsm_stripe_offset);
+ lmm->lmm_ost_count = (lov->desc.ld_tgt_count);
+
+ /* Only fill in the object ids which we are actually using.
+ * Assumes lmm_objects is otherwise zero-filled. */
+ for (i = 0, loi = lsm->lsm_oinfo; i < stripe_count; i++, loi++)
+ /* XXX call down to osc_packmd() to do the packing */
+ lmm->lmm_objects[loi->loi_ost_idx].l_object_id = (loi->loi_id);
+
+ RETURN(lmm_size);
+}
+
+int lov_unpackmd(struct lustre_handle *conn, struct lov_stripe_md **lsmp,
+ struct lov_mds_md *lmm)
+{
+ struct obd_device *obd = class_conn2obd(conn);
+ struct lov_obd *lov = &obd->u.lov;
+ struct lov_stripe_md *lsm;
+ struct lov_oinfo *loi;
+ int ost_count = lov->desc.ld_active_tgt_count;
+ int ost_offset = 0;
+ int stripe_count = 0;
+ int lsm_size;
+ int i;
+ ENTRY;
+
+ if (lmm)
+ /* endianness */
+ stripe_count = (lmm->lmm_stripe_count);
+
+ if (!stripe_count)
+ stripe_count = lov->desc.ld_default_stripe_count;
+ if (!stripe_count || stripe_count > ost_count)
+ stripe_count = ost_count;
+
+ /* XXX LOV STACKING call into osc for sizes */
+ lsm_size = lov_stripe_md_size(stripe_count);
+
+ if (!lsmp)
+ RETURN(lsm_size);
+
+ if (*lsmp && !lmm) {
+ stripe_count = (*lsmp)->lsm_stripe_count;
+ OBD_FREE(*lsmp, lov_stripe_md_size(stripe_count));
+ *lsmp = NULL;
+ RETURN(0);
+ }
+
+ if (!*lsmp) {
+ OBD_ALLOC(*lsmp, lsm_size);
+ if (!*lsmp)
+ RETURN(-ENOMEM);
+ }
+
+ lsm = *lsmp;
+
+ lsm->lsm_stripe_count = stripe_count;
+ if (!lmm)
+ RETURN(lsm_size);
+
+ /* XXX endianness */
+ ost_offset = lsm->lsm_stripe_offset = (lmm->lmm_stripe_offset);
+ lsm->lsm_magic = (lmm->lmm_magic);
+ lsm->lsm_object_id = (lmm->lmm_object_id);
+ lsm->lsm_stripe_size = (lmm->lmm_stripe_size);
+ lsm->lsm_stripe_pattern = (lmm->lmm_stripe_pattern);
+
+ for (i = 0, loi = lsm->lsm_oinfo; i < ost_count; i++, ost_offset++) {
+ ost_offset %= ost_count;
+
+ if (!lmm->lmm_objects[ost_offset].l_object_id)
+ continue;
+
+ LASSERT(loi - lsm->lsm_oinfo < stripe_count);
+ /* XXX LOV STACKING call down to osc_unpackmd() */
+ loi->loi_id = (lmm->lmm_objects[ost_offset].l_object_id);
+ loi->loi_ost_idx = ost_offset;
+ loi++;
+ }
+
+ RETURN(lsm_size);
+}
modulefs_DATA = mdc.o
EXTRA_PROGRAMS = mdc
-LINX= mds_updates.c ll_pack.c lov_pack.c client.c
+LINX= mds_updates.c ll_pack.c client.c
mdc_SOURCES = mdc_request.c mdc_reint.c lproc_mdc.c $(LINX)
-lov_pack.c:
- test -e lov_pack.c || ln -sf $(top_srcdir)/lib/lov_pack.c .
ll_pack.c:
test -e ll_pack.c || ln -sf $(top_srcdir)/lib/ll_pack.c .
mds_updates.c:
client.c:
test -e client.c || ln -sf $(top_srcdir)/lib/client.c .
-
include $(top_srcdir)/Rules
#include <linux/obd_class.h>
#include <linux/lustre_mds.h>
-#include <linux/obd_lov.h>
static int mdc_reint(struct ptlrpc_request *request, int level)
{
RETURN(rc);
}
-int mdc_create(struct lustre_handle *conn,
- struct inode *dir, const char *name, int namelen,
- const char *tgt, int tgtlen, int mode, __u32 uid,
- __u32 gid, __u64 time, __u64 rdev, struct lov_stripe_md *lsm,
+int mdc_create(struct lustre_handle *conn, struct inode *dir,
+ const char *name, int namelen, const void *data, int datalen,
+ int mode, __u32 uid, __u32 gid, __u64 time, __u64 rdev,
struct ptlrpc_request **request)
{
struct ptlrpc_request *req;
int level, bufcount = 2;
ENTRY;
- if (S_ISLNK(mode)) {
- size[2] = tgtlen + 1;
- bufcount = 3;
+ if (data && datalen) {
+ size[bufcount] = datalen;
+ bufcount++;
}
req = ptlrpc_prep_req(class_conn2cliimp(conn), MDS_REINT, bufcount,
RETURN(-ENOMEM);
/* mds_create_pack fills msg->bufs[1] with name
- * and msg->bufs[2] with tgt, for symlinks */
+ * and msg->bufs[2] with tgt, for symlinks or lov MD data */
mds_create_pack(req, 0, dir, mode, rdev, uid, gid, time,
- name, namelen, tgt, tgtlen);
+ name, namelen, data, datalen);
size[0] = sizeof(struct mds_body);
req->rq_replen = lustre_msg_size(1, size);
rc = mdc_reint(req, LUSTRE_CONN_FULL);
*request = req;
- if (rc == -ERESTARTSYS )
+ if (rc == -ERESTARTSYS)
rc = 0;
RETURN(rc);
#include <linux/lustre_lite.h>
#include <linux/lustre_dlm.h>
#include <linux/init.h>
-#include <linux/obd_lov.h>
#include <linux/lprocfs_status.h>
#define REQUEST_MINOR 244
ll_ino2fid(&body->fid1, ino, 0, type);
body->valid = valid;
- if (S_ISREG(type)) {
- struct client_obd *mdc = &class_conn2obd(conn)->u.cli;
- bufcount = 2;
- size[1] = mdc->cl_max_mds_easize;
- } else if (valid & OBD_MD_LINKNAME) {
- bufcount = 2;
- size[1] = ea_size;
+ if (ea_size) {
+ size[bufcount] = ea_size;
+ bufcount++;
body->size = ea_size;
- CDEBUG(D_INODE, "allocating %d bytes for symlink in packet\n",
+ CDEBUG(D_INODE, "reserving %d bytes for MD/symlink in packet\n",
ea_size);
}
req->rq_replen = lustre_msg_size(bufcount, size);
de->d_name.name, de->d_name.len);
req->rq_replen = lustre_msg_size(3, repsize);
- } else if (it->it_op & (IT_GETATTR | IT_RENAME | IT_LINK |
- IT_OPEN | IT_SETATTR | IT_LOOKUP | IT_READLINK)) {
+ } else if (it->it_op & (IT_GETATTR | IT_RENAME | IT_LINK |
+ IT_OPEN | IT_SETATTR | IT_LOOKUP | IT_READLINK)) {
size[2] = sizeof(struct mds_body);
size[3] = de->d_name.len + 1;
}
int mdc_open(struct lustre_handle *conn, obd_id ino, int type, int flags,
- struct lov_stripe_md *lsm, struct lustre_handle *fh,
+ struct lov_mds_md *lmm, int lmm_size, struct lustre_handle *fh,
struct ptlrpc_request **request)
{
struct mds_body *body;
struct ptlrpc_request *req;
ENTRY;
- if (lsm) {
+ if (lmm && lmm_size) {
bufcount = 3;
size[2] = size[1]; /* shuffle the spare data along */
-
- size[1] = lsm->lsm_mds_easize;
+ size[1] = lmm_size;
}
req = ptlrpc_prep_req(class_conn2cliimp(conn), MDS_OPEN, bufcount, size,
if (!req)
GOTO(out, rc = -ENOMEM);
- if (lsm)
- lustre_msg_set_op_flags(req->rq_reqmsg, MDS_OPEN_HAS_EA);
-
-
req->rq_flags |= PTL_RPC_FL_REPLAY;
body = lustre_msg_buf(req->rq_reqmsg, 0);
body->flags = HTON__u32(flags);
memcpy(&body->handle, fh, sizeof(body->handle));
- if (lsm)
- lov_packmd(lustre_msg_buf(req->rq_reqmsg, 1), lsm);
+ if (lmm && lmm_size) {
+ CDEBUG(D_INODE, "sending %u bytes MD for ino LPU64\n",
+ lmm_size, ino);
+ lustre_msg_set_op_flags(req->rq_reqmsg, MDS_OPEN_HAS_EA);
+ memcpy(lustre_msg_buf(req->rq_reqmsg, 1), lmm, lmm_size);
+ body->flags |= HTON__u32(OBD_MD_FLEASIZE);
+ }
req->rq_replen = lustre_msg_size(1, size);
/* If open is replayed, we need to fix up the fh. */
req->rq_replay_cb = mdc_replay_open;
- replay_data = lustre_msg_buf(req->rq_reqmsg, lsm ? 2 : 1);
+ replay_data = lustre_msg_buf(req->rq_reqmsg, lmm ? 2 : 1);
replay_data->fh = fh;
-
+
EXIT;
out:
*request = req;
static int __init ptlrpc_request_init(void)
{
- return class_register_type(&mdc_obd_ops, status_class_var,
+ return class_register_type(&mdc_obd_ops, status_class_var,
LUSTRE_MDC_NAME);
}
modulefs_DATA = mds.o $(FSMOD).o
EXTRA_PROGRAMS = mds $(FSMOD)
-LINX= mds_updates.c simple.c ll_pack.c lov_pack.c target.c
+LINX= mds_updates.c simple.c ll_pack.c target.c
ll_pack.c:
test -e ll_pack.c || ln -sf $(top_srcdir)/lib/ll_pack.c
test -e mds_updates.c || ln -sf $(top_srcdir)/lib/mds_updates.c
simple.c:
test -e simple.c || ln -sf $(top_srcdir)/lib/simple.c
-lov_pack.c:
- test -e lov_pack.c || ln -sf $(top_srcdir)/lib/lov_pack.c
target.c:
test -e target.c || ln -sf $(top_srcdir)/lib/target.c
#if (LINUX_VERSION_CODE >= KERNEL_VERSION(2,5,0))
#include <linux/buffer_head.h>
#endif
+#include <linux/obd_lov.h>
#include <linux/lprocfs_status.h>
static kmem_cache_t *mds_file_cache;
RETURN(0);
}
- mds->mds_max_mdsize = sizeof(struct lov_mds_md) +
- tgt_count * sizeof(struct lov_object_id);
+ /* XXX the MDS should not really know about this */
+ mds->mds_max_mdsize = lov_mds_md_size(tgt_count);
rc = mds_get_lovtgts(mds, tgt_count,
lustre_msg_buf(req->rq_repmsg, 1));
if (rc) {
RETURN(0);
}
+int mds_pack_md(struct mds_obd *mds, struct ptlrpc_request *req,
+ int offset, struct mds_body *body, struct inode *inode)
+{
+ struct lov_mds_md *lmm;
+ int lmm_size = req->rq_repmsg->buflens[offset];
+ int rc;
+
+ if (lmm_size == 0) {
+ CERROR("no space reserved for inode %u MD\n", inode->i_ino);
+ RETURN(0);
+ }
+
+ lmm = lustre_msg_buf(req->rq_repmsg, offset);
+
+ /* I don't really like this, but it is a sanity check on the client
+ * MD request. However, if the client doesn't know how much space
+ * to reserve for the MD, this shouldn't be fatal either...
+ */
+ if (lmm_size > mds->mds_max_mdsize) {
+ CERROR("Reading MD for inode %u of %d bytes > max %d\n",
+ inode->i_ino, lmm_size, mds->mds_max_mdsize);
+ // RETURN(-EINVAL);
+ }
+
+ /* We don't need to store the reply size, because this buffer is
+ * discarded right after unpacking, and the LOV can figure out the
+ * size itself from the ost count.
+ */
+ if ((rc = mds_fs_get_md(mds, inode, lmm, lmm_size)) < 0) {
+ CDEBUG(D_INFO, "No md for ino %u: rc = %d\n", inode->i_ino, rc);
+ } else {
+ body->valid |= OBD_MD_FLEASIZE;
+ rc = 0;
+ }
+
+ return rc;
+}
+
static int mds_getattr_internal(struct mds_obd *mds, struct dentry *dentry,
struct ptlrpc_request *req,
struct mds_body *reqbody, int reply_off)
{
struct mds_body *body;
struct inode *inode = dentry->d_inode;
- int rc;
+ int rc = 0;
ENTRY;
if (inode == NULL)
mds_pack_inode2body(body, inode);
if (S_ISREG(inode->i_mode)) {
- struct lov_mds_md *lmm;
-
- lmm = lustre_msg_buf(req->rq_repmsg, reply_off + 1);
- lmm->lmm_easize = mds->mds_max_mdsize;
- rc = mds_fs_get_md(mds, inode, lmm);
-
- if (rc < 0) {
- if (rc == -ENODATA)
- RETURN(0);
- CERROR("mds_fs_get_md failed: %d\n", rc);
- RETURN(rc);
- }
- body->valid |= OBD_MD_FLEASIZE;
+ rc = mds_pack_md(mds, req, reply_off + 1, body, inode);
} else if (S_ISLNK(inode->i_mode) && reqbody->valid & OBD_MD_LINKNAME) {
char *symname = lustre_msg_buf(req->rq_repmsg, reply_off + 1);
int len = req->rq_repmsg->buflens[reply_off + 1];
rc = inode->i_op->readlink(dentry, symname, len);
if (rc < 0) {
CERROR("readlink failed: %d\n", rc);
- RETURN(rc);
- } else
+ } else {
CDEBUG(D_INODE, "read symlink dest %s\n", symname);
-
- body->valid |= OBD_MD_LINKNAME;
+ body->valid |= OBD_MD_LINKNAME;
+ }
}
- RETURN(0);
+ RETURN(rc);
}
static int mds_getattr_name(int offset, struct ptlrpc_request *req)
inode = de->d_inode;
if (S_ISREG(body->fid1.f_type)) {
- bufcount = 2;
- size[1] = mds->mds_max_mdsize;
+ int rc = mds_fs_get_md(mds, inode, NULL, 0);
+ CDEBUG(D_INODE, "got %d bytes MD data for inode %u\n",
+ rc, inode->i_ino);
+ if (rc < 0) {
+ if (rc != -ENODATA)
+ CERROR("error getting inode %u MD: rc = %d\n",
+ inode->i_ino, rc);
+ size[bufcount] = 0;
+ } else if (rc < mds->mds_max_mdsize) {
+ size[bufcount] = 0;
+ CERROR("MD size %d larger than maximum possible %u\n",
+ rc, mds->mds_max_mdsize);
+ } else
+ size[bufcount] = rc;
+ bufcount++;
} else if (body->valid & OBD_MD_LINKNAME) {
- bufcount = 2;
- size[1] = MIN(inode->i_size + 1, body->size);
+ size[bufcount] = MIN(inode->i_size + 1, body->size);
+ bufcount++;
CDEBUG(D_INODE, "symlink size: %d, reply space: %d\n",
inode->i_size + 1, body->size);
}
if (OBD_FAIL_CHECK(OBD_FAIL_MDS_GETATTR_PACK)) {
- CERROR("failed GETATTR_PACK test\n");
+ CERROR("failed MDS_GETATTR_PACK test\n");
req->rq_status = -ENOMEM;
GOTO(out, rc = -ENOMEM);
}
rc = lustre_pack_msg(bufcount, size, NULL, &req->rq_replen,
&req->rq_repmsg);
if (rc) {
- CERROR("out of memory or FAIL_MDS_GETATTR_PACK\n");
+ CERROR("out of memoryK\n");
req->rq_status = rc;
GOTO(out, rc);
}
return mfd;
}
-static int mds_store_ea(struct mds_obd *mds, struct ptlrpc_request *req,
- struct mds_body *body, struct dentry *de,
- struct lov_mds_md *lmm)
+static int mds_store_md(struct mds_obd *mds, struct ptlrpc_request *req,
+ int offset, struct mds_body *body, struct inode *inode)
{
+ struct lov_mds_md *lmm = lustre_msg_buf(req->rq_reqmsg, offset);
+ int lmm_size = req->rq_reqmsg->buflens[offset];
struct obd_run_ctxt saved;
struct obd_ucred uc;
void *handle;
int rc, rc2;
+ ENTRY;
+
+ /* I don't really like this, but it is a sanity check on the client
+ * MD request.
+ */
+ if (lmm_size > mds->mds_max_mdsize) {
+ CERROR("Saving MD for inode %u of %d bytes > max %d\n",
+ inode->i_ino, lmm_size, mds->mds_max_mdsize);
+ //RETURN(-EINVAL);
+ }
+ CDEBUG(D_INODE, "storing %d bytes MD for inode %u\n",
+ lmm_size, inode->i_ino);
uc.ouc_fsuid = body->fsuid;
uc.ouc_fsgid = body->fsgid;
uc.ouc_cap = body->capability;
- push_ctxt(&saved, &mds->mds_ctxt, &uc);
- handle = mds_fs_start(mds, de->d_inode, MDS_FSOP_SETATTR);
+ push_ctxt(&saved, &mds->mds_ctxt, &uc);
+ handle = mds_fs_start(mds, inode, MDS_FSOP_SETATTR);
if (!handle)
GOTO(out_ea, rc = -ENOMEM);
- rc = mds_fs_set_md(mds, de->d_inode, handle, lmm);
+ rc = mds_fs_set_md(mds, inode, handle, lmm, lmm_size);
if (!rc)
rc = mds_update_last_rcvd(mds, handle, req);
- rc2 = mds_fs_commit(mds, de->d_inode, handle);
+ rc2 = mds_fs_commit(mds, inode, handle);
if (rc2 && !rc)
rc = rc2;
out_ea:
pop_ctxt(&saved);
- return rc;
+ RETURN(rc);
}
static int mds_open(struct ptlrpc_request *req)
/* check if this inode has seen a delayed object creation */
if (lustre_msg_get_op_flags(req->rq_reqmsg) & MDS_OPEN_HAS_EA) {
- struct lov_mds_md *lmm = lustre_msg_buf(req->rq_reqmsg, 1);
-
- rc = mds_store_ea(mds, req, body, de, lmm);
+ rc = mds_store_md(mds, req, 1, body, de->d_inode);
if (rc) {
l_dput(de);
mntput(mnt);
RETURN(0);
}
-int mds_reint(int offset, struct ptlrpc_request *req)
+int mds_reint(struct ptlrpc_request *req, int offset)
{
int rc;
struct mds_update_record rec;
req->rq_status = rc;
break;
}
- rc = mds_reint(0, req);
+ rc = mds_reint(req, 0);
OBD_FAIL_RETURN(OBD_FAIL_MDS_REINT_NET_REP, 0);
break;
}
/* execute policy */
switch ((long)it->opc) {
case IT_CREAT|IT_OPEN:
- rc = mds_reint(2, req);
+ rc = mds_reint(req, 2);
if (rc || (req->rq_status != 0 &&
req->rq_status != -EEXIST)) {
rep->lock_policy_res2 = req->rq_status;
case IT_RMDIR:
case IT_SYMLINK:
case IT_UNLINK:
- rc = mds_reint(2, req);
+ rc = mds_reint(req, 2);
if (rc || (req->rq_status != 0 &&
req->rq_status != -EISDIR &&
req->rq_status != -ENOTDIR)) {
/* If the client is about to open a file that doesn't have an MD
* stripe record, it's going to need a write lock. */
- if (it->opc & IT_OPEN) {
- struct lov_mds_md *lmm =
- lustre_msg_buf(req->rq_repmsg, 2);
- if (lmm->lmm_easize == 0) {
- LDLM_DEBUG(lock, "open with no EA; returning PW"
- " lock");
- lock->l_req_mode = LCK_PW;
- }
+ if (it->opc & IT_OPEN &&
+ !(lustre_msg_get_op_flags(req->rq_reqmsg)&MDS_OPEN_HAS_EA)){
+ LDLM_DEBUG(lock, "open with no EA; returning PW lock");
+ lock->l_req_mode = LCK_PW;
}
if (flags & LDLM_FL_INTENT_ONLY) {
#include <linux/lustre_mds.h>
#include <linux/obd.h>
#include <linux/module.h>
-#include <linux/obd_lov.h>
static struct mds_fs_operations mds_ext3_fs_ops;
static kmem_cache_t *mcb_cache;
}
static int mds_ext3_set_md(struct inode *inode, void *handle,
- struct lov_mds_md *lmm)
+ struct lov_mds_md *lmm, int lmm_size)
{
int rc;
down(&inode->i_sem);
lock_kernel();
rc = ext3_xattr_set(handle, inode, EXT3_XATTR_INDEX_LUSTRE,
- XATTR_LUSTRE_MDS_OBJID, lmm,
- lmm ? lmm->lmm_easize : 0, 0);
+ XATTR_LUSTRE_MDS_OBJID, lmm, lmm_size, 0);
unlock_kernel();
up(&inode->i_sem);
return rc;
}
-static int mds_ext3_get_md(struct inode *inode, struct lov_mds_md *lmm)
+static int mds_ext3_get_md(struct inode *inode, struct lov_mds_md *lmm,int size)
{
int rc;
- int size = lmm->lmm_easize;
down(&inode->i_sem);
lock_kernel();
#include <linux/lustre_mds.h>
#include <linux/obd.h>
#include <linux/module.h>
-#include <linux/obd_lov.h>
static struct mds_fs_operations mds_extN_fs_ops;
static kmem_cache_t *mcb_cache;
#define EXTN_XATTR_INDEX_LUSTRE 5
#define XATTR_LUSTRE_MDS_OBJID "system.lustre_mds_objid"
-#define XATTR_MDS_MO_MAGIC 0xEA0BD047
-
/*
* We don't currently need any additional blocks for rmdir and
* unlink transactions because we are storing the OST oa_id inside
}
static int mds_extN_set_md(struct inode *inode, void *handle,
- struct lov_mds_md *lmm)
+ struct lov_mds_md *lmm, int lmm_size)
{
int rc;
down(&inode->i_sem);
lock_kernel();
rc = extN_xattr_set(handle, inode, EXTN_XATTR_INDEX_LUSTRE,
- XATTR_LUSTRE_MDS_OBJID, lmm,
- lmm ? lmm->lmm_easize : 0, 0);
+ XATTR_LUSTRE_MDS_OBJID, lmm, lmm_size, 0);
unlock_kernel();
up(&inode->i_sem);
return rc;
}
-static int mds_extN_get_md(struct inode *inode, struct lov_mds_md *lmm)
+static int mds_extN_get_md(struct inode *inode, struct lov_mds_md *lmm,int size)
{
int rc;
- int size = lmm->lmm_easize;
down(&inode->i_sem);
lock_kernel();
EXIT;
return;
}
- if (mds_extN_set_md(inode, handle, NULL))
+ if (mds_extN_set_md(inode, handle, NULL, 0))
CERROR("error clearing objid on %ld\n", inode->i_ino);
if (mds_extN_fs_ops.cl_delete_inode)
#include <linux/obd_lov.h>
#include <linux/lustre_lib.h>
+/* lov_unpackdesc() is in lov/lov_pack.c */
+
+void lov_packdesc(struct lov_desc *ld)
+{
+ ld->ld_tgt_count = HTON__u32(ld->ld_tgt_count);
+ ld->ld_default_stripe_count = HTON__u32(ld->ld_default_stripe_count);
+ ld->ld_default_stripe_size = HTON__u32(ld->ld_default_stripe_size);
+ ld->ld_pattern = HTON__u32(ld->ld_pattern);
+}
+
int mds_set_lovdesc(struct obd_device *obd, struct lov_desc *desc,
obd_uuid_t *uuidarray)
{
body = lustre_msg_buf(req->rq_repmsg, offset);
mds_pack_inode2fid(&body->fid1, inode);
mds_pack_inode2body(body, inode);
- if (S_ISREG(inode->i_mode)) {
- struct lov_mds_md *lmm;
-
- lmm = lustre_msg_buf(req->rq_repmsg, offset + 1);
- lmm->lmm_easize = mds->mds_max_mdsize;
-
- if (mds_fs_get_md(mds, inode, lmm) < 0) {
- CDEBUG(D_INFO,"No md for %ld: rc %d\n",
- inode->i_ino, rc);
- memset(lmm, 0, lmm->lmm_easize);
- } else
- body->valid |= OBD_MD_FLEASIZE;
- }
+ if (S_ISREG(inode->i_mode))
+ rc = mds_pack_md(mds, req, offset + 1, body, inode);
/* This isn't an error for RECREATE. */
if (rec->ur_opcode & REINT_REPLAYING) {
break;
case S_IFREG:
/* get OBD EA data first so client can also destroy object */
- if ((inode->i_mode & S_IFMT) == S_IFREG && offset) {
- struct lov_mds_md *lmm;
-
- lmm = lustre_msg_buf(req->rq_repmsg, offset + 1);
- lmm->lmm_easize = mds->mds_max_mdsize;
- if ((rc = mds_fs_get_md(mds, inode, lmm)) < 0) {
- CDEBUG(D_INFO, "No md for ino %ld: rc = %d\n",
- inode->i_ino, rc);
- memset(lmm, 0, lmm->lmm_easize);
- } else
- body->valid |= OBD_MD_FLEASIZE;
- }
+ if ((inode->i_mode & S_IFMT) == S_IFREG && offset)
+ rc = mds_pack_md(mds, req, offset + 1, body, inode);
/* no break */
case S_IFLNK:
case S_IFCHR:
mds_pack_inode2fid(&body->fid1, inode);
mds_pack_inode2body(body, inode);
- if (S_ISREG(inode->i_mode)) {
- struct lov_mds_md *lmm;
-
- lmm = lustre_msg_buf(req->rq_repmsg, 2);
- lmm->lmm_easize = mds->mds_max_mdsize;
- if ((rc = mds_fs_get_md(mds, inode, lmm)) < 0) {
- CDEBUG(D_INFO,"No md for %ld: rc %d\n",
- inode->i_ino, rc);
- memset(lmm, 0, lmm->lmm_easize);
- } else
- body->valid |= OBD_MD_FLEASIZE;
- }
+ if (S_ISREG(inode->i_mode))
+ rc = mds_pack_md(mds, req, 2, body, inode);
}
if (rec->ur_opcode & REINT_REPLAYING) {
/* XXX verify that the link is to the the right file? */
} else {
mds_pack_inode2fid(&body->fid1, inode);
mds_pack_inode2body(body, inode);
- if (S_ISREG(inode->i_mode)) {
- struct lov_mds_md *lmm;
-
- lmm = lustre_msg_buf(req->rq_repmsg, 2);
- lmm->lmm_easize = mds->mds_max_mdsize;
- if ((rc = mds_fs_get_md(mds, inode, lmm)) < 0) {
- CDEBUG(D_INFO,"No md for %ld: rc %d\n",
- inode->i_ino, rc);
- memset(lmm, 0, lmm->lmm_easize);
- } else
- body->valid |= OBD_MD_FLEASIZE;
- }
+ if (S_ISREG(inode->i_mode))
+ rc = mds_pack_md(mds, req, 2, body, inode);
}
}
RETURN(0);
}
-
static inline void obd_data2conn(struct lustre_handle *conn,
struct obd_ioctl_data *data)
{
conn->cookie = data->ioc_cookie;
}
-
static inline void obd_conn2data(struct obd_ioctl_data *data,
struct lustre_handle *conn)
{
unsigned int cmd, unsigned long arg)
{
char *buf = NULL;
- int len = 0;
struct obd_ioctl_data *data;
struct obd_device *obd = filp->private_data;
-
struct lustre_handle conn;
- int rw = OBD_BRW_READ;
- int err = 0;
- int serialised = 0;
-
+ int err = 0, len = 0;
ENTRY;
- switch (cmd)
- {
- case OBD_IOC_BRW_WRITE:
- case OBD_IOC_BRW_READ:
- case OBD_IOC_GETATTR:
- break;
- default:
- down(&obd_conf_sem);
- serialised = 1;
- break;
- }
+ down(&obd_conf_sem);
if (!obd && cmd != OBD_IOC_DEVICE && cmd != TCGETS &&
cmd != OBD_IOC_LIST &&
cmd != OBD_IOC_NAME2DEV && cmd != OBD_IOC_NEWDEV) {
CERROR("OBD ioctl: No device\n");
- GOTO(out, err=-EINVAL);
+ GOTO(out, err = -EINVAL);
}
if (obd_ioctl_getdata(&buf, &len, (void *)arg)) {
CERROR("OBD ioctl: data error\n");
- GOTO(out, err=-EINVAL);
+ GOTO(out, err = -EINVAL);
}
data = (struct obd_ioctl_data *)buf;
if(data->ioc_inlbuf2)
OBD_FREE(obd->obd_name, strlen(obd->obd_name)+1);
obd->obd_type = NULL;
-
- } else {
+ } else {
obd->obd_flags |= OBD_ATTACHED;
type->typ_refcnt++;
GOTO(out, err=0);
}
- case OBD_IOC_CREATE: {
- struct lov_stripe_md *lsm = NULL;
- obd_data2conn(&conn, data);
-
-#warning FIXME: save lsm into file handle for other ops, release on close
- err = obd_create(&conn, &data->ioc_obdo1, &lsm);
- if (!err) {
- err = copy_to_user((void *)arg, data, sizeof(*data));
- if (err)
- err = -EFAULT;
- }
- GOTO(out, err);
- }
-
- case OBD_IOC_GETATTR: {
- obd_data2conn(&conn, data);
- err = obd_getattr(&conn, &data->ioc_obdo1, NULL);
- if (!err) {
- err = copy_to_user((void *)arg, data, sizeof(*data));
- if (err)
- err = -EFAULT;
- }
- GOTO(out, err);
- }
-
- case OBD_IOC_SETATTR: {
- obd_data2conn(&conn, data);
- err = obd_setattr(&conn, &data->ioc_obdo1, NULL);
- if (!err) {
- err = copy_to_user((void *)arg, data, sizeof(*data));
- if (err)
- err = -EFAULT;
- }
- GOTO(out, err);
- }
-
- case OBD_IOC_DESTROY: {
- //void *ea;
- obd_data2conn(&conn, data);
-
- err = obd_destroy(&conn, &data->ioc_obdo1, NULL);
- if (!err) {
- err = copy_to_user((void *)arg, data, sizeof(*data));
- if (err)
- err = -EFAULT;
- }
- GOTO(out, err);
- }
-
- case OBD_IOC_OPEN: {
- struct lov_stripe_md *lsm = NULL; // XXX fill in from create
-
- obd_data2conn(&conn, data);
- err = obd_open(&conn, &data->ioc_obdo1, lsm);
- if (!err) {
- err = copy_to_user((void *)arg, data, sizeof(*data));
- if (err)
- err = -EFAULT;
- }
- GOTO(out, err);
- }
-
- case OBD_IOC_CLOSE: {
- struct lov_stripe_md *lsm = NULL; // XXX fill in from create
-
- obd_data2conn(&conn, data);
- err = obd_close(&conn, &data->ioc_obdo1, lsm);
- GOTO(out, err);
- }
-
- case OBD_IOC_BRW_WRITE:
- rw = OBD_BRW_WRITE;
- case OBD_IOC_BRW_READ: {
- struct lov_stripe_md tmp_lsm; // XXX fill in from create
- struct lov_stripe_md *lsm = &tmp_lsm; // XXX fill in from create
- struct brw_cb_data *brw_cbd = ll_init_brw_cb_data();
- obd_count pages = 0;
- struct brw_page *pga, *pgp;
- __u64 id = data->ioc_obdo1.o_id;
- int gfp_mask = (id & 1) ? GFP_HIGHUSER : GFP_KERNEL;
- int verify = (id != 0);
- __u64 off;
- int j;
-
- if (!brw_cbd)
- GOTO(out, err = -ENOMEM);
-
- obd_data2conn(&conn, data);
-
- pages = data->ioc_count / PAGE_SIZE;
- off = data->ioc_offset;
-
- CDEBUG(D_INODE, "BRW %s with %d pages @ "LPX64"\n",
- rw == OBD_BRW_READ ? "read" : "write", pages, off);
- OBD_ALLOC(pga, pages * sizeof(*pga));
- if (!pga) {
- CERROR("no memory for %d BRW per-page data\n", pages);
- GOTO(brw_free, err = -ENOMEM);
- }
-
- memset(lsm, 0, sizeof(*lsm)); // XXX don't do this later
- lsm->lsm_object_id = id; // ensure id == lsm->lsm_object_id
-
- for (j = 0, pgp = pga; j < pages; j++, off += PAGE_SIZE, pgp++){
- pgp->pg = alloc_pages(gfp_mask, 0);
- if (!pgp->pg) {
- CERROR("no memory for brw pages\n");
- GOTO(brw_cleanup, err = -ENOMEM);
- }
- pgp->count = PAGE_SIZE;
- pgp->off = off;
- pgp->flag = 0;
-
- if (verify) {
- void *addr = kmap(pgp->pg);
-
- if (rw == OBD_BRW_WRITE)
- page_debug_setup(addr, pgp->count,
- pgp->off, id);
- else
- page_debug_setup(addr, pgp->count,
- 0xdeadbeef00c0ffee,
- 0xdeadbeef00c0ffee);
- kunmap(pgp->pg);
- }
- }
-
- err = obd_brw(rw, &conn, lsm, j, pga, ll_sync_brw_cb, brw_cbd);
- if (err)
- CERROR("test_brw: error from obd_brw: err = %d\n", err);
- EXIT;
- brw_cleanup:
- for (j = 0, pgp = pga; j < pages; j++, pgp++) {
- if (pgp->pg != NULL) {
- if (verify && !err) {
- void *addr = kmap(pgp->pg);
-
- err = page_debug_check("test_brw",
- addr,
- PAGE_SIZE,
- pgp->off,id);
- kunmap(pgp->pg);
- }
- __free_pages(pgp->pg, 0);
- }
- }
- brw_free:
- OBD_FREE(pga, pages * sizeof(*pga));
- GOTO(out, err);
- }
default:
obd_data2conn(&conn, data);
out:
if (buf)
OBD_FREE(buf, len);
- if (serialised)
- up(&obd_conf_sem);
+ up(&obd_conf_sem);
RETURN(err);
} /* obd_class_ioctl */
/* declare character device */
static struct file_operations obd_psdev_fops = {
- ioctl: obd_class_ioctl, /* ioctl */
+ ioctl: obd_class_ioctl, /* ioctl */
open: obd_class_open, /* open */
- release: obd_class_release, /* release */
+ release: obd_class_release, /* release */
};
/* modules setup */
void (*class_signal_connection_failure)(struct ptlrpc_connection *);
#ifdef CONFIG_HIGHMEM
-#warning "using kmap accounting for deadlock avoidance"
/* Allow at most 3/4 of the kmap mappings to be consumed by vector I/O
* requests. This avoids deadlocks on servers which have a lot of clients
* doing vector I/O. We don't need to do this for non-vector I/O requests
EXPORT_SYMBOL(class_conn2ldlmimp);
EXPORT_SYMBOL(class_disconnect);
EXPORT_SYMBOL(class_disconnect_all);
-//EXPORT_SYMBOL(class_uuid_parse);
EXPORT_SYMBOL(class_uuid_unparse);
-//EXPORT_SYMBOL(class_multi_setup);
-//EXPORT_SYMBOL(class_multi_cleanup);
EXPORT_SYMBOL(class_signal_connection_failure);
-EXPORT_SYMBOL(ll_sync_brw_cb);
-EXPORT_SYMBOL(ll_init_brw_cb_data);
EXPORT_SYMBOL(class_nm_to_type);
static int __init init_obdclass(void)
/* -*- mode: c; c-basic-offset: 8; indent-tabs-mode: nil; -*-
* vim:expandtab:shiftwidth=8:tabstop=8:
*
- * lustre/obdclass/genops.c
- * Copyright (C) 2001-2002 Cluster File Systems, Inc.
+ * Copyright (c) 2001, 2002 Cluster File Systems, Inc.
*
- * This code is issued under the GNU General Public License.
- * See the file COPYING in this distribution
+ * This file is part of Lustre, http://www.lustre.org.
+ *
+ * Lustre is free software; you can redistribute it and/or
+ * modify it under the terms of version 2 of the GNU General Public
+ * License as published by the Free Software Foundation.
+ *
+ * Lustre is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Lustre; if not, write to the Free Software
+ * Foundation, Inc., 675 Mass Ave, Cambridge, MA 02139, USA.
*
* These are the only exported functions, they provide some generic
* infrastructure for managing object devices
- *
*/
#define DEBUG_SUBSYSTEM S_CLASS
int (*ptlrpc_put_connection_superhack)(struct ptlrpc_connection *c);
-/* I would prefer if these next four functions were in ptlrpc, to be honest,
- * but obdclass uses them for the netregression ioctls. -phil */
-static int ll_sync_brw_timeout(void *data)
-{
- struct brw_cb_data *brw_cbd = data;
- struct ptlrpc_bulk_desc *desc;
- ENTRY;
-
- LASSERT(brw_cbd);
- desc = brw_cbd->brw_desc;
-
- if (!desc) {
- CERROR("no desc for timed-out BRW, reopen Bugzilla 214!\n");
- RETURN(0); /* back to sleep -- someone had better wake us up! */
- }
-
- LASSERT(desc->bd_connection);
-
- CERROR("IO of %d pages to/from %s:%d (conn %p) timed out\n",
- desc->bd_page_count, desc->bd_connection->c_remote_uuid,
- desc->bd_portal, desc->bd_connection);
- desc->bd_connection->c_level = LUSTRE_CONN_RECOVD;
- desc->bd_flags |= PTL_RPC_FL_TIMEOUT;
- if (desc->bd_connection && class_signal_connection_failure) {
- class_signal_connection_failure(desc->bd_connection);
-
- /* We go back to sleep, until we're resumed or interrupted. */
- RETURN(0);
- }
-
- /* If we can't be recovered, just abort the syscall with -ETIMEDOUT. */
- RETURN(1);
-}
-
-static int ll_sync_brw_intr(void *data)
-{
- struct brw_cb_data *brw_cbd = data;
- struct ptlrpc_bulk_desc *desc = brw_cbd->brw_desc;
-
- ENTRY;
- desc->bd_flags |= PTL_RPC_FL_INTR;
- RETURN(1); /* ignored, as of this writing */
-}
-
-int ll_sync_brw_cb(struct brw_cb_data *brw_cbd, int err, int phase)
-{
- int ret;
- ENTRY;
-
- if (phase == CB_PHASE_START) {
- struct l_wait_info lwi;
- lwi = LWI_TIMEOUT_INTR(obd_timeout * HZ, ll_sync_brw_timeout,
- ll_sync_brw_intr, brw_cbd);
- ret = l_wait_event(brw_cbd->brw_waitq, brw_cbd->brw_complete, &lwi);
- if (atomic_dec_and_test(&brw_cbd->brw_refcount))
- OBD_FREE(brw_cbd, sizeof(*brw_cbd));
- if (ret == -EINTR)
- RETURN(ret);
- } else if (phase == CB_PHASE_FINISH) {
- brw_cbd->brw_err = err;
- brw_cbd->brw_complete = 1;
- wake_up(&brw_cbd->brw_waitq);
- if (atomic_dec_and_test(&brw_cbd->brw_refcount))
- OBD_FREE(brw_cbd, sizeof(*brw_cbd));
- RETURN(err);
- } else
- LBUG();
- EXIT;
- return 0;
-}
-
-struct brw_cb_data *ll_init_brw_cb_data(void)
-{
- struct brw_cb_data *brw_cbd;
-
- OBD_ALLOC(brw_cbd, sizeof(*brw_cbd));
- if (brw_cbd) {
- init_waitqueue_head(&brw_cbd->brw_waitq);
- atomic_set(&brw_cbd->brw_refcount, 2);
- }
- RETURN(brw_cbd);
-}
-
/*
* support functions: we could use inter-module communication, but this
* is more portable to other OS's
return type;
}
-int class_register_type(struct obd_ops *ops, struct lprocfs_vars* vars, char *nm)
+int class_register_type(struct obd_ops *ops, struct lprocfs_vars *vars,
+ char *nm)
{
struct obd_type *type;
int rc;
conn.addr = (__u64)(unsigned long)export;
conn.cookie = export->exp_cookie;
spin_unlock(&obddev->obd_dev_lock);
- CERROR("force disconnecting export %p\n", export);
+ CERROR("force disconnecting %s:%s export %p\n",
+ export->exp_obd->obd_type->typ_name,
+ export->exp_uuid, export);
rc = obd_disconnect(&conn);
if (rc < 0) {
/* AED: not so sure about this... We can't
EXTRA_PROGRAMS = obdecho
LINX=
-obdecho_SOURCES = echo.c lproc_echo.c $(LINX)
+obdecho_SOURCES = echo.c echo_client.c lproc_echo.c $(LINX)
include $(top_srcdir)/Rules
/* -*- mode: c; c-basic-offset: 8; indent-tabs-mode: nil; -*-
* vim:expandtab:shiftwidth=8:tabstop=8:
*
- * linux/fs/obdecho/echo.c
+ * Copyright (c) 2001, 2002 Cluster File Systems, Inc.
+ * Author: Peter Braam <braam@clusterfs.com>
+ * Author: Andreas Dilger <adilger@clusterfs.com>
*
- * Copyright (C) 2001, 2002 Cluster File Systems, Inc.
+ * This file is part of Lustre, http://www.lustre.org.
*
- * This code is issued under the GNU General Public License.
- * See the file COPYING in this distribution
+ * Lustre is free software; you can redistribute it and/or
+ * modify it under the terms of version 2 of the GNU General Public
+ * License as published by the Free Software Foundation.
*
- * by Peter Braam <braam@clusterfs.com>
- * and Andreas Dilger <adilger@clusterfs.com>
+ * Lustre is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Lustre; if not, write to the Free Software
+ * Foundation, Inc., 675 Mass Ave, Cambridge, MA 02139, USA.
*/
#define OBDECHO_VERSION "1.0"
return lprocfs_dereg_obd(dev);
}
-
-struct obd_ops echo_obd_ops = {
+static struct obd_ops echo_obd_ops = {
o_attach: echo_attach,
o_detach: echo_detach,
o_connect: echo_connect,
o_cleanup: echo_cleanup
};
+extern int echo_client_init(void);
+extern void echo_client_cleanup(void);
+
static int __init obdecho_init(void)
{
int rc;
-
printk(KERN_INFO "Echo OBD driver " OBDECHO_VERSION
" info@clusterfs.com\n");
echo_proc_init();
- rc = class_register_type(&echo_obd_ops, status_class_var,
+ rc = class_register_type(&echo_obd_ops, status_class_var,
OBD_ECHO_DEVICENAME);
- RETURN(rc);
-
+ if (rc)
+ RETURN(rc);
+
+ rc = echo_client_init();
+ if (rc)
+ class_unregister_type(OBD_ECHO_DEVICENAME);
+ RETURN(rc);
}
static void __exit obdecho_exit(void)
{
-
echo_proc_fini();
+ echo_client_cleanup();
class_unregister_type(OBD_ECHO_DEVICENAME);
}
--- /dev/null
+/* -*- mode: c; c-basic-offset: 8; indent-tabs-mode: nil; -*-
+ * vim:expandtab:shiftwidth=8:tabstop=8:
+ *
+ * Copyright (c) 2001, 2002 Cluster File Systems, Inc.
+ *
+ * This file is part of Lustre, http://www.lustre.org.
+ *
+ * Lustre is free software; you can redistribute it and/or
+ * modify it under the terms of version 2 of the GNU General Public
+ * License as published by the Free Software Foundation.
+ *
+ * Lustre is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Lustre; if not, write to the Free Software
+ * Foundation, Inc., 675 Mass Ave, Cambridge, MA 02139, USA.
+ */
+
+#include <linux/version.h>
+#include <linux/module.h>
+#include <linux/fs.h>
+
+#define DEBUG_SUBSYSTEM S_ECHO
+
+#include <linux/obd_support.h>
+#include <linux/obd_class.h>
+#include <linux/obd_echo.h>
+#include <linux/lustre_debug.h>
+#include <linux/lprocfs_status.h>
+
+static int echo_iocontrol(long cmd, struct lustre_handle *obdconn, int len,
+ void *karg, void *uarg)
+{
+ struct obd_device *obd = class_conn2obd(obdconn);
+ struct echo_client_obd *ec = &obd->u.echo_client;
+ struct obd_ioctl_data *data = karg;
+ int rw = OBD_BRW_READ, rc = 0;
+ ENTRY;
+
+ if (obd == NULL) {
+ CERROR("ioctl: No device\n");
+ GOTO(out, rc = -EINVAL);
+ }
+
+ switch (cmd) {
+ case OBD_IOC_CREATE: {
+ struct lov_stripe_md *lsm = NULL;
+#warning FIXME: save lsm into file handle for other ops, release on close
+ rc = obd_create(&ec->conn, &data->ioc_obdo1, &lsm);
+ GOTO(out, rc);
+ }
+
+ case OBD_IOC_GETATTR:
+ rc = obd_getattr(&ec->conn, &data->ioc_obdo1, NULL);
+ GOTO(out, rc);
+
+ case OBD_IOC_SETATTR:
+ rc = obd_setattr(&ec->conn, &data->ioc_obdo1, NULL);
+ GOTO(out, rc);
+
+ case OBD_IOC_DESTROY: {
+ //void *ea;
+ rc = obd_destroy(&ec->conn, &data->ioc_obdo1, NULL);
+ GOTO(out, rc);
+ }
+
+ case OBD_IOC_OPEN: {
+ struct lov_stripe_md *lsm = NULL; // XXX fill in from create
+
+ rc = obd_open(&ec->conn, &data->ioc_obdo1, lsm);
+ GOTO(out, rc);
+ }
+
+ case OBD_IOC_CLOSE: {
+ struct lov_stripe_md *lsm = NULL; // XXX fill in from create
+
+ rc = obd_close(&ec->conn, &data->ioc_obdo1, lsm);
+ GOTO(out, rc);
+ }
+
+ case OBD_IOC_BRW_WRITE:
+ rw = OBD_BRW_WRITE;
+ case OBD_IOC_BRW_READ: {
+ struct lov_stripe_md tmp_lsm; // XXX fill in from create
+ struct lov_stripe_md *lsm = &tmp_lsm; // XXX fill in from create
+ struct obd_brw_set *set;
+ obd_count pages = 0;
+ struct brw_page *pga, *pgp;
+ __u64 id = data->ioc_obdo1.o_id;
+ int gfp_mask = (id & 1) ? GFP_HIGHUSER : GFP_KERNEL;
+ int verify = (id != 0);
+ __u64 off;
+ int j;
+
+ set = obd_brw_set_new();
+ if (set == NULL)
+ GOTO(out, rc = -ENOMEM);
+
+ pages = data->ioc_count / PAGE_SIZE;
+ off = data->ioc_offset;
+
+ CDEBUG(D_INODE, "BRW %s with %d pages @ "LPX64"\n",
+ rw == OBD_BRW_READ ? "read" : "write", pages, off);
+ OBD_ALLOC(pga, pages * sizeof(*pga));
+ if (!pga) {
+ CERROR("no memory for %d BRW per-page data\n", pages);
+ GOTO(brw_free, rc = -ENOMEM);
+ }
+
+ memset(lsm, 0, sizeof(*lsm)); // XXX don't do this later
+ lsm->lsm_object_id = id; // ensure id == lsm->lsm_object_id
+
+ for (j = 0, pgp = pga; j < pages; j++, off += PAGE_SIZE, pgp++){
+ pgp->pg = alloc_pages(gfp_mask, 0);
+ if (!pgp->pg) {
+ CERROR("no memory for brw pages\n");
+ GOTO(brw_cleanup, rc = -ENOMEM);
+ }
+ pgp->count = PAGE_SIZE;
+ pgp->off = off;
+ pgp->flag = 0;
+
+ if (verify) {
+ void *addr = kmap(pgp->pg);
+
+ if (rw == OBD_BRW_WRITE)
+ page_debug_setup(addr, pgp->count,
+ pgp->off, id);
+ else
+ page_debug_setup(addr, pgp->count,
+ 0xdeadbeef00c0ffee,
+ 0xdeadbeef00c0ffee);
+ kunmap(pgp->pg);
+ }
+ }
+
+ set->brw_callback = ll_brw_sync_wait;
+ rc = obd_brw(rw, &ec->conn, lsm, j, pga, set);
+ if (rc)
+ CERROR("test_brw: error from obd_brw: rc = %d\n", rc);
+ else {
+ rc = ll_brw_sync_wait(set, CB_PHASE_START);
+ if (rc)
+ CERROR("test_brw: error from callback: rc = "
+ "%d\n", rc);
+ }
+ EXIT;
+ brw_cleanup:
+ for (j = 0, pgp = pga; j < pages; j++, pgp++) {
+ if (pgp->pg == NULL)
+ continue;
+
+ if (verify && !rc) {
+ void *addr = kmap(pgp->pg);
+
+ rc = page_debug_check("test_brw", addr,
+ PAGE_SIZE, pgp->off, id);
+ kunmap(pgp->pg);
+ }
+ __free_pages(pgp->pg, 0);
+ }
+ brw_free:
+ obd_brw_set_free(set);
+ OBD_FREE(pga, pages * sizeof(*pga));
+ GOTO(out, rc);
+ }
+ default:
+ return -ENOTTY;
+ }
+
+ out:
+ RETURN(rc);
+}
+
+static int echo_setup(struct obd_device *obddev, obd_count len, void *buf)
+{
+ struct obd_ioctl_data* data = buf;
+ struct echo_client_obd *ec = &obddev->u.echo_client;
+ struct obd_device *tgt;
+ int rc;
+ ENTRY;
+
+ if (data->ioc_inllen1 < 1) {
+ CERROR("requires a TARGET OBD UUID\n");
+ RETURN(-EINVAL);
+ }
+ if (data->ioc_inllen1 > 37) {
+ CERROR("OBD UUID must be less than 38 characters\n");
+ RETURN(-EINVAL);
+ }
+
+ MOD_INC_USE_COUNT;
+ tgt = class_uuid2obd(data->ioc_inlbuf1);
+ if (!tgt || !(tgt->obd_flags & OBD_ATTACHED) ||
+ !(tgt->obd_flags & OBD_SET_UP)) {
+ CERROR("device not attached or not set up (%d)\n",
+ data->ioc_dev);
+ GOTO(error_dec, rc = -EINVAL);
+ }
+
+ rc = obd_connect(&ec->conn, tgt, NULL, NULL, NULL);
+ if (rc) {
+ CERROR("fail to connect to device %d\n", data->ioc_dev);
+ GOTO(error_dec, rc = -EINVAL);
+ }
+ RETURN(rc);
+error_dec:
+ MOD_DEC_USE_COUNT;
+ RETURN(rc);
+}
+
+static int echo_cleanup(struct obd_device * obddev)
+{
+ struct echo_client_obd *ec = &obddev->u.echo_client;
+ int rc;
+ ENTRY;
+
+ if (!list_empty(&obddev->obd_exports)) {
+ CERROR("still has clients!\n");
+ RETURN(-EBUSY);
+ }
+
+ rc = obd_disconnect(&ec->conn);
+ if (rc) {
+ CERROR("fail to disconnect device: %d\n", rc);
+ RETURN(-EINVAL);
+ }
+
+ MOD_DEC_USE_COUNT;
+ RETURN(0);
+}
+
+static int echo_connect(struct lustre_handle *conn, struct obd_device *src,
+ obd_uuid_t cluuid, struct recovd_obd *recovd,
+ ptlrpc_recovery_cb_t recover)
+{
+ return class_connect(conn, src, cluuid);
+}
+
+static struct obd_ops echo_obd_ops = {
+ o_setup: echo_setup,
+ o_cleanup: echo_cleanup,
+ o_iocontrol: echo_iocontrol,
+ o_connect: echo_connect,
+ o_disconnect: class_disconnect
+};
+
+int echo_client_init(void)
+{
+ extern struct lprocfs_vars status_class_var[];
+
+ return class_register_type(&echo_obd_ops, status_class_var,
+ OBD_ECHO_CLIENT_DEVICENAME);
+}
+
+void echo_client_cleanup(void)
+{
+ class_unregister_type(OBD_ECHO_CLIENT_DEVICENAME);
+}
*
* linux/fs/obdfilter/filter.c
*
- * Copyright (C) 2001, 2002 Cluster File Systems, Inc.
+ * Copyright (c) 2001, 2002 Cluster File Systems, Inc.
+ * Author: Peter Braam <braam@clusterfs.com>
+ * Author: Andreas Dilger <adilger@clusterfs.com>
*
- * This code is issued under the GNU General Public License.
- * See the file COPYING in this distribution
+ * This file is part of Lustre, http://www.lustre.org.
*
- * by Peter Braam <braam@clusterfs.com>
- * and Andreas Dilger <adilger@clusterfs.com>
+ * Lustre is free software; you can redistribute it and/or
+ * modify it under the terms of version 2 of the GNU General Public
+ * License as published by the Free Software Foundation.
+ *
+ * Lustre is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Lustre; if not, write to the Free Software
+ * Foundation, Inc., 675 Mass Ave, Cambridge, MA 02139, USA.
*/
#define EXPORT_SYMTAB
oa->o_id, oa->o_mode);
}
+ if (IS_ERR(dentry)) {
+ CERROR("%s error looking up object: "LPX64"\n", what, oa->o_id);
+ RETURN(dentry);
+ }
+
if (!dentry->d_inode) {
- CERROR("%s on non-existent object: "LPU64"\n", what, oa->o_id);
+ CERROR("%s on non-existent object: "LPX64"\n", what, oa->o_id);
f_dput(dentry);
- dentry = ERR_PTR(-ENOENT);
+ RETURN(ERR_PTR(-ENOENT));
}
return dentry;
static int filter_pgcache_brw(int cmd, struct lustre_handle *conn,
struct lov_stripe_md *lsm, obd_count oa_bufs,
- struct brw_page *pga, brw_cb_t callback,
- struct brw_cb_data *brw_cbd)
+ struct brw_page *pga, struct obd_brw_set *set)
{
struct obd_export *export = class_conn2export(conn);
struct obd_run_ctxt saved;
*/
while (index < ((src->o_size + PAGE_SIZE - 1) >> PAGE_SHIFT)) {
struct brw_page pg;
- struct brw_cb_data *brw_cbd = ll_init_brw_cb_data();
+ struct obd_brw_set *set;
- if (!brw_cbd) {
+ set = obd_brw_set_new();
+ if (set == NULL) {
err = -ENOMEM;
EXIT;
break;
pg.flag = 0;
page->index = index;
- err = obd_brw(OBD_BRW_READ, src_conn, &srcmd, 1, &pg,
- ll_sync_brw_cb, brw_cbd);
-
- if ( err ) {
+ set->brw_callback = ll_brw_sync_wait;
+ err = obd_brw(OBD_BRW_READ, src_conn, &srcmd, 1, &pg, set);
+ obd_brw_set_free(set);
+ if (err) {
EXIT;
break;
}
- brw_cbd = ll_init_brw_cb_data();
- if (!brw_cbd) {
+ set = obd_brw_set_new();
+ if (set == NULL) {
err = -ENOMEM;
EXIT;
break;
pg.flag = OBD_BRW_CREATE;
CDEBUG(D_INFO, "Read page %ld ...\n", page->index);
- err = obd_brw(OBD_BRW_WRITE, dst_conn, &dstmd, 1, &pg,
- ll_sync_brw_cb, brw_cbd);
+ set->brw_callback = ll_brw_sync_wait;
+ err = obd_brw(OBD_BRW_WRITE, dst_conn, &dstmd, 1, &pg, set);
+ obd_brw_set_free(set);
/* XXX should handle dst->o_size, dst->o_blocks here */
- if ( err ) {
+ if (err) {
EXIT;
break;
}
/* -*- mode: c; c-basic-offset: 8; indent-tabs-mode: nil; -*-
* vim:expandtab:shiftwidth=8:tabstop=8:
*
- * Copyright (C) 2001, 2002 Cluster File Systems, Inc.
+ * Copyright (C) 2001, 2002 Cluster File Systems, Inc.
+ * Author Peter Braam <braam@clusterfs.com>
*
- * This code is issued under the GNU General Public License.
- * See the file COPYING in this distribution
+ * This file is part of Lustre, http://www.lustre.org.
*
- * Author Peter Braam <braam@clusterfs.com>
+ * Lustre is free software; you can redistribute it and/or
+ * modify it under the terms of version 2 of the GNU General Public
+ * License as published by the Free Software Foundation.
*
- * This server is single threaded at present (but can easily be multi
- * threaded). For testing and management it is treated as an
- * obd_device, although it does not export a full OBD method table
- * (the requests are coming in over the wire, so object target
- * modules do not have a full method table.)
+ * Lustre is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with Lustre; if not, write to the Free Software
+ * Foundation, Inc., 675 Mass Ave, Cambridge, MA 02139, USA.
+ *
+ * For testing and management it is treated as an obd_device,
+ * although * it does not export a full OBD method table (the
+ * requests are coming * in over the wire, so object target modules
+ * do not have a full * method table.)
*
*/
#include <linux/kp30.h>
#include <linux/lustre_mds.h> /* for mds_objid */
#include <linux/obd_ost.h>
-#include <linux/obd_lov.h>
#include <linux/ctype.h>
#include <linux/init.h>
#include <linux/lustre_ha.h>
extern struct lprocfs_vars status_var_nm_1[];
extern struct lprocfs_vars status_class_var[];
+int osc_attach(struct obd_device *dev, obd_count len, void *data)
+{
+ return lprocfs_reg_obd(dev, status_var_nm_1, dev);
+}
+
+int osc_detach(struct obd_device *dev)
+{
+ return lprocfs_dereg_obd(dev);
+}
+
+/* Pack OSC object metadata for shipment to the MDS. */
+static int osc_packmd(struct lustre_handle *conn, struct lov_mds_md **lmmp,
+ struct lov_stripe_md *lsm)
+{
+ int lmm_size;
+
+ lmm_size = sizeof(**lmmp);
+ if (!lmmp)
+ RETURN(lmm_size);
+
+ if (*lmmp && !lsm) {
+ OBD_FREE(*lmmp, lmm_size);
+ *lmmp = NULL;
+ RETURN(0);
+ }
+
+ if (!*lmmp) {
+ OBD_ALLOC(*lmmp, lmm_size);
+ if (!*lmmp)
+ RETURN(-ENOMEM);
+ }
+ if (lsm)
+ (*lmmp)->lmm_object_id = (lsm->lsm_object_id);
+
+ return lmm_size;
+}
+
+static int osc_unpackmd(struct lustre_handle *conn, struct lov_stripe_md **lsmp,
+ struct lov_mds_md *lmm)
+{
+ int lsm_size;
+
+ lsm_size = sizeof(**lsmp);
+ if (!lsmp)
+ RETURN(lsm_size);
+
+ if (*lsmp && !lmm) {
+ OBD_FREE(*lsmp, lsm_size);
+ *lsmp = NULL;
+ RETURN(0);
+ }
+
+ if (!*lsmp) {
+ OBD_ALLOC(*lsmp, lsm_size);
+ if (!*lsmp)
+ RETURN(-ENOMEM);
+ }
+
+ /* XXX endianness */
+ if (lmm)
+ (*lsmp)->lsm_object_id = (lmm->lmm_object_id);
+
+ return lsm_size;
+}
+
static int osc_getattr(struct lustre_handle *conn, struct obdo *oa,
struct lov_stripe_md *md)
{
lsm = *ea;
if (!lsm) {
- // XXX check oa->o_valid & OBD_MD_FLEASIZE first...
- OBD_ALLOC(lsm, oa->o_easize);
- if (!lsm)
- RETURN(-ENOMEM);
- lsm->lsm_mds_easize = oa->o_easize;
+ rc = obd_alloc_memmd(conn, &lsm);
+ if (rc < 0)
+ RETURN(rc);
}
request = ptlrpc_prep_req(class_conn2cliimp(conn), OST_CREATE, 1, &size,
ptlrpc_req_finished(request);
out:
if (rc && !*ea)
- OBD_FREE(lsm, oa->o_easize);
+ obd_free_memmd(conn, &lsm);
return rc;
}
return rc;
}
-struct osc_brw_cb_data {
- brw_cb_t callback;
- void *cb_data;
- void *obd_data;
- size_t obd_size;
-};
-
/* Our bulk-unmapping bottom half. */
static void unmap_and_decref_bulk_desc(void *data)
{
/* this is the callback function which is invoked by the Portals
* event handler associated with the bulk_sink queue and bulk_source queue.
*/
-
-static void osc_ptl_ev_hdlr(struct ptlrpc_bulk_desc *desc, void *data)
+static void osc_ptl_ev_hdlr(struct ptlrpc_bulk_desc *desc)
{
- struct osc_brw_cb_data *cb_data = data;
- int err = 0;
ENTRY;
- if (desc->bd_flags & PTL_RPC_FL_TIMEOUT) {
- err = (desc->bd_flags & PTL_RPC_FL_INTR ? -ERESTARTSYS :
- -ETIMEDOUT);
- }
+ LASSERT(desc->bd_brw_set != NULL);
+ LASSERT(desc->bd_brw_set->brw_callback != NULL);
- if (cb_data->callback)
- cb_data->callback(cb_data->cb_data, err, CB_PHASE_FINISH);
-
- if (cb_data->obd_data)
- OBD_FREE(cb_data->obd_data, cb_data->obd_size);
- OBD_FREE(cb_data, sizeof(*cb_data));
+ desc->bd_brw_set->brw_callback(desc->bd_brw_set, CB_PHASE_FINISH);
/* We can't kunmap the desc from interrupt context, so we do it from
* the bottom half above. */
static int osc_brw_read(struct lustre_handle *conn, struct lov_stripe_md *lsm,
obd_count page_count, struct brw_page *pga,
- brw_cb_t callback, struct brw_cb_data *data)
+ struct obd_brw_set *set)
{
struct ptlrpc_connection *connection =
client_conn2cli(conn)->cl_import.imp_connection;
struct ptlrpc_request *request = NULL;
struct ptlrpc_bulk_desc *desc = NULL;
struct ost_body *body;
- struct osc_brw_cb_data *cb_data = NULL;
- int rc, size[3] = {sizeof(*body)};
+ int rc, size[3] = {sizeof(*body)}, mapped = 0;
void *iooptr, *nioptr;
- int mapped = 0;
__u32 xid;
ENTRY;
GOTO(out_req, rc = -ENOMEM);
desc->bd_portal = OST_BULK_PORTAL;
desc->bd_ptl_ev_hdlr = osc_ptl_ev_hdlr;
- OBD_ALLOC(cb_data, sizeof(*cb_data));
- if (!cb_data)
- GOTO(out_desc, rc = -ENOMEM);
-
- cb_data->callback = callback;
- cb_data->cb_data = data;
- CDEBUG(D_PAGE, "data(%p)->desc = %p\n", data, desc);
- data->brw_desc = desc;
- desc->bd_ptl_ev_data = cb_data;
+ CDEBUG(D_PAGE, "desc = %p\n", desc);
iooptr = lustre_msg_buf(request->rq_reqmsg, 1);
nioptr = lustre_msg_buf(request->rq_reqmsg, 2);
* Register the bulk first, because the reply could arrive out of order,
* and we want to be ready for the bulk data.
*
- * The reference is released when brw_finish is complete.
+ * One reference is released when brw_finish is complete, the other when
+ * the caller removes us from the "set" list.
*
* On error, we never do the brw_finish, so we handle all decrefs.
*/
rc = ptlrpc_register_bulk(desc);
if (rc)
GOTO(out_unmap, rc);
+ obd_brw_set_add(set, desc);
}
request->rq_replen = lustre_msg_size(1, size);
* callback, before any such cleanup-requiring error condition can
* be detected.
*/
- if (rc)
- GOTO(out_req, rc);
-
- /* Callbacks cause asynchronous handling. */
- rc = callback(data, 0, CB_PHASE_START);
-
-out_req:
+ out_req:
ptlrpc_req_finished(request);
RETURN(rc);
while (mapped-- > 0)
kunmap(pga[mapped].pg);
obd_kmap_put(page_count);
- OBD_FREE(cb_data, sizeof(*cb_data));
-out_desc:
ptlrpc_bulk_decref(desc);
goto out_req;
}
static int osc_brw_write(struct lustre_handle *conn, struct lov_stripe_md *md,
obd_count page_count, struct brw_page *pga,
- brw_cb_t callback, struct brw_cb_data *data)
+ struct obd_brw_set *set)
{
struct ptlrpc_connection *connection =
client_conn2cli(conn)->cl_import.imp_connection;
struct ost_body *body;
struct niobuf_local *local = NULL;
struct niobuf_remote *remote;
- struct osc_brw_cb_data *cb_data = NULL;
- int rc, j, size[3] = {sizeof(*body)};
+ int rc, j, size[3] = {sizeof(*body)}, mapped = 0;
void *iooptr, *nioptr;
- int mapped = 0;
ENTRY;
size[1] = sizeof(struct obd_ioobj);
GOTO(out_req, rc = -ENOMEM);
desc->bd_portal = OSC_BULK_PORTAL;
desc->bd_ptl_ev_hdlr = osc_ptl_ev_hdlr;
- OBD_ALLOC(cb_data, sizeof(*cb_data));
- if (!cb_data)
- GOTO(out_desc, rc = -ENOMEM);
-
- cb_data->callback = callback;
- cb_data->cb_data = data;
- CDEBUG(D_PAGE, "data(%p)->desc = %p\n", data, desc);
- data->brw_desc = desc;
- desc->bd_ptl_ev_data = cb_data;
+ CDEBUG(D_PAGE, "desc = %p\n", desc);
iooptr = lustre_msg_buf(request->rq_reqmsg, 1);
nioptr = lustre_msg_buf(request->rq_reqmsg, 2);
OBD_ALLOC(local, page_count * sizeof(*local));
if (!local)
- GOTO(out_cb, rc = -ENOMEM);
-
- cb_data->obd_data = local;
- cb_data->obd_size = page_count * sizeof(*local);
+ GOTO(out_desc, rc = -ENOMEM);
obd_kmap_get(page_count, 0);
if (OBD_FAIL_CHECK(OBD_FAIL_OSC_BRW_WRITE_BULK))
GOTO(out_unmap, rc = 0);
- /* Our reference is released when brw_finish is complete. */
+ OBD_FREE(local, page_count * sizeof(*local));
+
+ /* One reference is released when brw_finish is complete, the other
+ * when the caller removes it from the "set" list. */
+ obd_brw_set_add(set, desc);
rc = ptlrpc_send_bulk(desc);
/* XXX: Mike, same question as in osc_brw_read. */
- if (rc)
- GOTO(out_req, rc);
-
- /* Callbacks cause asynchronous handling. */
- rc = callback(data, 0, CB_PHASE_START);
-
out_req:
ptlrpc_req_finished(request);
RETURN(rc);
obd_kmap_put(page_count);
OBD_FREE(local, page_count * sizeof(*local));
-out_cb:
- OBD_FREE(cb_data, sizeof(*cb_data));
out_desc:
ptlrpc_bulk_decref(desc);
goto out_req;
static int osc_brw(int cmd, struct lustre_handle *conn,
struct lov_stripe_md *md, obd_count page_count,
- struct brw_page *pga, brw_cb_t callback,
- struct brw_cb_data *data)
+ struct brw_page *pga, struct obd_brw_set *set)
{
ENTRY;
pages_per_brw = page_count;
if (cmd & OBD_BRW_WRITE)
- rc = osc_brw_write(conn, md, pages_per_brw, pga,
- callback, data);
+ rc = osc_brw_write(conn, md, pages_per_brw, pga, set);
else
- rc = osc_brw_read(conn, md, pages_per_brw, pga,
- callback, data);
+ rc = osc_brw_read(conn, md, pages_per_brw, pga, set);
if (rc != 0)
RETURN(rc);
* fixup the lock to start and end on page boundaries. */
if (extent->end != OBD_OBJECT_EOF) {
extent->start &= PAGE_MASK;
- extent->end = (extent->end + PAGE_SIZE - 1) & PAGE_MASK;
+ extent->end = (extent->end & PAGE_MASK) + PAGE_SIZE - 1;
}
/* Next, search for already existing extent locks that will cover us */
memcpy(uuidp, obddev->obd_uuid, sizeof(*uuidp));
err = copy_to_user((void *)uarg, buf, len);
+ if (err)
+ err = -EFAULT;
OBD_FREE(buf, len);
GOTO(out, err);
}
return err;
}
-int osc_attach(struct obd_device *dev, obd_count len, void *data)
-{
- return lprocfs_reg_obd(dev, status_var_nm_1, dev);
-}
-
-int osc_detach(struct obd_device *dev)
-{
- return lprocfs_dereg_obd(dev);
-}
struct obd_ops osc_obd_ops = {
o_attach: osc_attach,
o_detach: osc_detach,
o_setup: client_obd_setup,
o_cleanup: client_obd_cleanup,
+ o_connect: client_obd_connect,
+ o_disconnect: client_obd_disconnect,
o_statfs: osc_statfs,
+ o_packmd: osc_packmd,
+ o_unpackmd: osc_unpackmd,
o_create: osc_create,
o_destroy: osc_destroy,
o_getattr: osc_getattr,
o_setattr: osc_setattr,
o_open: osc_open,
o_close: osc_close,
- o_connect: client_obd_connect,
- o_disconnect: client_obd_disconnect,
o_brw: osc_brw,
o_punch: osc_punch,
o_enqueue: osc_enqueue,
static int __init osc_init(void)
{
- int rc;
- rc = class_register_type(&osc_obd_ops, status_class_var,
- LUSTRE_OSC_NAME);
- RETURN(rc);
-
+ RETURN(class_register_type(&osc_obd_ops, status_class_var,
+ LUSTRE_OSC_NAME));
}
static void __exit osc_exit(void)
atomic_set(&desc->bd_refcount, 1);
init_waitqueue_head(&desc->bd_waitq);
INIT_LIST_HEAD(&desc->bd_page_list);
+ INIT_LIST_HEAD(&desc->bd_set_chain);
ptl_set_inv_handle(&desc->bd_md_h);
ptl_set_inv_handle(&desc->bd_me_h);
}
return desc;
}
+int ptlrpc_bulk_error(struct ptlrpc_bulk_desc *desc)
+{
+ int rc = 0;
+ if (desc->bd_flags & PTL_RPC_FL_TIMEOUT) {
+ rc = (desc->bd_flags & PTL_RPC_FL_INTR ? -ERESTARTSYS :
+ -ETIMEDOUT);
+ }
+ return rc;
+}
+
struct ptlrpc_bulk_page *ptlrpc_prep_bulk_page(struct ptlrpc_bulk_desc *desc)
{
struct ptlrpc_bulk_page *bulk;
return;
}
+ LASSERT(list_empty(&desc->bd_set_chain));
+
list_for_each_safe(tmp, next, &desc->bd_page_list) {
struct ptlrpc_bulk_page *bulk;
bulk = list_entry(tmp, struct ptlrpc_bulk_page, bp_link);
EXIT;
}
+static int ll_sync_brw_timeout(void *data)
+{
+ struct obd_brw_set *set = data;
+ struct list_head *tmp;
+ int failed = 0;
+ ENTRY;
+
+ LASSERT(set);
+
+ set->brw_flags |= PTL_RPC_FL_TIMEOUT;
+
+ list_for_each(tmp, &set->brw_desc_head) {
+ struct ptlrpc_bulk_desc *desc =
+ list_entry(tmp, struct ptlrpc_bulk_desc, bd_set_chain);
+
+ /* Skip descriptors that were completed successfully. */
+ if (desc->bd_flags & (PTL_BULK_FL_RCVD | PTL_BULK_FL_SENT))
+ continue;
+
+ LASSERT(desc->bd_connection);
+
+ /* If PtlMDUnlink succeeds, then it hasn't completed yet. If it
+ * fails, the bulk finished _just_ in time (after the timeout
+ * fired but before we got this far) and we'll let it live.
+ */
+ if (PtlMDUnlink(desc->bd_md_h) != 0) {
+ CERROR("Near-miss on OST %s -- need to adjust "
+ "obd_timeout?\n",
+ desc->bd_connection->c_remote_uuid);
+ continue;
+ }
+
+ CERROR("IO of %d pages to/from %s:%d (conn %p) timed out\n",
+ desc->bd_page_count, desc->bd_connection->c_remote_uuid,
+ desc->bd_portal, desc->bd_connection);
+ desc->bd_connection->c_level = LUSTRE_CONN_RECOVD;
+
+ /* This one will "never" arrive, don't wait for it. */
+ if (atomic_dec_and_test(&set->brw_refcount))
+ wake_up(&set->brw_waitq);
+
+ if (class_signal_connection_failure)
+ class_signal_connection_failure(desc->bd_connection);
+ else
+ failed = 1;
+ }
+
+ /* 0 = We go back to sleep, until we're resumed or interrupted */
+ /* 1 = We can't be recovered, just abort the syscall with -ETIMEDOUT */
+ RETURN(failed);
+}
+
+static int ll_sync_brw_intr(void *data)
+{
+ struct obd_brw_set *set = data;
+
+ ENTRY;
+ set->brw_flags |= PTL_RPC_FL_INTR;
+ RETURN(1); /* ignored, as of this writing */
+}
+
+int ll_brw_sync_wait(struct obd_brw_set *set, int phase)
+{
+ struct l_wait_info lwi;
+ struct list_head *tmp, *next;
+ int rc = 0;
+ ENTRY;
+
+ switch(phase) {
+ case CB_PHASE_START:
+ lwi = LWI_TIMEOUT_INTR(obd_timeout * HZ, ll_sync_brw_timeout,
+ ll_sync_brw_intr, set);
+ rc = l_wait_event(set->brw_waitq,
+ atomic_read(&set->brw_refcount) == 0, &lwi);
+
+ list_for_each_safe(tmp, next, &set->brw_desc_head) {
+ struct ptlrpc_bulk_desc *desc =
+ list_entry(tmp, struct ptlrpc_bulk_desc,
+ bd_set_chain);
+ list_del_init(&desc->bd_set_chain);
+ ptlrpc_bulk_decref(desc);
+ }
+ break;
+ case CB_PHASE_FINISH:
+ if (atomic_dec_and_test(&set->brw_refcount))
+ wake_up(&set->brw_waitq);
+ break;
+ default:
+ LBUG();
+ }
+
+ RETURN(rc);
+}
+
struct ptlrpc_request *ptlrpc_prep_req(struct obd_import *imp, int opcode,
int count, int *lengths, char **bufs)
{
/* Requests that aren't from replayable imports, or which don't have
* transno information, can be "committed" early.
+ *
+ * But don't commit anything that's kept indefinitely for replay (has
+ * the PTL_RPC_FL_REPLAY flag set), such as open requests.
*/
-
if ((req->rq_import->imp_flags & IMP_REPLAYABLE) == 0 ||
- req->rq_repmsg->transno == 0) {
+ (req->rq_repmsg->transno == 0 &&
+ (req->rq_flags & PTL_RPC_FL_REPLAY) == 0)) {
/* This import doesn't support replay, so we can just "commit"
* this request now.
*/
desc->bd_flags |= PTL_BULK_FL_SENT;
wake_up(&desc->bd_waitq);
if (desc->bd_ptl_ev_hdlr != NULL)
- desc->bd_ptl_ev_hdlr(desc, desc->bd_ptl_ev_data);
+ desc->bd_ptl_ev_hdlr(desc);
}
RETURN(0);
desc->bd_flags |= PTL_BULK_FL_RCVD;
wake_up(&desc->bd_waitq);
if (desc->bd_ptl_ev_hdlr != NULL)
- desc->bd_ptl_ev_hdlr(desc, desc->bd_ptl_ev_data);
+ desc->bd_ptl_ev_hdlr(desc);
} else {
CERROR("Unexpected event type!\n");
LBUG();
return 0;
}
+void obd_brw_set_add(struct obd_brw_set *set, struct ptlrpc_bulk_desc *desc)
+{
+ atomic_inc(&desc->bd_refcount);
+ atomic_inc(&set->brw_refcount);
+ desc->bd_brw_set = set;
+ list_add(&desc->bd_set_chain, &set->brw_desc_head);
+}
+
+struct obd_brw_set *obd_brw_set_new(void)
+{
+ struct obd_brw_set *set;
+
+ OBD_ALLOC(set, sizeof(*set));
+
+ if (set != NULL) {
+ init_waitqueue_head(&set->brw_waitq);
+ INIT_LIST_HEAD(&set->brw_desc_head);
+ atomic_set(&set->brw_refcount, 0);
+ }
+
+ return set;
+}
+
+void obd_brw_set_free(struct obd_brw_set *set)
+{
+ struct list_head *tmp, *next;
+ ENTRY;
+
+ if (!list_empty(&set->brw_desc_head)) {
+ EXIT;
+ return;
+ }
+
+ list_for_each_safe(tmp, next, &set->brw_desc_head) {
+ struct ptlrpc_bulk_desc *desc =
+ list_entry(tmp, struct ptlrpc_bulk_desc, bd_set_chain);
+
+ CERROR("Unfinished bulk descriptor: %p\n", desc);
+
+ ptlrpc_abort_bulk(desc);
+ }
+ OBD_FREE(set, sizeof(*set));
+ EXIT;
+ return;
+}
+
int ptlrpc_reply(struct ptlrpc_service *svc, struct ptlrpc_request *req)
{
if (req->rq_repmsg == NULL) {
{
return class_connect(conn, src, cluuid);
}
+
int connmgr_attach(struct obd_device *dev, obd_count len, void *data)
{
return lprocfs_reg_obd(dev, status_var_nm_1, dev);
EXPORT_SYMBOL(ptlrpc_resend_req);
EXPORT_SYMBOL(ptl_send_rpc);
EXPORT_SYMBOL(ptlrpc_link_svc_me);
+EXPORT_SYMBOL(obd_brw_set_free);
+EXPORT_SYMBOL(obd_brw_set_new);
+EXPORT_SYMBOL(obd_brw_set_add);
/* client.c */
EXPORT_SYMBOL(ptlrpc_init_client);
EXPORT_SYMBOL(ptlrpc_prep_bulk_page);
EXPORT_SYMBOL(ptlrpc_free_bulk_page);
EXPORT_SYMBOL(ptlrpc_check_status);
+EXPORT_SYMBOL(ll_brw_sync_wait);
/* service.c */
EXPORT_SYMBOL(ptlrpc_init_svc);
openclose
createdestroy
lovstripe
+lov.xml
+uml.xml
echo.xml
local.xml
+stat
pkgexampledir = '${exec_prefix}/usr/lib/$(PACKAGE)/examples'
pkgexample_SCRIPTS = llmount.sh llmountcleanup.sh llecho.sh local.sh uml.sh lov.sh
noinst_SCRIPTS = llsetup.sh llrsetup.sh llcleanup.sh
-noinst_DATA = ext2_10000.gz ext2_25000.gz ext3_10000.gz lov.xml lustre.cfg
+noinst_DATA = lustre.cfg
noinst_SCRIPTS += fs.sh intent-test.sh intent-test2.sh leak_finder.pl \
lldlm.sh llecho.sh llext3.sh llmodules.sh llmount-client.sh \
llmount-server.sh llmount.sh llmountcleanup.sh llrext3.sh \
runtests runvmstat snaprun.sh tbox.sh common.sh
noinst_PROGRAMS = openunlink testreq truncate directio openme writeme mcreate
noinst_PROGRAMS += munlink tchmod toexcl fsx test_brw openclose createdestroy
-noinst_PROGRAMS += lovstripe stat createmany # ldaptest
+noinst_PROGRAMS += lovstripe stat createmany mkdirmany # ldaptest
# ldaptest_SOURCES = ldaptest.c
tchmod_SOURCES = tchmod.c
lovstripe_SOURCES = lovstripe.c
stat_SOURCES = stat.c
createmany_SOURCES = createmany.c
+mkdirmany_SOURCES = mkdirmany.c
include $(top_srcdir)/Rules
# force the osc to be configured (this is normally done when it is mounted)
$LMC -m $config --node $CLIENT --osc OSC_$SERVER || exit 3
+$LMC -m $config --node $CLIENT --echo_client OSC_${SERVER} || exit 3
$LCONF --gdb $config || exit 4
cat <<EOF
run getattr tests as:
-../utils/lctl --device '\$OSC_$SERVER' test_getattr 1000000
+../utils/lctl --device \$ECHO_$SERVER test_getattr 1000000
EOF
TMP=${TMP:-/tmp}
MDSDEV=$TMP/mds1
-MDSSIZE=50000
+MDSSIZE=100000
OSTDEV=$TMP/ost1
-OSTSIZE=50000
+OSTSIZE=250000
kver=`uname -r | cut -d "." -f 1,2`
OSTDEV1=$TMP/ost1
OSTDEV2=$TMP/ost2
+OSTDEV3=$TMP/ost3
OSTSIZE=100000
STRIPE_BYTES=65536
-STRIPES_PER_OBJ=0 # 0 means stripe over all OSTs
+STRIPES_PER_OBJ=2 # 0 means stripe over all OSTs
# create nodes
${LMC} -o $config --node localhost --net localhost tcp || exit 1
${LMC} -m $config --lov lov1 mds1 $STRIPE_BYTES $STRIPES_PER_OBJ 0 || exit 20
${LMC} -m $config --node localhost --lov lov1 --ost $OSTDEV1 $OSTSIZE || exit 21
${LMC} -m $config --node localhost --lov lov1 --ost $OSTDEV2 $OSTSIZE || exit 22
+${LMC} -m $config --node localhost --lov lov1 --ost $OSTDEV3 $OSTSIZE || exit 23
# create client config
${LMC} -m $config --node localhost --mtpt /mnt/lustre mds1 lov1 || exit 30
/****************** Custom includes ********************/
#include <linux/lustre_lite.h>
+#include <linux/lustre_idl.h>
/****************** Functions ******************/
-int write_file(char *name, struct lov_user_md *striping, int bufsize,
+int write_file(char *name, struct lov_mds_md *striping, int bufsize,
char *buf1, char *buf2);
int main(int argc, char *argv[])
{
- struct lov_user_md a_striping;
+ struct lov_mds_md a_striping;
long bufsize = sizeof(long) * STRIPE_SIZE;
char *rbuf, *wbuf;
int data, *dp;
*dp = data;
/* Init defaults on striping info */
- a_striping.lum_stripe_size = STRIPE_SIZE;
- a_striping.lum_stripe_pattern = 0;
+ a_striping.lmm_magic = LOV_MAGIC;
+ a_striping.lmm_stripe_size = STRIPE_SIZE;
+ a_striping.lmm_stripe_pattern = 0;
/* Write file for OST1 only */
/* Start at OST 0, and use only 1 OST */
- a_striping.lum_stripe_offset = 0;
- a_striping.lum_stripe_count = 1;
+ a_striping.lmm_stripe_offset = 0;
+ a_striping.lmm_stripe_count = 1;
result = write_file("/mnt/lustre/ost1", &a_striping, bufsize,
wbuf, rbuf);
/* Write file for OST2 only */
/* Start at OST 1, and use only 1 OST */
- a_striping.lum_stripe_offset = 1;
- a_striping.lum_stripe_count = 1;
+ a_striping.lmm_stripe_offset = 1;
+ a_striping.lmm_stripe_count = 1;
result = write_file("/mnt/lustre/ost2", &a_striping, bufsize,
wbuf, rbuf);
/* Write file across both OST1 and OST2 */
/* Start at OST 0, and use only 2 OSTs */
- a_striping.lum_stripe_offset = 0;
- a_striping.lum_stripe_count = 2;
+ a_striping.lmm_stripe_offset = 0;
+ a_striping.lmm_stripe_count = 2;
result = write_file("/mnt/lustre/ost1and2", &a_striping, bufsize,
wbuf, rbuf);
}
-int write_file(char *name, struct lov_user_md *striping, int bufsize,
+int write_file(char *name, struct lov_mds_md *striping, int bufsize,
char *wbuf, char *rbuf)
{
int fd, result;
--- /dev/null
+#include <stdio.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <time.h>
+#include <errno.h>
+#include <string.h>
+#include <fcntl.h>
+#include <unistd.h>
+#include <stdlib.h>
+
+int main(int argc, char ** argv)
+{
+ int i, rc, count;
+ char dirname[4096];
+
+ if (argc < 3) {
+ printf("Usage %s dirnamebase count\n", argv[0]);
+ return 1;
+ }
+
+ if (strlen(argv[1]) > 4080) {
+ printf("name too long\n");
+ return 1;
+ }
+
+ count = strtoul(argv[2], NULL, 0);
+
+ for (i=0 ; i < count ; i++) {
+ sprintf(dirname, "%s-%d", argv[1], i);
+ rc = mkdir(dirname, S_IFREG| 0444);
+ if (rc) {
+ printf("mkdir(%s) error: %s\n",
+ dirname, strerror(errno));
+ break;
+ }
+ if ((i % 10000) == 0)
+ printf(" - created %d (time %ld)\n", i, time(0));
+ }
+ return rc;
+}
export PATH=/sbin:/usr/sbin:$SRCDIR:$PATH
cleanup() {
- $LCONF --cleanup --dump /tmp/debug $OPTS
- trap 0
+ sync
+ $LCONF --cleanup --dump /tmp/debug $OPTS
+ sync
+ #trap 0
}
ERROR=
OSCMT="`mount | awk '/ lustre_lite / { print $3 }' | tail -1`"
if [ -z "$OSCMT" ]; then
[ -z "$*" ] && fail "usage: $0 [--reformat] <conf>.xml" 1
- $LCONF $@ || exit 1
- trap cleanup 0
+ $LCONF --gdb $@ || exit 1
+ #trap cleanup 0
OSCMT="`mount | awk '/ lustre_lite / { print $3 }' | tail -1`"
[ -z "$OSCMT" ] && fail "no lustre filesystem mounted" 1
+ I_MOUNTED="yes"
fi
while [ "$1" ]; do
[ "$ERROR" ] && fail "old and new files are different" $ERROR
cleanup || exit 19
-sync
-
$LCONF $OPTS || exit 20
echo "comparing previously copied files"
[ "$ERROR" ] && fail "old and new files are different on second diff" $ERROR
cleanup || exit 29
-sync
$LCONF $OPTS || exit 30
echo "renaming $HOSTS.ren to $HOSTS"
echo "This is normal on BA OSTs, because of subdirectories." 1>&2
fi
-cleanup $OPTS || exit 29
-sync
+[ "$I_MOUNTED" = "yes" ] && cleanup $OPTS || exit 29
#!/bin/bash
-#CLEAN=umount /mnt/lustre
-#START=../utils/lconf --minlevel 70 local.xml
-CLEAN="sh llmountcleanup.sh"
-START="sh llmount.sh"
-
-
-
-echo '==== touch /mnt/lustre/f ; rm /mnt/lustre/* ==== test 19'
+export NAME=$NAME
+clean() {
+ echo -n "cleanup..."
+ sh llmountcleanup.sh > /dev/null
+ dmesg | grep leaked | grep -v " 0 bytes"
+ dmesg | grep -i destruct
+}
+CLEAN=clean
+start() {
+ echo -n "mounting..."
+ sh llrmount.sh > /dev/null
+ echo -n "mounted"
+ echo 0 > /proc/sys/portals/debug
+}
+START=start
+
+echo '== touch .../f ; rm .../f ========== test 0'
touch /mnt/lustre/f
-rm /mnt/lustre/*
+rm /mnt/lustre/f
$CLEAN
-dmesg | grep -i destruct
$START
-
-echo '=============================== test 1'
+echo '== mkdir .../d1; mkdir .../d1/d2 == test 1'
mkdir /mnt/lustre/d1
mkdir /mnt/lustre/d1/d2
$CLEAN
-dmesg | grep -i destruct
$START
-
-echo '=============================== test 2'
+echo '== mkdir .../d1; touch .../d1/f === test 2'
mkdir /mnt/lustre/d2
touch /mnt/lustre/d2/f
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 3
+echo '== mkdir .../d3 =================== test 3'
mkdir /mnt/lustre/d3
$CLEAN
$START
+echo '== touch .../d3/f ================= test 3b'
touch /mnt/lustre/d3/f
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 4
+echo '== mkdir .../d4 =================== test 4'
mkdir /mnt/lustre/d4
$CLEAN
$START
+echo '== mkdir .../d4/d2 ================ test 4b'
mkdir /mnt/lustre/d4/d2
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 5
+echo '== mkdir .../d5; mkdir .../d5/d2; chmod .../d5/d2 == test 5'
mkdir /mnt/lustre/d5
mkdir /mnt/lustre/d5/d2
chmod 0666 /mnt/lustre/d5/d2
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 6
+echo '== touch .../f6; chmod .../f6 ===== test 6'
touch /mnt/lustre/f6
chmod 0666 /mnt/lustre/f6
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 7
+echo '== mkdir .../d7; mcreate .../d7/f; chmod .../d7/f == test 7'
mkdir /mnt/lustre/d7
./mcreate /mnt/lustre/d7/f
chmod 0666 /mnt/lustre/d7/f
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 8
+echo '== mkdir .../d8; touch .../d8/f; chmod .../d8/f == test 8'
mkdir /mnt/lustre/d8
touch /mnt/lustre/d8/f
chmod 0666 /mnt/lustre/d8/f
$CLEAN
-dmesg | grep -i destruct
$START
-echo '=============9=================' test 9
+echo '== mkdir .../d9; mkdir .../d9/d2; mkdir .../d9/d2/d3 == test 9'
mkdir /mnt/lustre/d9
mkdir /mnt/lustre/d9/d2
mkdir /mnt/lustre/d9/d2/d3
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 10
+echo '=============================== test 10'
mkdir /mnt/lustre/d10
mkdir /mnt/lustre/d10/d2
touch /mnt/lustre/d10/d2/f
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 11
+echo '=============================== test 11'
mkdir /mnt/lustre/d11
mkdir /mnt/lustre/d11/d2
chmod 0666 /mnt/lustre/d11/d2
chmod 0555 /mnt/lustre/d11/d2
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 12
+echo '=============================== test 12'
mkdir /mnt/lustre/d12
touch /mnt/lustre/d12/f
chmod 0666 /mnt/lustre/d12/f
chmod 0555 /mnt/lustre/d12/f
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 13
+echo '=============================== test 13'
mkdir /mnt/lustre/d13
cp /etc/passwd /mnt/lustre/d13/f
> /mnt/lustre/d13/f
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 14
+echo '=============================== test 14'
mkdir /mnt/lustre/d14
touch /mnt/lustre/d14/f
rm /mnt/lustre/d14/f
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 15
+echo '=============================== test 15'
mkdir /mnt/lustre/d15
touch /mnt/lustre/d15/f
mv /mnt/lustre/d15/f /mnt/lustre/d15/f2
$CLEAN
-dmesg | grep -i destruct
$START
-echo '===============================' test 16
+echo '=============================== test 16'
mkdir /mnt/lustre/d16
touch /mnt/lustre/d16/f
rm -rf /mnt/lustre/d16/f
$CLEAN
-dmesg | grep -i destruct
$START
echo '====== symlinks: create, remove symlinks (dangling and real) =====' test 17
rm -f /mnt/lustre/l-dangle
rm -f /mnt/lustre/l-exist
$CLEAN
-dmesg | grep -i destruct
$START
echo '==== touch /mnt/lustre/f ; ls /mnt/lustre ==== test 18'
touch /mnt/lustre/f
ls /mnt/lustre
$CLEAN
-dmesg | grep -i destruct
$START
echo '==== touch /mnt/lustre/f ; ls -l /mnt/lustre ==== test 19'
touch /mnt/lustre/f
ls -l /mnt/lustre
+rm /mnt/lustre/f
$CLEAN
-dmesg | grep -i destruct
$START
-echo '==== touch /mnt/lustre/f ; ls -l /mnt/lustre ==== test 19'
+echo '==== touch /mnt/lustre/f ; ls -l /mnt/lustre ==== test 20'
touch /mnt/lustre/f
rm /mnt/lustre/f
echo "1 done"
rm /mnt/lustre/f
echo "3 done"
$CLEAN
-dmesg | grep -i destruct
$START
+echo '=========== finished ==========='
exit
TAGS
obdctl
lctl
+lfind
+lstripe
CPPFLAGS = $(HAVE_LIBREADLINE)
obdctl_LDADD := $(LIBREADLINE)
lctl_LDADD := $(LIBREADLINE) -lptlctl
-sbin_PROGRAMS = lctl lfind obdctl
+sbin_PROGRAMS = lctl lfind lstripe obdctl
sbin_SCRIPTS = lconf lmc
obdctl_SOURCES = parser.c obdctl.c obd.c parser.h obdctl.h
lctl_SOURCES = parser.c obd.c lctl.c parser.h
lfind_SOURCES = lfind.c
+lstripe_SOURCES = lstripe.c
lfind_CPPFLAGS = -D_XOPEN_SOURCE=500
EXTRA_DIST = $(sbin_SCRIPTS)
Module.cleanup(self)
+class ECHO_CLIENT(Module):
+ def __init__(self,dom_node):
+ Module.__init__(self, 'ECHO_CLIENT', dom_node)
+ self.obd_uuid = get_first_ref(dom_node, 'osc')
+ debug("HERE",self.obd_uuid)
+ self.add_module('lustre/obdecho', 'obdecho')
+
+ def prepare(self):
+ if is_prepared(self.uuid):
+ return
+ self.info(self.obd_uuid)
+
+ lctl.newdev(attach="echo_client %s %s" % (self.name, self.uuid),
+ setup = self.obd_uuid)
+
+ def cleanup(self):
+ if not is_prepared(self.uuid):
+ return
+ Module.cleanup(self)
+
+
class Mountpoint(Module):
def __init__(self,dom_node):
Module.__init__(self, 'MTPT', dom_node)
ret = 50
elif type in ('lov', 'lovconfig'):
ret = 60
- elif type in ('mountpoint',):
+ elif type in ('mountpoint', 'echo_client'):
ret = 70
if ret < config.minlevel() or ret > config.maxlevel():
n = MDS(dom_node)
elif type == 'osc':
n = VOSC(dom_node)
+ elif type == 'echo_client':
+ n = ECHO_CLIENT(dom_node)
elif type == 'mdc':
n = MDC(dom_node)
elif type == 'mountpoint':
"usage: modules <path>"},
{"panic", jt_dbg_panic, 0, "force the kernel to panic\n"
"usage: panic"},
-
+
/* User interface commands */
{"======= control ========", jt_noop, 0, "control commands"},
{"help", Parser_help, 0, "help"},
-int main(int argc, char **argv)
+int main(int argc, char **argv)
{
int rc;
#define printk printf
#include <linux/lustre_lib.h>
#include <linux/lustre_lite.h>
+#include <linux/obd_lov.h>
#warning Max obds per lov currently hardcoded to 1000 in lov/lov_obd.c
#define MAX_LOV_UUID_COUNT 1000
#define OBD_NOT_FOUND ((__u32)-1)
-#define debugMsg if (debug) printf
char * cmd;
-int debug;
struct option longOpts[] = {
- {"debug", 0, 0, 'd'},
{"help", 0, 0, 'h'},
{"obd", 1, 0, 'o'},
{"query", 0, 0, 'o'},
+ {"verbose", 0, 0, 'v'},
{0, 0, 0, 0}
};
int query;
-char * shortOpts = "dho:qv";
+int verbose;
+char * shortOpts = "ho:qv";
char * usageMsg = "[ --obd <obd uuid> | --query ] <dir|file> ...";
-int max_stripe_count = MAX_LOV_UUID_COUNT;
+int max_ost_count = MAX_LOV_UUID_COUNT;
obd_uuid_t * obduuid;
__u32 obdcount;
__u32 obdindex;
obd_uuid_t * uuids;
int uuidslen;
int cfglen;
-struct lov_user_md *lum;
-int lumlen;
+struct lov_mds_md *lmm;
+int lmmlen;
void init();
void usage(FILE *stream);
while ((c = getopt_long(argc, argv, shortOpts, longOpts, NULL)) != -1) {
switch (c) {
- case 'd':
- debug++;
- break;
case 'o':
if (obduuid) {
errMsg("obd '%s' already specified: '%s'.",
case 'q':
query++;
break;
+ case 'v':
+ verbose++;
+ break;
case '?':
usage(stderr);
exit(1);
datalen = size_round(sizeof(data));
desclen = size_round(sizeof(desc));
- uuidslen = size_round(max_stripe_count * sizeof(*uuids));
+ uuidslen = size_round(max_ost_count * sizeof(*uuids));
cfglen = datalen + desclen + uuidslen;
- lumlen = sizeof(*lum) + max_stripe_count * sizeof(*lum->lum_luoinfo);
- if (cfglen > lumlen)
+ lmmlen = lov_mds_md_size(max_ost_count);
+ if (cfglen > lmmlen)
buflen = cfglen;
else
- buflen = lumlen;
+ buflen = lmmlen;
#warning max ioctl buffer size currently hardcoded to 8192
if (buflen > 8192) {
remaining = nuuids * sizeof(*uuids);
if (uuidslen > remaining)
nuuids--;
- nluoinfos = (buflen - sizeof(*lum)) / sizeof(*lum->lum_luoinfo);
+ nluoinfos = (buflen - sizeof(*lmm)) / sizeof(*lmm->lmm_objects);
if (nuuids > nluoinfos)
- max_stripe_count = nluoinfos;
+ max_ost_count = nluoinfos;
else
- max_stripe_count = nuuids;
+ max_ost_count = nuuids;
cfglen = datalen + desclen + uuidslen;
- lumlen = sizeof(*lum) + max_stripe_count *
- sizeof(*lum->lum_luoinfo);
+ lmmlen = lov_mds_md_size(max_ost_count);
}
if ((buf = malloc(buflen)) == NULL) {
exit(1);
}
- lum = (struct lov_user_md *)buf;
+ lmm = (struct lov_mds_md *)buf;
uuids = (obd_uuid_t *)buf;
}
}
int
-processFile(const char *path,
- const struct stat *sp,
- int flag,
- struct FTW *ftwp
-) {
- struct lov_user_oinfo *luoinfo;
+processFile(const char *path, const struct stat *sp, int flag, struct FTW *ftwp)
+{
int fd;
int count;
int rc;
}
memset((void *)buf, 0, buflen);
- lum->lum_stripe_count = max_stripe_count;
+ lmm->lmm_magic = LOV_MAGIC;
+ lmm->lmm_ost_count = max_ost_count;
- if ((rc = ioctl(fd, LL_IOC_LOV_GETSTRIPE, (void *)lum)) < 0) {
+ if ((rc = ioctl(fd, LL_IOC_LOV_GETSTRIPE, (void *)lmm)) < 0) {
errMsg("LL_IOC_LOV_GETSTRIPE ioctl failed.");
perror("ioctl");
exit(1);
close(fd);
- count = lum->lum_stripe_count;
- luoinfo = lum->lum_luoinfo;
+ if (query || verbose)
+ printf("\n%s:\n", path);
- if (query) {
- printf("%s\n", path);
- for (i = 0; i < count; i++, luoinfo++) {
- printf("%4d: obdindex: %-4d objid: %lld\n",
- i, luoinfo->luo_idx, luoinfo->luo_id);
- }
- return(0);
+ if (verbose) {
+ printf("lmm_magic: 0x%x\n", lmm->lmm_magic);
+ printf("lmm_object_id: 0x%llx\n", lmm->lmm_object_id);
+ printf("lmm_stripe_offset: %d\n", lmm->lmm_stripe_offset);
+ printf("lmm_stripe_count: %d\n", lmm->lmm_stripe_count);
+ printf("lmm_ost_count: %d\n", lmm->lmm_ost_count);
+ printf("lmm_stripe_pattern: %d\n", lmm->lmm_stripe_pattern);
}
- debugMsg("LL_IOC_LOV_GETSTRIPE:%s: obdindex: %d count: %d\n",
- path, obdindex, count);
+ count = lmm->lmm_ost_count;
- for (i = 0; i < count; i++, luoinfo++) {
- debugMsg("%-4d: obdidx: %-4d objid: %lld\n",
- i, luoinfo->luo_idx, luoinfo->luo_id);
- if (luoinfo->luo_idx == obdindex) {
- printf("%s\n", path);
- return 0;
- }
+ if (query || verbose) {
+ struct lov_object_id *loi;
+ __u64 oid;
+
+ loi = lmm->lmm_objects;
+
+ printf("obdidx\tobjid\n");
+
+ for (i = 0; i < count; i++, loi++)
+ if ((oid = loi->l_object_id))
+ printf("%6d\t%5lld\n", i, oid);
+
+ if (query)
+ return(0);
}
+ if (lmm->lmm_objects[obdindex].l_object_id)
+ printf("%s\n", path);
+
return(0);
}
data.ioc_inllen3 = 0;
memset(&desc, 0, sizeof(desc));
- desc.ld_tgt_count = max_stripe_count;
+ desc.ld_tgt_count = max_ost_count;
if (obd_ioctl_pack(&data, &buf, buflen)) {
errMsg("internal buffering error.");
osc.appendChild(self.ref("obd", obd_uuid))
return osc
+ def echo_client(self, name, uuid, osc_uuid):
+ ec = self.newService("echo_client", name, uuid)
+ ec.appendChild(self.ref("osc", osc_uuid))
+ return ec
+
def ost(self, name, uuid, obd_uuid, net_uuid):
ost = self.newService("ost", name, uuid)
ost.appendChild(self.ref("network", net_uuid))
if not net_uuid:
error("NODE: ", node_name, "not found")
-
mds = gen.mds(mds_name, mds_uuid, fstype, devname, get_format_flag(options),
net_uuid, node_uuid, dev_size=size)
lustre.appendChild(mds)
node_add_profile(gen, node, 'osc', osc_uuid)
+#ditto
+def add_echo_client(gen, lustre, options, args):
+ """ add an echo client to the profile for this node. """
+ if len(args) < 1:
+ usage()
+ osc_name = args[0]
+ if options.has_key('node'):
+ node_name = options['node']
+ else:
+ error("--echo_client requires a --node argument")
+ node = findByName(lustre, node_name, "node")
+
+ echoname = new_name('ECHO_'+ node_name)
+ echo_uuid = new_uuid(echoname)
+ node_add_profile(gen, node, 'echo_client', echo_uuid)
+ osc_uuid = name2uuid(lustre, osc_name) # either 'osc' or 'lov'
+ echo = gen.echo_client(echoname, echo_uuid, osc_uuid)
+ lustre.appendChild(echo)
+
+
def add_lov(gen, lustre, options, args):
""" create a lov """
if len(args) < 4:
#
def parse_cmdline(argv):
short_opts = "ho:i:m:"
- long_opts = ["ost", "osc", "mtpt", "lov=", "node=", "mds=", "net", "tcpbuf=",
+ long_opts = ["ost", "osc", "mtpt", "lov=", "node=", "mds=", "net",
+ "echo_client", "tcpbuf=",
"route", "router", "merge=", "format", "reformat", "output=",
"obdtype=", "fstype=", "obduuid=", "in=", "help", "batch="]
opts = []
options['ost'] = 1
if o == "--osc":
options['osc'] = 1
+ if o == "--echo_client":
+ options['echo_client'] = 1
if o == "--mds":
options['mds'] = a
if o == "--net":
add_ost(gen, lustre, options, args)
elif options.has_key('osc'):
add_osc(gen, lustre, options, args)
+ elif options.has_key('echo_client'):
+ add_echo_client(gen, lustre, options, args)
elif options.has_key('mtpt'):
add_mtpt(gen, lustre, options, args)
elif options.has_key('mds'):
--- /dev/null
+#include <ctype.h>
+#include <errno.h>
+#include <fcntl.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/ioctl.h>
+#include <unistd.h>
+
+
+/****************** Custom includes ********************/
+#include <linux/lustre_lite.h>
+#include <linux/lustre_idl.h>
+
+
+/****************** Functions ******************/
+
+void usage(char *pgm)
+{
+ fprintf(stderr, "\nIncorrect parameters! Correct usage:\n\n" );
+ fprintf(stderr, "%s <output filename> <stripe size> <OST #> <stripe #>\n", pgm);
+
+ fprintf(stderr, "\n\nArgument explanations:\n---------------------\n\n");
+ fprintf(stderr, "<output filename> = the full name and path of the output file to create\n");
+ fprintf(stderr, "<stripe size> = the number of bytes to have in each stripe.\n");
+ fprintf(stderr, "<OST #> = the OST number to start the striping on.\n");
+ fprintf(stderr, "<stripe #> = the number of stripes to use.\n");
+
+ fprintf(stderr, "\n\nExamples:\n---------\n\n");
+
+ fprintf(stderr, "%s /mnt/lustre/ost1 131072 0 1\n", pgm);
+ fprintf(stderr, "\t\tcreates a file only on ost1.\n\n");
+
+ fprintf(stderr, "%s /mnt/lustre/ost2 131072 1 1\n", pgm);
+ fprintf(stderr, "\t\tcreates a file only on ost2.\n\n");
+
+ fprintf(stderr, "%s /mnt/lustre/ost1and2 131072 0 2\n", pgm);
+ fprintf(stderr, "\t\tcreates a 128k file with 2 stripes, on ost1 and ost2.\n");
+
+ fprintf(stderr, "%s /mnt/lustre/ost1and2 131072 1 2\n", pgm);
+ fprintf(stderr, "\t\tcreates a 128k file with 2 stripes, on ost2 and ost1.\n");
+}
+
+int create_file(char *name, long stripe_size, int stripe_offset,
+ int stripe_count)
+{
+ struct lov_mds_md a_striping;
+ int fd, result = 0;
+
+ /* Initialize IOCTL striping pattern structure */
+ a_striping.lmm_magic = LOV_MAGIC;
+ a_striping.lmm_stripe_pattern = 0;
+ a_striping.lmm_stripe_size = stripe_size;
+ a_striping.lmm_stripe_offset = stripe_offset;
+ a_striping.lmm_stripe_count = stripe_count;
+
+ fd = open(name, O_CREAT | O_RDWR | O_LOV_DELAY_CREATE, 0644);
+ if (fd < 0) {
+ fprintf(stderr, "\nUnable to open '%s': %s\n",
+ name, strerror(errno));
+ result = -errno;
+ } else if (ioctl(fd, LL_IOC_LOV_SETSTRIPE, &a_striping)) {
+ fprintf(stderr, "\nError on ioctl for '%s' (%d): %s\n",
+ name, fd, strerror(errno));
+ result = -errno;
+ } else if (close(fd) < 0) {
+ fprintf(stderr, "\nError on close for '%s' (%d): %s\n",
+ name, fd, strerror(errno));
+ result = -errno;
+ }
+
+ return result;
+}
+
+int main(int argc, char *argv[])
+{
+ int result;
+ long st_size;
+ int st_offset,
+ st_count;
+
+ /* Check to make sure we have enough parameters */
+ if (argc != 5) {
+ usage(argv[0]);
+ return(-1);
+ }
+
+ /* Get the stripe size */
+ st_size = atol(argv[2]);
+
+ /* Get the stripe offset*/
+ st_offset = atoi(argv[3]);
+
+ /* Get the stripe count */
+ st_count = atoi(argv[4]);
+
+ /* Create the file, as specified. Return and display any errors. */
+ result = create_file(argv[1], st_size, st_offset, st_count);
+
+ return result;
+}
uuid CDATA #REQUIRED">\r
\r
<!-- main elements -->\r
-<!ELEMENT lustre (node | mountpoint | ldlm |\r
+<!ELEMENT lustre (node | mountpoint | ldlm | echo_client |\r
mds | mdc | obd | ost | osc | lov | lovconfig)*>\r
\r
<!ELEMENT node (network | profile)*>\r
hi CDATA #IMPLIED >\r
\r
<!ELEMENT profile (ldlm_ref | network_ref | obd_ref | ost_ref | osc_ref |\r
- mds_ref | mdc_ref | lov_ref | lovconfig_ref| mountpoint_ref)*>\r
+ echo_client_ref | mds_ref | mdc_ref | lov_ref |\r
+ lovconfig_ref| mountpoint_ref)*>\r
<!ATTLIST profile >\r
\r
<!ELEMENT mountpoint (path | fileset | mds_ref | osc_ref)*>\r
<!ATTLIST mountpoint %tag.attr;>\r
+<!ELEMENT echo_client (osc_ref)*>\r
+<!ATTLIST echo_client %tag.attr;>\r
<!ELEMENT ldlm EMPTY>\r
<!ATTLIST ldlm %tag.attr;>\r
\r
<!ATTLIST mdc_ref %tag.ref;>\r
<!ELEMENT mountpoint_ref %tag.content;>\r
<!ATTLIST mountpoint_ref %tag.ref;>\r
+<!ELEMENT echo_client_ref %tag.content;>\r
+<!ATTLIST echo_client_ref %tag.ref;>\r
<!ELEMENT server_ref %tag.content;>\r
<!ATTLIST server_ref %tag.ref;>\r
<!ELEMENT failover_ref %tag.content;>\r