public inbox for systemtap@sourceware.org
 help / color / mirror / Atom feed
* [1/3] Userspace probes prototype-take2
@ 2006-02-08 14:10 Prasanna S Panchamukhi
  2006-02-08 14:12 ` [2/3] " Prasanna S Panchamukhi
  0 siblings, 1 reply; 11+ messages in thread
From: Prasanna S Panchamukhi @ 2006-02-08 14:10 UTC (permalink / raw)
  To: systemtap

Here is take-2 on the user space probes prototype.
I have taken care of most of the comments from Yanmin Zhang.
Presently this patch set does not work with Config PREEMPT
enabled kernels. Next patch release will fix this.

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

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 address should be
			 substracted from the beginning 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.

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.

	const char *pname = "/home/prasanna/bin/myapp";

	p.kp.pre_handler=handler_pre;
	p.kp.post_handler=handler_post;
	p.kp.fault_handler=handler_fault;

	/* Specify the offset within the application/executable*/
	p.offset = (unsigned long)0x4d4;
	/* Secify the probe address */
	/* $nm appln |grep func1 */
	p.kp.addr = (kprobe_opcode_t *)0x080484d4;
	/*pointer to the pathname of the application */
	p.pathname = pname;
	/* Now register the userspace probe */
	register_uprobe(&p);

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

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

diff -puN include/linux/kprobes.h~kprobes_userspace_probes-base-interface include/linux/kprobes.h

 include/linux/namei.h   |    1 

diff -puN include/linux/kprobes.h~kprobes_userspace_probes-base-interface include/linux/kprobes.h


 fs/namei.c              |   11 -
 include/linux/kprobes.h |   36 +++
 include/linux/namei.h   |    1 
 kernel/kprobes.c        |  465 +++++++++++++++++++++++++++++++++++++++++++++---
 4 files changed, 489 insertions(+), 24 deletions(-)

diff -puN fs/namei.c~kprobes_userspace_probes-base-interface fs/namei.c
--- linux-2.6.16-rc1-mm5/fs/namei.c~kprobes_userspace_probes-base-interface	2006-02-08 19:20:27.000000000 +0530
+++ linux-2.6.16-rc1-mm5-prasanna/fs/namei.c	2006-02-08 19:20:27.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-rc1-mm5/include/linux/kprobes.h~kprobes_userspace_probes-base-interface	2006-02-08 19:20:27.000000000 +0530
+++ linux-2.6.16-rc1-mm5-prasanna/include/linux/kprobes.h	2006-02-08 19:20:27.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>
@@ -117,6 +121,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,6 +192,9 @@ 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, unsigned long *address);
+extern void arch_arm_uprobe(unsigned long *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);
@@ -194,6 +227,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-rc1-mm5/include/linux/namei.h~kprobes_userspace_probes-base-interface	2006-02-08 19:20:27.000000000 +0530
+++ linux-2.6.16-rc1-mm5-prasanna/include/linux/namei.h	2006-02-08 19:20:27.000000000 +0530
@@ -82,6 +82,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-rc1-mm5/kernel/kprobes.c~kprobes_userspace_probes-base-interface	2006-02-08 19:20:27.000000000 +0530
+++ linux-2.6.16-rc1-mm5-prasanna/kernel/kprobes.c	2006-02-08 19:20:27.000000000 +0530
@@ -47,10 +47,13 @@
 
 static struct hlist_head kprobe_table[KPROBE_TABLE_SIZE];
 static struct hlist_head kretprobe_inst_table[KPROBE_TABLE_SIZE];
+static struct list_head uprobe_module_list;
 
 DEFINE_MUTEX(kprobe_mutex);		/* Protects kprobe_table */
+DEFINE_MUTEX(uprobe_mutex);		/* Protects uprobe_module */
 DEFINE_SPINLOCK(kretprobe_lock);	/* Protects kretprobe_inst_table */
 static DEFINE_PER_CPU(struct kprobe *, kprobe_instance) = NULL;
+extern DEFINE_PER_CPU(struct uprobe *, current_uprobe);
 
 #ifdef __ARCH_WANT_KPROBES_INSN_SLOT
 /*
@@ -166,26 +169,6 @@ static inline void reset_kprobe_instance
 }
 
 /*
- * This routine is called either:
- * 	- under the kprobe_mutex - during kprobe_[un]register()
- * 				OR
- * 	- with preemption disabled - from arch/xxx/kernel/kprobes.c
- */
-struct kprobe __kprobes *get_kprobe(void *addr)
-{
-	struct hlist_head *head;
-	struct hlist_node *node;
-	struct kprobe *p;
-
-	head = &kprobe_table[hash_ptr(addr, KPROBE_HASH_BITS)];
-	hlist_for_each_entry_rcu(p, node, head, hlist) {
-		if (p->addr == addr)
-			return p;
-	}
-	return NULL;
-}
-
-/*
  * Aggregate handlers for multiple kprobes support - these handlers
  * take care of invoking the individual kprobe handlers on p->list
  */
@@ -248,6 +231,107 @@ static int __kprobes aggr_break_handler(
 	return ret;
 }
 
+/**
+ * This routine looks for an existing uprobe at the given offset and inode.
+ * If it's found, returns the corresponding kprobe pointer.
+ */
+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 = &kprobe_table[hash_ptr((kprobe_opcode_t *)
+			(((unsigned long)inode) * offset), KPROBE_HASH_BITS)];
+
+	hlist_for_each_entry(p, node, head, hlist) {
+		if (kernel_text_address((unsigned long)p->addr))
+			continue;
+
+		if (p->pre_handler == aggr_pre_handler) {
+			kpr = list_entry(rcu_dereference(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.
+ */
+static 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;
+
+	down_read(&mm->mmap_sem);
+	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) {
+		up_read(&mm->mmap_sem);
+		return NULL;
+	}
+
+	inode = vma->vm_file->f_dentry->d_inode;
+	up_read(&mm->mmap_sem);
+
+	p = get_kprobe_user(inode, offset);
+	if (!p)
+		return NULL;
+
+	if (p->pre_handler == aggr_pre_handler) {
+		kpr = list_entry(rcu_dereference(p)->list.next, typeof(*kpr),
+									list);
+		uprobe = container_of(kpr, struct uprobe, kp);
+	} else
+                uprobe = container_of(p, struct uprobe, kp);
+
+	if (uprobe)
+		 __get_cpu_var(current_uprobe) = uprobe;
+
+	return p;
+}
+
+/*
+ * This routine is called either:
+ *	- under the kprobe_mutex - during kprobe_[un]register()
+ *				OR
+ *	- with preemption disabled - from arch/xxx/kernel/kprobes.c
+ */
+struct kprobe __kprobes *get_kprobe(void *addr)
+{
+	struct hlist_head *head;
+	struct hlist_node *node;
+	struct kprobe *p;
+
+	if (!kernel_text_address((unsigned long)addr))
+		return get_uprobe(addr);
+
+	head = &kprobe_table[hash_ptr(addr, KPROBE_HASH_BITS)];
+	hlist_for_each_entry_rcu(p, node, head, hlist) {
+		if (p->addr == addr)
+			return p;
+	}
+	return NULL;
+}
+
 /* Walks the list and increments nmissed count for multiprobe case */
 void __kprobes kprobes_inc_nmissed_count(struct kprobe *p)
 {
@@ -559,6 +643,342 @@ void __kprobes unregister_jprobe(struct 
 	unregister_kprobe(&jp->kp);
 }
 
+typedef int (*process_uprobe_func_t)(struct uprobe *uprobe,
+				unsigned long *address, struct page *page);
+
+/**
+ * Adds the kprobe structure for the specified uprobe to either the
+ * kprobe_table or to the aggregate hash list for a given inode and offset.
+ * Also copies the instructions and inserts the breakpoint.
+ */
+int __kprobes insert_kprobe_user(struct uprobe *uprobe, unsigned long *address,
+							struct page *page)
+{
+	struct kprobe *old_p;
+	struct hlist_head *head;
+
+	uprobe->kp.nmissed = 0;
+	old_p = get_kprobe_user(uprobe->inode, uprobe->offset);
+	if (old_p)
+		return register_aggr_kprobe(old_p, &uprobe->kp);
+
+	head = &kprobe_table[hash_ptr((kprobe_opcode_t *)(uprobe->offset *
+			(unsigned long)uprobe->inode), KPROBE_HASH_BITS)];
+
+	INIT_HLIST_NODE(&uprobe->kp.hlist);
+	hlist_add_head_rcu(&uprobe->kp.hlist, head);
+
+	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;
+	unsigned long *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 = kmap(page);
+	uprobe_address = (unsigned long *)((unsigned long)uprobe_address +
+				(unsigned long) (uprobe->offset & ~PAGE_MASK));
+	ret = (*process_kprobe_user)(uprobe, uprobe_address, page);
+	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;
+
+	vma_prio_tree_foreach(vma, &iter, head, offset, offset) {
+		mm = vma->vm_mm;
+		down_read(&mm->mmap_sem);
+		spin_lock(&mapping->i_mmap_lock);
+
+
+		spin_lock(&mm->page_table_lock);
+		start = vma->vm_start - (vma->vm_pgoff << PAGE_SHIFT);
+		end = vma->vm_end - (vma->vm_pgoff << PAGE_SHIFT);
+		spin_unlock(&mm->page_table_lock);
+
+		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);
+	up_read(&mm->mmap_sem);
+	}
+}
+
+/**
+ * Check if the given offset lies within the given page range.
+ */
+static inline int find_page_probe(unsigned long offset,
+						unsigned long page_start)
+{
+	unsigned long page_end = page_start + PAGE_SIZE;
+
+	if ((offset >= page_start) && (offset < page_end))
+		return 1;
+
+	return 0;
+}
+
+/**
+ * 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;
+}
+
+/**
+ * Get the inode operations. This function leaves with the dentry held
+ * and taking with the inode writelock held to ensure that the file on
+ * which probes are currently active does not change from under us. 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)
+{
+	INIT_HLIST_HEAD(&umodule->ulist_head);
+	hlist_add_head(&uprobe->ulist, &umodule->ulist_head);
+	list_add(&umodule->mlist, &uprobe_module_list);
+}
+
+int __kprobes remove_kprobe_user(struct uprobe *uprobe, unsigned long *address,
+				struct page *page)
+{
+	struct kprobe *old_p, *list_p, *p;
+	int cleanup_p;
+
+	p = &uprobe->kp;
+	mutex_lock(&kprobe_mutex);
+	old_p = get_kprobe_user(uprobe->inode, uprobe->offset);
+	if (unlikely(!old_p)) {
+		mutex_unlock(&kprobe_mutex);
+		return 0;
+	}
+
+	if (p != old_p) {
+		list_for_each_entry_rcu(list_p, &old_p->list, list)
+			if (list_p == p)
+			/* kprobe p is a valid probe */
+				goto valid_p;
+		mutex_unlock(&kprobe_mutex);
+		return 0;
+	}
+
+valid_p:
+	if ((old_p == p) || ((old_p->pre_handler == aggr_pre_handler) &&
+		(p->list.next == &old_p->list) &&
+		(p->list.prev == &old_p->list))) {
+		/* Only probe on the hash list */
+		arch_disarm_uprobe(p, (kprobe_opcode_t *)address);
+		hlist_del_rcu(&old_p->hlist);
+		cleanup_p = 1;
+	} else {
+		list_del_rcu(&p->list);
+		cleanup_p = 0;
+	}
+
+	mutex_unlock(&kprobe_mutex);
+
+	synchronize_sched();
+	if (cleanup_p) {
+		if (p != old_p) {
+			list_del_rcu(&p->list);
+			kfree(old_p);
+		}
+	}
+
+	return 0;
+}
+
+/**
+ * unregister_uprobe: Disarms the probe, removes the kprobe and uprobe
+ * pointers from the hash lists. Unhooks readpage routines.
+ */
+void __kprobes unregister_uprobe(struct uprobe *uprobe)
+{
+	struct address_space *mapping;
+	struct uprobe_module *umodule;
+	struct page *page;
+	int ret = 0;
+
+	if (!uprobe->inode)
+		return;
+
+	mapping = uprobe->inode->i_mapping;
+
+	page = find_get_page(mapping, uprobe->offset >> PAGE_CACHE_SHIFT);
+
+	ret = map_uprobe_page(page, uprobe, remove_kprobe_user);
+	/*
+	 * TODO: unregister_uprobe should not fail, need to handle if it fails.
+	 */
+	flush_vma(mapping, page, uprobe);
+
+	if (page)
+		page_cache_release(page);
+
+	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);
+}
+
+/**
+ * register_uprobe(): combination of inode and offset is used to identify each
+ * probe uniquely. Each uprobe can be found from the kprobes_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.
+	 * uprobe->offset = (unsigned long)uprobe->kp.addr & UPROBE_OFFSET_MASK;
+	 */
+	if ((error = path_lookup(uprobe->pathname, LOOKUP_FOLLOW, &nd)))
+		return error;
+
+	inode = nd.dentry->d_inode;
+
+	error = ex_write_lock(inode);
+	if (error) {
+		path_release(&nd);
+		goto out;
+	}
+
+	mutex_lock(&uprobe_mutex);
+	/*
+	 * 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) {
+		/*
+		 * Allocate a uprobe_module structure for this application
+		 * if not allocated before.
+		 */
+		umodule = kzalloc(sizeof(struct uprobe_module), GFP_KERNEL);
+		if (!umodule) {
+			error = -ENOMEM;
+			path_release(&nd);
+			ex_write_unlock(inode);
+			goto out;
+		}
+		memcpy(&umodule->nd, &nd, sizeof(struct nameidata));
+		get_inode_ops(uprobe, umodule);
+	} else {
+		ex_write_unlock(inode);
+		hlist_add_head(&uprobe->ulist, &umodule->ulist_head);
+	}
+
+	uprobe->inode = inode;
+	mapping = inode->i_mapping;
+	mutex_lock(&kprobe_mutex);
+	page = find_get_page(mapping, (uprobe->offset >> PAGE_CACHE_SHIFT));
+
+	/*
+	 * If error == -EINVAL, return success, probes will inserted by
+	 * readpage hooks.
+	 * TODO: Use a more suitable errno?
+	 */
+	error = map_uprobe_page(page, uprobe, insert_kprobe_user);
+	if (error == -EINVAL)
+		error = 0;
+	flush_vma(mapping, page, uprobe);
+
+	if (page)
+		page_cache_release(page);
+
+	mutex_unlock(&kprobe_mutex);
+out:
+	mutex_unlock(&uprobe_mutex);
+
+	return error;
+}
+
 #ifdef ARCH_SUPPORTS_KRETPROBES
 
 /*
@@ -650,6 +1070,8 @@ static int __init init_kprobes(void)
 		INIT_HLIST_HEAD(&kretprobe_inst_table[i]);
 	}
 
+	/* initialize uprobe_module_list */
+	INIT_LIST_HEAD(&uprobe_module_list);
 	err = arch_init_kprobes();
 	if (!err)
 		err = register_die_notifier(&kprobe_exceptions_nb);
@@ -666,4 +1088,5 @@ 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] 11+ messages in thread

* Re: [2/3] Userspace probes prototype-take2
  2006-02-08 14:10 [1/3] Userspace probes prototype-take2 Prasanna S Panchamukhi
@ 2006-02-08 14:12 ` Prasanna S Panchamukhi
  2006-02-08 14:13   ` [3/3] " Prasanna S Panchamukhi
  0 siblings, 1 reply; 11+ messages in thread
From: Prasanna S Panchamukhi @ 2006-02-08 14:12 UTC (permalink / raw)
  To: systemtap


This patch provides the feature to insert the probes on the pages
that are not present in the memory during registeration.

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

User space probe also allows the probes to be inserted within the
pages even that are not present in the memory at the time of
registration. This is done by adding hooks to readpage and readpages
routines. During registration, the address space operation object is
modified by replacing with userspace probes's specific readpage and
readpages routines. So that when the pages are readinto the memory
through the readpage and readpages address space operations, the
probes can be automatically inserted into those pages. These user
space readpage and readpages routines internally call the original
readpage() and readpages() routines and then check if the probes are
to be added to these pages and then insert the probes on these pages.
Overhead of adding these hooks are limited to the application on
which probes are inserted.

While unregiteration, care should be taken to replace the readpage and
readpages() hooks by original routines if no probes exists on that
application.

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

 include/linux/namei.h |    1 

diff -puN kernel/kprobes.c~kprobes_userspace_probes_hook_readpage kernel/kprobes.c

diff -puN kernel/kprobes.c~kprobes_userspace_probes_hook_readpage kernel/kprobes.c


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

diff -puN kernel/kprobes.c~kprobes_userspace_probes_hook_readpage kernel/kprobes.c
--- linux-2.6.16-rc1-mm5/kernel/kprobes.c~kprobes_userspace_probes_hook_readpage	2006-02-08 19:21:09.000000000 +0530
+++ linux-2.6.16-rc1-mm5-prasanna/kernel/kprobes.c	2006-02-08 19:21:09.000000000 +0530
@@ -769,6 +769,110 @@ static struct uprobe_module __kprobes *g
 	return NULL;
 }
 
+static inline void insert_readpage_uprobe(struct page *page,
+	struct address_space *mapping, struct uprobe *uprobe)
+{
+	if (find_page_probe(uprobe->offset >> PAGE_CACHE_SHIFT,
+				page->index << PAGE_CACHE_SHIFT)) {
+		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) {
+		printk("uprobe_readpages: we don't have a module \
+				associated with this file.. Aborting\n");
+		retval = -EINVAL;
+		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().
+	 */
+	mutex_lock(&kprobe_mutex);
+	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);
+	}
+	mutex_unlock(&kprobe_mutex);
+
+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) {
+		printk("uprobe_readpages: we don't have a module \
+				associated with this file.. Aborting\n");
+		retval = -EINVAL;
+		goto out;
+	}
+
+	/* call original readpage() */
+	retval = umodule->ori_a_ops->readpage(file, page);
+	if (retval < 0)
+		goto out;
+
+	mutex_lock(&kprobe_mutex);
+	hlist_for_each_entry(uprobe, node, &umodule->ulist_head, ulist) {
+		if (!uprobe->kp.opcode)
+			insert_readpage_uprobe(page, mapping, uprobe);
+	}
+	mutex_unlock(&kprobe_mutex);
+
+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,
@@ -799,9 +903,17 @@ static inline int ex_write_unlock(struct
 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;
 }
 
 int __kprobes remove_kprobe_user(struct uprobe *uprobe, unsigned long *address,
@@ -811,6 +923,7 @@ int __kprobes remove_kprobe_user(struct 
 	int cleanup_p;
 
 	p = &uprobe->kp;
+	 /*TODO: change it to spin lock, we enter here holding i_mmap_lock */
 	mutex_lock(&kprobe_mutex);
 	old_p = get_kprobe_user(uprobe->inode, uprobe->offset);
 	if (unlikely(!old_p)) {
@@ -887,6 +1000,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] 11+ messages in thread

* Re: [3/3] Userspace probes prototype-take2
  2006-02-08 14:12 ` [2/3] " Prasanna S Panchamukhi
