public inbox for systemtap@sourceware.org
 help / color / mirror / Atom feed
* Re: Patch [2/3] Userspace probes readpage hooks
  2006-01-19 14:38 Patch [1/3] Userspace probes new interfaces Prasanna S Panchamukhi
@ 2006-01-19 14:37 ` Prasanna S Panchamukhi
  2006-01-19 14:38   ` Patch [3/3] Userspace probes single stepping out-of-line Prasanna S Panchamukhi
  2006-01-19 16:14 ` Patch [1/3] Userspace probes new interfaces Frank Ch. Eigler
  1 sibling, 1 reply; 13+ messages in thread
From: Prasanna S Panchamukhi @ 2006-01-19 14:37 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>


 fs/namei.c            |   11 ++-
 include/linux/namei.h |    1 
 kernel/kprobes.c      |  141 ++++++++++++++++++++++++++++++++++++++++++++++++++
 3 files changed, 150 insertions(+), 3 deletions(-)

diff -puN kernel/kprobes.c~kprobes_userspace_probes_hook_readpage kernel/kprobes.c
--- linux-2.6.15/kernel/kprobes.c~kprobes_userspace_probes_hook_readpage	2006-01-19 19:37:48.000000000 +0530
+++ linux-2.6.15-prasanna/kernel/kprobes.c	2006-01-19 19:37:49.000000000 +0530
@@ -815,6 +815,133 @@ static struct uprobe_module __kprobes *g
 	return NULL;
 }
 
+static inline void insert_readpage_uprobe(struct page *page,
+	struct address_space *mapping, struct uprobe *uprobe)
+{
+	struct vm_area_struct *vma = NULL;
+
+	if (find_page_probe(uprobe->offset >> PAGE_CACHE_SHIFT,
+				page->index << PAGE_CACHE_SHIFT)) {
+		spin_lock(&mapping->i_mmap_lock);
+		vma = find_get_vma(uprobe, page, mapping);
+		map_uprobe_page(page, vma, uprobe, insert_kprobe_user);
+
+		flush_vma(mapping, page, uprobe);
+		spin_unlock(&mapping->i_mmap_lock);
+	}
+}
+
+/**
+ *  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");
+		mutex_unlock(&uprobe_mutex);
+		return -EINVAL;
+	}
+
+	/* 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;
+
+		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");
+		mutex_unlock(&uprobe_mutex);
+		return -EINVAL;
+	}
+
+	/* 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)
+		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,
+ * 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
@@ -828,11 +955,21 @@ static int __kprobes get_inode_ops(struc
 	struct address_space *as;
 	int error = 0;
 
+	error = ex_write_lock(umodule->nd.dentry->d_inode);
+	if (error)
+		return error;
+
 	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;
+
 	return error;
 }
 
@@ -863,6 +1000,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)) {
@@ -946,6 +1084,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);
 	}
diff -puN fs/namei.c~kprobes_userspace_probes_hook_readpage fs/namei.c
--- linux-2.6.15/fs/namei.c~kprobes_userspace_probes_hook_readpage	2006-01-19 19:37:49.000000000 +0530
+++ linux-2.6.15-prasanna/fs/namei.c	2006-01-19 19:37:49.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/namei.h~kprobes_userspace_probes_hook_readpage include/linux/namei.h
--- linux-2.6.15/include/linux/namei.h~kprobes_userspace_probes_hook_readpage	2006-01-19 19:37:49.000000000 +0530
+++ linux-2.6.15-prasanna/include/linux/namei.h	2006-01-19 19:37:49.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)
 {

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

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

* Patch [1/3] Userspace probes new interfaces
@ 2006-01-19 14:38 Prasanna S Panchamukhi
  2006-01-19 14:37 ` Patch [2/3] Userspace probes readpage hooks Prasanna S Panchamukhi
  2006-01-19 16:14 ` Patch [1/3] Userspace probes new interfaces Frank Ch. Eigler
  0 siblings, 2 replies; 13+ messages in thread
From: Prasanna S Panchamukhi @ 2006-01-19 14:38 UTC (permalink / raw)
  To: systemtap

Hi,

Please find the set of patches that implement the provide the following features
Patch [1/3] - new interfaces
Patch [2/3] - readpage hooks
Patch [3/3] - single stepping out_of_line 

Please review and provide your feedback.

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>


 include/linux/kprobes.h |   36 +++
 kernel/kprobes.c        |  509 ++++++++++++++++++++++++++++++++++++++++++++++--
 2 files changed, 524 insertions(+), 21 deletions(-)

diff -puN include/linux/kprobes.h~kprobes_userspace_probes-base-interface include/linux/kprobes.h
--- linux-2.6.15/include/linux/kprobes.h~kprobes_userspace_probes-base-interface	2006-01-19 18:48:04.000000000 +0530
+++ linux-2.6.15-prasanna/include/linux/kprobes.h	2006-01-19 19:35:53.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 kernel/kprobes.c~kprobes_userspace_probes-base-interface kernel/kprobes.c
--- linux-2.6.15/kernel/kprobes.c~kprobes_userspace_probes-base-interface	2006-01-19 18:48:04.000000000 +0530
+++ linux-2.6.15-prasanna/kernel/kprobes.c	2006-01-19 19:37:25.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;
+
+	spin_lock(&mm->page_table_lock);
+	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) {
+		spin_unlock(&mm->page_table_lock);
+		return NULL;
+	}
+
+	inode = vma->vm_file->f_dentry->d_inode;
+	spin_unlock(&mm->page_table_lock);
+
+	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)
 {
@@ -576,6 +660,386 @@ void __kprobes unregister_jprobe(struct 
 	unregister_kprobe(&jp->kp);
 }
 
+typedef int (*process_uprobe_func_t)(struct uprobe *uprobe,
+				unsigned long *address, struct page *page,
+						struct vm_area_struct *vma);
+
+/**
+ * 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 vm_area_struct *vma)
+{
+	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);
+	flush_icache_user_range(vma, page, (unsigned long)address,
+						sizeof(kprobe_opcode_t));
+
+	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 vm_area_struct *vma,
+				     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, vma);
+	kunmap(page);
+
+	unlock_page(page);
+
+	return ret;
+}
+
+
+/**
+ * find_get_vma walks through the list of process private mappings and
+ * gets the vma containing the offset.
+ */
+static struct vm_area_struct __kprobes *find_get_vma(struct uprobe *uprobe,
+			struct page *page, struct address_space *mapping)
+{
+	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;
+
+		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)
+			return vma;
+	}
+
+	return NULL;
+}
+
+/**
+ * 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;
+
+		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));
+	}
+}
+
+/**
+ * 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;
+}
+
+/**
+ * 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 int __kprobes get_inode_ops(struct uprobe *uprobe,
+				   struct uprobe_module *umodule)
+{
+	struct address_space *as;
+	int error = 0;
+
+	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;
+
+	return error;
+}
+
+/**
+ * Lookup the pathname and the get the inode and dentry.
+ */
+static inline int get_uprobe_inode(const char *pathname, struct inode **inode)
+{
+	int error;
+	struct nameidata nd;
+
+	if ((error = path_lookup(pathname, LOOKUP_FOLLOW, &nd))) {
+		path_release(&nd);
+		return error;
+	}
+
+	*inode = (struct inode *)nd.dentry->d_inode;
+	path_release(&nd);
+
+	return 0;
+}
+
+
+int __kprobes remove_kprobe_user(struct uprobe *uprobe, unsigned long *address,
+				struct page *page, struct vm_area_struct *vma)
+{
+	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);
+		flush_icache_user_range(vma, page, (unsigned long)address,
+						sizeof(kprobe_opcode_t));
+		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 vm_area_struct *vma = NULL;
+	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);
+
+	spin_lock(&mapping->i_mmap_lock);
+	vma = find_get_vma(uprobe, page, mapping);
+	ret = map_uprobe_page(page, vma, uprobe, remove_kprobe_user);
+	/*
+	 * TODO: unregister_uprobe should not fail, need to handle if it fails.
+	 */
+	flush_vma(mapping, page, uprobe);
+	spin_unlock(&mapping->i_mmap_lock);
+
+	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);
+		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 vm_area_struct *vma = NULL;
+	struct inode *inode;
+	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 = get_uprobe_inode(uprobe->pathname, &inode)))
+		return error;
+
+	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;
+			goto out;
+		}
+
+		error = path_lookup(uprobe->pathname, LOOKUP_FOLLOW,
+								&umodule->nd);
+		if (error) {
+			path_release(&umodule->nd);
+			goto out;
+		}
+
+		if ((error = get_inode_ops(uprobe, umodule))) {
+			path_release(&umodule->nd);
+			kfree(umodule);
+			goto out;
+		}
+	} else
+		hlist_add_head(&uprobe->ulist, &umodule->ulist_head);
+
+	uprobe->inode = umodule->nd.dentry->d_inode;
+	mapping = umodule->nd.dentry->d_inode->i_mapping;
+	mutex_lock(&kprobe_mutex);
+	page = find_get_page(mapping, (uprobe->offset >> PAGE_CACHE_SHIFT));
+
+	spin_lock(&mapping->i_mmap_lock);
+	vma = find_get_vma(uprobe, page, mapping);
+
+	/*
+	 * If error == -EINVAL, return success, probes will inserted by
+	 * readpage hooks.
+	 * TODO: Use a more suitable errno?
+	 */
+	error = map_uprobe_page(page, vma, uprobe, insert_kprobe_user);
+	if (error == -EINVAL)
+		error = 0;
+	flush_vma(mapping, page, uprobe);
+	spin_unlock(&mapping->i_mmap_lock);
+
+	if (page)
+		page_cache_release(page);
+
+	mutex_unlock(&kprobe_mutex);
+out:
+	mutex_unlock(&uprobe_mutex);
+
+	return error;
+}
+
 #ifdef ARCH_SUPPORTS_KRETPROBES
 
 int __kprobes register_kretprobe(struct kretprobe *rp)
@@ -650,6 +1114,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 +1132,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-25044636

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

* Re: Patch [3/3] Userspace probes single stepping out-of-line
  2006-01-19 14:37 ` Patch [2/3] Userspace probes readpage hooks Prasanna S Panchamukhi
