public inbox for systemtap@sourceware.org
 help / color / mirror / Atom feed
* [PATH 1/3] User space probes-take3-RFC
@ 2006-03-06 15:09 Prasanna S Panchamukhi
  2006-03-06 15:10 ` [PATH 2/3] User space probes-readpage hooks take3-RFC Prasanna S Panchamukhi
  2006-03-08  9:35 ` [PATH 1/3] User space probes-take3-RFC bibo,mao
  0 siblings, 2 replies; 5+ messages in thread
From: Prasanna S Panchamukhi @ 2006-03-06 15:09 UTC (permalink / raw)
  To: systemtap

Hi,

Below is the take 3 of user space probes, which
seems to work for probes on applications and libraries.
Needs more testing and code review. This implementation
uses a spin lock to serialize the probe execution.

Please provide your review comments on these patchs.
Thanks to Yanmin, Anil and Jim for their review commoments.

Thanks
Prasanna

This kprobes patch adds support for userspace probes.  It adds a new
struct, uprobe, to the kprobes API, along with register_uprobe and
unregister_uprobe functions.  The implementation uses another new
struct, uprobe_module.

Objects
--------
struct uprobe		- Allocated per probe by the user.
struct uprobe_module	- Allocated per application by the userspace probe
			  mechanism.
struct uprobe {
	/* pointer to the pathname of the application */
	char *pathname;
	/* kprobe structure with user specified handlers */
	struct kprobe kp;
	/* hlist of all the userspace probes per application */
	struct hlist_node ulist;
	/* inode of the probed application */
	struct inode *inode;
	/* probe offset within the file */
	unsigned long offset;
};

struct uprobe_module {
	/* hlist head of all userspace probes per application */
	struct hlist_head ulist_head;
	/* list of all uprobe_module for probed application */
	struct list_head mlist;
	/* to hold path/dentry etc. */
	struct nameidata nd;
	/* original readpage operations */
	struct address_space_operations *ori_a_ops;
	/* readpage hooks added operations */
	struct address_space_operations user_a_ops;
};

Explanation of struct members:

Before calling register_uprobe, the user sets the following members
of struct uprobe:
pathname - the pathname of the probed application's executable/library file
kp - the kprobe object that specifies the handler(s) to run when the
	probe is hit, and the virtual address (kp.addr) at which to
	place the probe.
offset - is the absolute offset of the probe point from the beginning
	 of the probed executable/library file.

The remaining members are for internal use.

uprobe members inode and offset uniquely identify each probe, where:
inode  - is the inode of the probed application.
	  of the file.

	When the probe is hit, get_uprobe() walks the kprobes hash table
	to find the uprobe structure with the matching inode and offset.
	This is more efficient than searching for the application's
	uprobe_module and then walking that uprobe_module's list of
	uprobes.

ulist_head and ulist - holds all uprobes for an executable/library file.
	During readpage() operations, it walks the per-executable/library file
	probe list and inserts the probes.

mlist - list of all the probed executable/library files. During readpage()
	operations, the module list is used to find the matching probed
	file based on the inode. This list is protected by uprobe_mutex.

nd -	holds the path of the probed executable/library file until all the
	inserted probes are removed for that executable/library file.

ori_a_ops and user_a_ops - are used to hold the readpage pointers and readpage()
	hooks.

Interfaces :

1. register_uprobe(struct uprobe *uprobe) : accepts a pointer to uprobe.
User has to allocate the uprobes structure and initialize following
elements:
	pathname	- points to the application's pathname
	offset		- offset of the probe from the file beginning;
[It's still the case that the user has to specify the offset as well
as the address (see TODO list)]
			  In case of library calls, the offset is the
			  relative offset from the beginning of the of
			  the mapped library.
	kp.addr		- virtual address within the executable.
	kp.pre_handler	- handler to be executed when probe is fired.
	kp.post_handler	- handler to be executed after single stepping
			  the original instruction.
	kp.fault_handler- handler to be executed if fault occurs while
                          executing the original instruction or the
                          handlers.

As with a kprobe, the user should not modify the uprobe while it is
registered. This routine returns zero on successful registeration.

2. unregister_uprobe(struct uprobe *uprobe) : accepts a pointer to uprobe.

Usage:
Usage is similar to kprobe.
	/* Allocate a uprobe structure */
	struct uprobe p;

	/* Define pre handler */
	int handler_pre(struct kprobe *p, struct pt_regs *regs)
	{
		<.............collect useful data..............>
		return 0;
	}

	void handler_post(struct kprobe *p, struct pt_regs *regs,
							unsigned long flags)
	{
		<.............collect useful data..............>
	}

	int handler_fault(struct kprobe *p, struct pt_regs *regs, int trapnr)
	{
		<.............collect useful data..............>
		[return ... what?]
	}

	Before inserting the probe, specify the pathname of the application
	on which the probe is to be inserted.

	/*pointer to the pathname of the application */
	p.pathname = "/home/prasanna/bin/myapp";
	p.kp.pre_handler=handler_pre;
	p.kp.post_handler=handler_post;
	p.kp.fault_handler=handler_fault;

	/* Secify the probe address */
	/* $nm appln |grep func1 */
	p.kp.addr = (kprobe_opcode_t *)0x080484d4;
	/* Specify the offset within the application/executable*/
	p.offset = (unsigned long)0x4d4;
	/* Now register the userspace probe */
	if (ret = register_uprobe(&p))
		printk("register_uprobe: unsuccessful ret= %d\n", ret);

	/* To unregister the registered probed, just call..*/
	unregister_uprobe(&p);

Signed-off-by : Prasanna S Panchamukhi <prasanna@in.ibm.com>

 include/linux/namei.h   |    1 

diff -puN fs/namei.c~kprobes_userspace_probes-base-interface fs/namei.c

 arch/i386/kernel/kprobes.c |   10 
 fs/namei.c                 |   11 
 include/linux/namei.h      |    1 

diff -puN fs/namei.c~kprobes_userspace_probes-base-interface fs/namei.c


 arch/i386/kernel/kprobes.c |   12 +
 fs/namei.c                 |   11 
 include/linux/kprobes.h    |   49 ++++
 include/linux/namei.h      |    1 
 kernel/kprobes.c           |  528 +++++++++++++++++++++++++++++++++++++++++++++
 5 files changed, 598 insertions(+), 3 deletions(-)

diff -puN arch/i386/kernel/kprobes.c~kprobes_userspace_probes-base-interface arch/i386/kernel/kprobes.c
--- linux-2.6.16-rc5-mm2/arch/i386/kernel/kprobes.c~kprobes_userspace_probes-base-interface	2006-03-06 20:09:21.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/arch/i386/kernel/kprobes.c	2006-03-06 20:09:21.000000000 +0530
@@ -99,6 +99,18 @@ static inline int is_IF_modifier(kprobe_
 	return 0;
 }
 