@ 2006-02-08 14:13   ` Prasanna S Panchamukhi
  0 siblings, 0 replies; 11+ messages in thread
From: Prasanna S Panchamukhi @ 2006-02-08 14:13 UTC (permalink / raw)
  To: systemtap


This patch handles the executing the registered callback
functions when probes is hit.

	Each userspace probe is uniquely identified by the
combination of inode and offset, hence during registeration the inode
and offset combination is added to kprobes hash table. Initially when
breakpoint instruction is hit, the kprobes hash table is looked up
for matching inode and offset. The pre_handlers are called in sequence
if multiple probes are registered. The original instruction is single
stepped out-of-line similar to kernel probes. In kernel space probes,
single stepping out-of-line is achieved by copying the instruction on
to some location within kernel address space and then single step
from that location. But for userspace probes, instruction copied
into kernel address space cannot be single stepped, hence the
instruction should 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. If the stack grows
downwards, the stack space below the stack pointer indicates the
unused stack free space and if the stack grows upwards, the stack
space above the stack pointer indicates the unused stack free space.

The instruction to be single stepped can modify the stack space, hence
before using the unused stack free 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 unused 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. This patch is not stable yet, should work for most conditions.

2. This patch works only with PREEMPT config option disabled, to work
in PREEMPT enabled condition handlers must be re-written and must
be seperated out from kernel probes allowing preemption.

3. 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.

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

5. Resume exectution should handle setting of proper eip and eflags
for special instructions similar to kernel probes.

6. Single stepping out-of-line expands the stack if there is no
enough stack space to copy the original instruction. Expansion of
stack should be shrinked back to the original size after single
stepping or the expanded stack should be reused for single stepping
out-of-line for other probes.

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

8. Handing of page faults while inthe kprobes_handler() and while
single stepping.

9. Accessing user space pages not present in memory, from the
registered callback routines.

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


 arch/i386/kernel/kprobes.c |  460 +++++++++++++++++++++++++++++++++++++++++++--
 include/asm-i386/kprobes.h |   13 +
 include/linux/kprobes.h    |    7 
 kernel/kprobes.c           |    3 
 4 files changed, 468 insertions(+), 15 deletions(-)

diff -puN arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line arch/i386/kernel/kprobes.c
--- linux-2.6.16-rc1-mm5/arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
+++ linux-2.6.16-rc1-mm5-prasanna/arch/i386/kernel/kprobes.c	2006-02-08 19:26:10.000000000 +0530
@@ -30,6 +30,7 @@
 
 #include <linux/config.h>
 #include <linux/kprobes.h>
+#include <linux/hash.h>
 #include <linux/ptrace.h>
 #include <linux/preempt.h>
 #include <asm/cacheflush.h>
@@ -38,8 +39,12 @@
 
 void jprobe_return_end(void);
 
+static struct uprobe_page *uprobe_page;
+static struct hlist_head uprobe_page_table[KPROBE_TABLE_SIZE];
 DEFINE_PER_CPU(struct kprobe *, current_kprobe) = NULL;
 DEFINE_PER_CPU(struct kprobe_ctlblk, kprobe_ctlblk);
+DEFINE_PER_CPU(struct uprobe *, current_uprobe) = NULL;
+DEFINE_PER_CPU(unsigned long, singlestep_addr);
 
 /* insert a jmp code */
 static inline void set_jmp_op(void *from, void *to)
@@ -125,6 +130,23 @@ void __kprobes arch_disarm_kprobe(struct
 			   (unsigned long) p->addr + sizeof(kprobe_opcode_t));
 }
 
+void __kprobes arch_disarm_uprobe(struct kprobe *p, kprobe_opcode_t *address)
+{
+	*address = p->opcode;
+}
+
+void __kprobes arch_arm_uprobe(unsigned long *address)
+{
+	*(kprobe_opcode_t *)address = BREAKPOINT_INSTRUCTION;
+}
+
+void __kprobes arch_copy_uprobe(struct kprobe *p, unsigned long *address)
+{
+	memcpy(p->ainsn.insn, (kprobe_opcode_t *)address,
+				MAX_INSN_SIZE * sizeof(kprobe_opcode_t));
+	p->opcode = *(kprobe_opcode_t *)address;
+}
+
 static inline void save_previous_kprobe(struct kprobe_ctlblk *kcb)
 {
 	kcb->prev_kprobe.kp = kprobe_running();
@@ -151,15 +173,326 @@ static inline void set_current_kprobe(st
 		kcb->kprobe_saved_eflags &= ~IF_MASK;
 }
 
+struct uprobe_page __kprobes *get_upage_current(struct task_struct *tsk)
+{
+	struct hlist_head *head;
+	struct hlist_node *node;
+	struct uprobe_page *upage;
+
+	head = &uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)];
+	hlist_for_each_entry(upage, node, head, hlist) {
+		if (upage->tsk == tsk)
+			return upage;
+        }
+	return NULL;
+}
+
+struct uprobe_page __kprobes *get_upage_free(struct task_struct *tsk)
+{
+	int cpu;
+
+	for_each_cpu(cpu) {
+		struct uprobe_page *upage;
+		upage = per_cpu_ptr(uprobe_page, cpu);
+		if (upage->status & UPROBE_PAGE_FREE)
+			return upage;
+	}
+	return NULL;
+}
+
+/**
+ * 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, it just maps a new page and copies the
+ *  new 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, *vaddr, stack_addr = regs->esp;
+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
+	struct uprobe_page *upage;
+	struct page *page;
+	pte_t *pte;
+
+
+	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;
+
+	preempt_enable_no_resched();
+
+	pte = get_uprobe_pte(addr);
+	preempt_disable();
+	if (!pte)
+		return -EFAULT;
+
+	upage = get_upage_free(current);
+	upage->status &= ~UPROBE_PAGE_FREE;
+	upage->tsk = current;
+	INIT_HLIST_NODE(&upage->hlist);
+	hlist_add_head(&upage->hlist,
+		&uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)]);
+
+	upage->orig_pte = pte;
+	upage->orig_pte_val =  pte_val(*pte);
+	set_pte(pte, (*(upage->alias_pte)));
+
+	page = pte_page(*pte);
+	vaddr = (unsigned long *)kmap_atomic(page, KM_USER1);
+	vaddr = (unsigned long *)((unsigned long)vaddr + (addr & ~PAGE_MASK));
+	memcpy(vaddr, (unsigned long *)uprobe->kp.ainsn.insn, size);
+	kunmap_atomic(vaddr, KM_USER1);
+	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, *vaddr, vm_addr;
+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
+	struct vm_area_struct *new_vma;
+	struct uprobe_page *upage;
+	struct mm_struct *mm = current->mm;
+	struct page *page;
+	pte_t *pte;
+
+
+	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
+		return -EFAULT;
+
+	preempt_enable_no_resched();
+
+	/* TODO: do we need to expand stack if extend_vma fails? */
+	new_vma = find_extend_vma(mm, vm_addr);
+	preempt_disable();
+	if (!new_vma)
+		return -ENOMEM;
+
+	/*
+	 * TODO: Expanding stack for every probe is not a good idea, stack must
+	 * either be shrunk to its original size after single stepping or the
+	 * expanded stack should be kept track of, for the probed application,
+	 * so it can be reused to single step out-of-line
+	 */
+	if (new_vma->vm_flags & VM_GROWSDOWN)
+		addr = new_vma->vm_start;
+	else
+		addr = new_vma->vm_end - size;
+
+	preempt_enable_no_resched();
+	pte = get_uprobe_pte(addr);
+	preempt_disable();
+	if (!pte)
+		return -EFAULT;
+
+	upage = get_upage_free(current);
+	upage->status &= ~UPROBE_PAGE_FREE;
+	upage->tsk = current;
+	INIT_HLIST_NODE(&upage->hlist);
+	hlist_add_head(&upage->hlist,
+		&uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)]);
+	upage->orig_pte = pte;
+	upage->orig_pte_val =  pte_val(*pte);
+	set_pte(pte, (*(upage->alias_pte)));
+
+	page = pte_page(*pte);
+	vaddr = (unsigned long *)kmap_atomic(page, KM_USER1);
+	vaddr = (unsigned long *)((unsigned long)vaddr + (addr & ~PAGE_MASK));
+	memcpy(vaddr, (unsigned long *)uprobe->kp.ainsn.insn, size);
+	kunmap_atomic(vaddr, KM_USER1);
+	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 no 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);
+	lock_page(page);
+
+	addr = (kprobe_opcode_t *)kmap_atomic(page, KM_USER0);
+	addr = (kprobe_opcode_t *)((unsigned long)addr +
+				 (unsigned long)(uprobe->offset & ~PAGE_MASK));
+	*addr = opcode;
+	/*TODO: flush vma ? */
+	kunmap_atomic(addr, KM_USER0);
+
+	unlock_page(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 uprobe_single_step(struct kprobe *p, struct pt_regs *regs)
+{
+	unsigned long stack_addr = regs->esp, flags;
+	struct vm_area_struct *vma = NULL;
+	struct uprobe *uprobe =  __get_cpu_var(current_uprobe);
+	struct kprobe_ctlblk *kcb = get_kprobe_ctlblk();
+	int err = 0;
+
+	down_read(&current->mm->mmap_sem);
+
+	vma = find_vma(current->mm, (stack_addr & PAGE_MASK));
+	if (!vma) {
+		/* TODO: Need better error reporting? */
+		printk("No vma found\n");
+		up_read(&current->mm->mmap_sem);
+		return -ENOENT;
+	}
+	flags = vma->vm_flags;
+	up_read(&current->mm->mmap_sem);
+
+	kcb->kprobe_status |= UPROBE_SS_STACK;
+	err = copy_insn_onstack(uprobe, regs, flags);
+
+	down_write(&current->mm->mmap_sem);
+
+	if (err) {
+		kcb->kprobe_status |= UPROBE_SS_NEW_STACK;
+		err = copy_insn_on_new_page(uprobe, regs, vma);
+	}
+	if (err) {
+		kcb->kprobe_status |= UPROBE_SS_EXPSTACK;
+		err = copy_insn_onexpstack(uprobe, regs, vma);
+	}
+
+	up_write(&current->mm->mmap_sem);
+
+	if (err) {
+		kcb->kprobe_status |= UPROBE_SS_INLINE;
+		replace_original_insn(uprobe, regs, uprobe->kp.opcode);
+	}
+
+	 __get_cpu_var(singlestep_addr) = regs->eip;
+
+
+	return 0;
+}
+
 static inline void prepare_singlestep(struct kprobe *p, struct pt_regs *regs)
 {
 	regs->eflags |= TF_MASK;
 	regs->eflags &= ~IF_MASK;
 	/*single step inline if the instruction is an int3*/
+
 	if (p->opcode == BREAKPOINT_INSTRUCTION)
 		regs->eip = (unsigned long)p->addr;
-	else
-		regs->eip = (unsigned long)&p->ainsn.insn;
+	else {
+		if (!kernel_text_address((unsigned long)p->addr))
+			uprobe_single_step(p, regs);
+		else
+			regs->eip = (unsigned long)&p->ainsn.insn;
+	}
 }
 
 /* Called with kretprobe_lock held */
@@ -194,6 +527,7 @@ static int __kprobes kprobe_handler(stru
 	kprobe_opcode_t *addr = NULL;
 	unsigned long *lp;
 	struct kprobe_ctlblk *kcb;
+	unsigned seg = regs->xcs & 0xffff;
 #ifdef CONFIG_PREEMPT
 	unsigned pre_preempt_count = preempt_count();
 #endif /* CONFIG_PREEMPT */
@@ -208,14 +542,21 @@ static int __kprobes kprobe_handler(stru
 	/* Check if the application is using LDT entry for its code segment and
 	 * calculate the address by reading the base address from the LDT entry.
 	 */
-	if ((regs->xcs & 4) && (current->mm)) {
+
+	if (regs->eflags & VM_MASK)
+		addr = (kprobe_opcode_t *)(((seg << 4) + regs->eip -
+			sizeof(kprobe_opcode_t)) & 0xffff);
+	else if ((regs->xcs & 4) && (current->mm)) {
+		local_irq_enable();
+		down(&current->mm->context.sem);
 		lp = (unsigned long *) ((unsigned long)((regs->xcs >> 3) * 8)
 					+ (char *) current->mm->context.ldt);
 		addr = (kprobe_opcode_t *) (get_desc_base(lp) + regs->eip -
 						sizeof(kprobe_opcode_t));
-	} else {
+		up(&current->mm->context.sem);
+		local_irq_disable();
+	} else
 		addr = (kprobe_opcode_t *)(regs->eip - sizeof(kprobe_opcode_t));
-	}
 	/* Check we're not actually recursing */
 	if (kprobe_running()) {
 		p = get_kprobe(addr);
@@ -235,7 +576,6 @@ static int __kprobes kprobe_handler(stru
 			save_previous_kprobe(kcb);
 			set_current_kprobe(p, regs, kcb);
 			kprobes_inc_nmissed_count(p);
-			prepare_singlestep(p, regs);
 			kcb->kprobe_status = KPROBE_REENTER;
 			return 1;
 		} else {
@@ -307,8 +647,8 @@ static int __kprobes kprobe_handler(stru
 	}
 
 ss_probe:
-	prepare_singlestep(p, regs);
 	kcb->kprobe_status = KPROBE_HIT_SS;
+	prepare_singlestep(p, regs);
 	return 1;
 
 no_kprobe:
@@ -498,6 +838,33 @@ no_change:
 	return;
 }
 
+static void __kprobes resume_execution_user(struct uprobe *uprobe,
+				struct pt_regs *regs, struct kprobe_ctlblk *kcb)
+{
+	unsigned long delta;
+	struct uprobe_page *upage;
+
+	/*
+	 * TODO :need to fixup special instructions as done with kernel probes.
+	 */
+	delta = regs->eip - __get_cpu_var(singlestep_addr);
+	regs->eip = (unsigned long)(uprobe->kp.addr + delta);
+
+	if ((kcb->kprobe_status & UPROBE_SS_EXPSTACK) ||
+			(kcb->kprobe_status & UPROBE_SS_NEW_STACK)) {
+		upage = get_upage_current(current);
+		set_pte(upage->orig_pte, __pte(upage->orig_pte_val));
+		pte_unmap(upage->orig_pte);
+
+		upage->status = UPROBE_PAGE_FREE;
+		hlist_del(&upage->hlist);
+
+	} else if (kcb->kprobe_status & UPROBE_SS_INLINE)
+		replace_original_insn(uprobe, regs,
+				(kprobe_opcode_t)BREAKPOINT_INSTRUCTION);
+	regs->eflags &= ~TF_MASK;
+}
+
 /*
  * Interrupts are disabled on entry as trap1 is an interrupt gate and they
  * remain disabled thoroughout this function.
@@ -510,16 +877,19 @@ static inline int post_kprobe_handler(st
 	if (!cur)
 		return 0;
 
-	if ((kcb->kprobe_status != KPROBE_REENTER) && cur->post_handler) {
-		kcb->kprobe_status = KPROBE_HIT_SSDONE;
+	if (!(kcb->kprobe_status & KPROBE_REENTER) && cur->post_handler) {
+		kcb->kprobe_status |= KPROBE_HIT_SSDONE;
 		cur->post_handler(cur, regs, 0);
 	}
 
-	resume_execution(cur, regs, kcb);
+	if (!kernel_text_address((unsigned long)cur->addr))
+		resume_execution_user(__get_cpu_var(current_uprobe), regs, kcb);
+	else
+		resume_execution(cur, regs, kcb);
 	regs->eflags |= kcb->kprobe_saved_eflags;
 
 	/*Restore back the original saved kprobes variables and continue. */
-	if (kcb->kprobe_status == KPROBE_REENTER) {
+	if (kcb->kprobe_status & KPROBE_REENTER) {
 		restore_previous_kprobe(kcb);
 		goto out;
 	}
@@ -547,7 +917,13 @@ static inline int kprobe_fault_handler(s
 		return 1;
 
 	if (kcb->kprobe_status & KPROBE_HIT_SS) {
-		resume_execution(cur, regs, kcb);
+		if (!kernel_text_address((unsigned long)cur->addr)) {
+			struct uprobe *uprobe =  __get_cpu_var(current_uprobe);
+			/* TODO: Proper handling of all instruction */
+			replace_original_insn(uprobe, regs, uprobe->kp.opcode);
+			regs->eflags &= ~TF_MASK;
+		} else
+			resume_execution(cur, regs, kcb);
 		regs->eflags |= kcb->kprobe_old_eflags;
 
 		reset_current_kprobe();
@@ -654,7 +1030,67 @@ int __kprobes longjmp_break_handler(stru
 	return 0;
 }
 
+static void free_alias(void)
+{
+	int cpu;
+
+	for_each_cpu(cpu) {
+		struct uprobe_page *upage;
+		upage = per_cpu_ptr(uprobe_page, cpu);
+
+		if (upage->alias_addr) {
+			set_pte(upage->alias_pte, __pte(upage->alias_pte_val));
+			kfree(upage->alias_addr);
+		}
+		upage->alias_pte = 0;
+	}
+	free_percpu(uprobe_page);
+	return;
+}
+
+static int alloc_alias(void)
+{
+	int cpu;
+
+	uprobe_page = __alloc_percpu(sizeof(struct uprobe_page));
+
+	for_each_cpu(cpu) {
+		struct uprobe_page *upage;
+		upage = per_cpu_ptr(uprobe_page, cpu);
+		upage->alias_addr = kmalloc(PAGE_SIZE, GFP_USER);
+		if (!upage->alias_addr) {
+			free_alias();
+			return -ENOMEM;
+		}
+		upage->alias_pte = lookup_address(
+					(unsigned long)upage->alias_addr);
+		upage->alias_pte_val = pte_val(*upage->alias_pte);
+		if (upage->alias_pte) {
+			upage->status = UPROBE_PAGE_FREE;
+			set_pte(upage->alias_pte,
+						pte_mkdirty(*upage->alias_pte));
+			set_pte(upage->alias_pte,
+						pte_mkexec(*upage->alias_pte));
+			set_pte(upage->alias_pte,
+						 pte_mkwrite(*upage->alias_pte));
+			set_pte(upage->alias_pte,
+						pte_mkyoung(*upage->alias_pte));
+		}
+	}
+	return 0;
+}
+
 int __init arch_init_kprobes(void)
 {
+	int ret = 0;
+	/*
+	 * user space probes requires a page to copy the original instruction
+	 * so that it can single step if there is no free stack space, allocate
+	 * per cpu page.
+	 */
+
+	if ((ret = alloc_alias()))
+		return ret;
+
 	return 0;
 }
diff -puN include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line include/asm-i386/kprobes.h
--- linux-2.6.16-rc1-mm5/include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
+++ linux-2.6.16-rc1-mm5-prasanna/include/asm-i386/kprobes.h	2006-02-08 19:26:10.000000000 +0530
@@ -42,6 +42,7 @@ typedef u8 kprobe_opcode_t;
 #define JPROBE_ENTRY(pentry)	(kprobe_opcode_t *)pentry
 #define ARCH_SUPPORTS_KRETPROBES
 #define arch_remove_kprobe(p)	do {} while (0)
+#define UPROBE_PAGE_FREE 0x00000001
 
 void kretprobe_trampoline(void);
 
@@ -74,6 +75,18 @@ struct kprobe_ctlblk {
 	struct prev_kprobe prev_kprobe;
 };
 
+/* per cpu uprobe page structure */
+struct uprobe_page {
+	struct hlist_node hlist;
+	pte_t *alias_pte;
+	pte_t *orig_pte;
+	unsigned long orig_pte_val;
+	unsigned long alias_pte_val;
+	void *alias_addr;
+	struct task_struct *tsk;
+	unsigned long status;
+};
+
 /* trap3/1 are intr gates for kprobes.  So, restore the status of IF,
  * if necessary, before executing the original int3/1 (trap) handler.
  */
diff -puN include/linux/kprobes.h~kprobes_userspace_probes_ss_out-of-line include/linux/kprobes.h
--- linux-2.6.16-rc1-mm5/include/linux/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
+++ linux-2.6.16-rc1-mm5-prasanna/include/linux/kprobes.h	2006-02-08 19:26:10.000000000 +0530
@@ -45,11 +45,18 @@
 #ifdef CONFIG_KPROBES
 #include <asm/kprobes.h>
 
+#define KPROBE_HASH_BITS 6
+#define KPROBE_TABLE_SIZE (1 << KPROBE_HASH_BITS)
+
 /* kprobe_status settings */
 #define KPROBE_HIT_ACTIVE	0x00000001
 #define KPROBE_HIT_SS		0x00000002
 #define KPROBE_REENTER		0x00000004
 #define KPROBE_HIT_SSDONE	0x00000008
+#define UPROBE_SS_STACK		0x00000010
+#define UPROBE_SS_EXPSTACK	0x00000020
+#define UPROBE_SS_INLINE	0x00000040
+#define UPROBE_SS_NEW_STACK	0x00000080
 
 /* Attach to insert probes on any functions which should be ignored*/
 #define __kprobes	__attribute__((__section__(".kprobes.text")))
diff -puN kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line kernel/kprobes.c
--- linux-2.6.16-rc1-mm5/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:10.000000000 +0530
+++ linux-2.6.16-rc1-mm5-prasanna/kernel/kprobes.c	2006-02-08 19:26:10.000000000 +0530
@@ -42,9 +42,6 @@
 #include <asm/errno.h>
 #include <asm/kdebug.h>
 
-#define KPROBE_HASH_BITS 6
-#define KPROBE_TABLE_SIZE (1 << KPROBE_HASH_BITS)
-
 static struct hlist_head kprobe_table[KPROBE_TABLE_SIZE];
 static struct hlist_head kretprobe_inst_table[KPROBE_TABLE_SIZE];
 static struct list_head uprobe_module_list;

_
-- 
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] 11+ messages in thread

* RE: [3/3] Userspace probes prototype-take2
@ 2006-02-20  5:48 Zhang, Yanmin
  0 siblings, 0 replies; 11+ messages in thread
From: Zhang, Yanmin @ 2006-02-20  5:48 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap

>>-----Original Message-----
>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Prasanna S Panchamukhi
>>Sent: 2006年2月20日 13:38
>>To: Zhang, Yanmin
>>Cc: systemtap@sources.redhat.com
>>Subject: Re: [3/3] Userspace probes prototype-take2
>>
>>Yanmin,
>>
>>
>>Please see my comments inline below.
>>
>>Thanks
>>Prasanna
>>> >>
>>> >>2. This patch works only with PREEMPT config option disabled, to work
>>> >>in PREEMPT enabled condition handlers must be re-written and must
>>> >>be seperated out from kernel probes allowing preemption.
>>> One of my old comments is an external device interrupt might happen when cpu is single-stepping the original instruction, then the
>>task might be switched to another cpu. If we disable irq when exiting to user space to single step the instruction, kernel might switch
>>the task off just on the exit kernel path. 1) uprobe_page; 2) kprobe_ctlblk, These 2 resources shouldn't be pre cpu, or we need get
>>another approach. How could you resolve the task switch issue?
>>
>>My new design does not use the kprobe handlers and per cpu kprobe data
>>structures itself  so that task switch issue will be resolved.
>>We register a separte set of uprobe handlers and use uprobe data structure.
>>Also now we will be handling uprobes serially and synchronize using some lock/mutex, but later on we have scale it up for better performance.
>>
>>> >>+static int __kprobes copy_insn_on_new_page(struct uprobe *uprobe ,
>>> >>+			struct pt_regs *regs, struct vm_area_struct *vma)
>>> >>+{
>>> >>+	unsigned long addr, *vaddr, stack_addr = regs->esp;
>>> >>+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
>>> >>+	struct uprobe_page *upage;
>>> >>+	struct page *page;
>>> >>+	pte_t *pte;
>>> >>+
>>> >>+
>>> >>+	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;
>>> >>+
>>> The multi-thread case is not resolved here. One of typical multi-thread model is that the all threads share the same vma and every
>>thread has 8-k stack.
>>
>>>If 2 threads trigger uprobe (although might be not the same uprobe) at the same time, one thread might erase single-step instruction
>>of another.
>>
>>Do these threads share the same stack pages?
[YM] No. They share the same vma. And copy_insm_onstack might happen to fail on the 2 threads at the same time.

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

* RE: [3/3] Userspace probes prototype-take2
@ 2006-02-20  5:48 Zhang, Yanmin
  0 siblings, 0 replies; 11+ messages in thread
From: Zhang, Yanmin @ 2006-02-20  5:48 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap

>>-----Original Message-----
>>From: Prasanna S Panchamukhi [mailto:prasanna@in.ibm.com]
>>Sent: 2006年2月20日 12:52
>>To: Zhang, Yanmin
>>Cc: systemtap@sources.redhat.com
>>Subject: Re: [3/3] Userspace probes prototype-take2
>>
>>Yanmin,
>>
>>On Mon, Feb 20, 2006 at 11:15:31AM +0800, Zhang, Yanmin wrote:
>>> I lost an important comment. The patch is not aware of signal processing. After kernel prepares the single-step-inst on the stack,
>>if a signal is delivered to the thread, kernel will save some states into stack and switch to signal handler function, so single-step-inst
>>on the stack might be erased.
>>
>>AFAIK this problem can be addressed in the following ways.
>>
>>1. Leave the sufficient stack space for the kernel to deliver the
>>signals and then copy the instructions on the stack.
[YM] signal action handler itself could be nested again, so it looks not a good approach.


>>
>>2. Synchronize usage of stack between signal processing and user space probes.
[YM] This approach looks not good. Another issue if doing so is that the single-step-insn itself might change esp.


>>
>>3. Block the signal processing by disabling interrupts and preemption from
>>the time we copy the instruction on the stack untill we single step on the
>>original instruction. Or even wait for the signal processing to be
>>complete and then setup the stack for single stepping on the original
>>instructions and single step.
[YM] We could do not check signal when exiting from kernel if trying to single-step. It's easy to be implemented, but community might argue. I prefer this one. For example, add a new flag, TIF_UPROBING, at thread_info->flags. And check it at appropriate time.


>>
>>Your suggestion are welcome to provide better solutions to this problem.
>>
>>Thanks
>>Prasanna

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

* Re: [3/3] Userspace probes prototype-take2
  2006-02-17  9:19 Zhang, Yanmin
@ 2006-02-20  5:36 ` Prasanna S Panchamukhi
  0 siblings, 0 replies; 11+ messages in thread