@ 2006-01-19 14:38   ` Prasanna S Panchamukhi
  2006-01-20  8:38     ` Uprobes: howto calculate the probe offset Prasanna S Panchamukhi
  0 siblings, 1 reply; 13+ messages in thread
From: Prasanna S Panchamukhi @ 2006-01-19 14:38 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. 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.

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

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

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

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

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

7. 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 |  352 +++++++++++++++++++++++++++++++++++++++++++--
 include/asm-i386/kprobes.h |    9 +
 include/linux/kprobes.h    |    3 
 3 files changed, 351 insertions(+), 13 deletions(-)

diff -puN arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line arch/i386/kernel/kprobes.c
--- linux-2.6.15/arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-01-19 19:37:53.000000000 +0530
+++ linux-2.6.15-prasanna/arch/i386/kernel/kprobes.c	2006-01-19 19:37:53.000000000 +0530
@@ -38,9 +38,11 @@
 
 void jprobe_return_end(void);
 
+static struct uprobe_page *uprobe_page;
 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);
 /*
  * returns non-zero if opcode modifies the interrupt flag.
  */
@@ -77,6 +79,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();
@@ -103,15 +122,223 @@ static inline void set_current_kprobe(st
 		kcb->kprobe_saved_eflags &= ~IF_MASK;
 }
 
