diff -urN linux.05/AIO-NOTES linux.10/AIO-NOTES
--- linux.05/AIO-NOTES	Wed Dec 31 19:00:00 1969
+++ linux.10/AIO-NOTES	Fri Jan 18 23:19:14 2002
@@ -0,0 +1,11 @@
+20020102
+	- ABI adjustment to io_getevents: the prototype is now
+		long io_getevents(aio_context_t ctx_id,
+                                  long nr,
+                                  struct io_event *events,
+                                  struct timespec *timeout)
+	  which does not affect binary compatibility on x86, only 
+	  for 64 bit machines.
+
+
+
diff -urN linux.05/fs/Makefile linux.10/fs/Makefile
--- linux.05/fs/Makefile	Fri Jan 18 23:18:40 2002
+++ linux.10/fs/Makefile	Fri Jan 18 23:19:14 2002
@@ -22,6 +22,8 @@
 obj-y += noquot.o
 endif
 
+obj-y += aio.o
+
 subdir-$(CONFIG_PROC_FS)	+= proc
 subdir-y			+= partitions
 
diff -urN linux.05/fs/aio.c linux.10/fs/aio.c
--- linux.05/fs/aio.c	Wed Dec 31 19:00:00 1969
+++ linux.10/fs/aio.c	Sat Jan 19 00:44:46 2002
@@ -0,0 +1,899 @@
+/* fs/aio.c
+ *	An async IO implementation for Linux
+ *	Written by Benjamin LaHaise <bcrl@redhat.com>
+ *
+ *	Implements an efficient asynchronous io interface.
+ *
+ *	Copyright 2000, 2001 Red Hat, Inc.  All Rights Reserved.
+ *
+ *   This program is free software; you can redistribute it and/or modify
+ *   it under the terms of the GNU General Public License as published by
+ *   the Free Software Foundation; either version 2 of the License, or
+ *   (at your option) any later version.
+ *
+ *   This program is distributed in the hope that it will be useful,
+ *   but WITHOUT ANY WARRANTY; without even the implied warranty of
+ *   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ *   GNU General Public License for more details.
+ *
+ *   You should have received a copy of the GNU General Public License along
+ *   with this program; if not, write to the Free Software Foundation, Inc.,
+ *   59 Temple Place, Suite 330, Boston, MA  02111-1307  USA
+ *
+ */
+//#define DEBUG 1
+
+#include <linux/sched.h>
+#include <linux/fs.h>
+#include <linux/file.h>
+#include <linux/mm.h>
+#include <linux/mman.h>
+#include <linux/vmalloc.h>
+#include <linux/iobuf.h>
+#include <linux/slab.h>
+#include <linux/timer.h>
+#include <linux/brlock.h>
+#include <linux/aio.h>
+#include <linux/smp_lock.h>
+#include <linux/compiler.h>
+#include <linux/poll.h>
+
+#include <asm/uaccess.h>
+
+#define static	/**/
+#define inline	/**/
+#define MAX_IOCTXS	0x800
+#if DEBUG > 1
+#define dprintk		printk
+#else
+#define dprintk(x...)	do { ; } while (0)
+#endif
+
+static spinlock_t aio_read_lock = SPIN_LOCK_UNLOCKED;
+static rwlock_t aio_req_lock = RW_LOCK_UNLOCKED;
+
+static kmem_cache_t	*kiocb_cachep;
+static kmem_cache_t	*kioctx_cachep;
+
+/* tunable.  Needs to be added to sysctl. */
+int max_aio_reqs = 0x10000;
+
+/* Used for rare fput completion. */
+static void aio_fput_routine(void *);
+static struct tq_struct	fput_tqueue = {
+	routine:	aio_fput_routine,
+};
+
+static struct kiocb	*fput_iocbs;
+static spinlock_t	fput_lock = SPIN_LOCK_UNLOCKED;
+
+/* aio_setup
+ *	Creates the slab caches used by the aio routines, panic on
+ *	failure as this is done early during the boot sequence.
+ */
+static int __init aio_setup(void)
+{
+	kiocb_cachep = kmem_cache_create("kiocb", sizeof(struct kiocb),
+				0, SLAB_HWCACHE_ALIGN, NULL, NULL);
+	if (!kiocb_cachep)
+		panic("unable to create kiocb cache\n");
+
+	kioctx_cachep = kmem_cache_create("kioctx", sizeof(struct kioctx),
+				0, SLAB_HWCACHE_ALIGN, NULL, NULL);
+	if (!kioctx_cachep)
+		panic("unable to create kioctx cache");
+
+	printk(KERN_NOTICE "aio_setup: okay!\n");
+	printk(KERN_NOTICE "aio_setup: sizeof(struct page) = %d\n", (int)sizeof(struct page));
+
+	return 0;
+}
+
+static void ioctx_free_reqs(struct kioctx *ctx)
+{
+	struct kiocb *iocb;
+	int nr = 0;
+	while ((iocb = ctx->free_req) != NULL) {
+		ctx->free_req = iocb->user_obj;
+		kmem_cache_free(kiocb_cachep, iocb);
+		nr++;
+	}
+}
+
+/* ioctx_alloc
+ *	Allocates and initializes an ioctx.  Returns an ERR_PTR if it failed.
+ */
+static struct kioctx *ioctx_alloc(unsigned nr_reqs)
+{
+	struct kioctx *ctx;
+	unsigned i;
+	long size;
+
+	/* Round off to a power of 2.  Needed for cheap mask operations */
+	for (i=1; i<nr_reqs; i<<=1)
+		;
+
+	nr_reqs = i;
+
+	/* Prevent overflows */
+	if ((nr_reqs > (0x10000000U / sizeof(struct io_event))) ||
+	    (nr_reqs > (0x10000000U / sizeof(struct kiocb)))) {
+		pr_debug("ENOMEM: nr_reqs too high\n");
+		return ERR_PTR(-ENOMEM);
+	}
+
+	ctx = kmem_cache_alloc(kioctx_cachep, GFP_KERNEL);
+	if (!ctx)
+		return ERR_PTR(-ENOMEM);
+
+	memset(ctx, 0, sizeof(*ctx));
+	ctx->max_reqs = nr_reqs;
+
+	atomic_set(&ctx->users, 1);
+	spin_lock_init(&ctx->lock);
+	init_waitqueue_head(&ctx->wait);
+
+	ctx->reqs = NULL;
+	ctx->free_req = NULL;
+
+	/* Allocate nr_reqs iocbs for io.  Free iocbs are chained off of 
+	 * ctx->free_req via the user_obj pointer.
+	 */
+	for (i=0; i<nr_reqs; i++) {
+		struct kiocb *iocb = kmem_cache_alloc(kiocb_cachep, GFP_KERNEL);
+		if (!iocb)
+			goto out_freereqs;
+		iocb->ctx = ctx;
+		iocb->filp = NULL;
+		iocb->cancel = NULL;
+		iocb->user_obj = ctx->free_req;
+		iocb->user_data = 0;
+		iocb->nr_atomic = 0;
+		iocb->key = i;
+		ctx->free_req = iocb;
+	}
+
+	/* Compensate for the ring buffer's head/tail overlap entry */
+	nr_reqs *= 2;
+
+	size = sizeof(struct aio_ring);
+	size += sizeof(struct io_event) * nr_reqs;
+
+	/* Try to use alloc_pages first... */
+	for (i=0; (PAGE_SIZE << i) < size; i++)
+		;
+	ctx->ring_order = i;
+	ctx->ring = (void *)__get_free_pages(GFP_KERNEL, i);
+	if (!ctx->ring) {
+		/* vmalloc it is... */
+		ctx->ring = vmalloc(size);
+		if (!ctx->ring)
+			goto out_freereqs;
+		ctx->ring_was_vmallocd = 1;
+	} else
+		ctx->ring_was_vmallocd = 0;
+
+	memset(ctx->ring, 0, size);
+	ctx->ring_mask = nr_reqs - 1;		/* trusted copy */
+	ctx->ring->mask = ctx->ring_mask;	/* user copy */
+
+	/* now link into global list.  kludge.  FIXME */
+	write_lock(&aio_req_lock);			/* FIXME */
+	ctx->ring->id = ctx->user_id = current->mm->new_ioctx_id++;
+	ctx->next = current->mm->ioctx_list;
+	current->mm->ioctx_list = ctx;
+	write_unlock(&aio_req_lock);			/* FIXME */
+
+	dprintk("aio: allocated ioctx %p[%ld]: mm=%p mask=0x%x\n",
+		ctx, ctx->user_id, current->mm, ctx->ring->mask);
+	return ctx;
+
+out_freereqs:
+	ioctx_free_reqs(ctx);
+out_freectx:
+	kmem_cache_free(kioctx_cachep, ctx);
+	ctx = ERR_PTR(-ENOMEM);
+
+	dprintk("aio: error allocating ioctx %p\n", ctx);
+	return ctx;
+}
+
+void exit_aio(struct mm_struct *mm)
+{
+	struct kioctx *ctx = mm->ioctx_list;
+
+	mm->ioctx_list = NULL;
+	while (ctx) {
+		struct kioctx *next = ctx->next;
+		put_ioctx(ctx);
+		ctx = next;
+	}
+}
+
+/* __put_ioctx
+ *	Called when the last user of an aio context has gone away,
+ *	and the struct needs to be freed.
+ */
+void __put_ioctx(struct kioctx *ctx)
+{
+	if (ctx->ring_was_vmallocd)
+		vfree(ctx->ring);
+	else
+		free_pages((unsigned long)ctx->ring, ctx->ring_order);
+
+	ioctx_free_reqs(ctx);
+	kmem_cache_free(kioctx_cachep, ctx);
+}
+
+/* aio_get_req
+ *	Allocate a slot for an aio request.  Increments the users count
+ * of the kioctx so that the kioctx stays around until all requests are
+ * complete.  Returns -EAGAIN if no requests are free.
+ */
+static inline struct kiocb *aio_get_req(struct kioctx *ctx)
+{
+	struct kiocb *req;
+
+	/* Use cmpxchg instead of spin_lock? */
+	spin_lock_irq(&ctx->lock);
+	req = ctx->free_req;
+	if (req && (ctx->reqs_active < aio_ring_avail(ctx->ring))) {
+		ctx->free_req = req->user_obj;
+		ctx->reqs_active++;
+		spin_unlock_irq(&ctx->lock);
+		req->user_obj = NULL;
+
+		get_ioctx(ctx);
+		return req;
+	}
+	spin_unlock_irq(&ctx->lock);
+
+	return NULL;
+}
+
+static void aio_fput_routine(void *data)
+{
+	struct kiocb *req;
+
+	spin_lock_irq(&fput_lock);
+	req = fput_iocbs;
+	fput_iocbs = NULL;
+	spin_unlock_irq(&fput_lock);
+
+	while (req) {
+		struct kioctx *ctx = req->ctx;
+		struct kiocb *next = req->user_obj;
+
+		/* Complete the fput */
+		__fput(req->filp);
+
+		/* Link the iocb into the context's free list */
+		spin_lock_irq(&ctx->lock);
+		req->filp = NULL;
+		req->user_obj = ctx->free_req;
+		ctx->free_req = req;
+		ctx->reqs_active--;
+		spin_unlock_irq(&ctx->lock);
+
+		put_ioctx(ctx);
+		req = next;
+	}
+}
+
+static void __aio_put_req(struct kioctx *ctx, struct kiocb *req)
+{
+	int put = 1;
+	dprintk(KERN_DEBUG "aio_put(%p): f_count=%d\n",
+		req, atomic_read(&req->filp->f_count));
+
+	req->cancel = NULL;
+
+	/* Must be done under the lock to serialise against cancellations */
+	if (unlikely(atomic_dec_and_test(&req->filp->f_count))) {
+		get_ioctx(ctx);
+		spin_lock(&fput_lock);
+		req->user_obj = fput_iocbs;
+		fput_iocbs = req;
+		spin_unlock(&fput_lock);
+		put = 0;
+		schedule_task(&fput_tqueue);
+	} else {
+		req->filp = NULL;
+		req->user_obj = ctx->free_req;
+		ctx->free_req = req;
+		ctx->reqs_active--;
+	}
+}
+
+static void aio_put_req(struct kioctx *ctx, struct kiocb *req)
+{
+	/* FIXME: use cmpxchg instead of spin_lock? */
+	spin_lock_irq(&ctx->lock);
+	__aio_put_req(ctx, req);
+	spin_unlock_irq(&ctx->lock);
+	put_ioctx(ctx);
+}
+
+/*	Lookup an ioctx id.  ioctx_list is lockless for reads.
+ *	FIXME: this is O(n) and is only suitable for development.
+ */
+static inline struct kioctx *lookup_ioctx(unsigned long ctx_id)
+{
+	struct kioctx *ioctx;
+	struct mm_struct *mm = current->mm;
+
+	read_lock(&aio_req_lock);
+	for (ioctx = current->mm->ioctx_list; ioctx; ioctx = ioctx->next)
+		if (likely(ioctx->user_id == ctx_id && !ioctx->dead)) {
+			get_ioctx(ioctx);
+			break;
+		}
+	read_unlock(&aio_req_lock);
+
+	return ioctx;
+}
+
+/* aio_complete
+ *	Called when the io request on the given iocb is complete.
+ */
+void aio_complete(struct kiocb *iocb, long res, long res2)
+{
+	struct kioctx	*ctx = iocb->ctx;
+	struct aio_ring	*ring = ctx->ring;
+	struct io_event	*event;
+	unsigned long	flags;
+	unsigned long	tail;
+
+	/* add a completion event to the ring buffer.
+	 * must be done holding ctx->lock to prevent
+	 * other code from messing with the tail
+	 * pointer since we might be called from irq
+	 * context.
+	 */
+	spin_lock_irqsave(&ctx->lock, flags);
+
+	tail = ring->tail;
+	event = &ring->io_events[tail];
+	tail = (tail + 1) & ring->mask;
+
+	event->obj = (u64)(unsigned long)iocb->user_obj;
+	event->data = iocb->user_data;
+	event->res = res;
+	event->res2 = res2;
+
+	dprintk("aio_complete: %p[%lu]: %p: %p %Lx %lx %lx\n",
+		ctx, tail, iocb, iocb->user_obj, iocb->user_data, res, res2);
+
+	/* after flagging the request as done, we
+	 * must never even look at it again
+	 */
+	barrier();
+
+	ring->tail = tail;
+
+	wmb();
+	if (!ring->woke)
+		ring->woke = 1;
+
+	pr_debug("added to ring %p at [%lu]\n", iocb, tail);
+#if 0
+	if (!wake) {
+		printk("kio_complete: should send user of %p a signal...\n", ctx);
+	}
+#endif
+
+	wake_up(&ctx->wait);
+
+	/* everything turned out well, dispose of the aiocb. */
+	__aio_put_req(ctx, iocb);
+
+	spin_unlock_irqrestore(&ctx->lock, flags);
+	put_ioctx(ctx);
+}
+
+/* aio_read_evt
+ *	Pull an event off of the ioctx's event ring.  Returns the number of 
+ *	events fetched (0 or 1 ;-)
+ *	FIXME: make this use cmpxchg.
+ *	TODO: make the ringbuffer user mmap()able (requires FIXME).
+ */
+static int aio_read_evt(struct kioctx *ioctx, struct io_event *ent)
+{
+	struct aio_ring *ring = ioctx->ring;
+	unsigned long head;
+	int ret = 0;
+
+	dprintk("in aio_read_evt h%lu t%lu m%lu\n",
+		 (unsigned long)ring->head, (unsigned long)ring->tail,
+		 (unsigned long)ring->mask);
+	barrier();
+	if (ring->head == ring->tail)
+		goto out;
+
+	spin_lock(&aio_read_lock);	/* investigate the value of making this per-ctx */
+
+	head = ring->head;
+	if (head != ring->tail) {
+		*ent = ring->io_events[head];
+		head = (head + 1) & ioctx->ring_mask;
+		barrier();
+		ring->head = head;
+		ret = 1;
+	}
+	spin_unlock(&aio_read_lock);
+
+out:
+	dprintk("leaving aio_read_evt: %d  h%lu t%lu\n", ret,
+		 (unsigned long)ring->head, (unsigned long)ring->tail);
+	return ret;
+}
+
+struct timeout {
+	struct timer_list	timer;
+	int			timed_out;
+	wait_queue_head_t	wait;
+};
+
+static void timeout_func(unsigned long data)
+{
+	struct timeout *to = (struct timeout *)data;
+
+	to->timed_out = 1;
+	wake_up(&to->wait);
+}
+
+static inline void init_timeout(struct timeout *to)
+{
+	init_timer(&to->timer);
+	to->timer.data = (unsigned long)to;
+	to->timer.function = timeout_func;
+	to->timed_out = 0;
+	init_waitqueue_head(&to->wait);
+}
+
+static inline void set_timeout(struct timeout *to, struct timespec *ts)
+{
+	unsigned long how_long;
+
+	if (!ts->tv_sec && !ts->tv_nsec) {
+		to->timed_out = 1;
+		return;
+	}
+
+	how_long = ts->tv_sec * HZ;
+#define HZ_NS (1000000000 / HZ)
+	how_long += (ts->tv_nsec + HZ_NS - 1) / HZ_NS;
+	
+	to->timer.expires = jiffies + how_long;
+	add_timer(&to->timer);
+}
+
+static inline void clear_timeout(struct timeout *to)
+{
+	del_timer_sync(&to->timer);
+}
+
+static int read_events(struct kioctx *ctx, int nr, struct io_event *event,
+			struct timespec *timeout)
+{
+	struct task_struct	*tsk = current;
+	DECLARE_WAITQUEUE(wait, tsk);
+	DECLARE_WAITQUEUE(to_wait, tsk);
+	int			ret;
+	int			i = 0;
+	struct io_event		ent;
+	struct timespec		ts;
+	struct timeout		to;
+
+	init_timeout(&to);
+
+	if (timeout) {
+		ret = -EFAULT;
+		if (copy_from_user(&ts, timeout, sizeof(ts)))
+			goto out;
+
+		set_timeout(&to, &ts);
+		if (to.timed_out)
+			timeout = 0;
+	}
+
+	memset(&ent, 0, sizeof(ent));
+	ret = 0;
+
+	while (i < nr) {
+		ret = aio_read_evt(ctx, &ent);
+		if (ret <= 0) {
+			if (i)
+				break;
+
+			ret = 0;
+			if (to.timed_out)
+				break;
+
+			add_wait_queue(&ctx->wait, &wait);
+			add_wait_queue(&to.wait, &to_wait);
+			do {
+				set_task_state(tsk, TASK_INTERRUPTIBLE);
+
+				ret = aio_read_evt(ctx, &ent);
+				if (ret <= 0)
+					break;
+				ret = 0;
+				if (to.timed_out)
+					break;
+				schedule();
+				if (to.timed_out)
+					break;
+				if (signal_pending(tsk)) {
+					ret = -EINTR;
+					break;
+				}
+				ret = aio_read_evt(ctx, &ent);
+			} while (ret <= 0) ;
+
+			set_task_state(tsk, TASK_RUNNING);
+			remove_wait_queue(&ctx->wait, &wait);
+			remove_wait_queue(&to.wait, &to_wait);
+		}
+
+		if (unlikely(ret <= 0))
+			break;
+
+		dprintk("read event: %Lx %Lx %Lx %Lx\n",
+			ent.data, ent.obj, ent.res, ent.res2);
+
+		/* FIXME: split checks in two */
+		ret = -EFAULT;
+		if (unlikely(copy_to_user(event, &ent, sizeof(ent)))) {
+			printk(KERN_DEBUG "aio: lost an event due to EFAULT.\n");
+			break;
+		}
+
+		/* Now complete the aio request and copy the result codes to userland. */
+		event ++;
+		i ++;
+	}
+
+	if (timeout)
+		clear_timeout(&to);
+out:
+	return i ? i : ret;
+}
+
+asmlinkage long sys_io_setup(unsigned nr_reqs, aio_context_t *ctxp)
+{
+	struct kioctx *ioctx = NULL;
+	unsigned long ctx;
+	long ret;
+
+	ret = get_user(ctx, ctxp);
+	if (unlikely(ret))
+		goto out;
+
+	ret = -EINVAL;
+	if (ctx || nr_reqs > max_aio_reqs) {
+		pr_debug("EINVAL: io_setup: !ctx or nr_reqs > max\n");
+		goto out;
+	}
+
+	ioctx = ioctx_alloc(nr_reqs);
+	ret = PTR_ERR(ioctx);
+	if (!IS_ERR(ioctx)) {
+		ret = put_user(ioctx->user_id, ctxp);
+		if (!ret)
+			return 0;
+		ioctx->dead = 1;
+		put_ioctx(ioctx);
+	}
+
+out:
+	return ret;
+}
+
+/* aio_release
+ *	Release the kioctx associated with the userspace handle.
+ */
+asmlinkage long sys_io_destroy(aio_context_t ctx)
+{
+	struct kioctx *ioctx = lookup_ioctx(ctx);
+	if (likely(NULL != ioctx)) {
+		struct kioctx **tmp;
+		int was_dead;
+
+		/* delete the entry from the list is someone else hasn't already */
+		write_lock(&aio_req_lock);
+		was_dead = ioctx->dead;
+		ioctx->dead = 1;
+		for (tmp = &current->mm->ioctx_list; *tmp && *tmp != ioctx;
+		     tmp = &(*tmp)->next)
+			;
+		if (*tmp)
+			*tmp = ioctx->next;
+		write_unlock(&aio_req_lock);
+
+		dprintk("aio_release(%p)\n", ioctx);
+		put_ioctx(ioctx);	/* once for the lookup */
+		if (likely(!was_dead))
+			put_ioctx(ioctx);	/* twice for the list */
+		return 0;
+	}
+	pr_debug("EINVAL: io_destroy: invalid context id\n");
+	return -EINVAL;
+}
+
+int generic_aio_poll(struct file *file, struct kiocb *req, struct iocb iocb)
+{
+	unsigned events = iocb.aio_buf;
+
+	/* Did the user set any bits they weren't supposed to? (The 
+	 * above is actually a cast.
+	 */
+	if (unlikely(events != iocb.aio_buf))
+		return -EINVAL;
+	
+	return async_poll(req, events);
+}
+
+/* sys_io_submit
+ *	Copy an aiocb from userspace into kernel space, then convert it to
+ *	a kiocb, submit and repeat until done.  Error codes on copy/submit
+ *	only get returned for the first aiocb copied as otherwise the size
+ *	of aiocbs copied is returned (standard write sematics).
+ */
+asmlinkage long sys_io_submit(aio_context_t ctx_id, long nr, struct iocb **iocbpp)
+{
+	struct kioctx *ctx;
+	long ret = 0;
+	int i;
+
+	ctx = lookup_ioctx(ctx_id);
+	if (unlikely(!ctx)) {
+		pr_debug("EINVAL: io_submit: invalid context id\n");
+		return -EINVAL;
+	}
+
+	for (i=0; i<nr; i++) {
+		int (*op)(struct file *, struct kiocb *, struct iocb);
+		struct iocb *iocbp, tmp;
+		struct kiocb *req;
+		struct file *file;
+
+		ret = get_user(iocbp, iocbpp + i);
+		if (ret)
+			break;
+
+		ret = copy_from_user(&tmp, iocbp, sizeof(tmp));
+		if (ret)
+			break;
+
+		ret = -EINVAL;
+		/* enforce forwards compatibility on users */
+		if (tmp.aio_reserved1 || tmp.aio_reserved2 || tmp.aio_reserved3) {
+			pr_debug("EINVAL: io_submit: reserve field set\n");
+			break;
+		}
+
+		/* prevent overflows */
+		if ((tmp.aio_buf != (unsigned long)tmp.aio_buf) ||
+		    (tmp.aio_nbytes != (size_t)tmp.aio_nbytes) ||
+		    ((ssize_t)tmp.aio_nbytes < 0)) {
+			pr_debug("EINVAL: io_submit: overflow check\n");
+			break;
+		}
+
+		file = fget(tmp.aio_fildes);
+		ret = -EBADF;
+		if (!file)
+			break;
+
+		req = aio_get_req(ctx);
+		ret = -EAGAIN;
+		if (!req)
+			goto out_fput;
+
+		req->filp = file;
+		tmp.aio_key = req->key;
+		ret = put_user(tmp.aio_key, &iocbp->aio_key);
+		if (ret)
+			goto out_put_req;
+
+		req->user_obj = iocbp;
+		req->user_data = tmp.aio_data;
+
+		switch (tmp.aio_lio_opcode) {
+		case IOCB_CMD_PREAD:	op = file->f_op->aio_read; break;
+		case IOCB_CMD_PREADX:	op = file->f_op->aio_readx; break;
+		case IOCB_CMD_PWRITE:	op = file->f_op->aio_write; break;
+		case IOCB_CMD_FSYNC:	op = file->f_op->aio_fsync; break;
+		case IOCB_CMD_POLL:	op = generic_aio_poll; break;
+		default:		op = NULL; break;
+		}
+		ret = -EINVAL;
+		if (!op) {
+			pr_debug("EINVAL: io_submit: no operation provided\n");
+			goto out_put_req;
+		}
+
+		ret = op(file, req, tmp);
+		if (!ret)
+			continue;
+
+		if (ret != 0) {
+			aio_complete(req, ret, 0);
+			continue;
+		}
+
+		pr_debug("io_submit: op returned %ld\n", ret);
+
+	out_put_req:
+		aio_put_req(ctx, req);
+	out_fput:
+		fput(file);
+		break;
+	}
+
+	put_ioctx(ctx);
+	run_task_queue(&tq_disk);
+	return i ? i : ret;
+}
+
+static inline void generic_aio_complete_rw(int rw, void *_iocb, struct kvec *vec, ssize_t res)
+{
+	struct kiocb *iocb = _iocb;
+	ssize_t total = iocb->nr_atomic;
+
+	if (res > 0)
+		total += res;
+
+	aio_complete(iocb, total ? total : res, 0);
+	//unmap_kvec(vec, rw);
+	//free_kvec(vec);
+}
+
+void generic_aio_complete_read(void *_iocb, struct kvec *vec, ssize_t res)
+{
+	generic_aio_complete_rw(1, _iocb, vec, res);
+}
+
+void generic_aio_complete_write(void *_iocb, struct kvec *vec, ssize_t res)
+{
+	generic_aio_complete_rw(0, _iocb, vec, res);
+}
+
+ssize_t generic_aio_read(struct file *file, struct kiocb *req, struct iocb iocb, size_t min_size)
+{
+	unsigned long buf = iocb.aio_buf;
+	size_t size = iocb.aio_nbytes;
+	ssize_t	nr_read = 0;
+	loff_t pos = iocb.aio_offset;
+	kvec_cb_t cb;
+
+	if (likely(NULL != file->f_op->new_read)) {
+		nr_read = file->f_op->new_read(file, (void *)buf, size,
+					       &pos, F_ATOMIC);
+		dprintk("from new_read: nr_read: %ld\n", (long)nr_read);
+		if ((-EAGAIN == nr_read) || (-EWOULDBLOCKIO == nr_read))
+			nr_read = 0;
+		else if ((nr_read >= min_size) || (nr_read < 0)) {
+			dprintk("returning nr_read: %ld\n", (long)nr_read);
+			return nr_read;
+		}
+	}
+	dprintk("nr_read: %ld\n", (long)nr_read);
+
+	req->nr_atomic = nr_read;
+	size -= nr_read;
+	buf += nr_read;
+	cb.vec = map_user_kvec(READ, buf, size);
+	cb.fn = generic_aio_complete_read;
+	cb.data = req;
+
+	dprintk("generic_aio_read: cb.vec=%p\n", cb.vec);
+	if (IS_ERR(cb.vec))
+		return nr_read ? nr_read : PTR_ERR(cb.vec);
+
+	return file->f_op->kvec_read(file, cb, size, pos);
+}
+
+ssize_t generic_file_aio_read(struct file *file, struct kiocb *req, struct iocb iocb)
+{
+	return generic_aio_read(file, req, iocb, iocb.aio_nbytes);	
+}
+
+ssize_t generic_sock_aio_read(struct file *file, struct kiocb *req, struct iocb iocb)
+{
+	return generic_aio_read(file, req, iocb, 1);	
+}
+
+ssize_t generic_aio_write(struct file *file, struct kiocb *req, struct iocb iocb, size_t min_size)
+{
+	unsigned long buf = iocb.aio_buf;
+	size_t size = iocb.aio_nbytes;
+	loff_t pos = iocb.aio_offset;
+	ssize_t	nr_written = 0;
+	kvec_cb_t cb;
+	long res;
+
+	if (likely(NULL != file->f_op->new_write)) {
+		nr_written = file->f_op->new_write(file, (void *)buf, size,
+					       &pos, F_ATOMIC);
+		pr_debug("generic_aio_write: new_write: %ld\n", (long)nr_written);
+		if (-EAGAIN == nr_written)
+			nr_written = 0;
+		if ((nr_written >= min_size) || (nr_written < 0))
+			return nr_written;
+	}
+
+	req->nr_atomic = nr_written;
+	size -= nr_written;
+	buf += nr_written;
+	cb.vec = map_user_kvec(WRITE, buf, size);
+	cb.fn = generic_aio_complete_write;
+	cb.data = req;
+
+	if (IS_ERR(cb.vec)) {
+		pr_debug("generic_aio_write: map_user_kvec: %ld\n", PTR_ERR(cb.vec));
+		return nr_written ? nr_written : PTR_ERR(cb.vec);
+	}
+
+	res = file->f_op->kvec_write(file, cb, size, iocb.aio_offset);
+	pr_debug("generic_aio_write: kvec_write: %ld\n", res);
+	if (unlikely(res < 0)) {
+		if (nr_written)
+			res = nr_written;
+	}
+	return res;
+}
+
+ssize_t generic_file_aio_write(struct file *file, struct kiocb *req, struct iocb iocb)
+{
+	return generic_aio_write(file, req, iocb, iocb.aio_nbytes);	
+}
+
+asmlinkage long sys_io_cancel(aio_context_t ctx, struct iocb *iocb)
+{
+	return -ENOSYS;
+}
+
+asmlinkage long sys_io_wait(aio_context_t ctx_id, struct iocb *iocb, struct timespec *timeout)
+{
+#if 0	/* FIXME.  later. */
+	struct kioctx *ioctx;
+	long ret = -EINVAL;
+	unsigned key;
+	long obj = (long)iocb;
+
+	ioctx = lookup_ioctx(ctx_id);
+	if (!ioctx)
+		goto out;
+
+	ret = get_user(key, &iocb->aio_key);
+	if (ret)
+		goto out;
+
+	ret = __aio_complete(ioctx, key, obj, !!timeout);
+	put_ioctx(ioctx);
+
+out:
+	return ret;
+#endif
+	return -ENOSYS;
+}
+
+asmlinkage long sys_io_getevents(aio_context_t ctx_id,
+				   long nr,
+				   struct io_event *events,
+				   struct timespec *timeout)
+{
+	struct kioctx *ioctx = lookup_ioctx(ctx_id);
+	long ret = -EINVAL;
+
+	if (likely(NULL != ioctx)) {
+		ret = read_events(ioctx, nr, events, timeout);
+		put_ioctx(ioctx);
+	}
+
+	return ret;
+}
+
+__initcall(aio_setup);
diff -urN linux.05/include/linux/aio.h linux.10/include/linux/aio.h
--- linux.05/include/linux/aio.h	Wed Dec 31 19:00:00 1969
+++ linux.10/include/linux/aio.h	Sat Jan 19 00:16:59 2002
@@ -0,0 +1,73 @@
+#ifndef __LINUX__AIO_H
+#define __LINUX__AIO_H
+
+#ifndef __LINUX__KIOVEC_H
+#include <linux/kiovec.h>
+#endif
+#ifndef __ASM__ATOMIC_H
+#include <asm/atomic.h>
+#endif
+
+#define AIO_MAXSEGS		4
+#define AIO_KIOGRP_NR_ATOMIC	8
+
+struct kioctx;
+
+struct kiocb {
+	void		(*cancel)(void *data, struct kioctx *ctx, int idx);
+	struct file	*filp;
+	struct kioctx	*ctx;
+	void		*user_obj;
+	__u64		user_data;
+	ssize_t		nr_atomic;
+	unsigned	key;		/* id of this request */
+};
+
+#define AIO_RING_PAGES	8
+#define AIO_RING_PAGES	8
+struct kioctx {
+	atomic_t		users;
+	int			dead;
+
+	/* This needs improving */
+	unsigned long		user_id;
+	struct kioctx		*next;
+
+	wait_queue_head_t	wait;
+
+	spinlock_t		lock;
+
+	int			reqs_active;
+	struct kiocb		*reqs;
+	struct kiocb		*free_req;
+
+	unsigned		max_reqs;
+	unsigned		ring_mask;
+	struct aio_ring		*ring;
+	int			ring_was_vmallocd;
+	int			ring_order;
+};
+
+extern struct file_operations aio_fops;
+
+extern void FASTCALL(aio_complete(struct kiocb *iocb, long res, long res2));
+extern void __put_ioctx(struct kioctx *ctx);
+struct mm_struct;
+extern void exit_aio(struct mm_struct *mm);
+
+#define get_ioctx(kioctx)	atomic_inc(&(kioctx)->users)
+#define put_ioctx(kioctx)	do { if (atomic_dec_and_test(&(kioctx)->users)) __put_ioctx(kioctx); } while (0)
+
+#ifndef __LINUX__AIO_ABI_H
+#include <linux/aio_abi.h>
+#endif
+
+struct file;
+extern int generic_aio_poll(struct file *file, struct kiocb *req, struct iocb iocb);
+extern ssize_t generic_aio_read(struct file *file, struct kiocb *req, struct iocb iocb, size_t min_size);
+extern ssize_t generic_aio_write(struct file *file, struct kiocb *req, struct iocb iocb, size_t min_size);
+extern ssize_t generic_file_aio_read(struct file *file, struct kiocb *req, struct iocb iocb);
+extern ssize_t generic_file_aio_write(struct file *file, struct kiocb *req, struct iocb iocb);
+extern ssize_t generic_sock_aio_read(struct file *file, struct kiocb *req, struct iocb iocb);
+
+#endif /* __LINUX__AIO_H */
diff -urN linux.05/include/linux/aio_abi.h linux.10/include/linux/aio_abi.h
--- linux.05/include/linux/aio_abi.h	Wed Dec 31 19:00:00 1969
+++ linux.10/include/linux/aio_abi.h	Sat Jan 19 00:16:59 2002
@@ -0,0 +1,101 @@
+/* linux/aio_abi.h
+ *
+ * Copyright 2000,2001,2002 Red Hat.
+ *
+ * Written by Benjamin LaHaise <bcrl@redhat.com>
+ *
+ * Permission to use, copy, modify, and distribute this software and its
+ * documentation is hereby granted, provided that the above copyright
+ * notice appears in all copies.  This software is provided without any
+ * warranty, express or implied.  Red Hat makes no representations about
+ * the suitability of this software for any purpose.
+ *
+ * IN NO EVENT SHALL RED HAT BE LIABLE TO ANY PARTY FOR DIRECT, INDIRECT,
+ * SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OF
+ * THIS SOFTWARE AND ITS DOCUMENTATION, EVEN IF RED HAT HAS BEEN ADVISED
+ * OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ * RED HAT DISCLAIMS ANY WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+ * PURPOSE.  THE SOFTWARE PROVIDED HEREUNDER IS ON AN "AS IS" BASIS, AND
+ * RED HAT HAS NO OBLIGATION TO PROVIDE MAINTENANCE, SUPPORT, UPDATES,
+ * ENHANCEMENTS, OR MODIFICATIONS.
+ */
+#ifndef __LINUX__AIO_ABI_H
+#define __LINUX__AIO_ABI_H
+
+#include <asm/byteorder.h>
+
+typedef unsigned long	aio_context_t;
+
+enum {
+	IOCB_CMD_PREAD = 0,
+	IOCB_CMD_PWRITE = 1,
+	IOCB_CMD_FSYNC = 2,
+	IOCB_CMD_FDSYNC = 3,
+	IOCB_CMD_PREADX = 4,
+	IOCB_CMD_POLL = 5,
+};
+
+/* read() from /dev/aio returns these structures. */
+struct io_event {
+	__u64		data;		/* the data field from the iocb */
+	__u64		obj;		/* what iocb this event came from */
+	__s64		res;		/* result code for this event */
+	__s64		res2;		/* secondary result */
+};
+
+#if defined(__LITTLE_ENDIAN)
+#define PADDED(x,y)	x, y
+#elif defined(__BIG_ENDIAN)
+#define PADDED(x,y)	y, x
+#else
+#error edit for your odd byteorder.
+#endif
+
+struct aio_ring {
+	__u32	PADDED(id,   pad1);	/* kernel internal index number */
+	__u32	PADDED(mask, pad2);	/* number of io_events - 1 */
+	__u32	PADDED(head, pad3);
+	__u32	PADDED(tail, pad4);
+
+	__u32	PADDED(woke, pad5);	/* set when a wakeup was sent */
+
+	__u32	pad6[22];		/* pad out to 128 bytes */
+
+	struct io_event		io_events[0];
+}; /* 128 bytes + ring size */
+
+#define aio_ring_avail(ring)	(((ring)->head + (ring)->mask - (ring)->tail) & (ring)->mask)
+
+/*
+ * we always use a 64bit off_t when communicating
+ * with userland.  its up to libraries to do the
+ * proper padding and aio_error abstraction
+ */
+
+struct iocb {
+	/* these are internal to the kernel/libc. */
+	__u64	aio_data;	/* data to be returned in event's data */
+	__u32	PADDED(aio_key, aio_reserved1);
+				/* the kernel sets aio_key to the req # */
+
+	/* common fields */
+	__u16	aio_lio_opcode;	/* see IOCB_CMD_ above */
+	__s16	aio_reqprio;
+	__u32	aio_fildes;
+
+	__u64	aio_buf;
+	__u64	aio_nbytes;
+	__s64	aio_offset;
+
+	/* extra parameters */
+	__u64	aio_reserved2;
+	__u64	aio_reserved3;
+}; /* 64 bytes */
+
+#undef IFBIG
+#undef IFLITTLE
+
+#endif /* __LINUX__AIO_ABI_H */
+
diff -urN linux.05/include/linux/sched.h linux.10/include/linux/sched.h
--- linux.05/include/linux/sched.h	Fri Jan 18 23:18:40 2002
+++ linux.10/include/linux/sched.h	Fri Jan 18 23:19:14 2002
@@ -206,6 +206,7 @@
 /* Number of map areas at which the AVL tree is activated. This is arbitrary. */
 #define AVL_MIN_MAP_COUNT	32
 
+struct kioctx;
 struct mm_struct {
 	struct vm_area_struct * mmap;		/* list of VMAs */
 	struct vm_area_struct * mmap_avl;	/* tree of VMAs */
@@ -234,6 +235,9 @@
 
 	/* Architecture-specific MM context */
 	mm_context_t context;
+
+	struct kioctx	*ioctx_list;
+	unsigned long	new_ioctx_id;
 };
 
 extern int mmlist_nr;
diff -urN linux.05/mm/mmap.c linux.10/mm/mmap.c
--- linux.05/mm/mmap.c	Fri Jan 18 23:18:40 2002
+++ linux.10/mm/mmap.c	Fri Jan 18 23:19:14 2002
@@ -926,6 +926,7 @@
 {
 	struct vm_area_struct * mpnt;
 
+	exit_aio(mm);
 	release_segments(mm);
 	spin_lock(&mm->page_table_lock);
 	mpnt = mm->mmap;