From: Prasanna S Panchamukhi @ 2006-02-20  5:36 UTC (permalink / raw)
  To: Zhang, Yanmin; +Cc: systemtap

Yanmin,


Please see my comments inline below.

Thanks
Prasanna
> >>
> >>2. This patch works only with PREEMPT config option disabled, to work
> >>in PREEMPT enabled condition handlers must be re-written and must
> >>be seperated out from kernel probes allowing preemption.
> One of my old comments is an external device interrupt might happen when cpu is single-stepping the original instruction, then the task might be switched to another cpu. If we disable irq when exiting to user space to single step the instruction, kernel might switch the task off just on the exit kernel path. 1) uprobe_page; 2) kprobe_ctlblk, These 2 resources shouldn't be pre cpu, or we need get another approach. How could you resolve the task switch issue?

My new design does not use the kprobe handlers and per cpu kprobe data
structures itself  so that task switch issue will be resolved.
We register a separte set of uprobe handlers and use uprobe data structure.
Also now we will be handling uprobes serially and synchronize using some lock/mutex, but later on we have scale it up for better performance.

> >>+static int __kprobes copy_insn_on_new_page(struct uprobe *uprobe ,
> >>+			struct pt_regs *regs, struct vm_area_struct *vma)
> >>+{
> >>+	unsigned long addr, *vaddr, stack_addr = regs->esp;
> >>+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
> >>+	struct uprobe_page *upage;
> >>+	struct page *page;
> >>+	pte_t *pte;
> >>+
> >>+
> >>+	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;
> >>+
> The multi-thread case is not resolved here. One of typical multi-thread model is that the all threads share the same vma and every thread has 8-k stack. 

>If 2 threads trigger uprobe (although might be not the same uprobe) at the same time, one thread might erase single-step instruction of another.

Do these threads share the same stack pages?


-- 
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] 11+ messages in thread

* Re: [3/3] Userspace probes prototype-take2
  2006-02-20  3:32 Zhang, Yanmin
@ 2006-02-20  5:07 ` Prasanna S Panchamukhi
  0 siblings, 0 replies; 11+ messages in thread
From: Prasanna S Panchamukhi @ 2006-02-20  5:07 UTC (permalink / raw)
  To: Zhang, Yanmin; +Cc: systemtap

Yanmin,

On Mon, Feb 20, 2006 at 11:32:31AM +0800, Zhang, Yanmin wrote:
> >>-----Original Message-----
> >>From: Zhang, Yanmin
> >>Sent: 2006年2月20日 11:16
> >>To: Zhang, Yanmin; prasanna@in.ibm.com; systemtap@sources.redhat.com
> >>Subject: RE: [3/3] Userspace probes prototype-take2
> >>
> >>I lost an important comment. The patch is not aware of signal processing. After kernel prepares the single-step-inst on the stack, if
> >>a signal is delivered to the thread, kernel will save some states into stack and switch to signal handler function, so single-step-inst
> >>on the stack might be erased.
> >>
> >>>>-----Original Message-----
> >>>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Zhang, Yanmin
> >>>>Sent: 2006年2月17日 17:20
> >>>>To: prasanna@in.ibm.com; systemtap@sources.redhat.com
> >>>>Subject: RE: [3/3] Userspace probes prototype-take2
> >>>>
> >>>>2 main issues:
> >>>>1) task switch caused by external interrupt when single-step;
> [YM] I think we could resolve this problem. Kernel probe has some differences from uprobe. One of them is that we couldn't estimate if kernel probe happens in process context, or interrupt context, while uprobe always happens in process context (user space). So from some points of view, uprobe could be simplified from kernel probe.
> a) Don't use kcb (kprobe_ctlblk) if uprobe is triggered. Create new functions, kprobe__handler_user, kprobe_fault_handler_user and other handlers. In the new functions, instead of kcb, we could use uprobe_page being allocated dynamically
.
Yes, I am trying to seperate out kprobes_handlers and uprobe_handlers
since user space probe handlers can preempt and might sleep. also given
that we might preempt, we cannot reuse the kprobe_handlers() that use rcu.
My next take will address this issues.

> Considering signal action handler (possible uprobe nested), a thread might have a list of uprobe_page.

Yes, reentrancy in this situation also need to be handled.

> b) Delete current_uprobe;

> 
> 
> 
> >>>>2) multi-thread:
> [YM] We could resolve this problem.
> a) Don't call replace_orignal_insn in function uprobe_single_step. It might cause a race condition.
> b) Delete copy_insn_on_new_page;
> c) Merge copy_insn_onstack and copy_insn_onexpstack. The single-step-insn address could be esp-sizeof(long long)-MAX_INSN_SIZE*sizeof(kprobe_opcode_t). 
This can be done.
> d) If the stack couldn't be expanded, just kill the thread. It's reasonable because the stack is used up.

We need to take a closer look at this issue

> 
> 
> >>>>
> >>>>See below inline comments.
> >>>>
> >>>>Yanmin
> >>>>
> >>>>>>-----Original Message-----
> >>>>>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Prasanna S Panchamukhi
> >>>>>>Sent: 2006年2月8日 22:14
> >>>>>>To: systemtap@sources.redhat.com
> >>>>>>Subject: Re: [3/3] Userspace probes prototype-take2
> >>>>>>
> >>>>>>
> >>>>>>This patch handles the executing the registered callback
> >>>>>>functions when probes is hit.

-- 
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] 11+ messages in thread

* Re: [3/3] Userspace probes prototype-take2
  2006-02-20  3:16 Zhang, Yanmin
@ 2006-02-20  4:51 ` Prasanna S Panchamukhi
  0 siblings, 0 replies; 11+ messages in thread
From: Prasanna S Panchamukhi @ 2006-02-20  4:51 UTC (permalink / raw)
  To: Zhang, Yanmin; +Cc: systemtap

Yanmin,

On Mon, Feb 20, 2006 at 11:15:31AM +0800, Zhang, Yanmin wrote:
> I lost an important comment. The patch is not aware of signal processing. After kernel prepares the single-step-inst on the stack, if a signal is delivered to the thread, kernel will save some states into stack and switch to signal handler function, so single-step-inst on the stack might be erased.

AFAIK this problem can be addressed in the following ways.

1. Leave the sufficient stack space for the kernel to deliver the
signals and then copy the instructions on the stack.

2. Synchronize usage of stack between signal processing and user space probes.

3. Block the signal processing by disabling interrupts and preemption from 
the time we copy the instruction on the stack untill we single step on the 
original instruction. Or even wait for the signal processing to be
complete and then setup the stack for single stepping on the original
instructions and single step. 

Your suggestion are welcome to provide better solutions to this problem.

Thanks
Prasanna