+int __kprobes arch_alloc_insn(struct kprobe *p)
+{
+	mutex_lock(&kprobe_mutex);
+	p->ainsn.insn = get_insn_slot();
+	mutex_unlock(&kprobe_mutex);
+
+	if (!p->ainsn.insn)
+		return -ENOMEM;
+
+	return 0;
+}
+
 int __kprobes arch_prepare_kprobe(struct kprobe *p)
 {
 	/* insn: must be on special executable page on i386. */
diff -puN fs/namei.c~kprobes_userspace_probes-base-interface fs/namei.c
--- linux-2.6.16-rc5-mm2/fs/namei.c~kprobes_userspace_probes-base-interface	2006-03-06 20:09:21.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/fs/namei.c	2006-03-06 20:09:21.000000000 +0530
@@ -322,10 +322,8 @@ int get_write_access(struct inode * inod
 	return 0;
 }
 
-int deny_write_access(struct file * file)
+int deny_write_access_to_inode(struct inode *inode)
 {
-	struct inode *inode = file->f_dentry->d_inode;
-
 	spin_lock(&inode->i_lock);
 	if (atomic_read(&inode->i_writecount) > 0) {
 		spin_unlock(&inode->i_lock);
@@ -337,6 +335,13 @@ int deny_write_access(struct file * file
 	return 0;
 }
 
+int deny_write_access(struct file * file)
+{
+	struct inode *inode = file->f_dentry->d_inode;
+
+	return deny_write_access_to_inode(inode);
+}
+
 void path_release(struct nameidata *nd)
 {
 	dput(nd->dentry);
diff -puN include/linux/kprobes.h~kprobes_userspace_probes-base-interface include/linux/kprobes.h
--- linux-2.6.16-rc5-mm2/include/linux/kprobes.h~kprobes_userspace_probes-base-interface	2006-03-06 20:09:21.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/include/linux/kprobes.h	2006-03-06 20:19:00.000000000 +0530
@@ -37,6 +37,10 @@
 #include <linux/spinlock.h>
 #include <linux/rcupdate.h>
 #include <linux/mutex.h>
+#include <linux/mm.h>
+#include <linux/dcache.h>
+#include <linux/namei.h>
+#include <linux/pagemap.h>
 
 #ifdef CONFIG_KPROBES
 #include <asm/kprobes.h>
@@ -54,6 +58,7 @@ struct kprobe;
 struct pt_regs;
 struct kretprobe;
 struct kretprobe_instance;
+extern struct uprobe *current_uprobe;
 typedef int (*kprobe_pre_handler_t) (struct kprobe *, struct pt_regs *);
 typedef int (*kprobe_break_handler_t) (struct kprobe *, struct pt_regs *);
 typedef void (*kprobe_post_handler_t) (struct kprobe *, struct pt_regs *,
@@ -117,6 +122,32 @@ struct jprobe {
 DECLARE_PER_CPU(struct kprobe *, current_kprobe);
 DECLARE_PER_CPU(struct kprobe_ctlblk, kprobe_ctlblk);
 
+struct uprobe {
+	/* pointer to the pathname of the application */
+	char *pathname;
+	/* kprobe structure with user specified handlers */
+	struct kprobe kp;
+	/* hlist of all the userspace probes per application */
+	struct hlist_node ulist;
+	/* inode of the probed application */
+	struct inode *inode;
+	/* probe offset within the file */
+	unsigned long offset;
+};
+
+struct uprobe_module {
+	/* hlist head of all userspace probes per application */
+	struct hlist_head ulist_head;
+	/* list of all uprobe_module for probed application */
+	struct list_head mlist;
+	/* to hold path/dentry etc. */
+	struct nameidata nd;
+	/* original readpage operations */
+	struct address_space_operations *ori_a_ops;
+	/* readpage hooks added operations */
+	struct address_space_operations user_a_ops;
+};
+
 #ifdef ARCH_SUPPORTS_KRETPROBES
 extern void arch_prepare_kretprobe(struct kretprobe *rp, struct pt_regs *regs);
 #else /* ARCH_SUPPORTS_KRETPROBES */
@@ -162,9 +193,14 @@ extern void show_registers(struct pt_reg
 extern kprobe_opcode_t *get_insn_slot(void);
 extern void free_insn_slot(kprobe_opcode_t *slot);
 extern void kprobes_inc_nmissed_count(struct kprobe *p);
+extern void arch_copy_uprobe(struct kprobe *p, kprobe_opcode_t *address);
+extern void arch_arm_uprobe(kprobe_opcode_t *address);
+extern void arch_disarm_uprobe(struct kprobe *p, kprobe_opcode_t *address);
 
 /* Get the kprobe at this addr (if any) - called with preemption disabled */
 struct kprobe *get_kprobe(void *addr);
+struct kprobe *get_uprobe(void *addr);
+extern int arch_alloc_insn(struct kprobe *p);
 struct hlist_head * kretprobe_inst_table_head(struct task_struct *tsk);
 
 /* kprobe_running() will just return the current_kprobe on this CPU */
@@ -183,6 +219,16 @@ static inline struct kprobe_ctlblk *get_
 	return (&__get_cpu_var(kprobe_ctlblk));
 }
 
+static inline void set_uprobe_instance(struct kprobe *p)
+{
+	current_uprobe = container_of(p, struct uprobe, kp);
+}
+
+static inline void reset_uprobe_instance(void)
+{
+	current_uprobe = NULL;
+}
+
 int register_kprobe(struct kprobe *p);
 void unregister_kprobe(struct kprobe *p);
 int setjmp_pre_handler(struct kprobe *, struct pt_regs *);
@@ -194,6 +240,9 @@ void jprobe_return(void);
 int register_kretprobe(struct kretprobe *rp);
 void unregister_kretprobe(struct kretprobe *rp);
 
+int register_uprobe(struct uprobe *uprobe);
+void unregister_uprobe(struct uprobe *uprobe);
+
 struct kretprobe_instance *get_free_rp_inst(struct kretprobe *rp);
 void add_rp_inst(struct kretprobe_instance *ri);
 void kprobe_flush_task(struct task_struct *tk);
diff -puN include/linux/namei.h~kprobes_userspace_probes-base-interface include/linux/namei.h
--- linux-2.6.16-rc5-mm2/include/linux/namei.h~kprobes_userspace_probes-base-interface	2006-03-06 20:09:21.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/include/linux/namei.h	2006-03-06 20:09:21.000000000 +0530
@@ -81,6 +81,7 @@ extern int follow_up(struct vfsmount **,
 
 extern struct dentry *lock_rename(struct dentry *, struct dentry *);
 extern void unlock_rename(struct dentry *, struct dentry *);
+extern int deny_write_access_to_inode(struct inode *inode);
 
 static inline void nd_set_link(struct nameidata *nd, char *path)
 {
diff -puN kernel/kprobes.c~kprobes_userspace_probes-base-interface kernel/kprobes.c
--- linux-2.6.16-rc5-mm2/kernel/kprobes.c~kprobes_userspace_probes-base-interface	2006-03-06 20:09:21.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/kernel/kprobes.c	2006-03-06 20:17:58.000000000 +0530
@@ -52,6 +52,13 @@ DEFINE_MUTEX(kprobe_mutex);		/* Protects
 DEFINE_SPINLOCK(kretprobe_lock);	/* Protects kretprobe_inst_table */
 static DEFINE_PER_CPU(struct kprobe *, kprobe_instance) = NULL;
 
+/* user space probes lists */
+static struct list_head uprobe_module_list;
+static struct hlist_head uprobe_table[KPROBE_TABLE_SIZE];
+/* Protects uprobe_table and uprobe_module_list*/
+DEFINE_SPINLOCK(uprobe_lock);
+DEFINE_MUTEX(uprobe_mutex);		/* Protects uprobe_module_table */
+
 #ifdef __ARCH_WANT_KPROBES_INSN_SLOT
 /*
  * kprobe->ainsn.insn points to the copy of the instruction to be
@@ -248,6 +255,128 @@ static int __kprobes aggr_break_handler(
 	return ret;
 }
 
+/*
+ * Aggregate handlers for multiple uprobes support - these handlers
+ * take care of invoking the individual uprobe handlers on p->list
+ */
+static int __kprobes aggr_user_pre_handler(struct kprobe *p,
+							 struct pt_regs *regs)
+{
+	struct kprobe *kp;
+
+	list_for_each_entry(kp, &p->list, list) {
+		if (kp->pre_handler) {
+			set_uprobe_instance(kp);
+			if (kp->pre_handler(kp, regs))
+				return 1;
+		}
+	}
+	return 0;
+}
+
+static void __kprobes aggr_user_post_handler(struct kprobe *p,
+				struct pt_regs *regs, unsigned long flags)
+{
+	struct kprobe *kp;
+
+	list_for_each_entry(kp, &p->list, list) {
+		if (kp->post_handler) {
+			set_uprobe_instance(kp);
+			kp->post_handler(kp, regs, flags);
+		}
+	}
+}
+
+static int __kprobes aggr_user_fault_handler(struct kprobe *p,
+					struct pt_regs *regs, int trapnr)
+{
+	struct kprobe *cur;
+
+	/*
+	 * if we faulted "during" the execution of a user specified
+	 * probe handler, invoke just that probe's fault handler
+	 */
+	cur = &current_uprobe->kp;
+	if (cur && cur->fault_handler)
+		if (cur->fault_handler(cur, regs, trapnr))
+			return 1;
+	return 0;
+}
+
+/**
+ * This routine looks for an existing uprobe at the given offset and inode.
+ * If it's found, returns the corresponding kprobe pointer.
+ * This should be called with uprobe_lock held.
+ */
+static struct kprobe __kprobes *get_kprobe_user(struct inode *inode,
+							unsigned long offset)
+{
+	struct hlist_head *head;
+	struct hlist_node *node;
+	struct kprobe *p, *kpr;
+	struct uprobe *uprobe;
+
+	head = &uprobe_table[hash_ptr((kprobe_opcode_t *)
+			(((unsigned long)inode) * offset), KPROBE_HASH_BITS)];
+
+	hlist_for_each_entry(p, node, head, hlist) {
+		if (p->pre_handler == aggr_user_pre_handler) {
+			kpr = list_entry(p->list.next, typeof(*kpr), list);
+			uprobe = container_of(kpr, struct uprobe, kp);
+		} else
+			uprobe = container_of(p, struct uprobe, kp);
+
+		if ((uprobe->inode == inode) && (uprobe->offset == offset))
+			return p;
+	}
+
+	return NULL;
+}
+
+/**
+ * Finds a uprobe at the specified user-space address in the current task.
+ * Points current_uprobe at that uprobe and returns the corresponding kprobe.
+ */
+struct kprobe __kprobes *get_uprobe(void *addr)
+{
+	struct mm_struct *mm = current->mm;
+	struct vm_area_struct *vma;
+	struct inode *inode;
+	unsigned long offset;
+	struct kprobe *p, *kpr;
+	struct uprobe *uprobe;
+
+	vma = find_vma(mm, (unsigned long)addr);
+
+	BUG_ON(!vma);	/* this should not happen, not in our memory map */
+
+	offset = (unsigned long)addr - vma->vm_start +
+						(vma->vm_pgoff << PAGE_SHIFT);
+	if (!vma->vm_file)
+		return NULL;
+
+	inode = vma->vm_file->f_dentry->d_inode;
+
+	p = get_kprobe_user(inode, offset);
+	if (!p)
+		return NULL;
+
+	if (p->pre_handler == aggr_user_pre_handler) {
+		/*
+		 * Walk the uprobe aggrigate list and return firt
+		 * element on aggrigate list.
+		 */
+		kpr = list_entry((p)->list.next, typeof(*kpr), list);
+		uprobe = container_of(kpr, struct uprobe, kp);
+	} else
+		uprobe = container_of(p, struct uprobe, kp);
+
+	if (uprobe)
+		current_uprobe = uprobe;
+
+	return p;
+}
+
 /* Walks the list and increments nmissed count for multiprobe case */
 void __kprobes kprobes_inc_nmissed_count(struct kprobe *p)
 {
@@ -382,6 +511,48 @@ static int __kprobes add_new_kprobe(stru
 }
 
 /*
+ * Fill in the required fields of the "manager uprobe". Replace the
+ * earlier kprobe in the hlist with the manager uprobe
+ */
+static inline void add_aggr_uprobe(struct kprobe *ap, struct kprobe *p)
+{
+	copy_kprobe(p, ap);
+	ap->addr = p->addr;
+	ap->pre_handler = aggr_user_pre_handler;
+	ap->post_handler = aggr_user_post_handler;
+	ap->fault_handler = aggr_user_fault_handler;
+
+	INIT_LIST_HEAD(&ap->list);
+	list_add(&p->list, &ap->list);
+
+	hlist_replace_rcu(&p->hlist, &ap->hlist);
+}
+
+/*
+ * This is the second or subsequent uprobe at the address - handle
+ * the intricacies
+ */
+static int __kprobes register_aggr_uprobe(struct kprobe *old_p,
+					  struct kprobe *p)
+{
+	int ret = 0;
+	struct kprobe *ap;
+
+	if (old_p->pre_handler == aggr_user_pre_handler) {
+		copy_kprobe(old_p, p);
+		list_add(&p->list, &old_p->list);
+	} else {
+		ap = kzalloc(sizeof(struct kprobe), GFP_ATOMIC);
+		if (!ap)
+			return -ENOMEM;
+		add_aggr_uprobe(ap, old_p);
+		copy_kprobe(ap, p);
+		list_add(&p->list, &old_p->list);
+	}
+	return ret;
+}
+
+/*
  * Fill in the required fields of the "manager kprobe". Replace the
  * earlier kprobe in the hlist with the manager kprobe
  */
@@ -559,6 +730,359 @@ void __kprobes unregister_jprobe(struct 
 	unregister_kprobe(&jp->kp);
 }
 
+typedef int (*process_uprobe_func_t)(struct uprobe *uprobe,
+				kprobe_opcode_t *address);
+
+/**
+ * Saves the original instruction in the uprobe structure and
+ * inserts the breakpoint at the given address.
+ */
+int __kprobes insert_kprobe_user(struct uprobe *uprobe,
+				kprobe_opcode_t *address)
+{
+	arch_copy_uprobe(&uprobe->kp, address);
+	arch_arm_uprobe(address);
+
+	return 0;
+}
+
+/**
+ * Wait for the page to be unlocked if someone else had locked it,
+ * then map the page and insert or remove the breakpoint.
+ */
+static int __kprobes map_uprobe_page(struct page *page, struct uprobe *uprobe,
+				     process_uprobe_func_t process_kprobe_user)
+{
+	int ret = 0;
+	kprobe_opcode_t *uprobe_address;
+
+	if (!page)
+		return -EINVAL; /* TODO: more suitable errno */
+
+	wait_on_page_locked(page);
+	/* could probably retry readpage here. */
+	if (!PageUptodate(page))
+		return -EINVAL; /* TODO: more suitable errno */
+
+	lock_page(page);
+
+	uprobe_address = (kprobe_opcode_t *)kmap(page);
+	uprobe_address = (kprobe_opcode_t *)((unsigned long)uprobe_address +
+						(uprobe->offset & ~PAGE_MASK));
+	ret = (*process_kprobe_user)(uprobe, uprobe_address);
+	kunmap(page);
+
+	unlock_page(page);
+
+	return ret;
+}
+
+/**
+ * flush_vma walks through the list of process private mappings,
+ * gets the vma containing the offset and flush all the vma's
+ * containing the probed page.
+ */
+static void __kprobes flush_vma(struct address_space *mapping,
+				struct page *page, struct uprobe *uprobe)
+{
+	struct vm_area_struct *vma = NULL;
+	struct prio_tree_iter iter;
+	struct prio_tree_root *head = &mapping->i_mmap;
+	struct mm_struct *mm;
+	unsigned long start, end, offset = uprobe->offset;
+
+	spin_lock(&mapping->i_mmap_lock);
+	vma_prio_tree_foreach(vma, &iter, head, offset, offset) {
+		mm = vma->vm_mm;
+		start = vma->vm_start - (vma->vm_pgoff << PAGE_SHIFT);
+		end = vma->vm_end - (vma->vm_pgoff << PAGE_SHIFT);
+
+		if ((start + offset) < end)
+			flush_icache_user_range(vma, page,
+					(unsigned long)uprobe->kp.addr,
+						sizeof(kprobe_opcode_t));
+	}
+	spin_unlock(&mapping->i_mmap_lock);
+}
+
+/**
+ * Walk the uprobe_module_list and return the uprobe module with matching
+ * inode.
+ */
+static struct uprobe_module __kprobes *get_module_by_inode(struct inode *inode)
+{
+	struct uprobe_module *umodule;
+
+	list_for_each_entry(umodule, &uprobe_module_list, mlist) {
+		if (umodule->nd.dentry->d_inode == inode)
+			return umodule;
+	}
+
+	return NULL;
+}
+
+/**
+ * Gets exclusive write access to the given inode to ensure that the file
+ * on which probes are currently applied does not change. Use the function,
+ * deny_write_access_to_inode() we added in fs/namei.c.
+ */
+static inline int ex_write_lock(struct inode *inode)
+{
+	return deny_write_access_to_inode(inode);
+}
+
+/**
+ * Called when removing user space probes to release the write lock on the
+ * inode.
+ */
+static inline int ex_write_unlock(struct inode *inode)
+{
+	atomic_inc(&inode->i_writecount);
+	return 0;
+}
+
+/**
+ * Add uprobe and uprobe_module to the appropriate hash list.
+ */
+static void __kprobes get_inode_ops(struct uprobe *uprobe,
+				   struct uprobe_module *umodule)
+{
+	INIT_HLIST_HEAD(&umodule->ulist_head);
+	hlist_add_head(&uprobe->ulist, &umodule->ulist_head);
+	list_add(&umodule->mlist, &uprobe_module_list);
+}
+
+/*
+ * Removes the specified uprobe from either aggrigate uprobe list
+ * or individual uprobe hash table.
+ */
+
+static int __kprobes remove_uprobe(struct uprobe *uprobe)
+{
+	struct kprobe *old_p, *list_p, *p;
+	int ret = 0;
+
+	p = &uprobe->kp;
+	old_p = get_kprobe_user(uprobe->inode, uprobe->offset);
+	if (unlikely(!old_p))
+		return 0;
+
+	if (p != old_p) {
+		list_for_each_entry(list_p, &old_p->list, list)
+			if (list_p == p)
+			/* kprobe p is a valid probe */
+				goto valid_p;
+		return 0;
+	}
+
+valid_p:
+	if ((old_p == p) ||
+			((old_p->pre_handler == aggr_user_pre_handler) &&
+			(p->list.next == &old_p->list) &&
+			(p->list.prev == &old_p->list))) {
+		/* Only probe on the hash list */
+			ret = 1;
+		hlist_del(&old_p->hlist);
+		if (p != old_p) {
+			list_del(&p->list);
+			kfree(old_p);
+		}
+	} else
+		list_del(&p->list);
+
+	return ret;
+}
+
+/*
+ * Disarms the probe and frees the corresponding instruction slot.
+ */
+static int __kprobes remove_kprobe_user(struct uprobe *uprobe,
+				kprobe_opcode_t *address)
+{
+	struct kprobe *p = &uprobe->kp;
+
+	arch_disarm_uprobe(p, address);
+	arch_remove_kprobe(p);
+
+	return 0;
+}
+
+/*
+ * Adds the given uprobe to the uprobe_hash table if it is
+ * the first probe to be inserted at the given address else
+ * adds to the aggrigate uprobe's list.
+ */
+static int __kprobes insert_uprobe(struct uprobe *uprobe)
+{
+	struct kprobe *old_p;
+	int ret = 0;
+	unsigned long offset = uprobe->offset;
+	unsigned long inode = (unsigned long) uprobe->inode;
+	struct hlist_head *head;
+	unsigned long flags;
+
+	spin_lock_irqsave(&uprobe_lock, flags);
+	uprobe->kp.nmissed = 0;
+
+	old_p = get_kprobe_user(uprobe->inode, uprobe->offset);
+
+	if (old_p)
+		register_aggr_uprobe(old_p, &uprobe->kp);
+	else {
+		head = &uprobe_table[hash_ptr((kprobe_opcode_t *)
+					(offset * inode), KPROBE_HASH_BITS)];
+		INIT_HLIST_NODE(&uprobe->kp.hlist);
+		hlist_add_head(&uprobe->kp.hlist, head);
+		ret = 1;
+	}
+
+	spin_unlock_irqrestore(&uprobe_lock, flags);
+
+	return ret;
+}
+
+/**
+ * unregister_uprobe: Disarms the probe, removes the uprobe
+ * pointers from the hash list and unhooks readpage routines.
+ */
+void __kprobes unregister_uprobe(struct uprobe *uprobe)
+{
+	struct address_space *mapping;
+	struct uprobe_module *umodule;
+	struct page *page;
+	unsigned long flags;
+	int ret = 0;
+
+	if (!uprobe->inode)
+		return;
+
+	mapping = uprobe->inode->i_mapping;
+
+	page = find_get_page(mapping, uprobe->offset >> PAGE_CACHE_SHIFT);
+
+	spin_lock_irqsave(&uprobe_lock, flags);
+	ret = remove_uprobe(uprobe);
+	spin_unlock_irqrestore(&uprobe_lock, flags);
+
+	mutex_lock(&uprobe_mutex);
+	if (!(umodule = get_module_by_inode(uprobe->inode)))
+		goto out;
+
+	hlist_del(&uprobe->ulist);
+	if (hlist_empty(&umodule->ulist_head)) {
+		list_del(&umodule->mlist);
+		ex_write_unlock(uprobe->inode);
+		path_release(&umodule->nd);
+		kfree(umodule);
+	}
+
+out:
+	mutex_unlock(&uprobe_mutex);
+	if (ret)
+		ret = map_uprobe_page(page, uprobe, remove_kprobe_user);
+
+	if (ret == -EINVAL)
+		return;
+	/*
+	 * TODO: unregister_uprobe should not fail, need to handle
+	 * if it fails.
+	 */
+	flush_vma(mapping, page, uprobe);
+
+	if (page)
+		page_cache_release(page);
+}
+
+/**
+ * register_uprobe(): combination of inode and offset is used to
+ * identify each probe uniquely. Each uprobe can be found from the
+ * uprobes_hash table by using inode and offset. register_uprobe(),
+ * inserts the breakpoint at the given address by locating and mapping
+ * the page. return 0 on success and error on failure.
+ */
+int __kprobes register_uprobe(struct uprobe *uprobe)
+{
+	struct address_space *mapping;
+	struct uprobe_module *umodule = NULL;
+	struct inode *inode;
+	struct nameidata nd;
+	struct page *page;
+	int error = 0;
+
+	INIT_HLIST_NODE(&uprobe->ulist);
+
+	/*
+	 * TODO: Need to calculate the absolute file offset for dynamic
+	 * shared libraries.
+	 */
+	if ((error = path_lookup(uprobe->pathname, LOOKUP_FOLLOW, &nd)))
+		return error;
+
+	mutex_lock(&uprobe_mutex);
+
+	inode = nd.dentry->d_inode;
+	error = ex_write_lock(inode);
+	if (error)
+		goto out;
+
+	/*
+	 * Check if there are probes already on this application and
+	 * add the corresponding uprobe to per application probe's list.
+	 */
+	umodule = get_module_by_inode(inode);
+	if (!umodule) {
+
+		error = arch_alloc_insn(&uprobe->kp);
+		if (error)
+			goto out;
+
+		/*
+		 * Allocate a uprobe_module structure for this
+		 * application if not allocated before.
+		 */
+		umodule = kzalloc(sizeof(struct uprobe_module), GFP_KERNEL);
+		if (!umodule) {
+			error = -ENOMEM;
+			ex_write_unlock(inode);
+			arch_remove_kprobe(&uprobe->kp);
+			goto out;
+		}
+		memcpy(&umodule->nd, &nd, sizeof(struct nameidata));
+		get_inode_ops(uprobe, umodule);
+	} else {
+		path_release(&nd);
+		ex_write_unlock(inode);
+		hlist_add_head(&uprobe->ulist, &umodule->ulist_head);
+	}
+	mutex_unlock(&uprobe_mutex);
+
+	uprobe->inode = inode;
+	mapping = inode->i_mapping;
+	page = find_get_page(mapping, (uprobe->offset >> PAGE_CACHE_SHIFT));
+
+	if (insert_uprobe(uprobe))
+		error = map_uprobe_page(page, uprobe, insert_kprobe_user);
+
+	/*
+	 * If error == -EINVAL, return success, probes will inserted by
+	 * readpage hooks.
+	 * TODO: Use a more suitable errno?
+	 */
+	if (error == -EINVAL)
+		error = 0;
+	flush_vma(mapping, page, uprobe);
+
+	if (page)
+		page_cache_release(page);
+
+	return error;
+out:
+	path_release(&nd);
+	mutex_unlock(&uprobe_mutex);
+
+	return error;
+}
+
 #ifdef ARCH_SUPPORTS_KRETPROBES
 
 /*
@@ -648,8 +1172,10 @@ static int __init init_kprobes(void)
 	for (i = 0; i < KPROBE_TABLE_SIZE; i++) {
 		INIT_HLIST_HEAD(&kprobe_table[i]);
 		INIT_HLIST_HEAD(&kretprobe_inst_table[i]);
+		INIT_HLIST_HEAD(&uprobe_table[i]);
 	}
 
+	INIT_LIST_HEAD(&uprobe_module_list);
 	err = arch_init_kprobes();
 	if (!err)
 		err = register_die_notifier(&kprobe_exceptions_nb);
@@ -666,4 +1192,6 @@ EXPORT_SYMBOL_GPL(unregister_jprobe);
 EXPORT_SYMBOL_GPL(jprobe_return);
 EXPORT_SYMBOL_GPL(register_kretprobe);
 EXPORT_SYMBOL_GPL(unregister_kretprobe);
+EXPORT_SYMBOL_GPL(register_uprobe);
+EXPORT_SYMBOL_GPL(unregister_uprobe);
 

_
-- 
Prasanna S Panchamukhi
Linux Technology Center
India Software Labs, IBM Bangalore
Email: prasanna@in.ibm.com
Ph: 91-80-51776329

^ permalink raw reply	[flat|nested] 5+ messages in thread

* Re: [PATH 2/3] User space probes-readpage hooks take3-RFC
  2006-03-06 15:09 [PATH 1/3] User space probes-take3-RFC Prasanna S Panchamukhi
@ 2006-03-06 15:10 ` Prasanna S Panchamukhi
  2006-03-06 15:11   ` [PATH 3/3] User space probes-single stepping out-of-line take3-RFC Prasanna S Panchamukhi
  2006-03-08  9:35 ` [PATH 1/3] User space probes-take3-RFC bibo,mao
  1 sibling, 1 reply; 5+ messages in thread
From: Prasanna S Panchamukhi @ 2006-03-06 15:10 UTC (permalink / raw)
  To: systemtap

This patch provides the feature of inserting probes on pages that are
not present in the memory during registration.

To add readpage and readpages() hooks, two new elements are added to
the uprobe_module object:
	struct address_space_operations *ori_a_ops;
	struct address_space_operations user_a_ops;

User-space probes allows probes to be inserted even in pages that are
not present in the memory at the time of registration. This is done
by adding hooks to the readpage and readpages routines. During
registration, the address space operation object is modified by
substituting user-space probes's specific readpage and readpages
routines. When the pages are read into memory through the readpage and
readpages address space operations, any associated probes are
automatically inserted into those pages. These user-space probes
readpage and readpages routines internally call the original
readpage() and readpages() routines, and then check whether probes are
to be added to these pages, inserting probes as necessary. The
overhead of adding these hooks is limited to the application on which
the probes are inserted.

During unregistration, care should be taken to replace the readpage and
readpages hooks with the original routines if no probes remain on that
application.

Signed-of-by: Prasanna S Panchamukhi <prasanna@in.ibm.com>


 kernel/kprobes.c |  121 +++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 files changed, 121 insertions(+)

diff -puN kernel/kprobes.c~kprobes_userspace_probes-hook-readpage kernel/kprobes.c
--- linux-2.6.16-rc5-mm2/kernel/kprobes.c~kprobes_userspace_probes-hook-readpage	2006-03-06 19:16:11.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/kernel/kprobes.c	2006-03-06 19:16:11.000000000 +0530
@@ -821,6 +821,115 @@ static struct uprobe_module __kprobes *g
 	return NULL;
 }
 
+static inline void insert_readpage_uprobe(struct page *page,
+	struct address_space *mapping, struct uprobe *uprobe)
+{
+	unsigned long page_start = page->index << PAGE_CACHE_SHIFT;
+	unsigned long page_end = page_start + PAGE_SIZE;
+
+	if ((uprobe->offset >= page_start) && (uprobe->offset < page_end)) {
+		map_uprobe_page(page, uprobe, insert_kprobe_user);
+		flush_vma(mapping, page, uprobe);
+	}
+}
+
+/**
+ *  This function hooks the readpages() of all modules that have active
+ *  probes on them. The original readpages() is called for the given
+ *  inode/address_space to actually read the pages into the memory.
+ *  Then all probes that are specified on these pages are inserted.
+ */
+static int __kprobes uprobe_readpages(struct file *file,
+				struct address_space *mapping,
+				struct list_head *pages, unsigned nr_pages)
+{
+	int retval = 0;
+	struct page *page;
+	struct uprobe_module *umodule;
+	struct uprobe *uprobe = NULL;
+	struct hlist_node *node;
+
+	mutex_lock(&uprobe_mutex);
+
+	umodule = get_module_by_inode(file->f_dentry->d_inode);
+	if (!umodule) {
+		/*
+		 * No module associated with this file, call the
+		 * original readpages().
+		 */
+		retval = mapping->a_ops->readpages(file, mapping,
+							pages, nr_pages);
+		goto out;
+	}
+
+	/* call original readpages() */
+	retval = umodule->ori_a_ops->readpages(file, mapping, pages, nr_pages);
+	if (retval < 0)
+		goto out;
+
+	/*
+	 * TODO: Walk through readpages page list and get
+	 * pages with probes instead of find_get_page().
+	 */
+	hlist_for_each_entry(uprobe, node, &umodule->ulist_head, ulist) {
+		page = find_get_page(mapping,
+				uprobe->offset >> PAGE_CACHE_SHIFT);
+		if (!page)
+			continue;
+
+		if (!uprobe->kp.opcode)
+			insert_readpage_uprobe(page, mapping, uprobe);
+		page_cache_release(page);
+	}
+
+out:
+	mutex_unlock(&uprobe_mutex);
+
+	return retval;
+}
+
+/**
+ *  This function hooks the readpage() of all modules that have active
+ *  probes on them. The original readpage() is called for the given
+ *  inode/address_space to actually read the pages into the memory.
+ *  Then all probes that are specified on this page are inserted.
+ */
+int __kprobes uprobe_readpage(struct file *file, struct page *page)
+{
+	int retval = 0;
+	struct uprobe_module *umodule;
+	struct uprobe *uprobe = NULL;
+	struct hlist_node *node;
+	struct address_space *mapping = file->f_dentry->d_inode->i_mapping;
+
+	mutex_lock(&uprobe_mutex);
+
+	umodule = get_module_by_inode(file->f_dentry->d_inode);
+	if (!umodule) {
+		/*
+		 * No module associated with this file, call the
+		 * original readpage().
+		 */
+		retval = mapping->a_ops->readpage(file, page);
+		goto out;
+	}
+
+	/* call original readpage() */
+	retval = umodule->ori_a_ops->readpage(file, page);
+	if (retval < 0)
+		goto out;
+
+	hlist_for_each_entry(uprobe, node, &umodule->ulist_head, ulist) {
+		if (!uprobe->kp.opcode)
+			insert_readpage_uprobe(page, mapping, uprobe);
+	}
+
+out:
+	mutex_unlock(&uprobe_mutex);
+
+	return retval;
+}
+
 /**
  * Gets exclusive write access to the given inode to ensure that the file
  * on which probes are currently applied does not change. Use the function,
@@ -843,13 +952,22 @@ static inline int ex_write_unlock(struct
 
 /**
  * Add uprobe and uprobe_module to the appropriate hash list.
+ * Also swithces i_op to hooks into readpage and readpages().
  */
 static void __kprobes get_inode_ops(struct uprobe *uprobe,
 				   struct uprobe_module *umodule)
 {
+	struct address_space *as;
+
 	INIT_HLIST_HEAD(&umodule->ulist_head);
 	hlist_add_head(&uprobe->ulist, &umodule->ulist_head);
 	list_add(&umodule->mlist, &uprobe_module_list);
+	as = umodule->nd.dentry->d_inode->i_mapping;
+	umodule->ori_a_ops = as->a_ops;
+	umodule->user_a_ops = *as->a_ops;
+	umodule->user_a_ops.readpage = uprobe_readpage;
+	umodule->user_a_ops.readpages = uprobe_readpages;
+	as->a_ops = &umodule->user_a_ops;
 }
 
 /*
@@ -971,6 +1089,9 @@ void __kprobes unregister_uprobe(struct 
 	hlist_del(&uprobe->ulist);
 	if (hlist_empty(&umodule->ulist_head)) {
 		list_del(&umodule->mlist);
+		umodule->nd.dentry->d_inode->i_mapping->a_ops =
+							umodule->ori_a_ops;
+
 		ex_write_unlock(uprobe->inode);
 		path_release(&umodule->nd);
 		kfree(umodule);

_
-- 
Prasanna S Panchamukhi
Linux Technology Center
India Software Labs, IBM Bangalore
Email: prasanna@in.ibm.com
Ph: 91-80-51776329

^ permalink raw reply	[flat|nested] 5+ messages in thread

* Re: [PATH 3/3] User space probes-single stepping out-of-line take3-RFC
  2006-03-06 15:10 ` [PATH 2/3] User space probes-readpage hooks take3-RFC Prasanna S Panchamukhi
@ 2006-03-06 15:11   ` Prasanna S Panchamukhi
  0 siblings, 0 replies; 5+ messages in thread
From: Prasanna S Panchamukhi @ 2006-03-06 15:11 UTC (permalink / raw)
  To: systemtap

This patch provides a mechanism for probe handling and
executing the user-specified handlers.

Each userspace probe is uniquely identified by the combination of
inode and offset, hence during registeration the inode and offset
combination is added to uprobes hash table. Initially when
breakpoint instruction is hit, the uprobes hash table is looked up
for matching inode and offset. The pre_handlers are called in
sequence if multiple probes are registered. Similar to kprobes,
uprobes also adopts to single step out-of-line, so that probe miss in
SMP environment can be avoided. But for userspace probes, instruction
copied into kernel address space cannot be single stepped, hence the
instruction must be copied to user address space. The solution is to
find free space in the current process address space and then copy the
original instruction and single step that instruction.

User processes use stack space to store local variables, agruments and
return values. Normally the stack space either below or above the
stack pointer indicates the free stack space.

The instruction to be single stepped can modify the stack space,
hence before using the free stack space, sufficient stack space should
be left. The instruction is copied to the bottom of the page and check
is made such that the copied instruction does not cross the page
boundry. The copied instruction is then single stepped. Several
architectures does not allow the instruction to be executed from the
stack location, since no-exec bit is set for the stack pages. In those
architectures, the page table entry corresponding to the stack page is
identified and the no-exec bit is unset making the instruction on that
stack page to be executed.

There are situations where even the free stack space is not enough for
the user instruction to be copied and single stepped. In such
situations, the virtual memory area(vma) can be expanded beyond the
current stack vma. This expaneded stack can be used to copy the
original instruction and single step out-of-line.

Even if the vma cannot be extended then the instruction much be
executed inline, by replacing the breakpoint instruction with original
instruction.

TODO list
--------
1. Execution of probe handlers are serialized using a uprobe_mutex,
   need to make them scalable.

2. As Yanmin mentioned, user-probes can reenter through signal
   handlers, need to support reentrancy similar to kprobes.

3. Synchornize usage of stack between signal handlers and user-space
   probe or prevent singnal handler using the stack space until the
   copied instruction is single stepped.

4. Insert probes on copy-on-write pages. Tracks all COW pages for the
   page containing the specified probe point and inserts/removes all
   the probe points for that page.

5. Optimize the insertion of probes through readpage hooks. Identify
   all the probes to be inserted on the read page and insert them at
   once.

6. A wrapper routines to calculate the offset from the probed file
   beginning. In case of dynamic shared library, the offset is
   calculated by subtracting the address of the probe point from the
   beginning of the file mapped address.

7. Robust fault handline to support faults while single stepping of
   original instruction.

8. To handle probes on special instructions like sub $64,%esp,
   sub %esi, %esp, mov %esi, %esp etc which can grow the stack.

9. We grap a mutex allowing it to sleep while probe processing, need
   to avoid sleeping while probe processing.

Signed-off-by: Prasanna S Panchamukhi <prasanna@in.ibm.com>

 arch/i386/mm/fault.c       |    3 
 include/linux/kprobes.h    |    8 

diff -puN arch/i386/kernel/kprobes.c~kprobes_userspace_probes-ss-out-of-line arch/i386/kernel/kprobes.c


 arch/i386/kernel/kprobes.c |  531 ++++++++++++++++++++++++++++++++++++++++++++-
 arch/i386/mm/fault.c       |    3 
 include/asm-i386/kprobes.h |   18 +
 include/linux/kprobes.h    |    8 
 4 files changed, 556 insertions(+), 4 deletions(-)

diff -puN arch/i386/kernel/kprobes.c~kprobes_userspace_probes-ss-out-of-line arch/i386/kernel/kprobes.c
--- linux-2.6.16-rc5-mm2/arch/i386/kernel/kprobes.c~kprobes_userspace_probes-ss-out-of-line	2006-03-06 19:16:45.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/arch/i386/kernel/kprobes.c	2006-03-06 19:28:07.000000000 +0530
@@ -40,6 +40,8 @@ void jprobe_return_end(void);
 
 DEFINE_PER_CPU(struct kprobe *, current_kprobe) = NULL;
 DEFINE_PER_CPU(struct kprobe_ctlblk, kprobe_ctlblk);
+static struct uprobe_ctlblk uprobe_ctlblk;
+struct uprobe *current_uprobe;
 
 /* insert a jmp code */
 static inline void set_jmp_op(void *from, void *to)
@@ -111,6 +113,22 @@ int __kprobes arch_alloc_insn(struct kpr
 	return 0;
 }
 
+void __kprobes arch_disarm_uprobe(struct kprobe *p, kprobe_opcode_t *address)
+{
+	*address = p->opcode;
+}
+
+void __kprobes arch_arm_uprobe(kprobe_opcode_t *address)
+{
+	*address = BREAKPOINT_INSTRUCTION;
+}
+
+void __kprobes arch_copy_uprobe(struct kprobe *p, kprobe_opcode_t *address)
+{
+	memcpy(p->ainsn.insn, address, MAX_INSN_SIZE * sizeof(kprobe_opcode_t));
+	p->opcode = *(kprobe_opcode_t *)address;
+}
+
 int __kprobes arch_prepare_kprobe(struct kprobe *p)
 {
 	/* insn: must be on special executable page on i386. */
@@ -579,8 +597,8 @@ int __kprobes kprobe_exceptions_notify(s
 	struct die_args *args = (struct die_args *)data;
 	int ret = NOTIFY_DONE;
 
-	if (user_mode(args->regs))
-		return ret;
+	if (args->regs && user_mode(args->regs))
+		return uprobe_exceptions_notify(self, val, data);
 
 	switch (val) {
 	case DIE_INT3:
@@ -671,6 +689,515 @@ int __kprobes longjmp_break_handler(stru
 	return 0;
 }
 
+/**
+ * This routines get the pte of the page containing the specified address.
+ */
+static pte_t  __kprobes *get_uprobe_pte(unsigned long address)
+{
+	pgd_t *pgd;
+	pud_t *pud;
+	pmd_t *pmd;
+	pte_t *pte = NULL;
+
+	pgd = pgd_offset(current->mm, address);
+	if (!pgd)
+		goto out;
+
+	pud = pud_offset(pgd, address);
+	if (!pud)
+		goto out;
+
+	pmd = pmd_offset(pud, address);
+	if (!pmd)
+		goto out;
+
+	pte = pte_alloc_map(current->mm, pmd, address);
+
+out:
+	return pte;
+}
+
+/**
+ *  This routine check for space in the current process's stack
+ *  address space. If enough address space is found, copy the original
+ *  instruction on that page for single stepping out-of-line.
+ */
+static int __kprobes copy_insn_on_new_page(struct uprobe *uprobe ,
+			struct pt_regs *regs, struct vm_area_struct *vma)
+{
+	unsigned long addr, stack_addr = regs->esp;
+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
+
+	if (vma->vm_flags & VM_GROWSDOWN) {
+		if (((stack_addr - sizeof(long long))) <
+						(vma->vm_start + size))
+			return -ENOMEM;
+		addr = vma->vm_start;
+	} else if (vma->vm_flags & VM_GROWSUP) {
+		if ((vma->vm_end - size) < (stack_addr + sizeof(long long)))
+			return -ENOMEM;
+		addr = vma->vm_end - size;
+	} else
+		return -EFAULT;
+
+	vma->vm_flags |= VM_LOCKED;
+
+	if (__copy_to_user_inatomic((unsigned long *)addr,
+				(unsigned long *)uprobe->kp.ainsn.insn, size))
+		return -EFAULT;
+
+	regs->eip = addr;
+
+	return 0;
+}
+
+/**
+ * This routine expands the stack beyond the present process address
+ * space and copies the instruction to that location, so that
+ * processor can single step out-of-line.
+ */
+static int __kprobes copy_insn_onexpstack(struct uprobe *uprobe,
+			struct pt_regs *regs, struct vm_area_struct *vma)
+{
+	unsigned long addr, vm_addr;
+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
+	struct vm_area_struct *new_vma;
+	struct mm_struct *mm = current->mm;
+
+
+	 if (!down_read_trylock(&current->mm->mmap_sem))
+		 return -ENOMEM;
+
+	if (vma->vm_flags & VM_GROWSDOWN)
+		vm_addr = vma->vm_start - size;
+	else if (vma->vm_flags & VM_GROWSUP)
+		vm_addr = vma->vm_end + size;
+	else {
+		up_read(&current->mm->mmap_sem);
+		return -EFAULT;
+	}
+
+	new_vma = find_extend_vma(mm, vm_addr);
+	if (!new_vma) {
+		up_read(&current->mm->mmap_sem);
+		return -ENOMEM;
+	}
+
+	if (new_vma->vm_flags & VM_GROWSDOWN)
+		addr = new_vma->vm_start;
+	else
+		addr = new_vma->vm_end - size;
+
+	new_vma->vm_flags |= VM_LOCKED;
+	up_read(&current->mm->mmap_sem);
+
+	if (__copy_to_user_inatomic((unsigned long *)addr,
+				(unsigned long *)uprobe->kp.ainsn.insn, size))
+		return -EFAULT;
+
+	regs->eip = addr;
+
+	return  0;
+}
+
+/**
+ * This routine checks for stack free space below the stack pointer
+ * and then copies the instructions at that location so that the
+ * processor can single step out-of-line. If there is not enough stack
+ * space or if copy_to_user fails or if the vma is invalid, it returns
+ * error.
+ */
+static int __kprobes copy_insn_onstack(struct uprobe *uprobe,
+			struct pt_regs *regs, unsigned long flags)
+{
+	unsigned long page_addr, stack_addr = regs->esp;
+	int  size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
+	unsigned long *source = (unsigned long *)uprobe->kp.ainsn.insn;
+
+	if (flags & VM_GROWSDOWN) {
+		page_addr = stack_addr & PAGE_MASK;
+
+		if (((stack_addr - sizeof(long long))) < (page_addr + size))
+			return -ENOMEM;
+
+		if (__copy_to_user_inatomic((unsigned long *)page_addr,
+								source, size))
+			return -EFAULT;
+
+		regs->eip = page_addr;
+	} else if (flags & VM_GROWSUP) {
+		page_addr = stack_addr & PAGE_MASK;
+
+		if (page_addr == stack_addr)
+			return -ENOMEM;
+		else
+			page_addr += PAGE_SIZE;
+
+		if ((page_addr - size) < (stack_addr + sizeof(long long)))
+			return -ENOMEM;
+
+		if (__copy_to_user_inatomic(
+			(unsigned long *)(page_addr - size), source, size))
+			return -EFAULT;
+
+		regs->eip = page_addr - size;
+	} else
+		return -EINVAL;
+
+	return 0;
+}
+
+/**
+ * This routines get the page containing the probe, maps it and
+ * replaced the instruction at the probed address with specified
+ * opcode.
+ */
+void __kprobes replace_original_insn(struct uprobe *uprobe,
+				struct pt_regs *regs, kprobe_opcode_t opcode)
+{
+	kprobe_opcode_t *addr;
+	struct page *page;
+
+	page = find_get_page(uprobe->inode->i_mapping,
+					uprobe->offset >> PAGE_CACHE_SHIFT);
+	BUG_ON(!page);
+
+	__lock_page(page);
+
+	addr = (kprobe_opcode_t *)kmap_atomic(page, KM_USER1);
+	addr = (kprobe_opcode_t *)((unsigned long)addr +
+				 (unsigned long)(uprobe->offset & ~PAGE_MASK));
+	*addr = opcode;
+	/*TODO: flush vma ? */
+	kunmap_atomic(addr, KM_USER1);
+
+	unlock_page(page);
+
+	if (page)
+		page_cache_release(page);
+	regs->eip = (unsigned long)uprobe->kp.addr;
+}
+
+/**
+ * This routine provides the functionality of single stepping
+ * out-of-line. If single stepping out-of-line cannot be achieved,
+ * it replaces with the original instruction allowing it to single
+ * step inline.
+ */
+static inline int prepare_singlestep_uprobe(struct uprobe *uprobe,
+				struct uprobe_ctlblk *ucb, struct pt_regs *regs)
+{
+	unsigned long stack_addr = regs->esp, flags;
+	struct vm_area_struct *vma = NULL;
+	int err = 0;
+
+	vma = find_vma(current->mm, (stack_addr & PAGE_MASK));
+	if (!vma) {
+		/* TODO: Need better error reporting? */
+		goto no_vma;
+	}
+	flags = vma->vm_flags;
+
+	regs->eflags |= TF_MASK;
+	regs->eflags &= ~IF_MASK;
+
+	/*
+	 * Copy_insn_on_stack tries to find some room for the instruction slot
+	 * in the same page as the current esp.
+	 */
+	err = copy_insn_onstack(uprobe, regs, flags);
+
+	/*
+	 * If copy_insn_on_stack() fails, copy_insn_on_new_page() is called to
+	 * try to find some room in the next pages below the current esp;
+	 */
+	if (err)
+		err = copy_insn_on_new_page(uprobe, regs, vma);
+	/*
+	 * If copy_insn_on_new_pagek() fails, copy_insn_on_expstack() is called to
+	 * try to grow the stack's VM area by one page.
+	 */
+	if (err)
+		err = copy_insn_onexpstack(uprobe, regs, vma);
+
+	ucb->uprobe_status = UPROBE_HIT_SS;
+
+	if (!err) {
+		ucb->upte = get_uprobe_pte(regs->eip);
+		if (!ucb->upte)
+			goto no_vma;
+		ucb->upage = pte_page(*ucb->upte);
+		__lock_page(ucb->upage);
+	}
+no_vma:
+	if (err) {
+		replace_original_insn(uprobe, regs, uprobe->kp.opcode);
+		ucb->uprobe_status = UPROBE_SS_INLINE;
+	}
+
+	ucb->singlestep_addr = regs->eip;
+
+	return 0;
+}
+
+/*
+ * uprobe_handler() executes the user specified handler and setup for
+ * single stepping the original instruction either out-of-line or inline.
+ */
+static int __kprobes uprobe_handler(struct pt_regs *regs)
+{
+	struct kprobe *p;
+	int ret = 0;
+	kprobe_opcode_t *addr = NULL;
+	struct uprobe_ctlblk *ucb = &uprobe_ctlblk;
+	unsigned long limit;
+
+	spin_lock_irqsave(&uprobe_lock, ucb->flags);
+	/* preemption is disabled, remains disabled
+	 * untill we single step on original instruction.
+	 */
+	preempt_disable();
+
+	addr = (kprobe_opcode_t *)(get_segment_eip(regs, &limit) - 1);
+
+	p = get_uprobe(addr);
+	if (!p) {
+
+		if (*addr != BREAKPOINT_INSTRUCTION) {
+			/*
+			 * The breakpoint instruction was removed right
+			 * after we hit it.  Another cpu has removed
+			 * either a probepoint or a debugger breakpoint
+			 * at this address.  In either case, no further
+			 * handling of this interrupt is appropriate.
+			 * Back up over the (now missing) int3 and run
+			 * the original instruction.
+			 */
+			regs->eip -= sizeof(kprobe_opcode_t);
+			ret = 1;
+		}
+		/* Not one of ours: let kernel handle it */
+		goto no_uprobe;
+	}
+
+	ucb->curr_p = p;
+	ucb->uprobe_status = UPROBE_HIT_ACTIVE;
+	ucb->uprobe_saved_eflags = (regs->eflags & (TF_MASK | IF_MASK));
+	ucb->uprobe_old_eflags = (regs->eflags & (TF_MASK | IF_MASK));
+	if (is_IF_modifier(p->opcode))
+		ucb->uprobe_saved_eflags &= ~IF_MASK;
+
+	if (p->pre_handler && p->pre_handler(p, regs))
+		/* handler has already set things up, so skip ss setup */
+		return 1;
+
+	prepare_singlestep_uprobe(current_uprobe, ucb, regs);
+	/*
+	 * Avoid scheduling the current while returning from
+	 * kernel to user mode.
+	 */
+	clear_need_resched();
+	return 1;
+
+no_uprobe:
+	spin_unlock_irqrestore(&uprobe_lock, ucb->flags);
+	preempt_enable_no_resched();
+
+	return ret;
+}
+
+/*
+ * Called after single-stepping.  p->addr is the address of the
+ * instruction whose first byte has been replaced by the "int 3"
+ * instruction.  To avoid the SMP problems that can occur when we
+ * temporarily put back the original opcode to single-step, we
+ * single-stepped a copy of the instruction.  The address of this
+ * copy is p->ainsn.insn.
+ *
+ * This function prepares to return from the post-single-step
+ * interrupt.  We have to fix up the stack as follows:
+ *
+ * 0) Typically, the new eip is relative to the copied instruction.  We
+ * need to make it relative to the original instruction.  Exceptions are
+ * return instructions and absolute or indirect jump or call instructions.
+ *
+ * 1) If the single-stepped instruction was pushfl, then the TF and IF
+ * flags are set in the just-pushed eflags, and may need to be cleared.
+ *
+ * 2) If the single-stepped instruction was a call, the return address
+ * that is atop the stack is the address following the copied instruction.
+ * We need to make it the address following the original instruction.
+ */
+static void __kprobes resume_execution_user(struct kprobe *p,
+		struct pt_regs *regs, struct uprobe_ctlblk *ucb)
+{
+	unsigned long *tos = (unsigned long *)regs->esp;
+	unsigned long next_eip = 0;
+	unsigned long copy_eip = ucb->singlestep_addr;
+	unsigned long orig_eip = (unsigned long)p->addr;
+
+	switch (p->ainsn.insn[0]) {
+	case 0x9c:		/* pushfl */
+		*tos &= ~(TF_MASK | IF_MASK);
+		*tos |= ucb->uprobe_old_eflags;
+		break;
+	case 0xc3:		/* ret/lret */
+	case 0xcb:
+	case 0xc2:
+	case 0xca:
+		regs->eflags &= ~TF_MASK;
+		next_eip = regs->eip;
+		/* eip is already adjusted, no more changes required*/
+		return;
+		break;
+	case 0xe8:		/* call relative - Fix return addr */
+		*tos = orig_eip + (*tos - copy_eip);
+		break;
+	case 0xff:
+		if ((p->ainsn.insn[1] & 0x30) == 0x10) {
+			/* call absolute, indirect */
+			/* Fix return addr; eip is correct. */
+			next_eip = regs->eip;
+			*tos = orig_eip + (*tos - copy_eip);
+		} else if (((p->ainsn.insn[1] & 0x31) == 0x20) ||
+			   ((p->ainsn.insn[1] & 0x31) == 0x21)) {
+			/* jmp near or jmp far  absolute indirect */
+			/* eip is correct. */
+			next_eip = regs->eip;
+		}
+		break;
+	case 0xea:		/* jmp absolute -- eip is correct */
+		next_eip = regs->eip;
+		break;
+	default:
+		break;
+	}
+
+	regs->eflags &= ~TF_MASK;
+	if (next_eip)
+		regs->eip = next_eip;
+	else
+		regs->eip = orig_eip + (regs->eip - copy_eip);
+}
+
+/*
+ * post_uprobe_handler(), executes the user specified handlers and
+ * resumes with the normal execution.
+ */
+static inline int post_uprobe_handler(struct pt_regs *regs)
+{
+	struct kprobe *cur;
+	struct uprobe_ctlblk *ucb;
+
+	if (!current_uprobe)
+		return 0;
+
+	ucb = &uprobe_ctlblk;
+	cur = ucb->curr_p;
+
+	if (!cur)
+		return 0;
+
+	if (cur->post_handler) {
+		if (ucb->uprobe_status == UPROBE_SS_INLINE)
+			ucb->uprobe_status = UPROBE_SSDONE_INLINE;
+		else
+			ucb->uprobe_status = UPROBE_HIT_SSDONE;
+		cur->post_handler(cur, regs, 0);
+	}
+
+	resume_execution_user(cur, regs, ucb);
+	regs->eflags |= ucb->uprobe_saved_eflags;
+
+	if (ucb->uprobe_status == UPROBE_SSDONE_INLINE)
+		replace_original_insn(current_uprobe, regs,
+						BREAKPOINT_INSTRUCTION);
+	else {
+		unlock_page(ucb->upage);
+		pte_unmap(ucb->upte);
+	}
+	current_uprobe = NULL;
+	spin_unlock_irqrestore(&uprobe_lock, ucb->flags);
+	preempt_enable_no_resched();
+	/*
+	 * if somebody else is singlestepping across a probe point, eflags
+	 * will have TF set, in which case, continue the remaining processing
+	 * of do_debug, as if this is not a probe hit.
+	 */
+	if (regs->eflags & TF_MASK)
+		return 0;
+
+	return 1;
+}
+
+static inline int uprobe_fault_handler(struct pt_regs *regs, int trapnr)
+{
+	struct kprobe *cur;
+	struct uprobe_ctlblk *ucb;
+	int ret = 0;
+
+	if (!current_uprobe)
+		return 0;
+
+	ucb = &uprobe_ctlblk;
+	cur = ucb->curr_p;
+
+	if (!cur)
+		return 0;
+
+	if ((ucb->uprobe_status == UPROBE_HIT_SS) ||
+				(ucb->uprobe_status == UPROBE_SS_INLINE)) {
+		if (cur->fault_handler && cur->fault_handler(cur, regs, trapnr))
+			return 1;
+
+		regs->eip = (unsigned long)cur->addr;
+		regs->eflags |= ucb->uprobe_old_eflags;
+		regs->eflags &= ~TF_MASK;
+		replace_original_insn(current_uprobe, regs,
+						BREAKPOINT_INSTRUCTION);
+		current_uprobe = NULL;
+		ret = 1;
+		spin_unlock_irqrestore(&uprobe_lock, ucb->flags);
+		preempt_enable_no_resched();
+	}
+	return ret;
+}
+
+/*
+ * Wrapper routine to for handling exceptions.
+ */
+int __kprobes uprobe_exceptions_notify(struct notifier_block *self,
+				       unsigned long val, void *data)
+{
+	struct die_args *args = (struct die_args *)data;
+	int ret = NOTIFY_DONE;
+
+	if (args->regs->eflags & VM_MASK) {
+		/* We are in virtual-8086 mode. Return NOTIFY_DONE */
+		return ret;
+	}
+
+	switch (val) {
+	case DIE_INT3:
+		if (uprobe_handler(args->regs))
+			ret = NOTIFY_STOP;
+		break;
+	case DIE_DEBUG:
+		if (post_uprobe_handler(args->regs))
+			ret = NOTIFY_STOP;
+		break;
+	case DIE_GPF:
+	case DIE_PAGE_FAULT:
+		if (current_uprobe &&
+		    uprobe_fault_handler(args->regs, args->trapnr))
+			ret = NOTIFY_STOP;
+		break;
+	default:
+		break;
+	}
+	return ret;
+}
+
 int __init arch_init_kprobes(void)
 {
 	return 0;
diff -puN arch/i386/mm/fault.c~kprobes_userspace_probes-ss-out-of-line arch/i386/mm/fault.c
--- linux-2.6.16-rc5-mm2/arch/i386/mm/fault.c~kprobes_userspace_probes-ss-out-of-line	2006-03-06 19:16:45.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/arch/i386/mm/fault.c	2006-03-06 19:16:45.000000000 +0530
@@ -71,8 +71,7 @@ void bust_spinlocks(int yes)
  * 
  * This is slow, but is very rarely executed.
  */
-static inline unsigned long get_segment_eip(struct pt_regs *regs,
-					    unsigned long *eip_limit)
+unsigned long get_segment_eip(struct pt_regs *regs, unsigned long *eip_limit)
 {
 	unsigned long eip = regs->eip;
 	unsigned seg = regs->xcs & 0xffff;
diff -puN include/asm-i386/kprobes.h~kprobes_userspace_probes-ss-out-of-line include/asm-i386/kprobes.h
--- linux-2.6.16-rc5-mm2/include/asm-i386/kprobes.h~kprobes_userspace_probes-ss-out-of-line	2006-03-06 19:16:45.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/include/asm-i386/kprobes.h	2006-03-06 19:16:45.000000000 +0530
@@ -26,6 +26,7 @@
  */
 #include <linux/types.h>
 #include <linux/ptrace.h>
+#include <asm/cacheflush.h>
 
 #define  __ARCH_WANT_KPROBES_INSN_SLOT
 
@@ -77,6 +78,18 @@ struct kprobe_ctlblk {
 	struct prev_kprobe prev_kprobe;
 };
 
+/* per user probe control block */
+struct uprobe_ctlblk {
+	unsigned long uprobe_status;
+	unsigned long uprobe_saved_eflags;
+	unsigned long uprobe_old_eflags;
+	unsigned long singlestep_addr;
+	unsigned long flags;
+	struct kprobe *curr_p;
+	pte_t *upte;
+	struct page *upage;
+};
+
 /* trap3/1 are intr gates for kprobes.  So, restore the status of IF,
  * if necessary, before executing the original int3/1 (trap) handler.
  */
@@ -88,4 +101,9 @@ static inline void restore_interrupts(st
 
 extern int kprobe_exceptions_notify(struct notifier_block *self,
 				    unsigned long val, void *data);
+int uprobe_exceptions_notify(struct notifier_block *self,
+						unsigned long val, void *data);
+extern unsigned long get_segment_eip(struct pt_regs *regs,
+						unsigned long *eip_limit);
+
 #endif				/* _ASM_KPROBES_H */
diff -puN include/linux/kprobes.h~kprobes_userspace_probes-ss-out-of-line include/linux/kprobes.h
--- linux-2.6.16-rc5-mm2/include/linux/kprobes.h~kprobes_userspace_probes-ss-out-of-line	2006-03-06 19:16:45.000000000 +0530
+++ linux-2.6.16-rc5-mm2-prasanna/include/linux/kprobes.h	2006-03-06 19:16:45.000000000 +0530
@@ -51,6 +51,13 @@
 #define KPROBE_REENTER		0x00000004
 #define KPROBE_HIT_SSDONE	0x00000008
 
+/* uprobe_status settings */
+#define UPROBE_HIT_ACTIVE	0x00000001
+#define UPROBE_HIT_SS		0x00000002
+#define UPROBE_HIT_SSDONE	0x00000004
+#define UPROBE_SS_INLINE	0x00000008
+#define UPROBE_SSDONE_INLINE	0x00000010
+
 /* Attach to insert probes on any functions which should be ignored*/
 #define __kprobes	__attribute__((__section__(".kprobes.text")))
 
@@ -183,6 +190,7 @@ struct kretprobe_instance {
 	struct task_struct *task;
 };
 
+extern spinlock_t uprobe_lock;
 extern spinlock_t kretprobe_lock;
 extern struct mutex kprobe_mutex;
 extern int arch_prepare_kprobe(struct kprobe *p);

_
-- 
Prasanna S Panchamukhi
Linux Technology Center
India Software Labs, IBM Bangalore
Email: prasanna@in.ibm.com
Ph: 91-80-51776329

^ permalink raw reply	[flat|nested] 5+ messages in thread

* Re: [PATH 1/3] User space probes-take3-RFC
  2006-03-06 15:09 [PATH 1/3] User space probes-take3-RFC Prasanna S Panchamukhi
  2006-03-06 15:10 ` [PATH 2/3] User space probes-readpage hooks take3-RFC Prasanna S Panchamukhi
@ 2006-03-08  9:35 ` bibo,mao
  2006-03-10 13:59   ` Prasanna S Panchamukhi
  1 sibling, 1 reply; 5+ messages in thread
From: bibo,mao @ 2006-03-08  9:35 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap

[-- Attachment #1: Type: text/plain, Size: 3728 bytes --]

prasanna,
I composed one example to test uprobe, it is in the attachment. And I 
compile uprobe.c with the command:
	gcc --static -g -o uprobe uprobe.c
	readelf --sections uprobe
And in kprobe-ex.c, some environment variant need modification.
	#define PROBE_FILENAME  "/root/test/uprobe"
	/* program text segment start address */
	#define TEXT_START      0x8048150
	/* program text segment offset */	
	#define TEXT_OFFSET     0x150
	/* probed instruction position */
	#define PROBE_POS       0x8048304
And there are some problems about current uprobe patch
1) it influences some operations about probed inode. Because there is 
hook function like readpage(s), when kprobe-ex.ko module is inserted, if 
  some operation like "cp uprobe uprobe1" is executed, file uprobe1 will 
include inserted "int3" instruction.

2) if probed instruction cause stack vma page fault, there will be some 
debug information like this:

void test_stack(){
    int i;
    int p[STACK_LIMIT];

/* set user probepoint here, and default there will
  * be do_page_fault handler to expand stack
  */
	p[0]=0;	
	i=0;
	i++;
}
passed: prehandler executed
BUG: sleeping function called from invalid context at mm/rmap.c:84
in_atomic():0, irqs_disabled():1
  <c014c72f> anon_vma_prepare+0x1b/0xbc   <c011b885> 
release_console_sem+0x1a1/0x1a9
  <c0148e1f> expand_stack+0x15/0xf1   <c02bb83c> do_page_fault+0x282/0x697
  <c02bb5ba> do_page_fault+0x0/0x697   <c0103423> error_code+0x4f/0x54
  <c01adbf6> __copy_to_user_ll+0xcf/0xda   <c02ba977> 
uprobe_exceptions_notify+0x166/0x609
  <c02bbc66> notifier_call_chain+0x15/0x25   <c02ba646> do_int3+0x3c/0x7b
  <c02ba332> int3+0x1e/0x24
passed: post handler executed

3)if probed instruction cause heap vma page fault, the applicatin will 
hangup.
void test_heap(){
    char *pchar;
	
	pchar = (char*) malloc(HEAP_LIMIT);
	pchar += (HEAP_LIMIT - 1);
	*pchar = 'c';  //set probe point here
}
passed: prehandler executed
passed: page fault executed
passed: prehandler executed
BUG: scheduling while atomic: uprobe/0x00000002/7199
  <c02b7dbf> schedule+0x43/0x73f   <c011be4f> vprintk+0x2be/0x2f6
  <c02b8e3f> __wait_on_bit_lock+0x4b/0x52   <c013b011> __lock_page+0x69/0x70
  <c02b8673> io_schedule+0x26/0x30   <c013b094> sync_page+0x0/0x3b
  <c013b0cc> sync_page+0x38/0x3b   <c02b8e1e> __wait_on_bit_lock+0x2a/0x52
  <c013b011> __lock_page+0x69/0x70   <c012bdf1> wake_bit_function+0x0/0x3c
  <c02bab7c> uprobe_exceptions_notify+0x36b/0x609   <c02bbc66> 
notifier_call_chain+0x15/0x25
  <c02ba646> do_int3+0x3c/0x7b   <c02ba332> int3+0x1e/0x24
  <c02b007b> __xfrm_policy_check+0x2a6/0x657
passed: page fault executed
BUG: spinlock wrong owner on CPU#1, syslog-ng/3683
  lock: c0362664, .magic: dead4ead, .owner: uprobe/7199, .owner_cpu: 0
  <c01ae774> _raw_spin_unlock+0x43/0x6c   <c02ba14a> 
_spin_unlock_irqrestore+0x8/0xc
  <c02badad> uprobe_exceptions_notify+0x59c/0x609   <c02bbc66> 
notifier_call_chain+0x15/0x25
  <c02bb5ba> do_page_fault+0x0/0x697   <c02bb775> do_page_fault+0x1bb/0x697
  <c015f786> sys_stat64+0x1e/0x23   <c01344ae> do_gettimeofday+0x31/0xda
  <c02bb5ba> do_page_fault+0x0/0x697   <c0103423> error_code+0x4f/0x54

4) if probed instruction is int3, system will crash.
void test_trap3()
{
/* set probepoint at here */	
	asm volatile (".byte 0xcc");
}
system will crash.


thanks
bibo,mao
Prasanna S Panchamukhi wrote:
> Hi,
> 
> Below is the take 3 of user space probes, which
> seems to work for probes on applications and libraries.
> Needs more testing and code review. This implementation
> uses a spin lock to serialize the probe execution.
> 
> Please provide your review comments on these patchs.
> Thanks to Yanmin, Anil and Jim for their review commoments.
> 
> Thanks
> Prasanna
> 

[-- Attachment #2: uprobe.c --]
[-- Type: text/x-csrc, Size: 1961 bytes --]

#include <stdlib.h>
#include <unistd.h>
#include <stdio.h>
#include <signal.h>
#include <errno.h>
#include <stdlib.h>
#include <string.h>

#define TRAP_FLAG 0x100
# define ENTER_KERNEL   "int $0x80\n\t"
#define STACK_LIMIT	90000
#define HEAP_LIMIT	0x200000

void test_libfunction(const char *ori)
{
   int result;

/* set probepoint at strcmp function, when program is
 * loaded to run, it will load first few page into page,
 * and then load page on demand
 */	
	result = strcmp(ori,"1234");	
}

void test_stack(){
   int i;
   int p[STACK_LIMIT];

/* set user probepoint here, and default there will
 * be do_page_fault handler to expand stack
 */
	p[0]=0;	
	i=0;
	i++;
}

void test_heap(){
   char *pchar;
	
	pchar = (char*) malloc(HEAP_LIMIT);
	pchar += (HEAP_LIMIT - 1); 
	*pchar = 'c';  //set probe point here

}

int test_dummy(int i)
{
	return i+1;
}

void test_functioncall()
{
    int i;
	i=0;
/* set probepoint at function call point */
	test_dummy(i);	
}

void test_functionreturn()
{
   int i;
	i=0;
/* set probepoint at function return point */
	return;	
}

void test_trap3()
{
/* set probepoint at here */	
	asm volatile (".byte 0xcc");
}

void test_syscall()
{
/* set probepoint at int 0x80 point */
	asm volatile (ENTER_KERNEL : : "a" (20) /* getpid() */);
}

void my_trap(int sig)
{
  printf("trap1 : PASS\n");
}

void test_fork(){
   int pid, status;
   char *pchar;

	pchar = (char*) malloc(HEAP_LIMIT);
	pchar += (HEAP_LIMIT - 1);
	pid= fork();
	if (pid == 0){
	/* here it will generate COW, and set probepoint here */
		*pchar = 'c';
		exit(0);
	}
	else{
		wait(&status);
	}
   
}

int main()
{
  int result,pid;
  int status;
  int i;
  char *pchar;

  	signal(SIGTRAP, my_trap);

/* set probepoint at i=i+1 point */
  	i=i+1;
	pchar = "12345";
	test_libfunction(pchar);

	test_stack();

	test_heap();

	test_functioncall();

	test_functionreturn();

	test_trap3();

	test_syscall();
	
	test_fork();	
	printf("finished\n");
	return 0;
}


[-- Attachment #3: kprobe-ex.c --]
[-- Type: text/x-csrc, Size: 1848 bytes --]

/*kprobe_example.c*/
#include <linux/kernel.h>
#include <linux/module.h>
#include <linux/kprobes.h>
#include <linux/kallsyms.h>
#include <linux/sched.h>

#define PROBE_FILENAME	"/root/test/uprobe"
#define TEXT_START 	0x8048150
#define TEXT_OFFSET 	0x150
#define PROBE_POS	0x8048304
/*For each probe you need to allocate a kprobe structure*/
struct uprobe  uprobe_ex;

/*kprobe pre_handler: called just before the probed instruction is executed*/
int handler_pre(struct kprobe *p, struct pt_regs *regs)
{
        printk("passed: prehandler executed \n");

        return 0;
}

/*kprobe post_handler: called after the probed instruction is executed*/
void handler_post(struct kprobe *p, struct pt_regs *regs, unsigned long flags)
{
	printk("passed: post handler executed \n");
}

/* fault_handler: this is called if an exception is generated for any
 * instruction within the pre- or post-handler, or when Kprobes
 * single-steps the probed instruction.
 */
int handler_fault(struct kprobe *p, struct pt_regs *regs, int trapnr)
{
        printk("passed: page fault executed \n");
        return 0;
}

int init_module(void)
{
        int ret;
	char *pname = PROBE_FILENAME;

	uprobe_ex.kp.pre_handler = handler_pre;
        uprobe_ex.kp.post_handler = handler_post;
        uprobe_ex.kp.fault_handler = handler_fault;
	//uprobe_ex.kp.opcode = 0;

        uprobe_ex.kp.addr = (kprobe_opcode_t*)PROBE_POS;
	uprobe_ex.offset = (PROBE_POS - TEXT_START + TEXT_OFFSET);
	uprobe_ex.pathname = pname;

        if ((ret = register_uprobe(&uprobe_ex) < 0)) {
                printk("register_uprobe failed, returned %d\n", ret);
                return -1;
        }
        printk("uprobe registered\n");
        return 0;
}

void cleanup_module(void)
{
        unregister_uprobe(&uprobe_ex);
        printk("uprobe unregistered\n");
}

MODULE_LICENSE("GPL");


^ permalink raw reply	[flat|nested] 5+ messages in thread

* Re: [PATH 1/3] User space probes-take3-RFC
  2006-03-08  9:35 ` [PATH 1/3] User space probes-take3-RFC bibo,mao
@ 2006-03-10 13:59   ` Prasanna S Panchamukhi
  0 siblings, 0 replies; 5+ messages in thread
From: Prasanna S Panchamukhi @ 2006-03-10 13:59 UTC (permalink / raw)
  To: bibo,mao; +Cc: systemtap


Bibo,

Thanks for writting excellent test cases.
Please find my comments inline below.

On Wed, Mar 08, 2006 at 05:27:50PM +0800, bibo,mao wrote:
> prasanna,
> I composed one example to test uprobe, it is in the attachment. And I 
> compile uprobe.c with the command:
> 	gcc --static -g -o uprobe uprobe.c
> 	readelf --sections uprobe
> And in kprobe-ex.c, some environment variant need modification.
> 	#define PROBE_FILENAME  "/root/test/uprobe"
> 	/* program text segment start address */
> 	#define TEXT_START      0x8048150
> 	/* program text segment offset */	
> 	#define TEXT_OFFSET     0x150
> 	/* probed instruction position */
> 	#define PROBE_POS       0x8048304
> And there are some problems about current uprobe patch
> 1) it influences some operations about probed inode. Because there is 
> hook function like readpage(s), when kprobe-ex.ko module is inserted, if 
>  some operation like "cp uprobe uprobe1" is executed, file uprobe1 will 
> include inserted "int3" instruction.


Looks like this problem was addressed in the original dprobes, We are
investigating this issue.

> 
> 2) if probed instruction cause stack vma page fault, there will be some 
> debug information like this:
> 
> void test_stack(){
>    int i;
>    int p[STACK_LIMIT];
> 
> /* set user probepoint here, and default there will
>  * be do_page_fault handler to expand stack
>  */
> 	p[0]=0;	
> 	i=0;
> 	i++;
> }
> passed: prehandler executed
> BUG: sleeping function called from invalid context at mm/rmap.c:84
> in_atomic():0, irqs_disabled():1
>  <c014c72f> anon_vma_prepare+0x1b/0xbc   <c011b885> 
> release_console_sem+0x1a1/0x1a9
>  <c0148e1f> expand_stack+0x15/0xf1   <c02bb83c> do_page_fault+0x282/0x697
>  <c02bb5ba> do_page_fault+0x0/0x697   <c0103423> error_code+0x4f/0x54
>  <c01adbf6> __copy_to_user_ll+0xcf/0xda   <c02ba977> 
> uprobe_exceptions_notify+0x166/0x609
>  <c02bbc66> notifier_call_chain+0x15/0x25   <c02ba646> do_int3+0x3c/0x7b
>  <c02ba332> int3+0x1e/0x24
> passed: post handler executed


There are two solutions to the above problem ie page fault while *single
stepping*.

1. Reset the probe to the probed address and allow it to recurse
   'n' times. Disarm it on (n+1)th fault and release locks.

2. Disarm the probe permanently and inform the user right away using
   some flags and release locks.

> 
> 3)if probed instruction cause heap vma page fault, the applicatin will 
> hangup.
> void test_heap(){
>    char *pchar;
> 	
> 	pchar = (char*) malloc(HEAP_LIMIT);
> 	pchar += (HEAP_LIMIT - 1);
> 	*pchar = 'c';  //set probe point here
> }
> passed: prehandler executed
> passed: page fault executed
> passed: prehandler executed
> BUG: scheduling while atomic: uprobe/0x00000002/7199
>  <c02b7dbf> schedule+0x43/0x73f   <c011be4f> vprintk+0x2be/0x2f6
>  <c02b8e3f> __wait_on_bit_lock+0x4b/0x52   <c013b011> __lock_page+0x69/0x70
>  <c02b8673> io_schedule+0x26/0x30   <c013b094> sync_page+0x0/0x3b
>  <c013b0cc> sync_page+0x38/0x3b   <c02b8e1e> __wait_on_bit_lock+0x2a/0x52
>  <c013b011> __lock_page+0x69/0x70   <c012bdf1> wake_bit_function+0x0/0x3c
>  <c02bab7c> uprobe_exceptions_notify+0x36b/0x609   <c02bbc66> 
> notifier_call_chain+0x15/0x25
>  <c02ba646> do_int3+0x3c/0x7b   <c02ba332> int3+0x1e/0x24
>  <c02b007b> __xfrm_policy_check+0x2a6/0x657
> passed: page fault executed
> BUG: spinlock wrong owner on CPU#1, syslog-ng/3683
>  lock: c0362664, .magic: dead4ead, .owner: uprobe/7199, .owner_cpu: 0
>  <c01ae774> _raw_spin_unlock+0x43/0x6c   <c02ba14a> 
> _spin_unlock_irqrestore+0x8/0xc
>  <c02badad> uprobe_exceptions_notify+0x59c/0x609   <c02bbc66> 
> notifier_call_chain+0x15/0x25
>  <c02bb5ba> do_page_fault+0x0/0x697   <c02bb775> do_page_fault+0x1bb/0x697
>  <c015f786> sys_stat64+0x1e/0x23   <c01344ae> do_gettimeofday+0x31/0xda
>  <c02bb5ba> do_page_fault+0x0/0x697   <c0103423> error_code+0x4f/0x54
> 


Same as above.

> 4) if probed instruction is int3, system will crash.
> void test_trap3()
> {
> /* set probepoint at here */	
> 	asm volatile (".byte 0xcc");
> }
> system will crash.

This scenerio is similar to the case where in a debugger (like gdb)
puts a breakpoint and then the user also want to insert the probe
at the same address.

As was discussed a while back, we will not allow a probe to be inserted
at an address that already has a breakpoint. This ofcourse will need
assistance from other user-space debuggers.

As of now we will not allow the probes to be inserted on an existing
breakpoint.

I will send out a patch to fix the above problems.

Thanks
Prasanna

-- 
Prasanna S Panchamukhi
Linux Technology Center
India Software Labs, IBM Bangalore
Email: prasanna@in.ibm.com
Ph: 91-80-51776329

^ permalink raw reply	[flat|nested] 5+ messages in thread

end of thread, other threads:[~2006-03-10 13:59 UTC | newest]

Thread overview: 5+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2006-03-06 15:09 [PATH 1/3] User space probes-take3-RFC Prasanna S Panchamukhi
2006-03-06 15:10 ` [PATH 2/3] User space probes-readpage hooks take3-RFC Prasanna S Panchamukhi
2006-03-06 15:11   ` [PATH 3/3] User space probes-single stepping out-of-line take3-RFC Prasanna S Panchamukhi
2006-03-08  9:35 ` [PATH 1/3] User space probes-take3-RFC bibo,mao
2006-03-10 13:59   ` Prasanna S Panchamukhi

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for read-only IMAP folder(s) and NNTP newsgroup(s).