+/**
+ * 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 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 err = 0, 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;
+
+	upage = per_cpu_ptr(uprobe_page, smp_processor_id());
+
+	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;
+
+	spin_lock(&mm->page_table_lock);
+
+	/* TODO: do we need to expand stack if extend_vma fails? */
+	if (!(new_vma = find_extend_vma(mm, vm_addr)))
+		err = expand_stack(new_vma, vm_addr);
+
+	if (err) {
+		spin_unlock(&mm->page_table_lock);
+		return err;
+	}
+
+	spin_unlock(&mm->page_table_lock);
+
+	/*
+	 * 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;
+
+	pte = get_uprobe_pte(addr);
+	if (!pte)
+		return -EFAULT;
+
+	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);
+	err = __copy_to_user_inatomic(vaddr,
+			(unsigned long *)uprobe->kp.ainsn.insn, size);
+	kunmap_atomic(vaddr, KM_USER1);
+	if (err)
+		return err;
+
+	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, struct vm_area_struct *vma)
+{
+	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 (vma->vm_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 + size),
+								source, size))
+			return -EFAULT;
+
+		regs->eip = page_addr + size;
+	} else if (vma->vm_flags & VM_GROWSUP) {
+		page_addr = (stack_addr & PAGE_MASK) + PAGE_SIZE;
+
+		if ((page_addr - (stack_addr + sizeof(long long))) < size)
+			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;
+	struct vm_area_struct *vma = NULL;
+	struct uprobe *uprobe =  __get_cpu_var(current_uprobe);
+	struct kprobe_ctlblk *kcb = get_kprobe_ctlblk();
+	int err = 0;
+
+	vma = find_vma(current->mm, (stack_addr & PAGE_MASK));
+	if (!vma) {
+		/* TODO: Need better error reporting? */
+		printk("No vma found\n");
+		return -ENOENT;
+	}
+
+	kcb->kprobe_status |= UPROBE_SS_STACK;
+	if ((err = copy_insn_onstack(uprobe, regs, vma))) {
+		err = copy_insn_onexpstack(uprobe, regs, vma);
+		kcb->kprobe_status |= UPROBE_SS_EXPSTACK;
+	}
+
+	if (err) {
+		replace_original_insn(uprobe, regs, uprobe->kp.opcode);
+		kcb->kprobe_status |= UPROBE_SS_INLINE;
+	}
+
+	 __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 */
@@ -146,6 +373,7 @@ static int __kprobes kprobe_handler(stru
 	kprobe_opcode_t *addr = NULL;
 	unsigned long *lp;
 	struct kprobe_ctlblk *kcb;
+	unsigned seg = regs->xcs & 0xffff;
 
 	/*
 	 * We don't want to be preempted for the entire
@@ -157,14 +385,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);
@@ -184,7 +419,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 {
@@ -241,8 +475,8 @@ static int __kprobes kprobe_handler(stru
 		return 1;
 
 ss_probe:
-	prepare_singlestep(p, regs);
 	kcb->kprobe_status = KPROBE_HIT_SS;
+	prepare_singlestep(p, regs);
 	return 1;
 
 no_kprobe:
@@ -396,6 +630,29 @@ static void __kprobes resume_execution(s
 	}
 }
 
+static void __kprobes resume_execution_user(struct uprobe *uprobe,
+				struct pt_regs *regs, struct kprobe_ctlblk *kcb)
+{
+	unsigned long delta;
+	struct uprobe_page *upage;
+
+	upage = per_cpu_ptr(uprobe_page, smp_processor_id());
+	/*
+	 * 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) {
+		set_pte(upage->orig_pte, __pte(upage->orig_pte_val));
+		pte_unmap(upage->orig_pte);
+	} 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.
@@ -408,16 +665,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;
 	}
@@ -445,7 +705,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();
@@ -557,7 +823,67 @@ static struct kprobe trampoline_p = {
 	.pre_handler = trampoline_probe_handler
 };
 
+
+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) {
+			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 register_kprobe(&trampoline_p);
 }
diff -puN include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line include/asm-i386/kprobes.h
--- linux-2.6.15/include/asm-i386/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-01-19 19:37:53.000000000 +0530
+++ linux-2.6.15-prasanna/include/asm-i386/kprobes.h	2006-01-19 19:37:53.000000000 +0530
@@ -68,6 +68,15 @@ struct kprobe_ctlblk {
 	struct prev_kprobe prev_kprobe;
 };
 
+/* per cpu uprobe page structure */
+struct uprobe_page {
+	pte_t *alias_pte;
+	pte_t *orig_pte;
+	unsigned long orig_pte_val;
+	unsigned long alias_pte_val;
+	void *alias_addr;
+};
+
 /* 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.15/include/linux/kprobes.h~kprobes_userspace_probes_ss_out-of-line	2006-01-19 19:37:53.000000000 +0530
+++ linux-2.6.15-prasanna/include/linux/kprobes.h	2006-01-19 19:37:53.000000000 +0530
@@ -50,6 +50,9 @@
 #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
 
 /* Attach to insert probes on any functions which should be ignored*/
 #define __kprobes	__attribute__((__section__(".kprobes.text")))

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

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

* Re: Patch [1/3] Userspace probes new interfaces
  2006-01-19 14:38 Patch [1/3] Userspace probes new interfaces Prasanna S Panchamukhi
  2006-01-19 14:37 ` Patch [2/3] Userspace probes readpage hooks Prasanna S Panchamukhi
@ 2006-01-19 16:14 ` Frank Ch. Eigler
  1 sibling, 0 replies; 13+ messages in thread
From: Frank Ch. Eigler @ 2006-01-19 16:14 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap


prasanna@in.ibm.com wrote:

> [...] Please review and provide your feedback. [...]

The API looks usable.  Did you folks come up with some test probe
modules to demonstrate & stress-test it?

- FChE

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

* Re:  Uprobes: howto calculate the probe offset
  2006-01-19 14:38   ` Patch [3/3] Userspace probes single stepping out-of-line Prasanna S Panchamukhi
@ 2006-01-20  8:38     ` Prasanna S Panchamukhi
  2006-01-20 16:58       ` Ulrich Drepper
  2006-01-20 17:17       ` Frank Ch. Eigler
  0 siblings, 2 replies; 13+ messages in thread
From: Prasanna S Panchamukhi @ 2006-01-20  8:38 UTC (permalink / raw)
  To: systemtap

Please find the details of calculating uprobe offset below and
let me know if you need more information.

Thanks
Prasanna

1. Allocate a uprobe structure.
	struct uprobe uprobe;

2. Calculate the offset for the application and initialize the
   pathname, address and offset elements.

    Eg : Probe on function foo();
	/* Specify the pathname of the application */
	$char pname[] ="/home/prasanna/MOD/appln";
	$urpobe.pathname = &pname[0];

	/*Speciy the virtual address */
	$cc -o appln app.c
	$nm appln |grep foo()
	$080484a9 T foo

	$uprobe.kp.addr = (kprobe_opcode_t *) 0x080484a9

	/* calculate the offset for function foo() */
	uprobe.offset = uprobe.kp.addr & 0x00000fff;

    Eg: Probe on function printf() defined in libc-2.3.4.so
	/* Specify the pathname of the library*/
	$char pname[] ="/lib/tls/libc-2.3.4.so";
	$uprobe.pathname = &pname[0];

	/*Speciy the virtual address */
	$nm /lib/tls/libc-2.3.4.so |grep printf
	$4ecc9940 W obstack_printf
	$4ecc9830 W obstack_vprintf
	$4ecaf320 T parse_printf_format
	$4ecb1420 T printf
	^^^^^^
	$4ed96088 b __printf_arginfo_table
	$4ed4a050 T __printf_chk
	$4ecacd60 T __printf_fp

	uprobe.kp.addr = (kprobe_opcode_t *) 0x4ecb1420;

	/*Calculate the offset */
	$cat /proc/1/mmaps
	$08048000-0804f000 r-xp 00000000 08:02 1144758    /sbin/init
	$0804f000-08050000 rw-p 00007000 08:02 1144758    /sbin/init
	$09d5d000-09d7e000 rw-p 09d5d000 00:00 0
	$4ec56000-4ec6b000 r-xp 00000000 08:02 1111946    /lib/ld-2.3.4.so
	$4ec6b000-4ec6c000 r--p 00015000 08:02 1111946    /lib/ld-2.3.4.so
	$4ec6c000-4ec6d000 rw-p 00016000 08:02 1111946    /lib/ld-2.3.4.so
	$4ec6f000-4ed93000 r-xp 00000000 08:02 1111959    /lib/tls/libc-2.3.4.so
	^^^^^^^^
	$4ed93000-4ed94000 r--p 00124000 08:02 1111959    /lib/tls/libc-2.3.4.so
	$4ed94000-4ed97000 rw-p 00125000 08:02 1111959    /lib/tls/libc-2.3.4.so
	$4ed97000-4ed99000 rw-p 4ed97000 00:00 0

	/* Lookup where the libc-2.3.4.so is mapped. From above you can see
	   the beginning map address is 0x4ec6f000 */
	$uprobe.offset  =
		(unsigned long)(((unsinged long)urpobe.kp.address) - 0x4ec6f000)
			= 0x4ecb1420 -  0x4ec6f000;
			= 0x42420;
3. Now initialize handlers apart from offset, address and pathname.

        $uprobe.kp.pre_handler = handler_pre;
        $uprobe.kp.post_handler = handler_post;
        $uprobe.kp.fault_handler = handler_fault;

4. Insert probes by registering userspace probe in the init_module.
	register_uprobe(&uprobe);

5. Remove probes by unregistering userspace probe in cleanup_module.
	unregister_uprobe(&uprobe);
-- 
Prasanna S Panchamukhi
Linux Technology Center
India Software Labs, IBM Bangalore
Email: prasanna@in.ibm.com
Ph: 91-80-25044636

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

* Re: Uprobes: howto calculate the probe offset
  2006-01-20  8:38     ` Uprobes: howto calculate the probe offset Prasanna S Panchamukhi
@ 2006-01-20 16:58       ` Ulrich Drepper
  2006-01-23 14:26         ` Prasanna S Panchamukhi
  2006-01-20 17:17       ` Frank Ch. Eigler
  1 sibling, 1 reply; 13+ messages in thread
From: Ulrich Drepper @ 2006-01-20 16:58 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap

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

Prasanna S Panchamukhi wrote:
> 	/*Calculate the offset */
> 	$cat /proc/1/mmaps
> 	$08048000-0804f000 r-xp 00000000 08:02 1144758    /sbin/init
> 	$0804f000-08050000 rw-p 00007000 08:02 1144758    /sbin/init
> 	$09d5d000-09d7e000 rw-p 09d5d000 00:00 0
> 	$4ec56000-4ec6b000 r-xp 00000000 08:02 1111946    /lib/ld-2.3.4.so
> 	$4ec6b000-4ec6c000 r--p 00015000 08:02 1111946    /lib/ld-2.3.4.so
> 	$4ec6c000-4ec6d000 rw-p 00016000 08:02 1111946    /lib/ld-2.3.4.so
> 	$4ec6f000-4ed93000 r-xp 00000000 08:02 1111959    /lib/tls/libc-2.3.4.so
> 	^^^^^^^^
> 	$4ed93000-4ed94000 r--p 00124000 08:02 1111959    /lib/tls/libc-2.3.4.so
> 	$4ed94000-4ed97000 rw-p 00125000 08:02 1111959    /lib/tls/libc-2.3.4.so
> 	$4ed97000-4ed99000 rw-p 4ed97000 00:00 0
> 
> 	/* Lookup where the libc-2.3.4.so is mapped. From above you can see
> 	   the beginning map address is 0x4ec6f000 */
> 	$uprobe.offset  =
> 		(unsigned long)(((unsinged long)urpobe.kp.address) - 0x4ec6f000)
> 			= 0x4ecb1420 -  0x4ec6f000;
> 			= 0x42420;

That's not how you in general can compute the offset.

You have to look at each loaded segment in the program header (PT_LOAD
entries).  For each segment you can determine an offset which is the
difference between the actual address used when loading the DSO and the
p_vaddr in the file.

Then, whenever you look up a symbol, determine which segment the symbol
is in (compare the address with [p_vaddr,p_vaddr+p_memsz) and add the
offset for that segment to the address of the symbol as defined in the
symbol table.

It'll for simple cases create the same result as you have above.  But
your's is a lucky guess which can fail.

-- 
➧ Ulrich Drepper ➧ Red Hat, Inc. ➧ 444 Castro St ➧ Mountain View, CA ❖


[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 251 bytes --]

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

* Re: Uprobes: howto calculate the probe offset
  2006-01-20  8:38     ` Uprobes: howto calculate the probe offset Prasanna S Panchamukhi
  2006-01-20 16:58       ` Ulrich Drepper
@ 2006-01-20 17:17       ` Frank Ch. Eigler
  2006-01-23 14:47         ` Prasanna S Panchamukhi
  1 sibling, 1 reply; 13+ messages in thread
From: Frank Ch. Eigler @ 2006-01-20 17:17 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap


prasanna wrote:

> [...for an executable...]
> 	/* calculate the offset for function foo() */
> 	uprobe.offset = uprobe.kp.addr & 0x00000fff;

> [...for a shared library...]
> 	uprobe.kp.addr = (kprobe_opcode_t *) 0x4ecb1420;
> 	/*Calculate the offset */
> 	$cat /proc/1/mmaps
> [...]
> 	$4ec6f000-4ed93000 r-xp 00000000 08:02 1111959    /lib/tls/libc-2.3.4.so
> [...]
> 	$uprobe.offset  = [...]
> 			= 0x4ecb1420 -  0x4ec6f000;
> 			= 0x42420;

Could you explain why the uprobe.offset number is sometimes simply an
intra-page offset (which kprobes could compute), and sometimes a
mapping-relative one?

Also, given that it's possible to map a shared library at different
addresses within different processes, does the latter computation
imply that a uprobe only works for exactly those threads that have the
same shlib mapped at the same address?  What happens to other threads?


- FChE

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

* Re: Uprobes: howto calculate the probe offset
  2006-01-20 16:58       ` Ulrich Drepper
@ 2006-01-23 14:26         ` Prasanna S Panchamukhi
  0 siblings, 0 replies; 13+ messages in thread
From: Prasanna S Panchamukhi @ 2006-01-23 14:26 UTC (permalink / raw)
  To: Ulrich Drepper; +Cc: systemtap

Ulrich,

> That's not how you in general can compute the offset.
> 
> You have to look at each loaded segment in the program header (PT_LOAD
> entries).  For each segment you can determine an offset which is the
> difference between the actual address used when loading the DSO and the
> p_vaddr in the file.
> 
> Then, whenever you look up a symbol, determine which segment the symbol
> is in (compare the address with [p_vaddr,p_vaddr+p_memsz) and add the
> offset for that segment to the address of the symbol as defined in the
> symbol table.
> 

Thanks for clarifying this.

Can we then use the bfd helpers (such as bfd_canonicalize_symtab()) to
get the symbol and section info and calculate the offset as you have
indicated above?

Thanks
Prasanna

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

* Re: Uprobes: howto calculate the probe offset
  2006-01-20 17:17       ` Frank Ch. Eigler
@ 2006-01-23 14:47         ` Prasanna S Panchamukhi
  0 siblings, 0 replies; 13+ messages in thread
From: Prasanna S Panchamukhi @ 2006-01-23 14:47 UTC (permalink / raw)
  To: Frank Ch. Eigler; +Cc: systemtap

Hi Frank,
> 
> Could you explain why the uprobe.offset number is sometimes simply an
> intra-page offset (which kprobes could compute), and sometimes a
> mapping-relative one?

Normally the address specified for the applications are absolute,
hence we map the page that contains the probed address, and add just the
page offset to insert/remove probes.

> 
> Also, given that it's possible to map a shared library at different
> addresses within different processes, does the latter computation
> imply that a uprobe only works for exactly those threads that have the
> same shlib mapped at the same address?  What happens to other threads?

There will be only one on disk image and we put the probes based on
that. If the same image is mapped at different addresses, the offset will
remain the same, but we have to explicitly track all such maps and
insert/remove probes on them as well.

Thanks
Prasanna

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

* RE: Patch [3/3] Userspace probes single stepping out-of-line
@ 2006-02-02  7:46 Zhang, Yanmin
  0 siblings, 0 replies; 13+ messages in thread
From: Zhang, Yanmin @ 2006-02-02  7:46 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap, Keshavamurthy, Anil S, Mao, Bibo

>>-----Original Message-----
>>From: Prasanna S Panchamukhi [mailto:prasanna@in.ibm.com]
>>Sent: 2006年1月30日 16:45
>>To: Zhang, Yanmin
>>Cc: systemtap@sources.redhat.com; Keshavamurthy, Anil S; Mao, Bibo
>>Subject: Re: Patch [3/3] Userspace probes single stepping out-of-line
>>
>>Yanmin,
>>
>>Please see my comments inline below.
>>
>>> >>7. Accessing user space pages not present in memory, from the
>>> >>registered callback routines.
>>> The patch uses the page_addr aligned with stack pointer to store instructions for single step.
>>> It doesn't consider scenarios of multi-thread process. For example, 1 process has 10 threads
>>> and every thread has an 8kb stack.
>>
>>Initially this patch checks if there is enough free space in the current stack
>>page below %esp before storing the instructions for single stepping. I think this
>>should work even for multi threaded processes.
>>
>> All the stacks share the same vma. Just near the end of
>>> the first 4kb page, threads might try to extend the same vma at the same time while every
>>> thread still has a stack page available. I suggest to use stack_addr - sizeof(long long) - size,
>>> if the result is bigger than vma->vm_start  at VM_GROWDOWN case.
>>
>>If there is no free space in the current stack page, we can check
>>for some space before vma->vm_start, and then expand beyond vm_start
>>if there is no space before vma->vm_start as you suggested.
>>We can synchronize among multiple-threads using mmap_sem, I will try and
>>implement this in the next set.
[YM] All threads of the process will grow the stack vma for ever as they couldn't withdraw.
And the address space of the process is used up in the end.

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

* RE: Patch [3/3] Userspace probes single stepping out-of-line
@ 2006-02-02  7:38 Zhang, Yanmin
  0 siblings, 0 replies; 13+ messages in thread
From: Zhang, Yanmin @ 2006-02-02  7:38 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap, Keshavamurthy, Anil S, Mao, Bibo

>>-----Original Message-----
>>From: Prasanna S Panchamukhi [mailto:prasanna@in.ibm.com]
>>Sent: 2006年1月30日 16:45
>>To: Zhang, Yanmin
>>Cc: systemtap@sources.redhat.com; Keshavamurthy, Anil S; Mao, Bibo
>>Subject: Re: Patch [3/3] Userspace probes single stepping out-of-line
>>
>>> >>+		if (((stack_addr - sizeof(long long)) - page_addr) < size)
>>> If stack_addr==page_addr, above is always false because they are unsigned long. Pls. change it to:
>>> 	if ((stack_addr - sizeof(long long)) < (page_addr + size))
>>
>>ok.
>>
>>> >>+			return -ENOMEM;
>>> >>+
>>> >>+		if (__copy_to_user_inatomic((unsigned long *)(page_addr + size),
>>> Should (page_addr + size) is just page_addr?
>>
>>yes, it should be page_addr.
>>
>>> >>+	} else if (vma->vm_flags & VM_GROWSUP) {
>>> >>+		page_addr = (stack_addr & PAGE_MASK) + PAGE_SIZE;
>>> It's weird to get the page align by this approach. Not accurate.
>>
>>Could you please suggest appropriate approach for this?
[YM] page_addr = (stack_addr + PAGE_SIZE -1) & PAGE_MASK;

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

* Re: Patch [3/3] Userspace probes single stepping out-of-line
  2006-01-26  7:30 Patch [3/3] Userspace probes single stepping out-of-line Zhang, Yanmin
@ 2006-01-30  8:42 ` Prasanna S Panchamukhi
  0 siblings, 0 replies; 13+ messages in thread
From: Prasanna S Panchamukhi @ 2006-01-30  8:42 UTC (permalink / raw)
  To: Zhang, Yanmin; +Cc: systemtap, Keshavamurthy, Anil S, Mao, Bibo

Yanmin,

Please see my comments inline below.

> >>7. Accessing user space pages not present in memory, from the
> >>registered callback routines.
> The patch uses the page_addr aligned with stack pointer to store instructions for single step.
> It doesn't consider scenarios of multi-thread process. For example, 1 process has 10 threads
> and every thread has an 8kb stack.

Initially this patch checks if there is enough free space in the current stack
page below %esp before storing the instructions for single stepping. I think this
should work even for multi threaded processes. 

 All the stacks share the same vma. Just near the end of
> the first 4kb page, threads might try to extend the same vma at the same time while every
> thread still has a stack page available. I suggest to use stack_addr - sizeof(long long) - size,
> if the result is bigger than vma->vm_start  at VM_GROWDOWN case.

If there is no free space in the current stack page, we can check
for some space before vma->vm_start, and then expand beyond vm_start 
if there is no space before vma->vm_start as you suggested. 
We can synchronize among multiple-threads using mmap_sem, I will try and
implement this in the next set.

> >>+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 err = 0, 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;
> >>+
> >>+	upage = per_cpu_ptr(uprobe_page, smp_processor_id());
> Upage is per cpu data. If just jumping back to user space to execute the single step,
> device interrupt might disrupt the current thread, then current thread might be schedule
> to another processor. When the thread is woken up again, the upage is not the original.
> How about changing it to task_struct related?

yes, this is a good suggestion.

> >>+	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;
> >>+
> >>+	spin_lock(&mm->page_table_lock);
> >>+
> >>+	/* TODO: do we need to expand stack if extend_vma fails? */
> >>+	if (!(new_vma = find_extend_vma(mm, vm_addr)))
> >>+		err = expand_stack(new_vma, vm_addr);
> If new_vma==NULL, panic?

It should be just vma and not new_vma in the expand_stack call,
I will fix it.

> >>+		if (((stack_addr - sizeof(long long)) - page_addr) < size)
> If stack_addr==page_addr, above is always false because they are unsigned long. Pls. change it to:
> 	if ((stack_addr - sizeof(long long)) < (page_addr + size))

ok.

> >>+			return -ENOMEM;
> >>+
> >>+		if (__copy_to_user_inatomic((unsigned long *)(page_addr + size),
> Should (page_addr + size) is just page_addr?

yes, it should be page_addr.

> >>+	} else if (vma->vm_flags & VM_GROWSUP) {
> >>+		page_addr = (stack_addr & PAGE_MASK) + PAGE_SIZE;
> It's weird to get the page align by this approach. Not accurate.

Could you please suggest appropriate approach for this?

> >>+		if ((page_addr - (stack_addr + sizeof(long long))) < size)
> Similar unsigned long calculation issue.

This will be taken care in the next patch release.


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

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

* RE: Patch [3/3] Userspace probes single stepping out-of-line
@ 2006-01-26  7:30 Zhang, Yanmin
  2006-01-30  8:42 ` Prasanna S Panchamukhi
  0 siblings, 1 reply; 13+ messages in thread
From: Zhang, Yanmin @ 2006-01-26  7:30 UTC (permalink / raw)
  To: prasanna; +Cc: systemtap, Keshavamurthy, Anil S, Mao, Bibo

>>-----Original Message-----
>>From: systemtap-owner@sourceware.org [mailto:systemtap-owner@sourceware.org] On Behalf Of Prasanna S Panchamukhi
>>Sent: 2006年1月19日 22:41
>>To: systemtap@sources.redhat.com
>>Subject: Re: Patch [3/3] Userspace probes single stepping out-of-line
>>
>>
>>This patch handles the executing the registered callback
>>functions when probes is hit.
>>stepping or the expanded stack should be reused for single stepping
>>out-of-line for other probes.
>>
>>5. 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.
>>
>>6. Handing of page faults while inthe kprobes_handler() and while
>>single stepping.
>>
>>7. Accessing user space pages not present in memory, from the
>>registered callback routines.
The patch uses the page_addr aligned with stack pointer to store instructions for single step.
It doesn't consider scenarios of multi-thread process. For example, 1 process has 10 threads
and every thread has an 8kb stack. All the stacks share the same vma. Just near the end of
the first 4kb page, threads might try to extend the same vma at the same time while every
thread still has a stack page available. I suggest to use stack_addr - sizeof(long long) - size,
if the result is bigger than vma->vm_start  at VM_GROWDOWN case.


>>
>>Signed-off-by: Prasanna S Panchamukhi <prasanna@in.ibm.com>
>>
>>
>> arch/i386/kernel/kprobes.c |  352 +++++++++++++++++++++++++++++++++++++++++++--
>> include/asm-i386/kprobes.h |    9 +
>> include/linux/kprobes.h    |    3
>> 3 files changed, 351 insertions(+), 13 deletions(-)
>>
>>diff -puN arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line arch/i386/kernel/kprobes.c
>>--- linux-2.6.15/arch/i386/kernel/kprobes.c~kprobes_userspace_probes_ss_out-of-line	2006-01-19 19:37:53.000000000 +0530
>>+++ linux-2.6.15-prasanna/arch/i386/kernel/kprobes.c	2006-01-19 19:37:53.000000000 +0530
>>@@ -38,9 +38,11 @@
>>
>> void jprobe_return_end(void);
>>
>>+static struct uprobe_page *uprobe_page;
>> 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);
>> /*
>>  * returns non-zero if opcode modifies the interrupt flag.
>>  */
>>@@ -77,6 +79,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();
>>@@ -103,15 +122,223 @@ static inline void set_current_kprobe(st
>> 		kcb->kprobe_saved_eflags &= ~IF_MASK;
>> }
>>
>>+/**
>>+ * 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 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 err = 0, 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;
>>+
>>+	upage = per_cpu_ptr(uprobe_page, smp_processor_id());
Upage is per cpu data. If just jumping back to user space to execute the single step,
device interrupt might disrupt the current thread, then current thread might be schedule
to another processor. When the thread is woken up again, the upage is not the original.
How about changing it to task_struct related?


>>+
>>+	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;
>>+
>>+	spin_lock(&mm->page_table_lock);
>>+
>>+	/* TODO: do we need to expand stack if extend_vma fails? */
>>+	if (!(new_vma = find_extend_vma(mm, vm_addr)))
>>+		err = expand_stack(new_vma, vm_addr);
If new_vma==NULL, panic?


>>+
>>+	if (err) {
>>+		spin_unlock(&mm->page_table_lock);
>>+		return err;
>>+	}
>>+
>>+	spin_unlock(&mm->page_table_lock);
>>+
>>+	/*
>>+	 * 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;
>>+
>>+	pte = get_uprobe_pte(addr);
>>+	if (!pte)
>>+		return -EFAULT;
>>+
>>+	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);
>>+	err = __copy_to_user_inatomic(vaddr,
>>+			(unsigned long *)uprobe->kp.ainsn.insn, size);
>>+	kunmap_atomic(vaddr, KM_USER1);
>>+	if (err)
>>+		return err;
>>+
>>+	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, struct vm_area_struct *vma)
>>+{
>>+	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 (vma->vm_flags & VM_GROWSDOWN) {
>>+		page_addr = stack_addr & PAGE_MASK;
>>+
>>+		if (((stack_addr - sizeof(long long)) - page_addr) < size)
If stack_addr==page_addr, above is always false because they are unsigned long. Pls. change it to:
	if ((stack_addr - sizeof(long long)) < (page_addr + size))


>>+			return -ENOMEM;
>>+
>>+		if (__copy_to_user_inatomic((unsigned long *)(page_addr + size),
Should (page_addr + size) is just page_addr?


>>+								source, size))
>>+			return -EFAULT;
>>+
>>+		regs->eip = page_addr + size;
Same issue like before.


>>+	} else if (vma->vm_flags & VM_GROWSUP) {
>>+		page_addr = (stack_addr & PAGE_MASK) + PAGE_SIZE;
It's weird to get the page align by this approach. Not accurate.

>>+
>>+		if ((page_addr - (stack_addr + sizeof(long long))) < size)
Similar unsigned long calculation issue.


>>+			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;
>>+}

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

end of thread, other threads:[~2006-02-02  7:46 UTC | newest]

Thread overview: 13+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2006-01-19 14:38 Patch [1/3] Userspace probes new interfaces Prasanna S Panchamukhi
2006-01-19 14:37 ` Patch [2/3] Userspace probes readpage hooks Prasanna S Panchamukhi
2006-01-19 14:38   ` Patch [3/3] Userspace probes single stepping out-of-line Prasanna S Panchamukhi
2006-01-20  8:38     ` Uprobes: howto calculate the probe offset Prasanna S Panchamukhi
2006-01-20 16:58       ` Ulrich Drepper
2006-01-23 14:26         ` Prasanna S Panchamukhi
2006-01-20 17:17       ` Frank Ch. Eigler
2006-01-23 14:47         ` Prasanna S Panchamukhi
2006-01-19 16:14 ` Patch [1/3] Userspace probes new interfaces Frank Ch. Eigler
2006-01-26  7:30 Patch [3/3] Userspace probes single stepping out-of-line Zhang, Yanmin
2006-01-30  8:42 ` Prasanna S Panchamukhi
2006-02-02  7:38 Zhang, Yanmin
2006-02-02  7:46 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).