Design Notes on Asynchronous I/O (aio) for Linux ----------------------------------------------- Date: 1/30/2002 - Based on Benjamin LaHaise's in kernel aio patches http://www.kernel.org/pub/linux/kernel/people/bcrl/aio http://www.kvack.org/~blah/aio http://people.redhat.com/bcrl/aio (Last version up at the point of writing this: aio-0.3.7) - Refers to some other earlier aio implementations, interfaces on some other operating systems, and DAFS specifications for context/comparison. - Includes several inputs and/or review comments from Ben LaHaise (overall) Andi Kleen (experiences from an alternative aio design) Al Viro (i/o multiplexor namespace) John Myers (concurrency control, prioritized event delivery) but any errors/inaccuracies are mine, and feedback or further inputs are more than welcome. Regards Suparna Bhattacharya (suparna@in.ibm.com) Linux Technology Centre IBM Software Labs, India Contents: ---------- 1. Motivation 1.1 Where aio could be used 1.2 Things that aio helps with 1.3 Alternatives to aio 2. Design Philosophy and Interface Design 2.1 System and interface design philosophy 2.2 Approaches for implementing aio 2.3 Extent of true async behaviour - queue depth/throttle points 2.4 Sequencing of aio operations 2.5 Completion/readiness notification 2.6 Wakeup policies for event notification 2.7 Other goals 3. Interfaces 3.1 Interfaces provided by this implementation 3.2 Extending the interfaces 4. Design Internals 4.1 Low level primitives 4.2 Generic async event handling pieces 4.3 In-kernel interfaces 4.4 Async poll 4.5 Raw disk aio 4.6 Filesystem/buffered aio 4.7 [Placeholder for network aio] 4.8 Extending aio to other operations (todo) 5. Placeholder for performance characteristics 6. Todo Items/Pending Issues ------------------------------------------------------------------ 1. Motivation Asynchronous i/o overlaps application processing with i/o operations for improved utilization of CPU and devices, and improved application performance, in a dynamic/adaptive manner, especially under high loads involving large numbers of i/o operations. 1.1 Where aio could be used: Application performance and scalable connection management: (a) Communications aio: Web Servers, Proxy servers, LDAP servers, X-server (b) Disk/File aio: Databases, I/O intensive applications (c) Combination Streaming content servers (video/audio/web/ftp) (transfering/serving data/files directly between disk and network) Note: The POSIX spec has examples of using aio in a journalization model, a data acquisition model and in supercomputing applications. It mentions that supercomputing and database architectures may often have specialized h/w that can provide true asynchrony underlying the logical aio interface. Aio enables an application to keep a device busy (e.g. raw i/o), potentially improving throughput. While maximum gains are likely to be for unbuffered i/o case, aio should be supported by all types of files and devices in the same standard manner. Besides, being able to do things like hands off zero-copy async sendfile can be quite useful for streaming content servers. 1.2 Things that aio helps with: - Ability for a thread to initiate operations or trigger actions without having to wait for them to complete. - Ability to queue up batches of operations and later issue a single wait to wait for completion of any of operations or at least a certain number of operations (Note: Currently its only "at least one" that's supported). - Multiplexing large no of connections or input sources in a scalable manner typically into an event driven service model. [This can significantly reduce the cost of idle connections, which could be important in protocols like IMAP or IRC for example where connections may be idle for most of the time] - Flexible/dynamic concurrency control tuning and load balancing. - Performance implications (a) Application thread gets to utilize its CPU time better (b) Avoids overhead of extra threads (8KB per kernel thread in linux) (c) System throughput helped by reducing context switches (since wait causes less than time-slice runs) - Ability to perform true zero-copy network i/o on arbitrary user buffers Currently sendfile or an in-kernel server is the only clean way to use the zero-copy networking features of 2.4. The async i/o api would enable extending this to arbitary user buffers. [Note: Standard O_NONBLOCK doesn't help as the API doesn't take the buffer away from the user. As a result the kernel can avoid a copy only if it MMU write protects the buffer and relies on COW to avoid overwrites while the buffer is in use. This would be rather expensive due to the TLB flushing requirements, especially as it involves IPIs on SMP. ] 1.2.1 Other expected features (aka POSIX): - Support for synchronous polling as well as asynchronous notification (signals/callbacks) of completion status, with ability to co-relate event(s) with the i/o request(s). [Note: Right now async notification is not performed by the core kernel aio implementation, but delivered via glibc userspace threads which wait for events and then signal the application. TBD: There are some suggestions for a direct signal delivery mechanism from the kernel for aio requests to avoid the pthreads overhead for some users of POSIX aio which use SIGEV_SIGNAL and do not link with the pthreads library. Possibly a SIGEV_EVENT opcode could be introduced to make the native API closer to a POSIX extension. ] - Allow multiple outstanding aio's to the same open instance and to multiple open instances (sequence might be affected by synchronized data integrity requirements or priorities) [Note: Currently there are firmer guarantees on ordering for sockets by the in-kernel aio, while for file/disk aio barrier operations may need to be added in the future] - Option to wait for notification of aio and non-aio events through a single interface [TBD: Ties in with Ben's recent idea of implementing userland wait queues ] - Support for cancellation of outstanding i/o requests [Note: Not implemented as yet but in plan (just done for aio_poll, others to follow). Cancellation can by its very nature only be best effort] - Specification of relative priorities of aio requests (optional) [Note: Not implemented as yet. Should be linked to the new priority based disk i/o scheduler when that happens] 1.2.2 Also Desirable: - Ability to drive certain sequences of related async operations/transfers in one shot from an application e.g. zero-copy async transfers across devices (zero-copy sendfile aio) 1.3 Alternatives to aio 1.Using more threads (has its costs) - static committed resource overhead per thread - potentially more context switches 2.Communications aio alternatives - /dev/*poll - specialized device node based interface for registration and notifications of events - suitable for readiness notification on sockets, but not for driving i/o. 3.Real-time signals - only a notification mechanism - requires fcntl (F_SETSIG) for edge triggered readiness notification enablement or aio interfaces (aio_sigevent settings: SIGEV_SIGNAL) for completion notification enablement through RT signals. - the mechanism has potential overflow issues (when signal queue limits are hit) where signals could get lost, especially with fasync route (which tends to generate a signal for every event rather than aggregate for an fd) and needs to be supplemented with some other form of polling over the sigtimedwait interface. The only way to tune the queue lengths is via sysctl. - relatively heavy when it comes to large numbers of events (btw, signal delivery with signal handlers is costly and not very suitable for this purpose because of the complications of locking against them in user space; so the sigtimedwait sort of interface is preferable) - there are some other problems with flexibility in setting the receipient of the signal via F_SETOWN (per process queues) which hinders concurrency. [Question to Ponder: More efficient implementation and extensions to RT signal interfaces, or have a different interface altogether ? ] Please refer to www.kegel.com/c10k.html for a far more detailed coverage of these mechanisms, and how they can be used by applications. Reasons for prefering aio: - Desirable to have a unified approach, rather than multiple isolated mechanisms if it can be done efficiently - Multiplexing across different kinds of operations and sources - Clear cut well-known system call interface preferable to more indirect interfaces - Driving optimizations from low level/core primitives can be more efficient and beneficial across multiple subsystems - Can separate the core event completion queue and notification mechanisms for flexiblity and efficiency. (Can have tunable wakeup semantics, tunable queue lengths, more efficient event ring buffer implementation) Note: There are synchronization concerns when the two are not unified from a caller's perpective though, so the interfaces need to be designed with that in mind. 2. Design Philosophy and Interface Design 2.1 System and Interface design philosophy: Alternatives: a. Entire system built on an asynchronous model, all the way through (e.g NT i/o subsystem). So most operations can be invoked in sync or async mode (sub-options of the same operation specific interface). Internally, the sync mode = async mode + wait for completion. b. Async operations are initiated through a separate interface, and could follow a separate path from the synchronous operations, to a degree (use common code, and low down things may be truly async and common for for both, but at the higher level the paths could be different) The POSIX aio interface is aligned with (b). This is the approach that the Linux implementation takes. Submission of all async i/o ops happens through a single call with different command options, and data used for representing different operations. Advantages: - No change in existing sync interfaces (can't afford to do that anyway) - Less impact on existing sync i/o path. This code does not have the overhead of maintaing async state (can use the stack), and can stay simple. Disadvantages: - Need to introduce interfaces or cmd structures for each operation that can be async. (A little akin to an ioctl style approach) - Different code paths implies some amount of duplication/maintenance concerns. Can be minimized by using as much common code as possible. 2.2 Approaches for implementing aio 2.2.1 Alternative ways of driving the operation to completion 1. Using threads to make things _look_ async to the application a. User level threads - glibc approach (one user thread per operation ?) poor scalability, performance b. Pool of threads - have a pool of threads servicing an aio request queue for the task - tradeof between degree of concurrency/utilization and resource consumption. 2. Hybrid approach (SGI kaio uses this) - If the underlying operation is async in nature, initiate it right away (better utilization of underlying device), and just handle waiting for completion via thread pool (could become a serialization point depending on load and number of threads) unless operation completes rightaway in a non-blocking manner. - If underlying operation is sync, then initiate it via the thread pool Note: - SGI kaio has internal async i/o initiation interfaces for raw i/o and generic read. - SGI kaio has these slave threads in the context of the aio task => at least one per task - SGI kaio slave threads perform a blocking wait for the operation just dequeued to complete before checking for completion of the next operation in the queue => number slave threads determines the degree of asynchrony. 3. Implement a true async state machine for each type of aio operation. (i.e a sequence of non-blocking steps, continuation driven by IRQ and event threads, based on low level primitives designed for this purpose) - Relatively harder to get right, and harder to debug, but provides more flexibility, and greater asynchrony This aio implementation takes approach 3 (with some caveats, as we shall see later). Andi Kleen had experimented with a new raw i/o device which would be truly async from the application's perspective until it had to block waiting for request queue slots. Instead of using a thread for completion as in approach 3 above, it sent RT signals to the application to signal i/o completion. The experience indicated that RT signals didn't seem to very suitable and synchronization was rather complicated. There also were problems with flow control with the elevator (application blocking on request queue slots and plugging issues). A paper from Univ of Wisconsin-Madison talks about a block async i/o implementation called BAIO. This scheme uses one slave thread per task similar to the SGI kaio approach, but in this case the BAIO service thread checks for completion in a non-blocking manner (it gets notified of i/o completion by the device driver) and in turn notifies the application. BAIO does not have to deal with synchronous underlying operations (doesn't access filesystems, as it only intends to expose a low level disk access mechanism enabling customized user level filesystems), and hence its async state machine is simple. 2.2.1.1 Optimization/Fast-path for non-blocking case In case an operation can complete in a non-blocking manner via the normal path, the additional async state path can be avoided. An F_ATOMIC flag check has been introduced down the sync i/o path to check for this, thus providing a fast path for aio. This idea comes from TUX. 2.2.2 Handling User Space Data Tranfer With asynchronous i/o, steps of the operation aren't guaranteed to execute in the caller's context. Hence transfers/copies to/from user space need to be handled carefully. Most of this discussion is relevant for buffered i/o, since direct i/o avoids user/kernel space data copies. In a thread pool approach, if a per-task thread pool is used, then such transfers can happen in the context of one of these threads. Typically the copy_to_user operations required to read transfered data into user space buffers after i/o completion would be handled by these aio threads. Both SGI kaio and BAIO rely on per-task service threads for this purpose. It may be possible to pass down all the user space data for the operation when initiating i/o while in the caller's context without blocking, though this is inherently likely to use extra kernel space memory. The same is true on the way up on i/o completion, where it may be possible to continue holding on to the in-kernel buffers until the caller actually gathers completion data, so that copy into user space can happen in the caller's context. However this again holds up additional memory resources which may not be suitable especially for large data transfers. [BTW, on windows NT, iirc this sort of stuff happens through APCs or asynchronous procedure calls, in a very crude sense somewhat like softirqs running in the context of a specified task] Instead, an approach similar to that taken with direct i/o has been adopted, where the user space buffers are represented in terms of physical memory descriptors (a list of tuples of the form ), called kvecs, rather than by virtual address, so that they are uniformly accessible in any process context. This required new in-kernel *kvec* interfaces which operate on this form of i/o currency or memory descriptors. Each entry/tuple in the kvec is called a kveclet, and represents a contiguous area of physical memory. A virtual address range or iovec (in the case readv/writev) would map to a set of such tuples which makes up a kvec. Note: ----- This fits in very nicely with the current multi-page bio implementation which also uses a similar vector representation, and also with the zero-copy network code implementation. Ben has submiited some patches to make this all a common data structure. TBD: Some simple changes are needed in the multi-page bio code to get this to work properly without requiring a copy of the descriptors. There is a discussion on various alternative representations that have been considered in the past in sec 1.2.2 of: http://lse.sourceforge.net/io/bionotes.txt The only possible drawback is that this approach does keep the user pages pinned in memory all the while until the i/o completes. However, it neatly avoids the per-task service thread requirement of other aio implementations. 2.3 Extent of true async behaviour - Queue depth/Throttle points There has been some discussion about the extent to which asynchronous behaviour should be supported in case the operation has to wait for some resource to become available (typically memory, or request queue slots). There obviously has to be some kind of throttling of requests by the system beyond which it cannot take in any more asynchronous io for processing. In such cases, it should return an error (as it does for non-blocking i/o) indicating temporary resource unavailability (-EAGAIN), rather than block waiting for resource (or could there be value in the the latter option ?). It seems appropriate for these bounds to be determined by the aio queue depth and associated resource limits, rather than by other system resources (though the allowable queue depth could be related to general resource availability). This would mean that ideally, when one initiates an async i/o operation, the operation gets queued without blocking anywhere, or returns an error in the event it hits the aio resource limits. [Note/TBD: This is the intended direction, but this aspect of the code is still under construction and is not complete. Currently async raw aio would probably block if it needs to wait for request queue slots. Async file i/o attempts to avoid blocking the app due to sub i/os for bmap kind of operations but it currently could block waiting for the inode semaphore. The long term direction is to convert this wait to an async state driven mechanism. The async state machine also has to be extended to the waits for bmap operations which has so far only been pushed out of the app's context to that of the event thread that drives the next step of the state machine (which means that it could block keventd temporarily).] 2.4 Sequencing of aio operations Specifying serialization restrictions or relative priorities: - posix_synchronized_io (for multiple requests to the same fd) says that reads should see data written by requests preceding it - enforces ordering to that extent, if specified. - aio_req_prio (not supported in the current implementation as yet) app can indicate some requests are lower priority than others, so the system can optimize system throughput and latency of other requests at the cost latency of such requests. Some Notes: - This feature should get linked to the priority based i/o scheduler when that goes in, in order to make sure that the i/os really get scheduled as per the priorities. - The priority of a request is specified relative to (and is lower than) the process priority, so it can't starve other process's requests etc when passed down to to the i/o scheduler for example. Besides the i/o scheduler would also have some kind of aging scheme of its own, or translate priorities to deadlines or latency estimates to handle things fairly. - [TBD: Priorities typically indicate hints or expectations unlike i/o barriers or synchronized i/o reqmts for strict ordering (except possibly for real time applications ?) ] - Posix, says that the same priority requests to a character device should be handled fifo. - As John Myers suggested, considering priorities on the event delivery path in itself may be useful even without control on i/o scheduling. This aspect could possibly be implemented early, since it would be needed in any case in the complete implementation to make sure that priorities are respected all the way through from initiation to completion processing. (See point 5 on prioritized event delivery under Sec 2.5) - To account for priorities at the intermediate steps in the async state machine, multiple priority task queues could be used instead of a single task queue to drive the steps. Beyond these restrictions and hints, sequencing is up to the system, with dual goals: - Maximize throughput (global decision) - Minimize latency (local, for a request) There are inherent tradeoffs between the above, though improving system throughput could help with average latency, provided pipeline startup time isn't significant. A balanced objective could be to maximize throughput within reasonable latency bounds. Since each operation may involve several steps which could potentially run into temporary resource contention or availability delay points, the sequence in which operations complete, or even reach the target device are affected by system scheduling decisions in terms of resource acquisition at each of these stages. Note/TBD: Since the current implementation uses event threads to drive stages of the async state machine, in situations where a sub-step isn't completely non-blocking (as desired), then the implementation ends up causing some degree of serialization, or rather further accentuating the order in which the requests reached the sub-step. This may be seem reasonable and possibly even beneficial for operations that are likely to contend for the same resources (e.g requests to the same device), but not optimal for requests that can proceeed in a relatively independent fashion. The eventual objective is to make sure that sub-steps are indeed non-blocking, and there is a plan to introduce some debugging aids to help enforce this. As discussed in Section 2.3, things like bmap, wait for request, and inode semaphore acquisition are still to be converted to non-blocking steps (currently a todo). 2.5 Completion/Readiness notification: Comment: Readiness notification can be treated as a completion of an asynchonous operation to await readiness. POSIX aio provides for waiting for completion of a particular request, or for an array of requests, either by means of polling, or asynchronously through signals. On some operating systems, there is a notion of an I/O Completion port (IOCP), which provides a flexible and scalable way of grouping completion events. One can associate multiple file descriptors with such a completion port, so that all completion events for requests on those files are sent to the completion port. The application can thus issue a wait on the completion port in order to get notified of any completion event for that group. The level of concurrency can be increased simply by increasing the number of threads waiting on the completion port. There are also certain additional concurrency control features that can be associated with IOCPs (as on NT), where the system decides how many threads to wakeup when completion events occur, depending on the concurrency limits set for the queue, and the actual number of runnable threads at that moment. Keeping the number of runnable threads constant in this manner protects against blocking due to page faults and other operations that cannot be performed asynchronously. On a similar note, the DAFS api spec incorportes completion groups for handling async i/o completion, the design being motivated by VI completion queues, NT IOCPs and the Solaris aiowait interfaces. Association of an i/o with a completion group (NULL would imply the default completion queue) happens at the time of i/o submission which lets the provider know where to place the event when it completes, contrary to aio_suspend style interface which specifies the grouping only when waiting on completion. This implementation for Linux makes use a similar notion to provide support for completion queues. There are api's to setup and destroy such completion queues, specifying the maximum queue lengths that a queue is configured for. Every asynchronous i/o request is associated with a completion queue when it is submitted (like the DAFS interfaces), and an application can issue a wait on a given queue to be notified of a completion event for any request associated with that queue. BSD kqueue (Jonathan Lemon) provides a very generic method for registering for and handling notification of events or conditions based on the concept of filters of different types. This covers a wide range of conditions including file/socket readiness notification (as in poll), directory/file (vnode) change notifications, process create/exit/stop notifications, signal notification, timer notification and also aio completion notification (via SIGEV_EVENT). The kqueue is equivalent to a completion queue, and the interface allows one to both register for events and wait for (and pick up) any events on the queue within the same call. It is rather flexible in terms of providing for various kinds of event registration/notification requirements, e.g oneshot or everytime, temporary disabling, clearing state if transitions need to be notifiied, and it supports both edge and level triggered types of filters. 2.5.1 Some Requirements which are addressed: 1. Efficient for large numbers of events and connections - The interface to register events to wait for should be separate from the interface used to actually poll/wait for the registered events to complete (unlike traditional poll/select), so that registrations can hold across multiple poll waits with minimum user-kernel transfers. (It is better to handle this at interface definition level than through some kind of an internal poll cache) The i/o submission routine takes a completion queue as a parameter, which associates/registers the events with a given completion group/queue. The application can issue multiple waits on the completion queue using a separate interface. - Ability to reap many events together (unlike current sigtimedwait and sigwaitinfo interfaces) The interface used to wait for and retrieve events, can return an array of completed events rather than just a single event. - Scalable/tunable queue limits - at least have a limit per queue rather than system wide limits Queue limits can be specified when creating a completion group. TBD: A control interface for changing queue parameters/limits (e.g io_queue_grow) might be useful - Room for more flexible/tunable wakeup semantics for better concurrency control Since the core event queue can be separated from the notification mechanism, the design allows one to provide for alternative wakeup semantics to optimize concurrency and reduce redundant or under-utilized context switches. Implementing these might require some additional parameters or interfaces to be defined. BTW, it is desirable to provide a unified interface for notification and event retrieval to a caller, to avoid synchronization complexities, even if the core policies are separable underneath in-kernel. [See the discussion in Sec 2.6 on wakeup policies for a more detailed discussion on this] 2. Enable flexible grouping of operations - Flexible grouping at the time of i/o submission (different operations on the same fd can belong to different groups, operations on different fds can belong to the same group) - Ability to wait for at least a specified number of operations from a specified group to complete (at least N vs at least 1 helps with batching on the way up, so that the application can perform its post processing activities in a batch, without redundant context switches) The DAFS api supports such a notion, both in its cg_batch_wait interface which returns when either N events have completed, or with less than N events in case of a timeout, and also in the form of a num_completions hint at the time of i/o submission. The latter is a hint that gets sent out to the server as a characteristic of the completion queue or session, so the server can use this hint to batch its responses accordingly. Knowing that the caller is interested only in batch completions helps with appropriate optimizations. Note: The Linux aio implementation today only supports "at least one" and not "at least N" (e.g the aio_nwait interface on AIX). The tradeoffs between responsiveness and fairness issues tend to to get amplified when considering "at least N" type of semantics, and this is one of the main concerns in supporting it. [See discussion on wakeup policies later] - Support dynamic additions to the group rather than a static or one time list passed through a single call Multiple i/o submissions can specify the same completion group, enabling events to be added to the group. [Question: Is the option of the completion group being different from the submission batch/group (i.e. per iocb grouping field) useful to have ? POSIX allows this] 3. Should also be able to wait for a specific operation to complete (without being very inefficient about it) One could either have low overhead group setup/teardown so such an operation may be assigned a group of its own (costs can be amortized across multiple such operations by reusing the same group if possible) or provide an interface to wait for a specific operation to complete. The latter would be more useful, though it requires a per-request wait queue or something similar. The current implementation has a syscall interface defined for this (io_wait), which hasn't been coded up as yet. The plan is to use hashed wait queues to conserve on space. There are also some semantics issues in terms of possibilities of another waiter on the queue picking up the corresponding completion event for this operation. To address this, the io_wait interface might be modified to include an argument for the returned event. BTW, there is an option of dealing with this using the group primitives either in user space, or even in kernel by waiting in a loop for any event in the group until the desired event occurs, but this could involve some extra interim wakeups / context switches under the covers, and a user level event distribution mechanism for the other events picked up in the meantime. 4. Enable Flexible distribution of responsibility across multiple threads/components Different threads can handle submission for different operations, and another pool of threads could wait on completion. The degree of concurrency can be improved simply by increasing threads in the pool that wait for and process completion of operations for that group. 5. Support for Prioritized Event Delivery This involves the basic infrastructure to be able to accord higher priority to the delivery of certain completion events over others, (e.g. depending on the request priority settings of the corresponding request), i.e. if multiple completion events have arrived on the queue, then the events for higher priorities should be picked up first by the application. TBD/Todo: One way of implementing this would be to have separate queues for different priorities and attempt to build an aggregate (virtual) queue. There are some design issues to be considered here as in any scheduling logic, and this needs to be looked at in totality in conjunction with some of the other requirements. For example, things like aging of events on the queue, could get a little complex to do. One of the approaches under consideration is to try to handle the interpretation of priorities in userspace, leaving some such decisions to the application. It is the application which decides the limits for each of the queues, so the kernel avoids having to handle that or balance space across the queues. Only kernel support for making a multiplexed wait on a group of completion queues possible might suffice to get this to work. Ben has in a mind a rather generic way of doing this (across not just completion queues, but also possibly across other sorts of waits) by providing primitives that expose the richness of the kernel's wait queue interfaces directly to userspace. The idea is that something like the following would become possible: user_wait_queue_t wait; int ret; add_wait_queue(high_pri_ctx, wait) add_wait_queue(low_pri_ctx, wait) ret = process_wait(); /* call it schedule() if you want */ while (vsys_getevents(high_pri_ctx...) > 0) ... ... ie, a very similar interface to what the kernel uses which can be mixed and matched across the different kinds of things that need to be waited upon (locks, io completion, etc). Such a mechanism can also be used for building the more complex locks that glibc needs to provide efficiently without sacrificing a rich and simple interface. Notice, that for true aio_req_prio, the kernel would have to be aware of completion queue priorities, but that it may still be possible for the order in which events are picked up (across the queues) to be handled by the application. BTW, another possibility is to maintain a userland queue (or set of queues, for each priority), into which events get drawn in whenever events are requested and then later distributed/picked up by the application's threads. One of the tricky issues with such multi-level queues is handling flow control, which is not very appealing. (Interestingly Viro's suggested interface (3.2) also deals with composite queues. Just one level of aggregation suffices for the prioritized delivery requirements, while Viro's interface supports multiple levels of aggregation. ) 2.6 Wakeup Policies for Event Notification 2.6.1 The wakeup policy used in this implementation The design is geared towards minimizing the latency of completion processing which directly related to the responsiveness of the (server) application to events. Ensuring fairness (or even starvation avoidance) is not expected to be an issue with the expected application model of symmetric server threads (i.e. threads which take the same actions on completion of given events), except in so far as it affects load balancing which in turn could affect latency. [TBD: I'm not sure of this, but starvation may be an issue when it comes to non-symmetric threads, where the event is a readiness indicator which the thread uses to decide on availability of space in order to push its data or something of that sort.] The wakeup policy in the current implementation is to wakeup a thread on the completion queue whenever an i/o completes. Any thread who picks up the event first (this could even be a new caller who wasn't already waiting on the queue) gets it, and if no events are available for a thread to pick up, it goes back to sleep again. By ensuring that the thread who gets to the event earlist picks it up, this keeps the latency minimal. Also in view of better cache utilization the wake queue mechanism is LIFO by default. (A new exclusive LIFO wakeup option has been introduced for this purpose) Making the wakeups exclusive reduces some contention or spurious wakeups. When events are not coming in rapidly enough for all the threads to get enough events that they can use up their full time slice processing, there is a likelihood of some contention and redundant or rather underutilized context switches. (While it just might happen that a thread is gets deprived of events as other threads keep picking them up instead, as discussed, that may not be significant, and probably just an indicator that the number of threads could be reduced. ) In the situation when there are a lot of events in the queue, then every thread tries to pick up as many events as it can (upto the number specified by the caller), but one at a time. The latter aspect (of holding the lock across the acquisition of only a single event at a time) helps with some amount of load balancing (for event distribution, or completion work) on SMP when these threads are running parallely multiple CPUs. 2.6.2 TBD: Note on at least N semantics: In some situations where an application is interested in batch results and overall throughput (vs responsiveness to individual events), an "at least N" kind of wakeup semantics, vs "at least one" can help amortize the cost of a wakeup/context switch across multiple completions. (This is better than just a time based sleep which doesn't have any co-relation with the i/o or event completion rates - one could have too many events building up or perhaps too little depending on the load). This makes sense when the amount of post-processing on receipt of an event is very small and the resulting latency is tolerable (combination of timeout + N lets one specify the bounds), so the application would rather receive notifications in batches. Things get a little tricky when trying to define the policies for "at least N" when multiple threads are involved, possibly with different values of N (though that is not a typical situation), in terms of event distribution, simply because the tradeoffs between latency and fairness tend to widen in this case. A natural extension of the current scheme to an at least N scheme, would be to wake up only waiters whose "N-value" matches or exceeds the number of events available, and then have them try to pick up their N events in one shot (i.e. atomically) if available or go back to sleep. If a thread finds more events available after it picks up its N events, or after it times out, then just as before it keeps picking up as many events as it can (upto the specified limit) but one at time. This helps reduce the load balancing vs batching conflict (the policy is batch upto n, balance beyond that). [TBD: Implementing the "check for N and wakeup" scheme above correctly in the presence of exclusive waits may require support in the wait queue wakeup logic to account for the status returned by a wait queue function to decide if the entry should be treated as done/woken up. The approach would be that the earliest waiter whose conditions are satisfied would get woken up] Obviously the possibility of starvation is relatively more glaring in this case, than with at-least-one, e.g. consider the case when 2N-1 events are just picked up by one thread, while the other thread is idle, and the 2Nth event comes in just then. As mentioned earlier, starvation is not an issue in itself, but the load balancing implication is worth keeping in mind. The maximum number of events requested in one shot and the timeout provide the bounds on this sort of a thing from an application's perspective. (BTW, The DAFS cg_batch_wait interface is "exactly N", which is one other way of handling this; actually it is exactly N or less on a timeout) Notice that trying to implement at-least-N semantics purely in user space above at-least-one primitives with multiple waits has latency issues in the multiple waiters case (besides the extra wakeups/context switches). In the worst case, with m threads, the latency for actual completion processing (where completion processing happens in batches of N events), could be delayed upto the arrival of the m*N-1 th event. Remark: "At least N" is still a TBD. 2.6.3 TBD: Load/Queue Length based wakeup semantics: This is another option, from a networking analogy, where the system could tune the N-value for wakeup on a queue, based on event rates or space available to queue more requests. This is however based on the expectation that completion processing would trigger a fresh batch of aio requests on the queue. Note: Being able to wait on a specific aio, or a submit and wait for all the submitted events to complete (the way it is supported in BSD kqueues) are other interfaces that could potentially reduce the number of context switches, and are useful in some situations (no implemented as yet). 2.6.3 TBD/Future: Per Completion Queue Concurrency Control There have been some thoughts about achieving IOCP concurrency control via associated scheduling group definitions, independently of aio completion queue semantics, so an application could possibly choose to use both aio and scheduling groups together. This might make sense because the system has no persistent association of the completion queue with threads that aren't waiting on that queue. Implicit grouping (e.g association with of a thread with the last ctx it invoked io_getevents on) is possible, but does make some assumptions (even if these might reflect the most typical cases) on the way the application threads handle completions and IOCP waits. On the other hand as John Myers indicated, a pure scheduling group feature that only looks at wakeups, without knowledge of the reason for the wakeup (the ability to distinguish between more events/work coming in which can be handled by any one from the set of threads, or indicating completion of synchronous actions meant for a specific thread) may not be able to take the kind of more informed decisions that a more tightly coupled feature or abstraction that operates at a slightly higher level can. One way to solve this would be for the scheduling group implementation (if and when it is implemented) to also allow for (in-kernel) priority indicators for waiters (or the wait queues, whichever seems appropriate), so that it can handle such decisions. Components like aio could take care of setting up such priorities as it sees fit (e.g accord lower priorities for the completion wait queue waits) to cause the desired behaviour. 2.7 Other Goals - Support POSIX as well as completion port style interfaces/wrappers The base kernel interfaces are deisgned to provide the minimum native support required for the library to implement both styles of interfaces, - Low overhead for kernel and user [Potential todos: Possibly via an mmaped ring buffer, vsyscalls] - Extensible to newer operations e.g. aio_sendfile, aio_readv/writev + anything else that seems useful in the future (semaphores, notifications etc ?) 3. Interfaces 3.1 The interfaces provided by this implementation The interfaces are based on a new set of system calls for aio: - Create/Setup a new completion context/queue. This completion context can only be shared across tasks that share the same mm (i.e. threads). __io_setup(int maxevents, io_context_t *ctxp) - Submit an aio operation. The iocb describes the kind of operation and the associated paramters. The completion queue to associate the operations with is specified too. __io_submit(io_context_t ctx, int nr, struct iocb **iocbs) - Retrieve completion events for operations associated with the completion queue. If no events are present, then wait for upto the timeout for at least one event to arrive. __io_getevents(io_context_t ctx, int nr, struct io_event *events, struct timespace *timeout) - Wait upto the timeout for the i/o described by the specific iocb to complete. [Ques: Should this interface be modified to retreive the event as well ?] --io_wait(io_context_t ctx, struct iocb *iocb, struct timespec *timeout) - Cancel the operation described by the specified iocb. __io_cancel(io_context_t ctx, void *iocb) - Teardown/Destroy a new completion context/queue (happens by default upon process/mm exit) Pending requests would be cancelled if possible, and the resources would get cleaned up when all in-flight requests get completed/cancelled. Naturally any unclaimed events would automatically be lost. __io_destroy(io_context_t ctx) The library interface that a user sees is built on top of the above system calls. It also provides a mechanism to associate callback routines with the iocb's which are invoked in user space as part of an event processing loop when the corresponding event arrives. There are helper routines (io_prep_read, io_prep_write, io_prep_poll, etc) which can be used for filling up an iocb structure for a given operation, before passing it down through io_submit. Please refer to the aio man pages for details on the interfaces and how to use them [Todo: Reference to man pages fron Ben] POSIX aio is implemented in user space library code over these basic system calls. This involves some amount of book-keeping and extra threads to handle some of the notification work (e.g. SIGEV_NOTIFY is handled by sending the notification signals to the kernel from the user space thread). (Note: The plan is to add support for direct signal delivery from the kernel for aio requests in which case this dependence on pthreads would change) 3.2 Extending the Interfaces Alternatives to using system calls for some of the aio interfaces, particularly the event polling/retreival pieces include implementing a pseudo device driver interface (like the /dev/poll and /dev/epoll approaches), or a pseudo file system interface. A system call approach appears to be a more direct and clear-cut interface than any specialized device driver ioctl or read/write operations approach, which was one of the reasons why the possibility of a /dev/aio was abandoned during aio development. TBD/Future: The filesystem namespace based approach that Al Viro has suggested for i/o multiplexors (for flexible and scalable event polling/notification), provides for some interesting features over aio completion queues like naming, sharing (across processes rather than just threads), access control, persistence, and hierarchial grouping (i.e. more than just a single level of grouping). The model uses AF_UNIX socket sendmsg/recvmsg calls with specific datagram formats (SCM_RIGHTS datagrams) on the namespace objects instead of any new apis for registration and polling of events. The interface is defined so that recvmsg gets a set of new open descriptors for each of the underlying channels with events. This makes it feasible to share event registrations across processes, since the fd used to register the event needn't be available when the event is picked up. However, it still would make sense to have a separate mechanism for async i/o and associated notifications. Possibly if something like the above is implemented, one could consider ways of associating aio completion queues with it, if that fits sematically, or move things like async poll out of aio in there. Most of the aio operations (other than async poll today, and possibly aio_sendfile later) involve user space buffers, so sharing across processes may not make much sense, except perhaps in the case of shared memory buffers. 4. Design Internals 4.1 Low Level Primitives : 4.1.1 wait_queue functions This primitive is based on an extension to the existing wait queue scheme. The idea is that both asynchronous and synchronous waiters just use the same wait queue associated with any given data structure transparent to the caller of wakeup. (This avoids the need to attach new notify/fasync sort of structures for every relevant operation/data structure involved in the async state machine) To support asynchronous waiters, the wait queue entry structure now contains a function pointer for the callback to be invoked for async notification. The default action, in case such a callback is not specified, is to assume that the entry corresponds to a synchronous waiter (as before) and to wake it up accordingly. The callback runs with interrupts disabled and with the internal wait queue spinlock held, so the amount of work done in the callback is expected to be very restricted. Additional spinlocks should be avoided. The right thing to do if more processing is required is to queue up a work-to-do action to be run in the context of an event thread (see next section). Extreme caution is recommended in using wait queue callbacks as it is rather prone to races if not used with care. There is a routine to atomically check for a condition and add a wait queue entry if the condition is not met (add_wait_queue_cond). The check for the condition happens with the internal wait queue spin lock held. This avoids missing events between the check and addition to the wait queue, which could be fatal for the async state machine. The standard way of handling the possibility of missed events with synchronous waiters was to add the wait queue entry before performing the check for the condition, and to just silently remove the entry thereafter if the condition has already been met. However in the case of async waiters where the follow on action happens in the wait queue function, this could lead to duplicate event detection, which could be a problem if the follow on action is not defined to be idempotent. The add_wait_queue_cond() feature helps guard against this. [Note: An associated implication of this is that checks for wait_queue_active outside of the internal wait queue lock are no longer appropriate, as it could lead to a missed event] The wait queue callback should check for the occurance of the condition before taking action just as in typical condition wait/signal scenarios. Notice that the callback is responsible for pulling the entry off the wait queue once it has been successfully signalled, unlike the synchronous case where queueing and dequeueing happens in the same context. 4.1.2 Work-to-dos (wtd) for async state machine Work-to-dos provide the basic abstraction for representing actions for driving the async state machine through all the steps needed to complete an async i/o operation. The design of work-to-dos in this aio implementation is based on suggestions from Jeff Merkey for implementing such async state machines, and is modelled on the approach taken in Ingo Molnar's implementation of TUX. As mentioned in the previous section, because of restricted conditions under which wait queue functions are called, it isn't always possible to drive steps of the async state machine purely through wait queue functions. Instead the wait queue function in turn could queue a work-to-do action to be invoked in a more suitable context, typically by a system worker thread. This is achieved using the task-queue primitives on Linux. Currently aio just uses the same task queue which is serviced by keventd (i.e. the context task queue). In the future this could possibly be handled by a pool of dedicated aio system worker threads. [TBD: Also, priorities may be supported by having multiple task queues of different priority] struct wtd_stack { void (*fn)(void *data); /* action */ void *data; /* context data for the action */ }; struct worktodo { wait_queue_t wait; /* this gets linked to the wait queue for the event which is expected to trigger/schedule this wtd */ struct tq_struct tq; /* this gets linked to the task queue on which the wtd has to be scheduled (context_tq today) */ void *data; /* for use by the wtd_ primatives */ /* The stack of actions */ int sp; struct wtd_stack stack[3]; }; A typical pattern for the asynchronous version corresponding to a synchronous operation consisting a set of non-blocking steps with synchronous waits between steps, could be something like the following: (Lets label this pattern as A) - Initiate step 1, and register an async waiter or callback - Async waiter completes and queues a work-to-do for the next step - The work-to-do initiates step 2 when it gets serviced, and registers an async waiter or callback to catch completion - Async waiter/callback initiates step 3 ... .. and so on till step n. Of course, there are other possible patterns, e.g where the operation can be split off into multiple independent sub-steps which can be initiated at the same time, and then use callbacks/async waiters to collect/consolidate the results and if required queue a work-to-do action after that to drive the follow up action. (Lets label this pattern B) The work-to-do structure is designed so that state information can be passed along from one step to the next (unlike synchronous operations, state can't be carried over on stack in this case). There is also support for stacking actions within the same work-to-do structure. This feature has been used in the network aio implementation (which is currently under a revamp) to enable calling routines to stack their post completion actions (and associated data) before invoking a routine that might involve an async wait. For example, consider a nested construct of the form: func1() { func2(); post_process1(); } func2() { func3(); post_process2(); } func3() { process; wait for completion; post_process3(); } The asynchronous version of the above could have the following pattern, assuming that a worktodo structure is shared/passed on in some manner down the levels of nesting: - func1 initializes the worktodo with the action post_process1(), before calling func2 - func2 pushes the action post_process2() on the worktodo stack before calling func3 - func3 pushes the action post_process3() on the worktodo stack - func3 then replaces its synchronous wait by setting up an asynchronous waiter which would schedule the worktodo sequence - the worktodo sequence simply pops each action by turn and exceutes it to achieve the desired effect. (Lets label this pattern C) Some caution is needed when using the async waiter + work-to-do combination, e.g maintaining the 1-1 association with an event and the queueing of the worktodo, and guards against duplication or event misses (as discussed in the previous section). Also, one needs to be very careful about recursions in the chained operations (can't have stack overflows in the kernel). 4.2 Generic async event handling pieces 4.2.1 The completion queue The in kernel representation of the completion queue structure (kioctx), contains a list of in-use (active) and free requests (where each request in the in-kernel iocb representation, i.e. kiocb), and also a circular ring buffer, where completion events are queued up as they arrive, and picked up in FIFO order. There is a per kioctx wait queue which is used to wait for events on that queue. The reference count of a kioctx is incremented when it is in use (i.e. when there are pending requests). A completion queue is associated with the mm struct for the concerned task, thus threads which share the same address space also share completion queues. The ctx_id is unique per-mm. The completion queues for a given address space are linked together with the list grounded in the mm struct. On process exit (i.e. when the mm users count goes to zero), the completion queue is released (the actual free could happen a little later depending on the reference count, i.e. in case the kioctx is in use). The ring buffer is designed to be virtually contiguous, so if necessary (i.e. if the higher order page allocation needed to accomdate the specified number of events fails) it may be vmalloc'ed. The requests/kiocbs are also preallocated when the kioctx is created, but these needn't be contigous and are allocated from slab. 4.2.2 I/O Submission, Completion and Event Pickup New requests can be submitted only if there is enough space left in the ring buffer to accomodate completion events for all pending requests as well as the new one in the ring buffer. The io_submit interface invokes the corresponding async file op based on the operation code specified in the iocb. The file descriptor reference count is incremented to protect against the case the process exits and closes the file while i/o is still in progress. In such a scenario the request, file descriptor and the kioctx state are not freed immediately, but in a deferred manner as and when the completions (or cancellation possibly once that is supported) happen, and it is safe to do so. When the operation completes, the corresponding completion path (via async waiters or worktodos) invokes aio_complete to takes care of queuing the completion status/event at the end the ring buffer, waking up any threads that may be waiting for events on the queue, releasing the request and other related cleanups (e.g decrementing the file descriptor reference count). When the io_getevents interface is invoked for harvesting events, it picks up completion events available in the circular ring buffer (i.e. from the head of queue), or waits for events to come in, depending on the wakeup and event distribution policies discussed in Sec 2.6. 4.2.3 TBD: User space memory mapping of the Ring Buffer The design allows for the possibility of modifying the implementation to allow for the events ring buffer to be mapped in user space, if that helps with performance (avoiding some memory copies and system call overheads). The current implementation prepares for avoiding the complexities of user-kernel locking in such a case by making sure that only one side updates any field (basically head and tail of the ring buffer), and also banking on the assumption that reading an old value won't cause any real harm. The Kernel/Producer updates the tail and the User/Consumer updates the head. If the User sees an old value of tail, it may not see some just arrived events, which is similar to the case when the events haven't arrived, and so harmless. If the Kernel sees an old value of Head, then it may think there isn't enough space in the queue and will try again later. TBD: As Andi Kleen observed, schemes like this could be rather fragile and hard to change as past experience with such optimizations in the networking code have indicated where proper spin locks had to be added eventually. So we need to understand how significant a performance benefit is acheived by moving to a user space mapped ring buffer to decide if it is worth it. 4.3 In-kernel interfaces 4.3.1 Operations The in-kernel interfaces that were added for aio implementation include the following: - New filesystem operations (fops) to support asynchronous read/write functions (f_op->aio_read/write) - Several helper routines for manipulating and operating on kvecs, the common i/o currency discussed in Sec 2.2.2 (e.g. mapping user space buffers to kvecs, copying data to/from/across kvecs i.e. *kvec_dst* routines) - New filesystem read/write operations (fop->kvec_read/write) which operate directly on kvec_cb's in asynchronous mode. These are the operations that have to be defined for different file types, e.g raw aio, buffered filesystem aio and network aio. The f_op->read/write operations are expected to be changed to support an F_ATOMIC flag which can be used to service an aio operation synchronously if it can be done in a non-blocking manner.This provides a fast path that avoids some of the overheads of async state machine when the operation can complete without blocking/waiting at all. Currently, F_ATOMIC is implemented via f_op->new_read/write for compatibility reasons. [Todo: The plan is to add f_ops->flags_supported to enable read/write operations to be converted wholesale with requiring additional code to check for supported operations in all callees.] The generic f_op->aio_read/write operations, first attempt the non-blocking synchronous path described above, and take the async route only if it fails (i.e. returns an error indicating that the operation might block). In that case they convert the user virtual address range to a kvec and then invoke the appropriate async kvec fops. Notice that this mechanism should be extendable to readv/writev in a relatively lightweight manner (compared to kiobufs), though aio readv/writev is still a Todo right now. 4.3.2 The i/o Container Data structure, kvec_cb The i/o unit which is passed around to the kvec fops is the kvec_cb structure, This contains a pointer to the kvec array discussed earlier plus associated callback state (i.e. callback routine and data pointer) for i/o completion. struct kveclet { struct page *page; unsigned offset; unsigned length; }; struct kvec { unsigned max_nr; unsigned nr; struct kveclet veclet[0]; }; struct kvec_cb { struct kvec *vec; void (*fn)(void *data, struct kvec *vec, ssize_t res); void *data; }; struct kvec_cb_list { struct list_head list; struct kvec_cb cb; }; The callback routine would typically be set to invoke aio_complete for performing completion notification. For a compound operation like aio_sendfile, which involves two i/os (input on one fd and output to the other), the callback could be used for driving the next stage of processing, i.e. to initiate the second i/o. [TBD: With this framework, callback chaining is not inherently supported. Intermediate layers could save pointers to higher layer callbacks as part of their callback data, and thus implement chaining themselves, but a standard mechanism would be preferable. ] The *kvec_dst* helper routines which are used for retrieving or transfering data from/to kvecs are designed to accept as argument a context structure (kvec_dst) to maintain state related to the remaining portions to transfer. Since a kvec contains fragments of non-uniform size, locating the portion to transfer given the offset in number of bytes from the start of the kvec is not a single step calculation, so its more efficient to maintain this information as part of the context structure. These routines also take care of performing temporary kmaps of veclets for memory copy operations, as needed. The map_user_kvec() routine is used to map a user space buffer to a kvec structure (it allocates the required number of veclet entries). It also takes care of bringing the corresponding physical pages if they are swapped out. It increases the reference count of the page, essentially pinning it in memory for the duration of the io. (TBD/Check: Where does unmap_kvec happen ?) 4.4 Async poll Async poll enables applications to make use of the advantages of aio completion queues for readiness notification, avoiding some of the scalability limitations and quirks of traditional poll/select. Instead of passing in an array of pairs, one prepares iocbs corresponding to each such pair, and then submits these iocbs using io_submit associating them with a completion queue. Notifications can now be obtained by waiting for events on the completion queue. Unlike select/poll, one does not need to rebuild the event set for every iteration of the event loop; the application just has to resubmit iocbs for the events it has already reaped, in case it needs to include them in the set again for the next poll wait. The implementation is a simple extension of the existing poll/select code, which associates an iocb with a poll table structure and replaces the synchronous wait on a poll table entry by an asynchronous completion sequence (using a wait queue function + worktodo construct) that issues aio_complete for the corresponding iocb thus affecting the notification. 4.5 Raw-disk aio The internal async kvec f_ops for raw disk i/o are implemented along the lines of pattern B discussed in Sec 4.1.2. The common raw_rw_kvec routine invokes brw_kvec_async, which shoots out all the i/o pieces to the low level block layer, and sets up the block i/o completion callbacks to take care of invoking the kvec_cb callback when all the pieces are done. The kvec_cb callback takes care of issuing aio_complete for completion notification. TBD/Todo: There is one problem with the implementation today, in that if the submit_bh/bio operation used by brw_kvec_async blocks waiting for request queue slots to become free, then it blocks the caller, so the operation wouldn't be truly async in that case. Fixing this is one of items in the current Todo list. For example, instead of the synchronous request wait, a non-blocking option supplemented with an async waiter for request queue slots, which in turn drives the corresponding i/o once requests are available, using state machine steps along the lines employed for file i/o, could be considered. [Note/Todo: In the aio patches based off 2.4, brw_kvec_async sets up buffer heads and keeps track of the io_count and the list of bhs (in a brw_cb structure, which also embeds the kvec_cb structure) in order to determine when all the pieces are done. In 2.5, it would allocate a bio struct to represent the entire i/o unless the size exceeds the maximum request size allowed, in which case multiple bios may need to be allocated. The bio struct could be set up to directly point to the veclet list in the kvec, avoiding the need to copy/translate descriptors in the process] 4.6 File-system/buffered aio The generic file kvec f_ops (generic_file_kvec_read/write), for buffered i/o on filesystems employ a state machine that can be considered close to pattern A (with a mix of pattern B) discussed in Sec 4.1.2. The state information required through all the iterative steps of this state machine is maintained in an iodesc structure that is setup in the beginning, and passed along as context data for the worktodo actions. The operation first maps the page cache pages corresponding to the specified range. These would form the source/target of the i/o operation. It maintains a list of these pages, as well as the kvec information representing the user buffer from/to which the transfer has to happen, as part of the iodesc structure, together with pointers or state information describing how much of the transfer has completed (i/o to/from the page cache pages, and the memcopy to/from the user buffer veclets). In case of read, the post processing action for completion of i/o on a particular page would involve copying the data into the user space buffer, while for write, the copy from the user space buffer to the page happens early before committing the writeout of the page (i.e. between prepare_write and commit_write). Notice that the potential blocking points down the typical read/write path involve: (a) Waiting to acquire locks on the concerned pages (page cache pages corresponding to the range where i/o is requested) before starting i/o (b) Waiting for the io to complete: - for read, this involves waiting for the page locks again (indicative that the page lock has been released after i/o completion), and then checking if the page is now uptodate - for write (O_SYNC case), this involves waiting for the page buffers, i.e. waiting for the writeout to complete. [TBD: Currently its really only O_DSYNC, and not meta-data sync that's affected ] Each of these waits has been converted to an async wait (wtd_wait_on_page and wtd_wait_on_buffer) operation, that triggers the next step of the i/o (i.e as in pattern A). Notice that this becomes multi-step when the i/o involves multiple pages and any of lock acquisitions is expected to require a wait. Some speedup is achieved by initiating as much work as possible, e.g. initiating as many readpage operations as possibly early on the readpath, and initiating all the writeouts together down the write path before waiting for completion of any (this is where the resemblance to pattern B comes in). Currently the filesystems modified to support aio include ext2, ext3 and nfs (nfs kvec f_ops internally make use of the generic_file_kvec* operations, after calling nfs_revalidate_inode). [Note/Todo: There is still some work to do to make the steps non-blocking. The bmap/extent determination operations performed by the filesystem are blocking, and the acquisition of the inode semaphore also needs to be converted to a wtd based operation] 4.7 Network aio [Todo: To be added later since the code is under a rewrite - pattern C in 4.1.2 ? ] 4.8 Extending aio to other operations (e.g sendfile) [Todo/Plan: The idea here is to make use of the kvec callbacks to kick the operation into the next state, i.e. on completion of input from the source fd, trigger the i/o to the output fd. ] 5. Performance Characteristics [Todo: Research/Inputs required] 6. Todo Items/Pending Issues - aio fsync - aio sendfile - direct aio path (reorder vfs paths to have a single rw_kvec interface from fs when it really needs to do i/o) - aio readv/writev - i/o cancellation implementation (best effort; cancel i/os on process exit ?) - io_wait implementation (needs hashed waitqueues) - check for any races in current filesystem implementation (?) - implementations for other filesystems - network aio rewrite - in-kernel signal delivery mechanism for aio requests - making sub-tasks truly async (waiting for request slots, bmap calls) - debugging aids to help detect drivers which aren't totally async (e.g use semaphores - need to check which) or other sub-tasks which aren't truly async - flow control in aio (address write throtting issue) - implementing io_queue_grow (changing queue lengths) - mmaped ring buffer (Could lockless approaches be more fragile than we forsee now ? Is it worth it ? How much does it save ?) - kernel memory pinning issue (pinning user buffers too early ? may be able to improve this with cross-memory descriptors once aio flow control is in place) - explore at-least-N - explore io_submit_wait - aio request priorities (get the basic scheme in place, later relate it to the priority based i/o scheduler when that happens) - user space grouping of multiple completion queues (handling priorities, concurrency control etc; expose wait-queue primitives to userspace) - interfacing with generic event namespace (pollfs) approach (viro's idea) 7. References/Related patches: 1. Dan Kegel's c10k site: (http://www.kegel.com/c10k.html) Talks about the /dev/epoll patch, RT signals, Signal-per-fd approach, BSD kqueues and lots of links and discussions on various programming models for handling large numbers of clients/connections, with comparative studies. 2. NT I/O completion ports, Solaris and AIX aio, POSIX aio specs 3. SGI's kaio implementation - (http://oss.sgi.com/projects/kaio) 4. Block Asynchronous I/O: A Flexible Infrastructure for User Level Filesystems - Muthian Sivathanu, Venkateshwaran Venkataramani, and Remzi H. Arapaci-Dusseau, Univ of Winsconsin-Madison (http://www.cs.wisc.edu/~muthian/baio-paper.pdf) 5. The Direct Access File System Protocol & API Specifications - DAFS Collaborative (http://www.dafscollaborative.org) 6. 2.5 block i/o design notes - (http://lse.sourceforge.net/io/bionotes.txt)