> 
> >>-----Original Message-----
> >>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Zhang, Yanmin
> >>Sent: 2006年2月17日 17:20
> >>To: prasanna@in.ibm.com; systemtap@sources.redhat.com
> >>Subject: RE: [3/3] Userspace probes prototype-take2
> >>
> >>2 main issues:
> >>1) task switch caused by external interrupt when single-step;
> >>2) multi-thread:
> >>
> >>See below inline comments.
> >>
> >>Yanmin
> >>
> >>>>-----Original Message-----
> >>>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Prasanna S Panchamukhi
> >>>>Sent: 2006年2月8日 22:14
> >>>>To: systemtap@sources.redhat.com
> >>>>Subject: Re: [3/3] Userspace probes prototype-take2
> >>>>
> >>>>
> >>>>This patch handles the executing the registered callback
> >>>>functions when probes is hit.
> >>>>
> >>>>	Each userspace probe is uniquely identified by the
> >>>>combination of inode and offset, hence during registeration the inode
> >>>>and offset combination is added to kprobes hash table. Initially when
> >>>>breakpoint instruction is hit, the kprobes hash table is looked up
> >>>>for matching inode and offset. The pre_handlers are called in sequence
> >>>>if multiple probes are registered. The original instruction is single
> >>>>stepped out-of-line similar to kernel probes. In kernel space probes,
> >>>>single stepping out-of-line is achieved by copying the instruction on
> >>>>to some location within kernel address space and then single step
> >>>>from that location. But for userspace probes, instruction copied
> >>>>into kernel address space cannot be single stepped, hence the
> >>>>instruction should 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. If the stack grows
> >>>>downwards, the stack space below the stack pointer indicates the
> >>>>unused stack free space and if the stack grows upwards, the stack
> >>>>space above the stack pointer indicates the unused stack free space.
> >>>>
> >>>>The instruction to be single stepped can modify the stack space, hence
> >>>>before using the unused stack free 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 unused 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. This patch is not stable yet, should work for most conditions.
> >>>>
> >>>>2. This patch works only with PREEMPT config option disabled, to work
> >>>>in PREEMPT enabled condition handlers must be re-written and must
> >>>>be seperated out from kernel probes allowing preemption.
> >>One of my old comments is an external device interrupt might happen when cpu is single-stepping the original instruction, then the task
> >>might be switched to another cpu. If we disable irq when exiting to user space to single step the instruction, kernel might switch the
> >>task off just on the exit kernel path. 1) uprobe_page; 2) kprobe_ctlblk, These 2 resources shouldn't be pre cpu, or we need get another
> >>approach. How could you resolve the task switch issue?
> >>
> >>
> >>
> >>>>
> >>>>3. 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.
> >>>>
> >>>>4. Optimize the insertion of probes through readpage hooks. Identify
> >>>>all the probes to be inserted on the read page and insert them at
> >>>>once.
> >>>>
> >>>>5. Resume exectution should handle setting of proper eip and eflags
> >>>>for special instructions similar to kernel probes.
> >>>>
> >>>>6. Single stepping out-of-line expands the stack if there is no
> >>>>enough stack space to copy the original instruction. Expansion of
> >>>>stack should be shrinked back to the original size after single
> >>>>stepping or the expanded stack should be reused for single stepping
> >>>>out-of-line for other probes.
> >>>>
> >>>>7. A wrapper routines to calculate the offset from the probed file
> >>>>beginning. In case of dynamic shared library, the offset is
> >>>>calculated by substracting the address of the probe point from the
> >>>>beginning of the file mapped address.
> >>>>
> >>>>8. Handing of page faults while inthe kprobes_handler() and while
> >>>>single stepping.
> >>>>
> >>>>9. Accessing user space pages not present in memory, from the
> >>>>registered callback routines.
> >>>>
> >>>>Signed-off-by: Prasanna S Panchamukhi <prasanna@in.ibm.com>
> >>>>
> >>>>
> >>>> arch/i386/kernel/kprobes.c |  460 +++++++++++++++++++++++++++++++++++++++++++--
> >>>> include/asm-i386/kprobes.h |   13 +
> >>>> include/linux/kprobes.h    |    7
> >>>> kernel/kprobes.c           |    3
> >>>> 4 files changed, 468 insertions(+), 15 deletions(-)
> >>>>
> >>>>diff -puN arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line arch/i386/kernel/kprobes.c
> >>>>--- linux-2.6.16-rc1-mm5/arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
> >>>>+++ linux-2.6.16-rc1-mm5-prasanna/arch/i386/kernel/kprobes.c	2006-02-08 19:26:10.000000000 +0530
> >>>>@@ -30,6 +30,7 @@
> >>>>
> >>>> #include <linux/config.h>
> >>>> #include <linux/kprobes.h>
> >>>>+#include <linux/hash.h>
> >>>> #include <linux/ptrace.h>
> >>>> #include <linux/preempt.h>
> >>>> #include <asm/cacheflush.h>
> >>>>@@ -38,8 +39,12 @@
> >>>>
> >>>> void jprobe_return_end(void);
> >>>>
> >>>>+static struct uprobe_page *uprobe_page;
> >>>>+static struct hlist_head uprobe_page_table[KPROBE_TABLE_SIZE];
> >>>> DEFINE_PER_CPU(struct kprobe *, current_kprobe) = NULL;
> >>>> DEFINE_PER_CPU(struct kprobe_ctlblk, kprobe_ctlblk);
> >>>>+DEFINE_PER_CPU(struct uprobe *, current_uprobe) = NULL;
> >>>>+DEFINE_PER_CPU(unsigned long, singlestep_addr);
> >>>>
> >>>> /* insert a jmp code */
> >>>> static inline void set_jmp_op(void *from, void *to)
> >>>>@@ -125,6 +130,23 @@ void __kprobes arch_disarm_kprobe(struct
> >>>> 			   (unsigned long) p->addr + sizeof(kprobe_opcode_t));
> >>>> }
> >>>>
> >>>>+void __kprobes arch_disarm_uprobe(struct kprobe *p, kprobe_opcode_t *address)
> >>>>+{
> >>>>+	*address = p->opcode;
> >>>>+}
> >>>>+
> >>>>+void __kprobes arch_arm_uprobe(unsigned long *address)
> >>>>+{
> >>>>+	*(kprobe_opcode_t *)address = BREAKPOINT_INSTRUCTION;
> >>>>+}
> >>>>+
> >>>>+void __kprobes arch_copy_uprobe(struct kprobe *p, unsigned long *address)
> >>>>+{
> >>>>+	memcpy(p->ainsn.insn, (kprobe_opcode_t *)address,
> >>>>+				MAX_INSN_SIZE * sizeof(kprobe_opcode_t));
> >>>>+	p->opcode = *(kprobe_opcode_t *)address;
> >>>>+}
> >>>>+
> >>>> static inline void save_previous_kprobe(struct kprobe_ctlblk *kcb)
> >>>> {
> >>>> 	kcb->prev_kprobe.kp = kprobe_running();
> >>>>@@ -151,15 +173,326 @@ static inline void set_current_kprobe(st
> >>>> 		kcb->kprobe_saved_eflags &= ~IF_MASK;
> >>>> }
> >>>>
> >>>>+struct uprobe_page __kprobes *get_upage_current(struct task_struct *tsk)
> >>>>+{
> >>>>+	struct hlist_head *head;
> >>>>+	struct hlist_node *node;
> >>>>+	struct uprobe_page *upage;
> >>>>+
> >>>>+	head = &uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)];
> >>>>+	hlist_for_each_entry(upage, node, head, hlist) {
> >>>>+		if (upage->tsk == tsk)
> >>>>+			return upage;
> >>>>+        }
> >>>>+	return NULL;
> >>>>+}
> >>>>+
> >>>>+struct uprobe_page __kprobes *get_upage_free(struct task_struct *tsk)
> >>>>+{
> >>>>+	int cpu;
> >>>>+
> >>>>+	for_each_cpu(cpu) {
> >>>>+		struct uprobe_page *upage;
> >>>>+		upage = per_cpu_ptr(uprobe_page, cpu);
> >>>>+		if (upage->status & UPROBE_PAGE_FREE)
> >>>>+			return upage;
> >>>>+	}
> >>>>+	return NULL;
> >>>>+}
> >>>>+
> >>>>+/**
> >>>>+ * 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, it just maps a new page and copies the
> >>>>+ *  new 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, *vaddr, stack_addr = regs->esp;
> >>>>+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
> >>>>+	struct uprobe_page *upage;
> >>>>+	struct page *page;
> >>>>+	pte_t *pte;
> >>>>+
> >>>>+
> >>>>+	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;
> >>>>+
> >>The multi-thread case is not resolved here. One of typical multi-thread model is that the all threads share the same vma and every thread
> >>has 8-k stack. If 2 threads trigger uprobe (although might be not the same uprobe) at the same time, one thread might erase single-step
> >>instruction of another.
> >>
> >>
> >>
> >>>>+	preempt_enable_no_resched();
> >>>>+
> >>>>+	pte = get_uprobe_pte(addr);
> >>>>+	preempt_disable();
> >>>>+	if (!pte)
> >>>>+		return -EFAULT;
> >>>>+
> >>>>+	upage = get_upage_free(current);
> >>>>+	upage->status &= ~UPROBE_PAGE_FREE;
> >>>>+	upage->tsk = current;
> >>>>+	INIT_HLIST_NODE(&upage->hlist);
> >>>>+	hlist_add_head(&upage->hlist,
> >>>>+		&uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)]);
> >>>>+
> >>>>+	upage->orig_pte = pte;
> >>>>+	upage->orig_pte_val =  pte_val(*pte);
> >>>>+	set_pte(pte, (*(upage->alias_pte)));
> >>>>+
> >>>>+	page = pte_page(*pte);
> >>>>+	vaddr = (unsigned long *)kmap_atomic(page, KM_USER1);
> >>>>+	vaddr = (unsigned long *)((unsigned long)vaddr + (addr & ~PAGE_MASK));
> >>>>+	memcpy(vaddr, (unsigned long *)uprobe->kp.ainsn.insn, size);
> >>>>+	kunmap_atomic(vaddr, KM_USER1);
> >>>>+	regs->eip = addr;
> >>So the temp page, upage->alias_addr, replaces the original one on the stack. If the replaced instruction is to operate stack, such like
> >>"push eax", the result might be on the new page. After the single step, the pte is restored to the original page which doesn't have
> >>the value of eax.
> >>
> >>
> >>
> >>>>+
> >>>>+	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)
> >>It has the same issues like function copy_insn_on_new_page.
> >>
> >>
> >>>>+{
> >>>>+	unsigned long addr, *vaddr, vm_addr;
> >>>>+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
> >>>>+	struct vm_area_struct *new_vma;
> >>>>+	struct uprobe_page *upage;
> >>>>+	struct mm_struct *mm = current->mm;
> >>>>+	struct page *page;
> >>>>+	pte_t *pte;
> >>>>+
> >>>>+
> >>>>+	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
> >>>>+		return -EFAULT;
> >>>>+
> >>>>+	preempt_enable_no_resched();
> >>>>+
> >>>>+	/* TODO: do we need to expand stack if extend_vma fails? */
> >>>>+	new_vma = find_extend_vma(mm, vm_addr);
> >>>>+	preempt_disable();
> >>>>+	if (!new_vma)
> >>>>+		return -ENOMEM;
> >>>>+
> >>>>+	/*
> >>>>+	 * TODO: Expanding stack for every probe is not a good idea, stack must
> >>>>+	 * either be shrunk to its original size after single stepping or the
> >>>>+	 * expanded stack should be kept track of, for the probed application,
> >>>>+	 * so it can be reused to single step out-of-line
> >>>>+	 */
> >>>>+	if (new_vma->vm_flags & VM_GROWSDOWN)
> >>>>+		addr = new_vma->vm_start;
> >>>>+	else
> >>>>+		addr = new_vma->vm_end - size;
> >>>>+
> >>>>+	preempt_enable_no_resched();
> >>>>+	pte = get_uprobe_pte(addr);
> >>>>+	preempt_disable();
> >>>>+	if (!pte)
> >>>>+		return -EFAULT;
> >>>>+
> >>>>+	upage = get_upage_free(current);
> >>>>+	upage->status &= ~UPROBE_PAGE_FREE;
> >>>>+	upage->tsk = current;
> >>>>+	INIT_HLIST_NODE(&upage->hlist);
> >>>>+	hlist_add_head(&upage->hlist,
> >>>>+		&uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)]);
> >>>>+	upage->orig_pte = pte;
> >>>>+	upage->orig_pte_val =  pte_val(*pte);
> >>>>+	set_pte(pte, (*(upage->alias_pte)));
> >>>>+
> >>>>+	page = pte_page(*pte);
> >>>>+	vaddr = (unsigned long *)kmap_atomic(page, KM_USER1);
> >>>>+	vaddr = (unsigned long *)((unsigned long)vaddr + (addr & ~PAGE_MASK));
> >>>>+	memcpy(vaddr, (unsigned long *)uprobe->kp.ainsn.insn, size);
> >>>>+	kunmap_atomic(vaddr, KM_USER1);
> >>>>+	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 no 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);
> >>>>+	lock_page(page);
> >>>>+
> >>>>+	addr = (kprobe_opcode_t *)kmap_atomic(page, KM_USER0);
> >>>>+	addr = (kprobe_opcode_t *)((unsigned long)addr +
> >>>>+				 (unsigned long)(uprobe->offset & ~PAGE_MASK));
> >>>>+	*addr = opcode;
> >>>>+	/*TODO: flush vma ? */
> >>>>+	kunmap_atomic(addr, KM_USER0);
> >>>>+
> >>>>+	unlock_page(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 uprobe_single_step(struct kprobe *p, struct pt_regs *regs)
> >>>>+{
> >>>>+	unsigned long stack_addr = regs->esp, flags;
> >>>>+	struct vm_area_struct *vma = NULL;
> >>>>+	struct uprobe *uprobe =  __get_cpu_var(current_uprobe);
> >>>>+	struct kprobe_ctlblk *kcb = get_kprobe_ctlblk();
> >>>>+	int err = 0;
> >>>>+
> >>>>+	down_read(&current->mm->mmap_sem);
> >>>>+
> >>>>+	vma = find_vma(current->mm, (stack_addr & PAGE_MASK));
> >>>>+	if (!vma) {
> >>>>+		/* TODO: Need better error reporting? */
> >>>>+		printk("No vma found\n");
> >>>>+		up_read(&current->mm->mmap_sem);
> >>>>+		return -ENOENT;
> >>>>+	}
> >>>>+	flags = vma->vm_flags;
> >>>>+	up_read(&current->mm->mmap_sem);
> >>>>+
> >>>>+	kcb->kprobe_status |= UPROBE_SS_STACK;
> >>>>+	err = copy_insn_onstack(uprobe, regs, flags);
> >>>>+
> >>>>+	down_write(&current->mm->mmap_sem);
> >>>>+
> >>>>+	if (err) {
> >>>>+		kcb->kprobe_status |= UPROBE_SS_NEW_STACK;
> >>>>+		err = copy_insn_on_new_page(uprobe, regs, vma);
> >>>>+	}
> >>>>+	if (err) {
> >>>>+		kcb->kprobe_status |= UPROBE_SS_EXPSTACK;
> >>>>+		err = copy_insn_onexpstack(uprobe, regs, vma);
> >>>>+	}
> >>>>+
> >>>>+	up_write(&current->mm->mmap_sem);
> >>>>+
> >>>>+	if (err) {
> >>>>+		kcb->kprobe_status |= UPROBE_SS_INLINE;
> >>>>+		replace_original_insn(uprobe, regs, uprobe->kp.opcode);
> >>>>+	}
> >>>>+
> >>>>+	 __get_cpu_var(singlestep_addr) = regs->eip;
> >>>>+
> >>>>+
> >>>>+	return 0;
> >>>>+}
> >>>>+
> >>>> static inline void prepare_singlestep(struct kprobe *p, struct pt_regs *regs)
> >>>> {
> >>>> 	regs->eflags |= TF_MASK;
> >>>> 	regs->eflags &= ~IF_MASK;
> >>>> 	/*single step inline if the instruction is an int3*/
> >>>>+
> >>>> 	if (p->opcode == BREAKPOINT_INSTRUCTION)
> >>>> 		regs->eip = (unsigned long)p->addr;
> >>>>-	else
> >>>>-		regs->eip = (unsigned long)&p->ainsn.insn;
> >>>>+	else {
> >>>>+		if (!kernel_text_address((unsigned long)p->addr))
> >>>>+			uprobe_single_step(p, regs);
> >>>>+		else
> >>>>+			regs->eip = (unsigned long)&p->ainsn.insn;
> >>>>+	}
> >>>> }
> >>>>
> >>>> /* Called with kretprobe_lock held */
> >>>>@@ -194,6 +527,7 @@ static int __kprobes kprobe_handler(stru
> >>>> 	kprobe_opcode_t *addr = NULL;
> >>>> 	unsigned long *lp;
> >>>> 	struct kprobe_ctlblk *kcb;
> >>>>+	unsigned seg = regs->xcs & 0xffff;
> >>>> #ifdef CONFIG_PREEMPT
> >>>> 	unsigned pre_preempt_count = preempt_count();
> >>>> #endif /* CONFIG_PREEMPT */
> >>>>@@ -208,14 +542,21 @@ static int __kprobes kprobe_handler(stru
> >>>> 	/* Check if the application is using LDT entry for its code segment and
> >>>> 	 * calculate the address by reading the base address from the LDT entry.
> >>>> 	 */
> >>>>-	if ((regs->xcs & 4) && (current->mm)) {
> >>>>+
> >>>>+	if (regs->eflags & VM_MASK)
> >>>>+		addr = (kprobe_opcode_t *)(((seg << 4) + regs->eip -
> >>>>+			sizeof(kprobe_opcode_t)) & 0xffff);
> >>>>+	else if ((regs->xcs & 4) && (current->mm)) {
> >>>>+		local_irq_enable();
> >>>>+		down(&current->mm->context.sem);
> >>>> 		lp = (unsigned long *) ((unsigned long)((regs->xcs >> 3) * 8)
> >>>> 					+ (char *) current->mm->context.ldt);
> >>>> 		addr = (kprobe_opcode_t *) (get_desc_base(lp) + regs->eip -
> >>>> 						sizeof(kprobe_opcode_t));
> >>>>-	} else {
> >>>>+		up(&current->mm->context.sem);
> >>>>+		local_irq_disable();
> >>>>+	} else
> >>>> 		addr = (kprobe_opcode_t *)(regs->eip - sizeof(kprobe_opcode_t));
> >>>>-	}
> >>>> 	/* Check we're not actually recursing */
> >>>> 	if (kprobe_running()) {
> >>>> 		p = get_kprobe(addr);
> >>>>@@ -235,7 +576,6 @@ static int __kprobes kprobe_handler(stru
> >>>> 			save_previous_kprobe(kcb);
> >>>> 			set_current_kprobe(p, regs, kcb);
> >>>> 			kprobes_inc_nmissed_count(p);
> >>>>-			prepare_singlestep(p, regs);
> >>>> 			kcb->kprobe_status = KPROBE_REENTER;
> >>>> 			return 1;
> >>>> 		} else {
> >>>>@@ -307,8 +647,8 @@ static int __kprobes kprobe_handler(stru
> >>>> 	}
> >>>>
> >>>> ss_probe:
> >>>>-	prepare_singlestep(p, regs);
> >>>> 	kcb->kprobe_status = KPROBE_HIT_SS;
> >>>>+	prepare_singlestep(p, regs);
> >>>> 	return 1;
> >>>>
> >>>> no_kprobe:
> >>>>@@ -498,6 +838,33 @@ no_change:
> >>>> 	return;
> >>>> }
> >>>>
> >>>>+static void __kprobes resume_execution_user(struct uprobe *uprobe,
> >>>>+				struct pt_regs *regs, struct kprobe_ctlblk *kcb)
> >>>>+{
> >>>>+	unsigned long delta;
> >>>>+	struct uprobe_page *upage;
> >>>>+
> >>>>+	/*
> >>>>+	 * TODO :need to fixup special instructions as done with kernel probes.
> >>>>+	 */
> >>>>+	delta = regs->eip - __get_cpu_var(singlestep_addr);
> >>>>+	regs->eip = (unsigned long)(uprobe->kp.addr + delta);
> >>>>+
> >>>>+	if ((kcb->kprobe_status & UPROBE_SS_EXPSTACK) ||
> >>>>+			(kcb->kprobe_status & UPROBE_SS_NEW_STACK)) {
> >>>>+		upage = get_upage_current(current);
> >>>>+		set_pte(upage->orig_pte, __pte(upage->orig_pte_val));
> >>>>+		pte_unmap(upage->orig_pte);
> >>>>+
> >>>>+		upage->status = UPROBE_PAGE_FREE;
> >>>>+		hlist_del(&upage->hlist);
> >>>>+
> >>>>+	} else if (kcb->kprobe_status & UPROBE_SS_INLINE)
> >>>>+		replace_original_insn(uprobe, regs,
> >>>>+				(kprobe_opcode_t)BREAKPOINT_INSTRUCTION);
> >>>>+	regs->eflags &= ~TF_MASK;
> >>>>+}
> >>>>+
> >>>> /*
> >>>>  * Interrupts are disabled on entry as trap1 is an interrupt gate and they
> >>>>  * remain disabled thoroughout this function.
> >>>>@@ -510,16 +877,19 @@ static inline int post_kprobe_handler(st
> >>>> 	if (!cur)
> >>>> 		return 0;
> >>>>
> >>>>-	if ((kcb->kprobe_status != KPROBE_REENTER) && cur->post_handler) {
> >>>>-		kcb->kprobe_status = KPROBE_HIT_SSDONE;
> >>>>+	if (!(kcb->kprobe_status & KPROBE_REENTER) && cur->post_handler) {
> >>>>+		kcb->kprobe_status |= KPROBE_HIT_SSDONE;
> >>>> 		cur->post_handler(cur, regs, 0);
> >>>> 	}
> >>>>
> >>>>-	resume_execution(cur, regs, kcb);
> >>>>+	if (!kernel_text_address((unsigned long)cur->addr))
> >>>>+		resume_execution_user(__get_cpu_var(current_uprobe), regs, kcb);
> >>>>+	else
> >>>>+		resume_execution(cur, regs, kcb);
> >>>> 	regs->eflags |= kcb->kprobe_saved_eflags;
> >>>>
> >>>> 	/*Restore back the original saved kprobes variables and continue. */
> >>>>-	if (kcb->kprobe_status == KPROBE_REENTER) {
> >>>>+	if (kcb->kprobe_status & KPROBE_REENTER) {
> >>>> 		restore_previous_kprobe(kcb);
> >>>> 		goto out;
> >>>> 	}
> >>>>@@ -547,7 +917,13 @@ static inline int kprobe_fault_handler(s
> >>>> 		return 1;
> >>>>
> >>>> 	if (kcb->kprobe_status & KPROBE_HIT_SS) {
> >>>>-		resume_execution(cur, regs, kcb);
> >>>>+		if (!kernel_text_address((unsigned long)cur->addr)) {
> >>>>+			struct uprobe *uprobe =  __get_cpu_var(current_uprobe);
> >>>>+			/* TODO: Proper handling of all instruction */
> >>>>+			replace_original_insn(uprobe, regs, uprobe->kp.opcode);
> >>>>+			regs->eflags &= ~TF_MASK;
> >>>>+		} else
> >>>>+			resume_execution(cur, regs, kcb);
> >>>> 		regs->eflags |= kcb->kprobe_old_eflags;
> >>>>
> >>>> 		reset_current_kprobe();
> >>>>@@ -654,7 +1030,67 @@ int __kprobes longjmp_break_handler(stru
> >>>> 	return 0;
> >>>> }
> >>>>
> >>>>+static void free_alias(void)
> >>>>+{
> >>>>+	int cpu;
> >>>>+
> >>>>+	for_each_cpu(cpu) {
> >>>>+		struct uprobe_page *upage;
> >>>>+		upage = per_cpu_ptr(uprobe_page, cpu);
> >>>>+
> >>>>+		if (upage->alias_addr) {
> >>>>+			set_pte(upage->alias_pte, __pte(upage->alias_pte_val));
> >>>>+			kfree(upage->alias_addr);
> >>>>+		}
> >>>>+		upage->alias_pte = 0;
> >>>>+	}
> >>>>+	free_percpu(uprobe_page);
> >>>>+	return;
> >>>>+}
> >>>>+
> >>>>+static int alloc_alias(void)
> >>>>+{
> >>>>+	int cpu;
> >>>>+
> >>>>+	uprobe_page = __alloc_percpu(sizeof(struct uprobe_page));
> >>[YM] Do here codes try to resolve the problem of task switch at single-step? If so, the per cpu data also might be used up although
> >>get_upage_free will go through all uprobe_page of all cpus. I suggest to allocate a series of uprobe_page, and allocate again when they
> >>are used up.
> >>
> >>
> >>
> >>
> >>>>+
> >>>>+	for_each_cpu(cpu) {
> >>>>+		struct uprobe_page *upage;
> >>>>+		upage = per_cpu_ptr(uprobe_page, cpu);
> >>>>+		upage->alias_addr = kmalloc(PAGE_SIZE, GFP_USER);
> >>[YM] Does kmalloc(PAGE_SIZE...) imply the result is aligned to page? How about using alloc_page?
> >>
> >>
> >>>>+		if (!upage->alias_addr) {
> >>>>+			free_alias();
> >>>>+			return -ENOMEM;
> >>>>+		}
> >>>>+		upage->alias_pte = lookup_address(
> >>>>+					(unsigned long)upage->alias_addr);
> >>>>+		upage->alias_pte_val = pte_val(*upage->alias_pte);
> >>>>+		if (upage->alias_pte) {
> >>[YM] If kmalloc returns a non-NULL address, upage->alias_pte is not equal to NULL. So delete above checking?
> >>
> >>
> >>>>+			upage->status = UPROBE_PAGE_FREE;
> >>>>+			set_pte(upage->alias_pte,
> >>>>+						pte_mkdirty(*upage->alias_pte));
> >>>>+			set_pte(upage->alias_pte,
> >>>>+						pte_mkexec(*upage->alias_pte));
> >>>>+			set_pte(upage->alias_pte,
> >>>>+						 pte_mkwrite(*upage->alias_pte));
> >>>>+			set_pte(upage->alias_pte,
> >>>>+						pte_mkyoung(*upage->alias_pte));
> >>>>+		}
> >>>>+	}
> >>>>+	return 0;
> >>>>+}
> >>>>+
> >>>> int __init arch_init_kprobes(void)
> >>>> {
> >>>>+	int ret = 0;
> >>>>+	/*
> >>>>+	 * user space probes requires a page to copy the original instruction
> >>>>+	 * so that it can single step if there is no free stack space, allocate
> >>>>+	 * per cpu page.
> >>>>+	 */
> >>>>+
> >>>>+	if ((ret = alloc_alias()))
> >>>>+		return ret;
> >>>>+
> >>>> 	return 0;
> >>>> }
> >>>>diff -puN include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line include/asm-i386/kprobes.h
> >>>>--- linux-2.6.16-rc1-mm5/include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
> >>>>+++ linux-2.6.16-rc1-mm5-prasanna/include/asm-i386/kprobes.h	2006-02-08 19:26:10.000000000 +0530
> >>>>@@ -42,6 +42,7 @@ typedef u8 kprobe_opcode_t;
> >>>> #define JPROBE_ENTRY(pentry)	(kprobe_opcode_t *)pentry
> >>>> #define ARCH_SUPPORTS_KRETPROBES
> >>>> #define arch_remove_kprobe(p)	do {} while (0)
> >>>>+#define UPROBE_PAGE_FREE 0x00000001
> >>>>
> >>>> void kretprobe_trampoline(void);
> >>>>
> >>>>@@ -74,6 +75,18 @@ struct kprobe_ctlblk {
> >>>> 	struct prev_kprobe prev_kprobe;
> >>>> };
> >>>>
> >>>>+/* per cpu uprobe page structure */
> >>>>+struct uprobe_page {
> >>>>+	struct hlist_node hlist;
> >>>>+	pte_t *alias_pte;
> >>>>+	pte_t *orig_pte;
> >>>>+	unsigned long orig_pte_val;
> >>>>+	unsigned long alias_pte_val;
> >>[YM] I think the patch doesn't support CONFIG_X86_PAE, because if CONFIG_X86_PAE=y, pte_t becomes 64 bits.
> >>How about changing above 2 members' type to pte_t directly?
> >>
> >>
> >>
> >>>>+	void *alias_addr;
> >>>>+	struct task_struct *tsk;
> >>>>+	unsigned long status;
> >>>>+};
> >>>>+
> >>>> /* trap3/1 are intr gates for kprobes.  So, restore the status of IF,
> >>>>  * if necessary, before executing the original int3/1 (trap) handler.
> >>>>  */
> >>>>diff -puN include/linux/kprobes.h~kprobes_userspace_probes_ss_out-of-line include/linux/kprobes.h
> >>>>--- linux-2.6.16-rc1-mm5/include/linux/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
> >>>>+++ linux-2.6.16-rc1-mm5-prasanna/include/linux/kprobes.h	2006-02-08 19:26:10.000000000 +0530
> >>>>@@ -45,11 +45,18 @@
> >>>> #ifdef CONFIG_KPROBES
> >>>> #include <asm/kprobes.h>
> >>>>
> >>>>+#define KPROBE_HASH_BITS 6
> >>>>+#define KPROBE_TABLE_SIZE (1 << KPROBE_HASH_BITS)
> >>>>+
> >>>> /* kprobe_status settings */
> >>>> #define KPROBE_HIT_ACTIVE	0x00000001
> >>>> #define KPROBE_HIT_SS		0x00000002
> >>>> #define KPROBE_REENTER		0x00000004
> >>>> #define KPROBE_HIT_SSDONE	0x00000008
> >>>>+#define UPROBE_SS_STACK		0x00000010
> >>>>+#define UPROBE_SS_EXPSTACK	0x00000020
> >>>>+#define UPROBE_SS_INLINE	0x00000040
> >>>>+#define UPROBE_SS_NEW_STACK	0x00000080
> >>>>
> >>>> /* Attach to insert probes on any functions which should be ignored*/
> >>>> #define __kprobes	__attribute__((__section__(".kprobes.text")))
> >>>>diff -puN kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line kernel/kprobes.c
> >>>>--- linux-2.6.16-rc1-mm5/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:10.000000000 +0530
> >>>>+++ linux-2.6.16-rc1-mm5-prasanna/kernel/kprobes.c	2006-02-08 19:26:10.000000000 +0530
> >>>>@@ -42,9 +42,6 @@
> >>>> #include <asm/errno.h>
> >>>> #include <asm/kdebug.h>
> >>>>
> >>>>-#define KPROBE_HASH_BITS 6
> >>>>-#define KPROBE_TABLE_SIZE (1 << KPROBE_HASH_BITS)
> >>>>-
> >>>> static struct hlist_head kprobe_table[KPROBE_TABLE_SIZE];
> >>>> static struct hlist_head kretprobe_inst_table[KPROBE_TABLE_SIZE];
> >>>> static struct list_head uprobe_module_list;
> >>>>
> >>>>_
> >>>>--
> >>>>Prasanna S Panchamukhi
> >>>>Linux Technology Center
> >>>>India Software Labs, IBM Bangalore
> >>>>Email: prasanna@in.ibm.com
> >>>>Ph: 91-80-51776329

-- 
Thanks & Regards
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] 11+ messages in thread

* RE: [3/3] Userspace probes prototype-take2
@ 2006-02-20  3:32 Zhang, Yanmin
  2006-02-20  5:07 ` Prasanna S Panchamukhi
  0 siblings, 1 reply; 11+ messages in thread
From: Zhang, Yanmin @ 2006-02-20  3:32 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap

>>-----Original Message-----
>>From: Zhang, Yanmin
>>Sent: 2006年2月20日 11:16
>>To: Zhang, Yanmin; prasanna@in.ibm.com; systemtap@sources.redhat.com
>>Subject: RE: [3/3] Userspace probes prototype-take2
>>
>>I lost an important comment. The patch is not aware of signal processing. After kernel prepares the single-step-inst on the stack, if
>>a signal is delivered to the thread, kernel will save some states into stack and switch to signal handler function, so single-step-inst
>>on the stack might be erased.
>>
>>>>-----Original Message-----
>>>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Zhang, Yanmin
>>>>Sent: 2006年2月17日 17:20
>>>>To: prasanna@in.ibm.com; systemtap@sources.redhat.com
>>>>Subject: RE: [3/3] Userspace probes prototype-take2
>>>>
>>>>2 main issues:
>>>>1) task switch caused by external interrupt when single-step;
[YM] I think we could resolve this problem. Kernel probe has some differences from uprobe. One of them is that we couldn't estimate if kernel probe happens in process context, or interrupt context, while uprobe always happens in process context (user space). So from some points of view, uprobe could be simplified from kernel probe.
a) Don't use kcb (kprobe_ctlblk) if uprobe is triggered. Create new functions, kprobe__handler_user, kprobe_fault_handler_user and other handlers. In the new functions, instead of kcb, we could use uprobe_page being allocated dynamically. Considering signal action handler (possible uprobe nested), a thread might have a list of uprobe_page.
b) Delete current_uprobe;



>>>>2) multi-thread:
[YM] We could resolve this problem.
a) Don't call replace_orignal_insn in function uprobe_single_step. It might cause a race condition.
b) Delete copy_insn_on_new_page;
c) Merge copy_insn_onstack and copy_insn_onexpstack. The single-step-insn address could be esp-sizeof(long long)-MAX_INSN_SIZE*sizeof(kprobe_opcode_t). 
d) If the stack couldn't be expanded, just kill the thread. It's reasonable because the stack is used up.


>>>>
>>>>See below inline comments.
>>>>
>>>>Yanmin
>>>>
>>>>>>-----Original Message-----
>>>>>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Prasanna S Panchamukhi
>>>>>>Sent: 2006年2月8日 22:14
>>>>>>To: systemtap@sources.redhat.com
>>>>>>Subject: Re: [3/3] Userspace probes prototype-take2
>>>>>>
>>>>>>
>>>>>>This patch handles the executing the registered callback
>>>>>>functions when probes is hit.

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

* RE: [3/3] Userspace probes prototype-take2
@ 2006-02-20  3:16 Zhang, Yanmin
  2006-02-20  4:51 ` Prasanna S Panchamukhi
  0 siblings, 1 reply; 11+ messages in thread
From: Zhang, Yanmin @ 2006-02-20  3:16 UTC (permalink / raw)
  To: Zhang, Yanmin, prasanna, systemtap

I lost an important comment. The patch is not aware of signal processing. After kernel prepares the single-step-inst on the stack, if a signal is delivered to the thread, kernel will save some states into stack and switch to signal handler function, so single-step-inst on the stack might be erased.

>>-----Original Message-----
>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Zhang, Yanmin
>>Sent: 2006年2月17日 17:20
>>To: prasanna@in.ibm.com; systemtap@sources.redhat.com
>>Subject: RE: [3/3] Userspace probes prototype-take2
>>
>>2 main issues:
>>1) task switch caused by external interrupt when single-step;
>>2) multi-thread:
>>
>>See below inline comments.
>>
>>Yanmin
>>
>>>>-----Original Message-----
>>>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Prasanna S Panchamukhi
>>>>Sent: 2006年2月8日 22:14
>>>>To: systemtap@sources.redhat.com
>>>>Subject: Re: [3/3] Userspace probes prototype-take2
>>>>
>>>>
>>>>This patch handles the executing the registered callback
>>>>functions when probes is hit.
>>>>
>>>>	Each userspace probe is uniquely identified by the
>>>>combination of inode and offset, hence during registeration the inode
>>>>and offset combination is added to kprobes hash table. Initially when
>>>>breakpoint instruction is hit, the kprobes hash table is looked up
>>>>for matching inode and offset. The pre_handlers are called in sequence
>>>>if multiple probes are registered. The original instruction is single
>>>>stepped out-of-line similar to kernel probes. In kernel space probes,
>>>>single stepping out-of-line is achieved by copying the instruction on
>>>>to some location within kernel address space and then single step
>>>>from that location. But for userspace probes, instruction copied
>>>>into kernel address space cannot be single stepped, hence the
>>>>instruction should 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. If the stack grows
>>>>downwards, the stack space below the stack pointer indicates the
>>>>unused stack free space and if the stack grows upwards, the stack
>>>>space above the stack pointer indicates the unused stack free space.
>>>>
>>>>The instruction to be single stepped can modify the stack space, hence
>>>>before using the unused stack free 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 unused 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. This patch is not stable yet, should work for most conditions.
>>>>
>>>>2. This patch works only with PREEMPT config option disabled, to work
>>>>in PREEMPT enabled condition handlers must be re-written and must
>>>>be seperated out from kernel probes allowing preemption.
>>One of my old comments is an external device interrupt might happen when cpu is single-stepping the original instruction, then the task
>>might be switched to another cpu. If we disable irq when exiting to user space to single step the instruction, kernel might switch the
>>task off just on the exit kernel path. 1) uprobe_page; 2) kprobe_ctlblk, These 2 resources shouldn't be pre cpu, or we need get another
>>approach. How could you resolve the task switch issue?
>>
>>
>>
>>>>
>>>>3. 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.
>>>>
>>>>4. Optimize the insertion of probes through readpage hooks. Identify
>>>>all the probes to be inserted on the read page and insert them at
>>>>once.
>>>>
>>>>5. Resume exectution should handle setting of proper eip and eflags
>>>>for special instructions similar to kernel probes.
>>>>
>>>>6. Single stepping out-of-line expands the stack if there is no
>>>>enough stack space to copy the original instruction. Expansion of
>>>>stack should be shrinked back to the original size after single
>>>>stepping or the expanded stack should be reused for single stepping
>>>>out-of-line for other probes.
>>>>
>>>>7. A wrapper routines to calculate the offset from the probed file
>>>>beginning. In case of dynamic shared library, the offset is
>>>>calculated by substracting the address of the probe point from the
>>>>beginning of the file mapped address.
>>>>
>>>>8. Handing of page faults while inthe kprobes_handler() and while
>>>>single stepping.
>>>>
>>>>9. Accessing user space pages not present in memory, from the
>>>>registered callback routines.
>>>>
>>>>Signed-off-by: Prasanna S Panchamukhi <prasanna@in.ibm.com>
>>>>
>>>>
>>>> arch/i386/kernel/kprobes.c |  460 +++++++++++++++++++++++++++++++++++++++++++--
>>>> include/asm-i386/kprobes.h |   13 +
>>>> include/linux/kprobes.h    |    7
>>>> kernel/kprobes.c           |    3
>>>> 4 files changed, 468 insertions(+), 15 deletions(-)
>>>>
>>>>diff -puN arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line arch/i386/kernel/kprobes.c
>>>>--- linux-2.6.16-rc1-mm5/arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
>>>>+++ linux-2.6.16-rc1-mm5-prasanna/arch/i386/kernel/kprobes.c	2006-02-08 19:26:10.000000000 +0530
>>>>@@ -30,6 +30,7 @@
>>>>
>>>> #include <linux/config.h>
>>>> #include <linux/kprobes.h>
>>>>+#include <linux/hash.h>
>>>> #include <linux/ptrace.h>
>>>> #include <linux/preempt.h>
>>>> #include <asm/cacheflush.h>
>>>>@@ -38,8 +39,12 @@
>>>>
>>>> void jprobe_return_end(void);
>>>>
>>>>+static struct uprobe_page *uprobe_page;
>>>>+static struct hlist_head uprobe_page_table[KPROBE_TABLE_SIZE];
>>>> DEFINE_PER_CPU(struct kprobe *, current_kprobe) = NULL;
>>>> DEFINE_PER_CPU(struct kprobe_ctlblk, kprobe_ctlblk);
>>>>+DEFINE_PER_CPU(struct uprobe *, current_uprobe) = NULL;
>>>>+DEFINE_PER_CPU(unsigned long, singlestep_addr);
>>>>
>>>> /* insert a jmp code */
>>>> static inline void set_jmp_op(void *from, void *to)
>>>>@@ -125,6 +130,23 @@ void __kprobes arch_disarm_kprobe(struct
>>>> 			   (unsigned long) p->addr + sizeof(kprobe_opcode_t));
>>>> }
>>>>
>>>>+void __kprobes arch_disarm_uprobe(struct kprobe *p, kprobe_opcode_t *address)
>>>>+{
>>>>+	*address = p->opcode;
>>>>+}
>>>>+
>>>>+void __kprobes arch_arm_uprobe(unsigned long *address)
>>>>+{
>>>>+	*(kprobe_opcode_t *)address = BREAKPOINT_INSTRUCTION;
>>>>+}
>>>>+
>>>>+void __kprobes arch_copy_uprobe(struct kprobe *p, unsigned long *address)
>>>>+{
>>>>+	memcpy(p->ainsn.insn, (kprobe_opcode_t *)address,
>>>>+				MAX_INSN_SIZE * sizeof(kprobe_opcode_t));
>>>>+	p->opcode = *(kprobe_opcode_t *)address;
>>>>+}
>>>>+
>>>> static inline void save_previous_kprobe(struct kprobe_ctlblk *kcb)
>>>> {
>>>> 	kcb->prev_kprobe.kp = kprobe_running();
>>>>@@ -151,15 +173,326 @@ static inline void set_current_kprobe(st
>>>> 		kcb->kprobe_saved_eflags &= ~IF_MASK;
>>>> }
>>>>
>>>>+struct uprobe_page __kprobes *get_upage_current(struct task_struct *tsk)
>>>>+{
>>>>+	struct hlist_head *head;
>>>>+	struct hlist_node *node;
>>>>+	struct uprobe_page *upage;
>>>>+
>>>>+	head = &uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)];
>>>>+	hlist_for_each_entry(upage, node, head, hlist) {
>>>>+		if (upage->tsk == tsk)
>>>>+			return upage;
>>>>+        }
>>>>+	return NULL;
>>>>+}
>>>>+
>>>>+struct uprobe_page __kprobes *get_upage_free(struct task_struct *tsk)
>>>>+{
>>>>+	int cpu;
>>>>+
>>>>+	for_each_cpu(cpu) {
>>>>+		struct uprobe_page *upage;
>>>>+		upage = per_cpu_ptr(uprobe_page, cpu);
>>>>+		if (upage->status & UPROBE_PAGE_FREE)
>>>>+			return upage;
>>>>+	}
>>>>+	return NULL;
>>>>+}
>>>>+
>>>>+/**
>>>>+ * 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, it just maps a new page and copies the
>>>>+ *  new 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, *vaddr, stack_addr = regs->esp;
>>>>+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
>>>>+	struct uprobe_page *upage;
>>>>+	struct page *page;
>>>>+	pte_t *pte;
>>>>+
>>>>+
>>>>+	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;
>>>>+
>>The multi-thread case is not resolved here. One of typical multi-thread model is that the all threads share the same vma and every thread
>>has 8-k stack. If 2 threads trigger uprobe (although might be not the same uprobe) at the same time, one thread might erase single-step
>>instruction of another.
>>
>>
>>
>>>>+	preempt_enable_no_resched();
>>>>+
>>>>+	pte = get_uprobe_pte(addr);
>>>>+	preempt_disable();
>>>>+	if (!pte)
>>>>+		return -EFAULT;
>>>>+
>>>>+	upage = get_upage_free(current);
>>>>+	upage->status &= ~UPROBE_PAGE_FREE;
>>>>+	upage->tsk = current;
>>>>+	INIT_HLIST_NODE(&upage->hlist);
>>>>+	hlist_add_head(&upage->hlist,
>>>>+		&uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)]);
>>>>+
>>>>+	upage->orig_pte = pte;
>>>>+	upage->orig_pte_val =  pte_val(*pte);
>>>>+	set_pte(pte, (*(upage->alias_pte)));
>>>>+
>>>>+	page = pte_page(*pte);
>>>>+	vaddr = (unsigned long *)kmap_atomic(page, KM_USER1);
>>>>+	vaddr = (unsigned long *)((unsigned long)vaddr + (addr & ~PAGE_MASK));
>>>>+	memcpy(vaddr, (unsigned long *)uprobe->kp.ainsn.insn, size);
>>>>+	kunmap_atomic(vaddr, KM_USER1);
>>>>+	regs->eip = addr;
>>So the temp page, upage->alias_addr, replaces the original one on the stack. If the replaced instruction is to operate stack, such like
>>"push eax", the result might be on the new page. After the single step, the pte is restored to the original page which doesn't have
>>the value of eax.
>>
>>
>>
>>>>+
>>>>+	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)
>>It has the same issues like function copy_insn_on_new_page.
>>
>>
>>>>+{
>>>>+	unsigned long addr, *vaddr, vm_addr;
>>>>+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
>>>>+	struct vm_area_struct *new_vma;
>>>>+	struct uprobe_page *upage;
>>>>+	struct mm_struct *mm = current->mm;
>>>>+	struct page *page;
>>>>+	pte_t *pte;
>>>>+
>>>>+
>>>>+	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
>>>>+		return -EFAULT;
>>>>+
>>>>+	preempt_enable_no_resched();
>>>>+
>>>>+	/* TODO: do we need to expand stack if extend_vma fails? */
>>>>+	new_vma = find_extend_vma(mm, vm_addr);
>>>>+	preempt_disable();
>>>>+	if (!new_vma)
>>>>+		return -ENOMEM;
>>>>+
>>>>+	/*
>>>>+	 * TODO: Expanding stack for every probe is not a good idea, stack must
>>>>+	 * either be shrunk to its original size after single stepping or the
>>>>+	 * expanded stack should be kept track of, for the probed application,
>>>>+	 * so it can be reused to single step out-of-line
>>>>+	 */
>>>>+	if (new_vma->vm_flags & VM_GROWSDOWN)
>>>>+		addr = new_vma->vm_start;
>>>>+	else
>>>>+		addr = new_vma->vm_end - size;
>>>>+
>>>>+	preempt_enable_no_resched();
>>>>+	pte = get_uprobe_pte(addr);
>>>>+	preempt_disable();
>>>>+	if (!pte)
>>>>+		return -EFAULT;
>>>>+
>>>>+	upage = get_upage_free(current);
>>>>+	upage->status &= ~UPROBE_PAGE_FREE;
>>>>+	upage->tsk = current;
>>>>+	INIT_HLIST_NODE(&upage->hlist);
>>>>+	hlist_add_head(&upage->hlist,
>>>>+		&uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)]);
>>>>+	upage->orig_pte = pte;
>>>>+	upage->orig_pte_val =  pte_val(*pte);
>>>>+	set_pte(pte, (*(upage->alias_pte)));
>>>>+
>>>>+	page = pte_page(*pte);
>>>>+	vaddr = (unsigned long *)kmap_atomic(page, KM_USER1);
>>>>+	vaddr = (unsigned long *)((unsigned long)vaddr + (addr & ~PAGE_MASK));
>>>>+	memcpy(vaddr, (unsigned long *)uprobe->kp.ainsn.insn, size);
>>>>+	kunmap_atomic(vaddr, KM_USER1);
>>>>+	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 no 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);
>>>>+	lock_page(page);
>>>>+
>>>>+	addr = (kprobe_opcode_t *)kmap_atomic(page, KM_USER0);
>>>>+	addr = (kprobe_opcode_t *)((unsigned long)addr +
>>>>+				 (unsigned long)(uprobe->offset & ~PAGE_MASK));
>>>>+	*addr = opcode;
>>>>+	/*TODO: flush vma ? */
>>>>+	kunmap_atomic(addr, KM_USER0);
>>>>+
>>>>+	unlock_page(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 uprobe_single_step(struct kprobe *p, struct pt_regs *regs)
>>>>+{
>>>>+	unsigned long stack_addr = regs->esp, flags;
>>>>+	struct vm_area_struct *vma = NULL;
>>>>+	struct uprobe *uprobe =  __get_cpu_var(current_uprobe);
>>>>+	struct kprobe_ctlblk *kcb = get_kprobe_ctlblk();
>>>>+	int err = 0;
>>>>+
>>>>+	down_read(&current->mm->mmap_sem);
>>>>+
>>>>+	vma = find_vma(current->mm, (stack_addr & PAGE_MASK));
>>>>+	if (!vma) {
>>>>+		/* TODO: Need better error reporting? */
>>>>+		printk("No vma found\n");
>>>>+		up_read(&current->mm->mmap_sem);
>>>>+		return -ENOENT;
>>>>+	}
>>>>+	flags = vma->vm_flags;
>>>>+	up_read(&current->mm->mmap_sem);
>>>>+
>>>>+	kcb->kprobe_status |= UPROBE_SS_STACK;
>>>>+	err = copy_insn_onstack(uprobe, regs, flags);
>>>>+
>>>>+	down_write(&current->mm->mmap_sem);
>>>>+
>>>>+	if (err) {
>>>>+		kcb->kprobe_status |= UPROBE_SS_NEW_STACK;
>>>>+		err = copy_insn_on_new_page(uprobe, regs, vma);
>>>>+	}
>>>>+	if (err) {
>>>>+		kcb->kprobe_status |= UPROBE_SS_EXPSTACK;
>>>>+		err = copy_insn_onexpstack(uprobe, regs, vma);
>>>>+	}
>>>>+
>>>>+	up_write(&current->mm->mmap_sem);
>>>>+
>>>>+	if (err) {
>>>>+		kcb->kprobe_status |= UPROBE_SS_INLINE;
>>>>+		replace_original_insn(uprobe, regs, uprobe->kp.opcode);
>>>>+	}
>>>>+
>>>>+	 __get_cpu_var(singlestep_addr) = regs->eip;
>>>>+
>>>>+
>>>>+	return 0;
>>>>+}
>>>>+
>>>> static inline void prepare_singlestep(struct kprobe *p, struct pt_regs *regs)
>>>> {
>>>> 	regs->eflags |= TF_MASK;
>>>> 	regs->eflags &= ~IF_MASK;
>>>> 	/*single step inline if the instruction is an int3*/
>>>>+
>>>> 	if (p->opcode == BREAKPOINT_INSTRUCTION)
>>>> 		regs->eip = (unsigned long)p->addr;
>>>>-	else
>>>>-		regs->eip = (unsigned long)&p->ainsn.insn;
>>>>+	else {
>>>>+		if (!kernel_text_address((unsigned long)p->addr))
>>>>+			uprobe_single_step(p, regs);
>>>>+		else
>>>>+			regs->eip = (unsigned long)&p->ainsn.insn;
>>>>+	}
>>>> }
>>>>
>>>> /* Called with kretprobe_lock held */
>>>>@@ -194,6 +527,7 @@ static int __kprobes kprobe_handler(stru
>>>> 	kprobe_opcode_t *addr = NULL;
>>>> 	unsigned long *lp;
>>>> 	struct kprobe_ctlblk *kcb;
>>>>+	unsigned seg = regs->xcs & 0xffff;
>>>> #ifdef CONFIG_PREEMPT
>>>> 	unsigned pre_preempt_count = preempt_count();
>>>> #endif /* CONFIG_PREEMPT */
>>>>@@ -208,14 +542,21 @@ static int __kprobes kprobe_handler(stru
>>>> 	/* Check if the application is using LDT entry for its code segment and
>>>> 	 * calculate the address by reading the base address from the LDT entry.
>>>> 	 */
>>>>-	if ((regs->xcs & 4) && (current->mm)) {
>>>>+
>>>>+	if (regs->eflags & VM_MASK)
>>>>+		addr = (kprobe_opcode_t *)(((seg << 4) + regs->eip -
>>>>+			sizeof(kprobe_opcode_t)) & 0xffff);
>>>>+	else if ((regs->xcs & 4) && (current->mm)) {
>>>>+		local_irq_enable();
>>>>+		down(&current->mm->context.sem);
>>>> 		lp = (unsigned long *) ((unsigned long)((regs->xcs >> 3) * 8)
>>>> 					+ (char *) current->mm->context.ldt);
>>>> 		addr = (kprobe_opcode_t *) (get_desc_base(lp) + regs->eip -
>>>> 						sizeof(kprobe_opcode_t));
>>>>-	} else {
>>>>+		up(&current->mm->context.sem);
>>>>+		local_irq_disable();
>>>>+	} else
>>>> 		addr = (kprobe_opcode_t *)(regs->eip - sizeof(kprobe_opcode_t));
>>>>-	}
>>>> 	/* Check we're not actually recursing */
>>>> 	if (kprobe_running()) {
>>>> 		p = get_kprobe(addr);
>>>>@@ -235,7 +576,6 @@ static int __kprobes kprobe_handler(stru
>>>> 			save_previous_kprobe(kcb);
>>>> 			set_current_kprobe(p, regs, kcb);
>>>> 			kprobes_inc_nmissed_count(p);
>>>>-			prepare_singlestep(p, regs);
>>>> 			kcb->kprobe_status = KPROBE_REENTER;
>>>> 			return 1;
>>>> 		} else {
>>>>@@ -307,8 +647,8 @@ static int __kprobes kprobe_handler(stru
>>>> 	}
>>>>
>>>> ss_probe:
>>>>-	prepare_singlestep(p, regs);
>>>> 	kcb->kprobe_status = KPROBE_HIT_SS;
>>>>+	prepare_singlestep(p, regs);
>>>> 	return 1;
>>>>
>>>> no_kprobe:
>>>>@@ -498,6 +838,33 @@ no_change:
>>>> 	return;
>>>> }
>>>>
>>>>+static void __kprobes resume_execution_user(struct uprobe *uprobe,
>>>>+				struct pt_regs *regs, struct kprobe_ctlblk *kcb)
>>>>+{
>>>>+	unsigned long delta;
>>>>+	struct uprobe_page *upage;
>>>>+
>>>>+	/*
>>>>+	 * TODO :need to fixup special instructions as done with kernel probes.
>>>>+	 */
>>>>+	delta = regs->eip - __get_cpu_var(singlestep_addr);
>>>>+	regs->eip = (unsigned long)(uprobe->kp.addr + delta);
>>>>+
>>>>+	if ((kcb->kprobe_status & UPROBE_SS_EXPSTACK) ||
>>>>+			(kcb->kprobe_status & UPROBE_SS_NEW_STACK)) {
>>>>+		upage = get_upage_current(current);
>>>>+		set_pte(upage->orig_pte, __pte(upage->orig_pte_val));
>>>>+		pte_unmap(upage->orig_pte);
>>>>+
>>>>+		upage->status = UPROBE_PAGE_FREE;
>>>>+		hlist_del(&upage->hlist);
>>>>+
>>>>+	} else if (kcb->kprobe_status & UPROBE_SS_INLINE)
>>>>+		replace_original_insn(uprobe, regs,
>>>>+				(kprobe_opcode_t)BREAKPOINT_INSTRUCTION);
>>>>+	regs->eflags &= ~TF_MASK;
>>>>+}
>>>>+
>>>> /*
>>>>  * Interrupts are disabled on entry as trap1 is an interrupt gate and they
>>>>  * remain disabled thoroughout this function.
>>>>@@ -510,16 +877,19 @@ static inline int post_kprobe_handler(st
>>>> 	if (!cur)
>>>> 		return 0;
>>>>
>>>>-	if ((kcb->kprobe_status != KPROBE_REENTER) && cur->post_handler) {
>>>>-		kcb->kprobe_status = KPROBE_HIT_SSDONE;
>>>>+	if (!(kcb->kprobe_status & KPROBE_REENTER) && cur->post_handler) {
>>>>+		kcb->kprobe_status |= KPROBE_HIT_SSDONE;
>>>> 		cur->post_handler(cur, regs, 0);
>>>> 	}
>>>>
>>>>-	resume_execution(cur, regs, kcb);
>>>>+	if (!kernel_text_address((unsigned long)cur->addr))
>>>>+		resume_execution_user(__get_cpu_var(current_uprobe), regs, kcb);
>>>>+	else
>>>>+		resume_execution(cur, regs, kcb);
>>>> 	regs->eflags |= kcb->kprobe_saved_eflags;
>>>>
>>>> 	/*Restore back the original saved kprobes variables and continue. */
>>>>-	if (kcb->kprobe_status == KPROBE_REENTER) {
>>>>+	if (kcb->kprobe_status & KPROBE_REENTER) {
>>>> 		restore_previous_kprobe(kcb);
>>>> 		goto out;
>>>> 	}
>>>>@@ -547,7 +917,13 @@ static inline int kprobe_fault_handler(s
>>>> 		return 1;
>>>>
>>>> 	if (kcb->kprobe_status & KPROBE_HIT_SS) {
>>>>-		resume_execution(cur, regs, kcb);
>>>>+		if (!kernel_text_address((unsigned long)cur->addr)) {
>>>>+			struct uprobe *uprobe =  __get_cpu_var(current_uprobe);
>>>>+			/* TODO: Proper handling of all instruction */
>>>>+			replace_original_insn(uprobe, regs, uprobe->kp.opcode);
>>>>+			regs->eflags &= ~TF_MASK;
>>>>+		} else
>>>>+			resume_execution(cur, regs, kcb);
>>>> 		regs->eflags |= kcb->kprobe_old_eflags;
>>>>
>>>> 		reset_current_kprobe();
>>>>@@ -654,7 +1030,67 @@ int __kprobes longjmp_break_handler(stru
>>>> 	return 0;
>>>> }
>>>>
>>>>+static void free_alias(void)
>>>>+{
>>>>+	int cpu;
>>>>+
>>>>+	for_each_cpu(cpu) {
>>>>+		struct uprobe_page *upage;
>>>>+		upage = per_cpu_ptr(uprobe_page, cpu);
>>>>+
>>>>+		if (upage->alias_addr) {
>>>>+			set_pte(upage->alias_pte, __pte(upage->alias_pte_val));
>>>>+			kfree(upage->alias_addr);
>>>>+		}
>>>>+		upage->alias_pte = 0;
>>>>+	}
>>>>+	free_percpu(uprobe_page);
>>>>+	return;
>>>>+}
>>>>+
>>>>+static int alloc_alias(void)
>>>>+{
>>>>+	int cpu;
>>>>+
>>>>+	uprobe_page = __alloc_percpu(sizeof(struct uprobe_page));
>>[YM] Do here codes try to resolve the problem of task switch at single-step? If so, the per cpu data also might be used up although
>>get_upage_free will go through all uprobe_page of all cpus. I suggest to allocate a series of uprobe_page, and allocate again when they
>>are used up.
>>
>>
>>
>>
>>>>+
>>>>+	for_each_cpu(cpu) {
>>>>+		struct uprobe_page *upage;
>>>>+		upage = per_cpu_ptr(uprobe_page, cpu);
>>>>+		upage->alias_addr = kmalloc(PAGE_SIZE, GFP_USER);
>>[YM] Does kmalloc(PAGE_SIZE...) imply the result is aligned to page? How about using alloc_page?
>>
>>
>>>>+		if (!upage->alias_addr) {
>>>>+			free_alias();
>>>>+			return -ENOMEM;
>>>>+		}
>>>>+		upage->alias_pte = lookup_address(
>>>>+					(unsigned long)upage->alias_addr);
>>>>+		upage->alias_pte_val = pte_val(*upage->alias_pte);
>>>>+		if (upage->alias_pte) {
>>[YM] If kmalloc returns a non-NULL address, upage->alias_pte is not equal to NULL. So delete above checking?
>>
>>
>>>>+			upage->status = UPROBE_PAGE_FREE;
>>>>+			set_pte(upage->alias_pte,
>>>>+						pte_mkdirty(*upage->alias_pte));
>>>>+			set_pte(upage->alias_pte,
>>>>+						pte_mkexec(*upage->alias_pte));
>>>>+			set_pte(upage->alias_pte,
>>>>+						 pte_mkwrite(*upage->alias_pte));
>>>>+			set_pte(upage->alias_pte,
>>>>+						pte_mkyoung(*upage->alias_pte));
>>>>+		}
>>>>+	}
>>>>+	return 0;
>>>>+}
>>>>+
>>>> int __init arch_init_kprobes(void)
>>>> {
>>>>+	int ret = 0;
>>>>+	/*
>>>>+	 * user space probes requires a page to copy the original instruction
>>>>+	 * so that it can single step if there is no free stack space, allocate
>>>>+	 * per cpu page.
>>>>+	 */
>>>>+
>>>>+	if ((ret = alloc_alias()))
>>>>+		return ret;
>>>>+
>>>> 	return 0;
>>>> }
>>>>diff -puN include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line include/asm-i386/kprobes.h
>>>>--- linux-2.6.16-rc1-mm5/include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
>>>>+++ linux-2.6.16-rc1-mm5-prasanna/include/asm-i386/kprobes.h	2006-02-08 19:26:10.000000000 +0530
>>>>@@ -42,6 +42,7 @@ typedef u8 kprobe_opcode_t;
>>>> #define JPROBE_ENTRY(pentry)	(kprobe_opcode_t *)pentry
>>>> #define ARCH_SUPPORTS_KRETPROBES
>>>> #define arch_remove_kprobe(p)	do {} while (0)
>>>>+#define UPROBE_PAGE_FREE 0x00000001
>>>>
>>>> void kretprobe_trampoline(void);
>>>>
>>>>@@ -74,6 +75,18 @@ struct kprobe_ctlblk {
>>>> 	struct prev_kprobe prev_kprobe;
>>>> };
>>>>
>>>>+/* per cpu uprobe page structure */
>>>>+struct uprobe_page {
>>>>+	struct hlist_node hlist;
>>>>+	pte_t *alias_pte;
>>>>+	pte_t *orig_pte;
>>>>+	unsigned long orig_pte_val;
>>>>+	unsigned long alias_pte_val;
>>[YM] I think the patch doesn't support CONFIG_X86_PAE, because if CONFIG_X86_PAE=y, pte_t becomes 64 bits.
>>How about changing above 2 members' type to pte_t directly?
>>
>>
>>
>>>>+	void *alias_addr;
>>>>+	struct task_struct *tsk;
>>>>+	unsigned long status;
>>>>+};
>>>>+
>>>> /* trap3/1 are intr gates for kprobes.  So, restore the status of IF,
>>>>  * if necessary, before executing the original int3/1 (trap) handler.
>>>>  */
>>>>diff -puN include/linux/kprobes.h~kprobes_userspace_probes_ss_out-of-line include/linux/kprobes.h
>>>>--- linux-2.6.16-rc1-mm5/include/linux/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
>>>>+++ linux-2.6.16-rc1-mm5-prasanna/include/linux/kprobes.h	2006-02-08 19:26:10.000000000 +0530
>>>>@@ -45,11 +45,18 @@
>>>> #ifdef CONFIG_KPROBES
>>>> #include <asm/kprobes.h>
>>>>
>>>>+#define KPROBE_HASH_BITS 6
>>>>+#define KPROBE_TABLE_SIZE (1 << KPROBE_HASH_BITS)
>>>>+
>>>> /* kprobe_status settings */
>>>> #define KPROBE_HIT_ACTIVE	0x00000001
>>>> #define KPROBE_HIT_SS		0x00000002
>>>> #define KPROBE_REENTER		0x00000004
>>>> #define KPROBE_HIT_SSDONE	0x00000008
>>>>+#define UPROBE_SS_STACK		0x00000010
>>>>+#define UPROBE_SS_EXPSTACK	0x00000020
>>>>+#define UPROBE_SS_INLINE	0x00000040
>>>>+#define UPROBE_SS_NEW_STACK	0x00000080
>>>>
>>>> /* Attach to insert probes on any functions which should be ignored*/
>>>> #define __kprobes	__attribute__((__section__(".kprobes.text")))
>>>>diff -puN kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line kernel/kprobes.c
>>>>--- linux-2.6.16-rc1-mm5/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:10.000000000 +0530
>>>>+++ linux-2.6.16-rc1-mm5-prasanna/kernel/kprobes.c	2006-02-08 19:26:10.000000000 +0530
>>>>@@ -42,9 +42,6 @@
>>>> #include <asm/errno.h>
>>>> #include <asm/kdebug.h>
>>>>
>>>>-#define KPROBE_HASH_BITS 6
>>>>-#define KPROBE_TABLE_SIZE (1 << KPROBE_HASH_BITS)
>>>>-
>>>> static struct hlist_head kprobe_table[KPROBE_TABLE_SIZE];
>>>> static struct hlist_head kretprobe_inst_table[KPROBE_TABLE_SIZE];
>>>> static struct list_head uprobe_module_list;
>>>>
>>>>_
>>>>--
>>>>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] 11+ messages in thread

* RE: [3/3] Userspace probes prototype-take2
@ 2006-02-17  9:19 Zhang, Yanmin
  2006-02-20  5:36 ` Prasanna S Panchamukhi
  0 siblings, 1 reply; 11+ messages in thread
From: Zhang, Yanmin @ 2006-02-17  9:19 UTC (permalink / raw)
  To: prasanna, systemtap

2 main issues:
1) task switch caused by external interrupt when single-step;
2) multi-thread:

See below inline comments.

Yanmin

>>-----Original Message-----
>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Prasanna S Panchamukhi
>>Sent: 2006年2月8日 22:14
>>To: systemtap@sources.redhat.com
>>Subject: Re: [3/3] Userspace probes prototype-take2
>>
>>
>>This patch handles the executing the registered callback
>>functions when probes is hit.
>>
>>	Each userspace probe is uniquely identified by the
>>combination of inode and offset, hence during registeration the inode
>>and offset combination is added to kprobes hash table. Initially when
>>breakpoint instruction is hit, the kprobes hash table is looked up
>>for matching inode and offset. The pre_handlers are called in sequence
>>if multiple probes are registered. The original instruction is single
>>stepped out-of-line similar to kernel probes. In kernel space probes,
>>single stepping out-of-line is achieved by copying the instruction on
>>to some location within kernel address space and then single step
>>from that location. But for userspace probes, instruction copied
>>into kernel address space cannot be single stepped, hence the
>>instruction should 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. If the stack grows
>>downwards, the stack space below the stack pointer indicates the
>>unused stack free space and if the stack grows upwards, the stack
>>space above the stack pointer indicates the unused stack free space.
>>
>>The instruction to be single stepped can modify the stack space, hence
>>before using the unused stack free 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 unused 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. This patch is not stable yet, should work for most conditions.
>>
>>2. This patch works only with PREEMPT config option disabled, to work
>>in PREEMPT enabled condition handlers must be re-written and must
>>be seperated out from kernel probes allowing preemption.
One of my old comments is an external device interrupt might happen when cpu is single-stepping the original instruction, then the task might be switched to another cpu. If we disable irq when exiting to user space to single step the instruction, kernel might switch the task off just on the exit kernel path. 1) uprobe_page; 2) kprobe_ctlblk, These 2 resources shouldn't be pre cpu, or we need get another approach. How could you resolve the task switch issue?



>>
>>3. 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.
>>
>>4. Optimize the insertion of probes through readpage hooks. Identify
>>all the probes to be inserted on the read page and insert them at
>>once.
>>
>>5. Resume exectution should handle setting of proper eip and eflags
>>for special instructions similar to kernel probes.
>>
>>6. Single stepping out-of-line expands the stack if there is no
>>enough stack space to copy the original instruction. Expansion of
>>stack should be shrinked back to the original size after single
>>stepping or the expanded stack should be reused for single stepping
>>out-of-line for other probes.
>>
>>7. A wrapper routines to calculate the offset from the probed file
>>beginning. In case of dynamic shared library, the offset is
>>calculated by substracting the address of the probe point from the
>>beginning of the file mapped address.
>>
>>8. Handing of page faults while inthe kprobes_handler() and while
>>single stepping.
>>
>>9. Accessing user space pages not present in memory, from the
>>registered callback routines.
>>
>>Signed-off-by: Prasanna S Panchamukhi <prasanna@in.ibm.com>
>>
>>
>> arch/i386/kernel/kprobes.c |  460 +++++++++++++++++++++++++++++++++++++++++++--
>> include/asm-i386/kprobes.h |   13 +
>> include/linux/kprobes.h    |    7
>> kernel/kprobes.c           |    3
>> 4 files changed, 468 insertions(+), 15 deletions(-)
>>
>>diff -puN arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line arch/i386/kernel/kprobes.c
>>--- linux-2.6.16-rc1-mm5/arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
>>+++ linux-2.6.16-rc1-mm5-prasanna/arch/i386/kernel/kprobes.c	2006-02-08 19:26:10.000000000 +0530
>>@@ -30,6 +30,7 @@
>>
>> #include <linux/config.h>
>> #include <linux/kprobes.h>
>>+#include <linux/hash.h>
>> #include <linux/ptrace.h>
>> #include <linux/preempt.h>
>> #include <asm/cacheflush.h>
>>@@ -38,8 +39,12 @@
>>
>> void jprobe_return_end(void);
>>
>>+static struct uprobe_page *uprobe_page;
>>+static struct hlist_head uprobe_page_table[KPROBE_TABLE_SIZE];
>> DEFINE_PER_CPU(struct kprobe *, current_kprobe) = NULL;
>> DEFINE_PER_CPU(struct kprobe_ctlblk, kprobe_ctlblk);
>>+DEFINE_PER_CPU(struct uprobe *, current_uprobe) = NULL;
>>+DEFINE_PER_CPU(unsigned long, singlestep_addr);
>>
>> /* insert a jmp code */
>> static inline void set_jmp_op(void *from, void *to)
>>@@ -125,6 +130,23 @@ void __kprobes arch_disarm_kprobe(struct
>> 			   (unsigned long) p->addr + sizeof(kprobe_opcode_t));
>> }
>>
>>+void __kprobes arch_disarm_uprobe(struct kprobe *p, kprobe_opcode_t *address)
>>+{
>>+	*address = p->opcode;
>>+}
>>+
>>+void __kprobes arch_arm_uprobe(unsigned long *address)
>>+{
>>+	*(kprobe_opcode_t *)address = BREAKPOINT_INSTRUCTION;
>>+}
>>+
>>+void __kprobes arch_copy_uprobe(struct kprobe *p, unsigned long *address)
>>+{
>>+	memcpy(p->ainsn.insn, (kprobe_opcode_t *)address,
>>+				MAX_INSN_SIZE * sizeof(kprobe_opcode_t));
>>+	p->opcode = *(kprobe_opcode_t *)address;
>>+}
>>+
>> static inline void save_previous_kprobe(struct kprobe_ctlblk *kcb)
>> {
>> 	kcb->prev_kprobe.kp = kprobe_running();
>>@@ -151,15 +173,326 @@ static inline void set_current_kprobe(st
>> 		kcb->kprobe_saved_eflags &= ~IF_MASK;
>> }
>>
>>+struct uprobe_page __kprobes *get_upage_current(struct task_struct *tsk)
>>+{
>>+	struct hlist_head *head;
>>+	struct hlist_node *node;
>>+	struct uprobe_page *upage;
>>+
>>+	head = &uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)];
>>+	hlist_for_each_entry(upage, node, head, hlist) {
>>+		if (upage->tsk == tsk)
>>+			return upage;
>>+        }
>>+	return NULL;
>>+}
>>+
>>+struct uprobe_page __kprobes *get_upage_free(struct task_struct *tsk)
>>+{
>>+	int cpu;
>>+
>>+	for_each_cpu(cpu) {
>>+		struct uprobe_page *upage;
>>+		upage = per_cpu_ptr(uprobe_page, cpu);
>>+		if (upage->status & UPROBE_PAGE_FREE)
>>+			return upage;
>>+	}
>>+	return NULL;
>>+}
>>+
>>+/**
>>+ * 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, it just maps a new page and copies the
>>+ *  new 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, *vaddr, stack_addr = regs->esp;
>>+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
>>+	struct uprobe_page *upage;
>>+	struct page *page;
>>+	pte_t *pte;
>>+
>>+
>>+	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;
>>+
The multi-thread case is not resolved here. One of typical multi-thread model is that the all threads share the same vma and every thread has 8-k stack. If 2 threads trigger uprobe (although might be not the same uprobe) at the same time, one thread might erase single-step instruction of another.



>>+	preempt_enable_no_resched();
>>+
>>+	pte = get_uprobe_pte(addr);
>>+	preempt_disable();
>>+	if (!pte)
>>+		return -EFAULT;
>>+
>>+	upage = get_upage_free(current);
>>+	upage->status &= ~UPROBE_PAGE_FREE;
>>+	upage->tsk = current;
>>+	INIT_HLIST_NODE(&upage->hlist);
>>+	hlist_add_head(&upage->hlist,
>>+		&uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)]);
>>+
>>+	upage->orig_pte = pte;
>>+	upage->orig_pte_val =  pte_val(*pte);
>>+	set_pte(pte, (*(upage->alias_pte)));
>>+
>>+	page = pte_page(*pte);
>>+	vaddr = (unsigned long *)kmap_atomic(page, KM_USER1);
>>+	vaddr = (unsigned long *)((unsigned long)vaddr + (addr & ~PAGE_MASK));
>>+	memcpy(vaddr, (unsigned long *)uprobe->kp.ainsn.insn, size);
>>+	kunmap_atomic(vaddr, KM_USER1);
>>+	regs->eip = addr;
So the temp page, upage->alias_addr, replaces the original one on the stack. If the replaced instruction is to operate stack, such like "push eax", the result might be on the new page. After the single step, the pte is restored to the original page which doesn't have the value of eax.



>>+
>>+	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)
It has the same issues like function copy_insn_on_new_page.


>>+{
>>+	unsigned long addr, *vaddr, vm_addr;
>>+	int size = MAX_INSN_SIZE * sizeof(kprobe_opcode_t);
>>+	struct vm_area_struct *new_vma;
>>+	struct uprobe_page *upage;
>>+	struct mm_struct *mm = current->mm;
>>+	struct page *page;
>>+	pte_t *pte;
>>+
>>+
>>+	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
>>+		return -EFAULT;
>>+
>>+	preempt_enable_no_resched();
>>+
>>+	/* TODO: do we need to expand stack if extend_vma fails? */
>>+	new_vma = find_extend_vma(mm, vm_addr);
>>+	preempt_disable();
>>+	if (!new_vma)
>>+		return -ENOMEM;
>>+
>>+	/*
>>+	 * TODO: Expanding stack for every probe is not a good idea, stack must
>>+	 * either be shrunk to its original size after single stepping or the
>>+	 * expanded stack should be kept track of, for the probed application,
>>+	 * so it can be reused to single step out-of-line
>>+	 */
>>+	if (new_vma->vm_flags & VM_GROWSDOWN)
>>+		addr = new_vma->vm_start;
>>+	else
>>+		addr = new_vma->vm_end - size;
>>+
>>+	preempt_enable_no_resched();
>>+	pte = get_uprobe_pte(addr);
>>+	preempt_disable();
>>+	if (!pte)
>>+		return -EFAULT;
>>+
>>+	upage = get_upage_free(current);
>>+	upage->status &= ~UPROBE_PAGE_FREE;
>>+	upage->tsk = current;
>>+	INIT_HLIST_NODE(&upage->hlist);
>>+	hlist_add_head(&upage->hlist,
>>+		&uprobe_page_table[hash_ptr(current, KPROBE_HASH_BITS)]);
>>+	upage->orig_pte = pte;
>>+	upage->orig_pte_val =  pte_val(*pte);
>>+	set_pte(pte, (*(upage->alias_pte)));
>>+
>>+	page = pte_page(*pte);
>>+	vaddr = (unsigned long *)kmap_atomic(page, KM_USER1);
>>+	vaddr = (unsigned long *)((unsigned long)vaddr + (addr & ~PAGE_MASK));
>>+	memcpy(vaddr, (unsigned long *)uprobe->kp.ainsn.insn, size);
>>+	kunmap_atomic(vaddr, KM_USER1);
>>+	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 no 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);
>>+	lock_page(page);
>>+
>>+	addr = (kprobe_opcode_t *)kmap_atomic(page, KM_USER0);
>>+	addr = (kprobe_opcode_t *)((unsigned long)addr +
>>+				 (unsigned long)(uprobe->offset & ~PAGE_MASK));
>>+	*addr = opcode;
>>+	/*TODO: flush vma ? */
>>+	kunmap_atomic(addr, KM_USER0);
>>+
>>+	unlock_page(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 uprobe_single_step(struct kprobe *p, struct pt_regs *regs)
>>+{
>>+	unsigned long stack_addr = regs->esp, flags;
>>+	struct vm_area_struct *vma = NULL;
>>+	struct uprobe *uprobe =  __get_cpu_var(current_uprobe);
>>+	struct kprobe_ctlblk *kcb = get_kprobe_ctlblk();
>>+	int err = 0;
>>+
>>+	down_read(&current->mm->mmap_sem);
>>+
>>+	vma = find_vma(current->mm, (stack_addr & PAGE_MASK));
>>+	if (!vma) {
>>+		/* TODO: Need better error reporting? */
>>+		printk("No vma found\n");
>>+		up_read(&current->mm->mmap_sem);
>>+		return -ENOENT;
>>+	}
>>+	flags = vma->vm_flags;
>>+	up_read(&current->mm->mmap_sem);
>>+
>>+	kcb->kprobe_status |= UPROBE_SS_STACK;
>>+	err = copy_insn_onstack(uprobe, regs, flags);
>>+
>>+	down_write(&current->mm->mmap_sem);
>>+
>>+	if (err) {
>>+		kcb->kprobe_status |= UPROBE_SS_NEW_STACK;
>>+		err = copy_insn_on_new_page(uprobe, regs, vma);
>>+	}
>>+	if (err) {
>>+		kcb->kprobe_status |= UPROBE_SS_EXPSTACK;
>>+		err = copy_insn_onexpstack(uprobe, regs, vma);
>>+	}
>>+
>>+	up_write(&current->mm->mmap_sem);
>>+
>>+	if (err) {
>>+		kcb->kprobe_status |= UPROBE_SS_INLINE;
>>+		replace_original_insn(uprobe, regs, uprobe->kp.opcode);
>>+	}
>>+
>>+	 __get_cpu_var(singlestep_addr) = regs->eip;
>>+
>>+
>>+	return 0;
>>+}
>>+
>> static inline void prepare_singlestep(struct kprobe *p, struct pt_regs *regs)
>> {
>> 	regs->eflags |= TF_MASK;
>> 	regs->eflags &= ~IF_MASK;
>> 	/*single step inline if the instruction is an int3*/
>>+
>> 	if (p->opcode == BREAKPOINT_INSTRUCTION)
>> 		regs->eip = (unsigned long)p->addr;
>>-	else
>>-		regs->eip = (unsigned long)&p->ainsn.insn;
>>+	else {
>>+		if (!kernel_text_address((unsigned long)p->addr))
>>+			uprobe_single_step(p, regs);
>>+		else
>>+			regs->eip = (unsigned long)&p->ainsn.insn;
>>+	}
>> }
>>
>> /* Called with kretprobe_lock held */
>>@@ -194,6 +527,7 @@ static int __kprobes kprobe_handler(stru
>> 	kprobe_opcode_t *addr = NULL;
>> 	unsigned long *lp;
>> 	struct kprobe_ctlblk *kcb;
>>+	unsigned seg = regs->xcs & 0xffff;
>> #ifdef CONFIG_PREEMPT
>> 	unsigned pre_preempt_count = preempt_count();
>> #endif /* CONFIG_PREEMPT */
>>@@ -208,14 +542,21 @@ static int __kprobes kprobe_handler(stru
>> 	/* Check if the application is using LDT entry for its code segment and
>> 	 * calculate the address by reading the base address from the LDT entry.
>> 	 */
>>-	if ((regs->xcs & 4) && (current->mm)) {
>>+
>>+	if (regs->eflags & VM_MASK)
>>+		addr = (kprobe_opcode_t *)(((seg << 4) + regs->eip -
>>+			sizeof(kprobe_opcode_t)) & 0xffff);
>>+	else if ((regs->xcs & 4) && (current->mm)) {
>>+		local_irq_enable();
>>+		down(&current->mm->context.sem);
>> 		lp = (unsigned long *) ((unsigned long)((regs->xcs >> 3) * 8)
>> 					+ (char *) current->mm->context.ldt);
>> 		addr = (kprobe_opcode_t *) (get_desc_base(lp) + regs->eip -
>> 						sizeof(kprobe_opcode_t));
>>-	} else {
>>+		up(&current->mm->context.sem);
>>+		local_irq_disable();
>>+	} else
>> 		addr = (kprobe_opcode_t *)(regs->eip - sizeof(kprobe_opcode_t));
>>-	}
>> 	/* Check we're not actually recursing */
>> 	if (kprobe_running()) {
>> 		p = get_kprobe(addr);
>>@@ -235,7 +576,6 @@ static int __kprobes kprobe_handler(stru
>> 			save_previous_kprobe(kcb);
>> 			set_current_kprobe(p, regs, kcb);
>> 			kprobes_inc_nmissed_count(p);
>>-			prepare_singlestep(p, regs);
>> 			kcb->kprobe_status = KPROBE_REENTER;
>> 			return 1;
>> 		} else {
>>@@ -307,8 +647,8 @@ static int __kprobes kprobe_handler(stru
>> 	}
>>
>> ss_probe:
>>-	prepare_singlestep(p, regs);
>> 	kcb->kprobe_status = KPROBE_HIT_SS;
>>+	prepare_singlestep(p, regs);
>> 	return 1;
>>
>> no_kprobe:
>>@@ -498,6 +838,33 @@ no_change:
>> 	return;
>> }
>>
>>+static void __kprobes resume_execution_user(struct uprobe *uprobe,
>>+				struct pt_regs *regs, struct kprobe_ctlblk *kcb)
>>+{
>>+	unsigned long delta;
>>+	struct uprobe_page *upage;
>>+
>>+	/*
>>+	 * TODO :need to fixup special instructions as done with kernel probes.
>>+	 */
>>+	delta = regs->eip - __get_cpu_var(singlestep_addr);
>>+	regs->eip = (unsigned long)(uprobe->kp.addr + delta);
>>+
>>+	if ((kcb->kprobe_status & UPROBE_SS_EXPSTACK) ||
>>+			(kcb->kprobe_status & UPROBE_SS_NEW_STACK)) {
>>+		upage = get_upage_current(current);
>>+		set_pte(upage->orig_pte, __pte(upage->orig_pte_val));
>>+		pte_unmap(upage->orig_pte);
>>+
>>+		upage->status = UPROBE_PAGE_FREE;
>>+		hlist_del(&upage->hlist);
>>+
>>+	} else if (kcb->kprobe_status & UPROBE_SS_INLINE)
>>+		replace_original_insn(uprobe, regs,
>>+				(kprobe_opcode_t)BREAKPOINT_INSTRUCTION);
>>+	regs->eflags &= ~TF_MASK;
>>+}
>>+
>> /*
>>  * Interrupts are disabled on entry as trap1 is an interrupt gate and they
>>  * remain disabled thoroughout this function.
>>@@ -510,16 +877,19 @@ static inline int post_kprobe_handler(st
>> 	if (!cur)
>> 		return 0;
>>
>>-	if ((kcb->kprobe_status != KPROBE_REENTER) && cur->post_handler) {
>>-		kcb->kprobe_status = KPROBE_HIT_SSDONE;
>>+	if (!(kcb->kprobe_status & KPROBE_REENTER) && cur->post_handler) {
>>+		kcb->kprobe_status |= KPROBE_HIT_SSDONE;
>> 		cur->post_handler(cur, regs, 0);
>> 	}
>>
>>-	resume_execution(cur, regs, kcb);
>>+	if (!kernel_text_address((unsigned long)cur->addr))
>>+		resume_execution_user(__get_cpu_var(current_uprobe), regs, kcb);
>>+	else
>>+		resume_execution(cur, regs, kcb);
>> 	regs->eflags |= kcb->kprobe_saved_eflags;
>>
>> 	/*Restore back the original saved kprobes variables and continue. */
>>-	if (kcb->kprobe_status == KPROBE_REENTER) {
>>+	if (kcb->kprobe_status & KPROBE_REENTER) {
>> 		restore_previous_kprobe(kcb);
>> 		goto out;
>> 	}
>>@@ -547,7 +917,13 @@ static inline int kprobe_fault_handler(s
>> 		return 1;
>>
>> 	if (kcb->kprobe_status & KPROBE_HIT_SS) {
>>-		resume_execution(cur, regs, kcb);
>>+		if (!kernel_text_address((unsigned long)cur->addr)) {
>>+			struct uprobe *uprobe =  __get_cpu_var(current_uprobe);
>>+			/* TODO: Proper handling of all instruction */
>>+			replace_original_insn(uprobe, regs, uprobe->kp.opcode);
>>+			regs->eflags &= ~TF_MASK;
>>+		} else
>>+			resume_execution(cur, regs, kcb);
>> 		regs->eflags |= kcb->kprobe_old_eflags;
>>
>> 		reset_current_kprobe();
>>@@ -654,7 +1030,67 @@ int __kprobes longjmp_break_handler(stru
>> 	return 0;
>> }
>>
>>+static void free_alias(void)
>>+{
>>+	int cpu;
>>+
>>+	for_each_cpu(cpu) {
>>+		struct uprobe_page *upage;
>>+		upage = per_cpu_ptr(uprobe_page, cpu);
>>+
>>+		if (upage->alias_addr) {
>>+			set_pte(upage->alias_pte, __pte(upage->alias_pte_val));
>>+			kfree(upage->alias_addr);
>>+		}
>>+		upage->alias_pte = 0;
>>+	}
>>+	free_percpu(uprobe_page);
>>+	return;
>>+}
>>+
>>+static int alloc_alias(void)
>>+{
>>+	int cpu;
>>+
>>+	uprobe_page = __alloc_percpu(sizeof(struct uprobe_page));
[YM] Do here codes try to resolve the problem of task switch at single-step? If so, the per cpu data also might be used up although get_upage_free will go through all uprobe_page of all cpus. I suggest to allocate a series of uprobe_page, and allocate again when they are used up.




>>+
>>+	for_each_cpu(cpu) {
>>+		struct uprobe_page *upage;
>>+		upage = per_cpu_ptr(uprobe_page, cpu);
>>+		upage->alias_addr = kmalloc(PAGE_SIZE, GFP_USER);
[YM] Does kmalloc(PAGE_SIZE...) imply the result is aligned to page? How about using alloc_page?


>>+		if (!upage->alias_addr) {
>>+			free_alias();
>>+			return -ENOMEM;
>>+		}
>>+		upage->alias_pte = lookup_address(
>>+					(unsigned long)upage->alias_addr);
>>+		upage->alias_pte_val = pte_val(*upage->alias_pte);
>>+		if (upage->alias_pte) {
[YM] If kmalloc returns a non-NULL address, upage->alias_pte is not equal to NULL. So delete above checking?


>>+			upage->status = UPROBE_PAGE_FREE;
>>+			set_pte(upage->alias_pte,
>>+						pte_mkdirty(*upage->alias_pte));
>>+			set_pte(upage->alias_pte,
>>+						pte_mkexec(*upage->alias_pte));
>>+			set_pte(upage->alias_pte,
>>+						 pte_mkwrite(*upage->alias_pte));
>>+			set_pte(upage->alias_pte,
>>+						pte_mkyoung(*upage->alias_pte));
>>+		}
>>+	}
>>+	return 0;
>>+}
>>+
>> int __init arch_init_kprobes(void)
>> {
>>+	int ret = 0;
>>+	/*
>>+	 * user space probes requires a page to copy the original instruction
>>+	 * so that it can single step if there is no free stack space, allocate
>>+	 * per cpu page.
>>+	 */
>>+
>>+	if ((ret = alloc_alias()))
>>+		return ret;
>>+
>> 	return 0;
>> }
>>diff -puN include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line include/asm-i386/kprobes.h
>>--- linux-2.6.16-rc1-mm5/include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
>>+++ linux-2.6.16-rc1-mm5-prasanna/include/asm-i386/kprobes.h	2006-02-08 19:26:10.000000000 +0530
>>@@ -42,6 +42,7 @@ typedef u8 kprobe_opcode_t;
>> #define JPROBE_ENTRY(pentry)	(kprobe_opcode_t *)pentry
>> #define ARCH_SUPPORTS_KRETPROBES
>> #define arch_remove_kprobe(p)	do {} while (0)
>>+#define UPROBE_PAGE_FREE 0x00000001
>>
>> void kretprobe_trampoline(void);
>>
>>@@ -74,6 +75,18 @@ struct kprobe_ctlblk {
>> 	struct prev_kprobe prev_kprobe;
>> };
>>
>>+/* per cpu uprobe page structure */
>>+struct uprobe_page {
>>+	struct hlist_node hlist;
>>+	pte_t *alias_pte;
>>+	pte_t *orig_pte;
>>+	unsigned long orig_pte_val;
>>+	unsigned long alias_pte_val;
[YM] I think the patch doesn't support CONFIG_X86_PAE, because if CONFIG_X86_PAE=y, pte_t becomes 64 bits.
How about changing above 2 members' type to pte_t directly?



>>+	void *alias_addr;
>>+	struct task_struct *tsk;
>>+	unsigned long status;
>>+};
>>+
>> /* trap3/1 are intr gates for kprobes.  So, restore the status of IF,
>>  * if necessary, before executing the original int3/1 (trap) handler.
>>  */
>>diff -puN include/linux/kprobes.h~kprobes_userspace_probes_ss_out-of-line include/linux/kprobes.h
>>--- linux-2.6.16-rc1-mm5/include/linux/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:09.000000000 +0530
>>+++ linux-2.6.16-rc1-mm5-prasanna/include/linux/kprobes.h	2006-02-08 19:26:10.000000000 +0530
>>@@ -45,11 +45,18 @@
>> #ifdef CONFIG_KPROBES
>> #include <asm/kprobes.h>
>>
>>+#define KPROBE_HASH_BITS 6
>>+#define KPROBE_TABLE_SIZE (1 << KPROBE_HASH_BITS)
>>+
>> /* kprobe_status settings */
>> #define KPROBE_HIT_ACTIVE	0x00000001
>> #define KPROBE_HIT_SS		0x00000002
>> #define KPROBE_REENTER		0x00000004
>> #define KPROBE_HIT_SSDONE	0x00000008
>>+#define UPROBE_SS_STACK		0x00000010
>>+#define UPROBE_SS_EXPSTACK	0x00000020
>>+#define UPROBE_SS_INLINE	0x00000040
>>+#define UPROBE_SS_NEW_STACK	0x00000080
>>
>> /* Attach to insert probes on any functions which should be ignored*/
>> #define __kprobes	__attribute__((__section__(".kprobes.text")))
>>diff -puN kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line kernel/kprobes.c
>>--- linux-2.6.16-rc1-mm5/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-02-08 19:26:10.000000000 +0530
>>+++ linux-2.6.16-rc1-mm5-prasanna/kernel/kprobes.c	2006-02-08 19:26:10.000000000 +0530
>>@@ -42,9 +42,6 @@
>> #include <asm/errno.h>
>> #include <asm/kdebug.h>
>>
>>-#define KPROBE_HASH_BITS 6
>>-#define KPROBE_TABLE_SIZE (1 << KPROBE_HASH_BITS)
>>-
>> static struct hlist_head kprobe_table[KPROBE_TABLE_SIZE];
>> static struct hlist_head kretprobe_inst_table[KPROBE_TABLE_SIZE];
>> static struct list_head uprobe_module_list;
>>
>>_
>>--
>>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] 11+ messages in thread

end of thread, other threads:[~2006-02-20  5:48 UTC | newest]

Thread overview: 11+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2006-02-08 14:10 [1/3] Userspace probes prototype-take2 Prasanna S Panchamukhi
2006-02-08 14:12 ` [2/3] " Prasanna S Panchamukhi
2006-02-08 14:13   ` [3/3] " Prasanna S Panchamukhi
2006-02-17  9:19 Zhang, Yanmin
2006-02-20  5:36 ` Prasanna S Panchamukhi
2006-02-20  3:16 Zhang, Yanmin
2006-02-20  4:51 ` Prasanna S Panchamukhi
2006-02-20  3:32 Zhang, Yanmin
2006-02-20  5:07 ` Prasanna S Panchamukhi
2006-02-20  5:48 Zhang, Yanmin
2006-02-20  5:48 Zhang, Yanmin

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).