dmaengine, async_tx: add a "no channel switch" allocator
[safe/jmp/linux-2.6] / drivers / dma / dmaengine.c
index 7a0594f..d5bc628 100644 (file)
  *
  * LOCKING:
  *
- * The subsystem keeps two global lists, dma_device_list and dma_client_list.
- * Both of these are protected by a mutex, dma_list_mutex.
+ * The subsystem keeps a global list of dma_device structs it is protected by a
+ * mutex, dma_list_mutex.
+ *
+ * A subsystem can get access to a channel by calling dmaengine_get() followed
+ * by dma_find_channel(), or if it has need for an exclusive channel it can call
+ * dma_request_channel().  Once a channel is allocated a reference is taken
+ * against its corresponding driver to disable removal.
  *
  * Each device has a channels list, which runs unlocked but is never modified
  * once the device is registered, it's just setup by the driver.
  *
- * Each client is responsible for keeping track of the channels it uses.  See
- * the definition of dma_event_callback in dmaengine.h.
- *
- * Each device has a kref, which is initialized to 1 when the device is
- * registered. A kref_get is done for each device registered.  When the
- * device is released, the corresponding kref_put is done in the release
- * method. Every time one of the device's channels is allocated to a client,
- * a kref_get occurs.  When the channel is freed, the corresponding kref_put
- * happens. The device's release function does a completion, so
- * unregister_device does a remove event, device_unregister, a kref_put
- * for the first reference, then waits on the completion for all other
- * references to finish.
- *
- * Each channel has an open-coded implementation of Rusty Russell's "bigref,"
- * with a kref and a per_cpu local_t.  A dma_chan_get is called when a client
- * signals that it wants to use a channel, and dma_chan_put is called when
- * a channel is removed or a client using it is unregistered.  A client can
- * take extra references per outstanding transaction, as is the case with
- * the NET DMA client.  The release function does a kref_put on the device.
- *     -ChrisL, DanW
+ * See Documentation/dmaengine.txt for more details
  */
 
 #include <linux/init.h>
 #include <linux/mutex.h>
 #include <linux/jiffies.h>
 #include <linux/rculist.h>
+#include <linux/idr.h>
 
 static DEFINE_MUTEX(dma_list_mutex);
 static LIST_HEAD(dma_device_list);
-static LIST_HEAD(dma_client_list);
 static long dmaengine_ref_count;
+static struct idr dma_idr;
 
 /* --- sysfs implementation --- */
 
+/**
+ * dev_to_dma_chan - convert a device pointer to the its sysfs container object
+ * @dev - device node
+ *
+ * Must be called under dma_list_mutex
+ */
+static struct dma_chan *dev_to_dma_chan(struct device *dev)
+{
+       struct dma_chan_dev *chan_dev;
+
+       chan_dev = container_of(dev, typeof(*chan_dev), device);
+       return chan_dev->chan;
+}
+
 static ssize_t show_memcpy_count(struct device *dev, struct device_attribute *attr, char *buf)
 {
-       struct dma_chan *chan = to_dma_chan(dev);
+       struct dma_chan *chan;
        unsigned long count = 0;
        int i;
+       int err;
 
-       for_each_possible_cpu(i)
-               count += per_cpu_ptr(chan->local, i)->memcpy_count;
+       mutex_lock(&dma_list_mutex);
+       chan = dev_to_dma_chan(dev);
+       if (chan) {
+               for_each_possible_cpu(i)
+                       count += per_cpu_ptr(chan->local, i)->memcpy_count;
+               err = sprintf(buf, "%lu\n", count);
+       } else
+               err = -ENODEV;
+       mutex_unlock(&dma_list_mutex);
 
-       return sprintf(buf, "%lu\n", count);
+       return err;
 }
 
 static ssize_t show_bytes_transferred(struct device *dev, struct device_attribute *attr,
                                      char *buf)
 {
-       struct dma_chan *chan = to_dma_chan(dev);
+       struct dma_chan *chan;
        unsigned long count = 0;
        int i;
+       int err;
 
-       for_each_possible_cpu(i)
-               count += per_cpu_ptr(chan->local, i)->bytes_transferred;
+       mutex_lock(&dma_list_mutex);
+       chan = dev_to_dma_chan(dev);
+       if (chan) {
+               for_each_possible_cpu(i)
+                       count += per_cpu_ptr(chan->local, i)->bytes_transferred;
+               err = sprintf(buf, "%lu\n", count);
+       } else
+               err = -ENODEV;
+       mutex_unlock(&dma_list_mutex);
 
-       return sprintf(buf, "%lu\n", count);
+       return err;
 }
 
 static ssize_t show_in_use(struct device *dev, struct device_attribute *attr, char *buf)
 {
-       struct dma_chan *chan = to_dma_chan(dev);
+       struct dma_chan *chan;
+       int err;
 
-       return sprintf(buf, "%d\n", chan->client_count);
+       mutex_lock(&dma_list_mutex);
+       chan = dev_to_dma_chan(dev);
+       if (chan)
+               err = sprintf(buf, "%d\n", chan->client_count);
+       else
+               err = -ENODEV;
+       mutex_unlock(&dma_list_mutex);
+
+       return err;
 }
 
 static struct device_attribute dma_attrs[] = {
@@ -118,18 +144,24 @@ static struct device_attribute dma_attrs[] = {
        __ATTR_NULL
 };
 
-static void dma_async_device_cleanup(struct kref *kref);
-
-static void dma_dev_release(struct device *dev)
+static void chan_dev_release(struct device *dev)
 {
-       struct dma_chan *chan = to_dma_chan(dev);
-       kref_put(&chan->device->refcount, dma_async_device_cleanup);
+       struct dma_chan_dev *chan_dev;
+
+       chan_dev = container_of(dev, typeof(*chan_dev), device);
+       if (atomic_dec_and_test(chan_dev->idr_ref)) {
+               mutex_lock(&dma_list_mutex);
+               idr_remove(&dma_idr, chan_dev->dev_id);
+               mutex_unlock(&dma_list_mutex);
+               kfree(chan_dev->idr_ref);
+       }
+       kfree(chan_dev);
 }
 
 static struct class dma_devclass = {
        .name           = "dma",
        .dev_attrs      = dma_attrs,
-       .dev_release    = dma_dev_release,
+       .dev_release    = chan_dev_release,
 };
 
 /* --- client and device registration --- */
@@ -189,7 +221,7 @@ static int dma_chan_get(struct dma_chan *chan)
 
        /* allocate upon first client reference */
        if (chan->client_count == 1 && err == 0) {
-               int desc_cnt = chan->device->device_alloc_chan_resources(chan, NULL);
+               int desc_cnt = chan->device->device_alloc_chan_resources(chan);
 
                if (desc_cnt < 0) {
                        err = desc_cnt;
@@ -218,44 +250,6 @@ static void dma_chan_put(struct dma_chan *chan)
                chan->device->device_free_chan_resources(chan);
 }
 
-/**
- * dma_client_chan_alloc - try to allocate channels to a client
- * @client: &dma_client
- *
- * Called with dma_list_mutex held.
- */
-static void dma_client_chan_alloc(struct dma_client *client)
-{
-       struct dma_device *device;
-       struct dma_chan *chan;
-       enum dma_state_client ack;
-
-       /* Find a channel */
-       list_for_each_entry(device, &dma_device_list, global_node) {
-               if (dma_has_cap(DMA_PRIVATE, device->cap_mask))
-                       continue;
-               /* Does the client require a specific DMA controller? */
-               if (client->slave && client->slave->dma_dev
-                               && client->slave->dma_dev != device->dev)
-                       continue;
-               if (!dma_device_satisfies_mask(device, client->cap_mask))
-                       continue;
-
-               list_for_each_entry(chan, &device->channels, device_node) {
-                       if (!chan->client_count)
-                               continue;
-                       ack = client->event_callback(client, chan,
-                                                    DMA_RESOURCE_AVAILABLE);
-
-                       /* we are done once this client rejects
-                        * an available resource
-                        */
-                       if (ack == DMA_NAK)
-                               return;
-               }
-       }
-}
-
 enum dma_status dma_sync_wait(struct dma_chan *chan, dma_cookie_t cookie)
 {
        enum dma_status status;
@@ -275,29 +269,6 @@ enum dma_status dma_sync_wait(struct dma_chan *chan, dma_cookie_t cookie)
 EXPORT_SYMBOL(dma_sync_wait);
 
 /**
- * dma_chan_cleanup - release a DMA channel's resources
- * @kref: kernel reference structure that contains the DMA channel device
- */
-void dma_chan_cleanup(struct kref *kref)
-{
-       struct dma_chan *chan = container_of(kref, struct dma_chan, refcount);
-       kref_put(&chan->device->refcount, dma_async_device_cleanup);
-}
-EXPORT_SYMBOL(dma_chan_cleanup);
-
-static void dma_chan_free_rcu(struct rcu_head *rcu)
-{
-       struct dma_chan *chan = container_of(rcu, struct dma_chan, rcu);
-
-       kref_put(&chan->refcount, dma_chan_cleanup);
-}
-
-static void dma_chan_release(struct dma_chan *chan)
-{
-       call_rcu(&chan->rcu, dma_chan_free_rcu);
-}
-
-/**
  * dma_cap_mask_all - enable iteration over all operation types
  */
 static dma_cap_mask_t dma_cap_mask_all;
@@ -347,7 +318,7 @@ static int __init dma_channel_table_init(void)
 
        return err;
 }
-subsys_initcall(dma_channel_table_init);
+arch_initcall(dma_channel_table_init);
 
 /**
  * dma_find_channel - find a channel to carry out the operation
@@ -358,9 +329,6 @@ struct dma_chan *dma_find_channel(enum dma_transaction_type tx_type)
        struct dma_chan *chan;
        int cpu;
 
-       WARN_ONCE(dmaengine_ref_count == 0,
-                 "client called %s without a reference", __func__);
-
        cpu = get_cpu();
        chan = per_cpu_ptr(channel_table[tx_type], cpu)->chan;
        put_cpu();
@@ -377,9 +345,6 @@ void dma_issue_pending_all(void)
        struct dma_device *device;
        struct dma_chan *chan;
 
-       WARN_ONCE(dmaengine_ref_count == 0,
-                 "client called %s without a reference", __func__);
-
        rcu_read_lock();
        list_for_each_entry_rcu(device, &dma_device_list, global_node) {
                if (dma_has_cap(DMA_PRIVATE, device->cap_mask))
@@ -483,10 +448,10 @@ static void dma_channel_rebalance(void)
                }
 }
 
-static struct dma_chan *private_candidate(dma_cap_mask_t *mask, struct dma_device *dev)
+static struct dma_chan *private_candidate(dma_cap_mask_t *mask, struct dma_device *dev,
+                                         dma_filter_fn fn, void *fn_param)
 {
        struct dma_chan *chan;
-       struct dma_chan *ret = NULL;
 
        if (!__dma_device_satisfies_mask(dev, mask)) {
                pr_debug("%s: wrong capabilities\n", __func__);
@@ -505,14 +470,18 @@ static struct dma_chan *private_candidate(dma_cap_mask_t *mask, struct dma_devic
        list_for_each_entry(chan, &dev->channels, device_node) {
                if (chan->client_count) {
                        pr_debug("%s: %s busy\n",
-                                __func__, dev_name(&chan->dev));
+                                __func__, dma_chan_name(chan));
                        continue;
                }
-               ret = chan;
-               break;
+               if (fn && !fn(chan, fn_param)) {
+                       pr_debug("%s: %s filter said false\n",
+                                __func__, dma_chan_name(chan));
+                       continue;
+               }
+               return chan;
        }
 
-       return ret;
+       return NULL;
 }
 
 /**
@@ -525,54 +494,41 @@ struct dma_chan *__dma_request_channel(dma_cap_mask_t *mask, dma_filter_fn fn, v
 {
        struct dma_device *device, *_d;
        struct dma_chan *chan = NULL;
-       enum dma_state_client ack;
        int err;
 
        /* Find a channel */
        mutex_lock(&dma_list_mutex);
        list_for_each_entry_safe(device, _d, &dma_device_list, global_node) {
-               chan = private_candidate(mask, device);
-               if (!chan)
-                       continue;
-
-               if (fn)
-                       ack = fn(chan, fn_param);
-               else
-                       ack = DMA_ACK;
-
-               if (ack == DMA_ACK) {
+               chan = private_candidate(mask, device, fn, fn_param);
+               if (chan) {
                        /* Found a suitable channel, try to grab, prep, and
                         * return it.  We first set DMA_PRIVATE to disable
                         * balance_ref_count as this channel will not be
                         * published in the general-purpose allocator
                         */
                        dma_cap_set(DMA_PRIVATE, device->cap_mask);
+                       device->privatecnt++;
                        err = dma_chan_get(chan);
 
                        if (err == -ENODEV) {
                                pr_debug("%s: %s module removed\n", __func__,
-                                        dev_name(&chan->dev));
+                                        dma_chan_name(chan));
                                list_del_rcu(&device->global_node);
                        } else if (err)
                                pr_err("dmaengine: failed to get %s: (%d)\n",
-                                      dev_name(&chan->dev), err);
+                                      dma_chan_name(chan), err);
                        else
                                break;
-               } else if (ack == DMA_DUP) {
-                       pr_debug("%s: %s filter said DMA_DUP\n",
-                                __func__, dev_name(&chan->dev));
-               } else if (ack == DMA_NAK) {
-                       pr_debug("%s: %s filter said DMA_NAK\n",
-                                __func__, dev_name(&chan->dev));
-                       break;
-               } else
-                       WARN_ONCE(1, "filter_fn: unknown response?\n");
-               chan = NULL;
+                       if (--device->privatecnt == 0)
+                               dma_cap_clear(DMA_PRIVATE, device->cap_mask);
+                       chan->private = NULL;
+                       chan = NULL;
+               }
        }
        mutex_unlock(&dma_list_mutex);
 
        pr_debug("%s: %s (%s)\n", __func__, chan ? "success" : "fail",
-                chan ? dev_name(&chan->dev) : NULL);
+                chan ? dma_chan_name(chan) : NULL);
 
        return chan;
 }
@@ -584,39 +540,23 @@ void dma_release_channel(struct dma_chan *chan)
        WARN_ONCE(chan->client_count != 1,
                  "chan reference count %d != 1\n", chan->client_count);
        dma_chan_put(chan);
+       /* drop PRIVATE cap enabled by __dma_request_channel() */
+       if (--chan->device->privatecnt == 0)
+               dma_cap_clear(DMA_PRIVATE, chan->device->cap_mask);
+       chan->private = NULL;
        mutex_unlock(&dma_list_mutex);
 }
 EXPORT_SYMBOL_GPL(dma_release_channel);
 
 /**
- * dma_chans_notify_available - broadcast available channels to the clients
- */
-static void dma_clients_notify_available(void)
-{
-       struct dma_client *client;
-
-       mutex_lock(&dma_list_mutex);
-
-       list_for_each_entry(client, &dma_client_list, global_node)
-               dma_client_chan_alloc(client);
-
-       mutex_unlock(&dma_list_mutex);
-}
-
-/**
- * dma_async_client_register - register a &dma_client
- * @client: ptr to a client structure with valid 'event_callback' and 'cap_mask'
+ * dmaengine_get - register interest in dma_channels
  */
-void dma_async_client_register(struct dma_client *client)
+void dmaengine_get(void)
 {
        struct dma_device *device, *_d;
        struct dma_chan *chan;
        int err;
 
-       /* validate client data */
-       BUG_ON(dma_has_cap(DMA_SLAVE, client->cap_mask) &&
-               !client->slave);
-
        mutex_lock(&dma_list_mutex);
        dmaengine_ref_count++;
 
@@ -632,7 +572,7 @@ void dma_async_client_register(struct dma_client *client)
                                break;
                        } else if (err)
                                pr_err("dmaengine: failed to get %s: (%d)\n",
-                                      dev_name(&chan->dev), err);
+                                      dma_chan_name(chan), err);
                }
        }
 
@@ -642,25 +582,18 @@ void dma_async_client_register(struct dma_client *client)
         */
        if (dmaengine_ref_count == 1)
                dma_channel_rebalance();
-       list_add_tail(&client->global_node, &dma_client_list);
        mutex_unlock(&dma_list_mutex);
 }
-EXPORT_SYMBOL(dma_async_client_register);
+EXPORT_SYMBOL(dmaengine_get);
 
 /**
- * dma_async_client_unregister - unregister a client and free the &dma_client
- * @client: &dma_client to free
- *
- * Force frees any allocated DMA channels, frees the &dma_client memory
+ * dmaengine_put - let dma drivers be removed when ref_count == 0
  */
-void dma_async_client_unregister(struct dma_client *client)
+void dmaengine_put(void)
 {
        struct dma_device *device;
        struct dma_chan *chan;
 
-       if (!client)
-               return;
-
        mutex_lock(&dma_list_mutex);
        dmaengine_ref_count--;
        BUG_ON(dmaengine_ref_count < 0);
@@ -671,24 +604,61 @@ void dma_async_client_unregister(struct dma_client *client)
                list_for_each_entry(chan, &device->channels, device_node)
                        dma_chan_put(chan);
        }
-
-       list_del(&client->global_node);
        mutex_unlock(&dma_list_mutex);
 }
-EXPORT_SYMBOL(dma_async_client_unregister);
+EXPORT_SYMBOL(dmaengine_put);
 
-/**
- * dma_async_client_chan_request - send all available channels to the
- * client that satisfy the capability mask
- * @client - requester
- */
-void dma_async_client_chan_request(struct dma_client *client)
+static bool device_has_all_tx_types(struct dma_device *device)
 {
+       /* A device that satisfies this test has channels that will never cause
+        * an async_tx channel switch event as all possible operation types can
+        * be handled.
+        */
+       #ifdef CONFIG_ASYNC_TX_DMA
+       if (!dma_has_cap(DMA_INTERRUPT, device->cap_mask))
+               return false;
+       #endif
+
+       #if defined(CONFIG_ASYNC_MEMCPY) || defined(CONFIG_ASYNC_MEMCPY_MODULE)
+       if (!dma_has_cap(DMA_MEMCPY, device->cap_mask))
+               return false;
+       #endif
+
+       #if defined(CONFIG_ASYNC_MEMSET) || defined(CONFIG_ASYNC_MEMSET_MODULE)
+       if (!dma_has_cap(DMA_MEMSET, device->cap_mask))
+               return false;
+       #endif
+
+       #if defined(CONFIG_ASYNC_XOR) || defined(CONFIG_ASYNC_XOR_MODULE)
+       if (!dma_has_cap(DMA_XOR, device->cap_mask))
+               return false;
+       #endif
+
+       #if defined(CONFIG_ASYNC_PQ) || defined(CONFIG_ASYNC_PQ_MODULE)
+       if (!dma_has_cap(DMA_PQ, device->cap_mask))
+               return false;
+       #endif
+
+       return true;
+}
+
+static int get_dma_id(struct dma_device *device)
+{
+       int rc;
+
+ idr_retry:
+       if (!idr_pre_get(&dma_idr, GFP_KERNEL))
+               return -ENOMEM;
        mutex_lock(&dma_list_mutex);
-       dma_client_chan_alloc(client);
+       rc = idr_get_new(&dma_idr, NULL, &device->dev_id);
        mutex_unlock(&dma_list_mutex);
+       if (rc == -EAGAIN)
+               goto idr_retry;
+       else if (rc != 0)
+               return rc;
+
+       return 0;
 }
-EXPORT_SYMBOL(dma_async_client_chan_request);
 
 /**
  * dma_async_device_register - registers DMA devices found
@@ -696,9 +666,9 @@ EXPORT_SYMBOL(dma_async_client_chan_request);
  */
 int dma_async_device_register(struct dma_device *device)
 {
-       static int id;
        int chancnt = 0, rc;
        struct dma_chan* chan;
+       atomic_t *idr_ref;
 
        if (!device)
                return -ENODEV;
@@ -708,8 +678,12 @@ int dma_async_device_register(struct dma_device *device)
                !device->device_prep_dma_memcpy);
        BUG_ON(dma_has_cap(DMA_XOR, device->cap_mask) &&
                !device->device_prep_dma_xor);
-       BUG_ON(dma_has_cap(DMA_ZERO_SUM, device->cap_mask) &&
-               !device->device_prep_dma_zero_sum);
+       BUG_ON(dma_has_cap(DMA_XOR_VAL, device->cap_mask) &&
+               !device->device_prep_dma_xor_val);
+       BUG_ON(dma_has_cap(DMA_PQ, device->cap_mask) &&
+               !device->device_prep_dma_pq);
+       BUG_ON(dma_has_cap(DMA_PQ_VAL, device->cap_mask) &&
+               !device->device_prep_dma_pq_val);
        BUG_ON(dma_has_cap(DMA_MEMSET, device->cap_mask) &&
                !device->device_prep_dma_memset);
        BUG_ON(dma_has_cap(DMA_INTERRUPT, device->cap_mask) &&
@@ -725,40 +699,55 @@ int dma_async_device_register(struct dma_device *device)
        BUG_ON(!device->device_issue_pending);
        BUG_ON(!device->dev);
 
-       init_completion(&device->done);
-       kref_init(&device->refcount);
+       /* note: this only matters in the
+        * CONFIG_ASYNC_TX_DISABLE_CHANNEL_SWITCH=y case
+        */
+       if (device_has_all_tx_types(device))
+               dma_cap_set(DMA_ASYNC_TX, device->cap_mask);
 
-       mutex_lock(&dma_list_mutex);
-       device->dev_id = id++;
-       mutex_unlock(&dma_list_mutex);
+       idr_ref = kmalloc(sizeof(*idr_ref), GFP_KERNEL);
+       if (!idr_ref)
+               return -ENOMEM;
+       rc = get_dma_id(device);
+       if (rc != 0) {
+               kfree(idr_ref);
+               return rc;
+       }
+
+       atomic_set(idr_ref, 0);
 
        /* represent channels in sysfs. Probably want devs too */
        list_for_each_entry(chan, &device->channels, device_node) {
+               rc = -ENOMEM;
                chan->local = alloc_percpu(typeof(*chan->local));
                if (chan->local == NULL)
-                       continue;
+                       goto err_out;
+               chan->dev = kzalloc(sizeof(*chan->dev), GFP_KERNEL);
+               if (chan->dev == NULL) {
+                       free_percpu(chan->local);
+                       chan->local = NULL;
+                       goto err_out;
+               }
 
                chan->chan_id = chancnt++;
-               chan->dev.class = &dma_devclass;
-               chan->dev.parent = device->dev;
-               dev_set_name(&chan->dev, "dma%dchan%d",
+               chan->dev->device.class = &dma_devclass;
+               chan->dev->device.parent = device->dev;
+               chan->dev->chan = chan;
+               chan->dev->idr_ref = idr_ref;
+               chan->dev->dev_id = device->dev_id;
+               atomic_inc(idr_ref);
+               dev_set_name(&chan->dev->device, "dma%dchan%d",
                             device->dev_id, chan->chan_id);
 
-               rc = device_register(&chan->dev);
+               rc = device_register(&chan->dev->device);
                if (rc) {
-                       chancnt--;
                        free_percpu(chan->local);
                        chan->local = NULL;
+                       kfree(chan->dev);
+                       atomic_dec(idr_ref);
                        goto err_out;
                }
-
-               /* One for the channel, one of the class device */
-               kref_get(&device->refcount);
-               kref_get(&device->refcount);
-               kref_init(&chan->refcount);
                chan->client_count = 0;
-               chan->slow_ref = 0;
-               INIT_RCU_HEAD(&chan->rcu);
        }
        device->chancnt = chancnt;
 
@@ -780,20 +769,30 @@ int dma_async_device_register(struct dma_device *device)
                        }
                }
        list_add_tail_rcu(&device->global_node, &dma_device_list);
+       if (dma_has_cap(DMA_PRIVATE, device->cap_mask))
+               device->privatecnt++;   /* Always private */
        dma_channel_rebalance();
        mutex_unlock(&dma_list_mutex);
 
-       dma_clients_notify_available();
-
        return 0;
 
 err_out:
+       /* if we never registered a channel just release the idr */
+       if (atomic_read(idr_ref) == 0) {
+               mutex_lock(&dma_list_mutex);
+               idr_remove(&dma_idr, device->dev_id);
+               mutex_unlock(&dma_list_mutex);
+               kfree(idr_ref);
+               return rc;
+       }
+
        list_for_each_entry(chan, &device->channels, device_node) {
                if (chan->local == NULL)
                        continue;
-               kref_put(&device->refcount, dma_async_device_cleanup);
-               device_unregister(&chan->dev);
-               chancnt--;
+               mutex_lock(&dma_list_mutex);
+               chan->dev->chan = NULL;
+               mutex_unlock(&dma_list_mutex);
+               device_unregister(&chan->dev->device);
                free_percpu(chan->local);
        }
        return rc;
@@ -801,20 +800,11 @@ err_out:
 EXPORT_SYMBOL(dma_async_device_register);
 
 /**
- * dma_async_device_cleanup - function called when all references are released
- * @kref: kernel reference object
- */
-static void dma_async_device_cleanup(struct kref *kref)
-{
-       struct dma_device *device;
-
-       device = container_of(kref, struct dma_device, refcount);
-       complete(&device->done);
-}
-
-/**
  * dma_async_device_unregister - unregister a DMA device
  * @device: &dma_device
+ *
+ * This routine is called by dma driver exit routines, dmaengine holds module
+ * references to prevent it being called while channels are in use.
  */
 void dma_async_device_unregister(struct dma_device *device)
 {
@@ -829,12 +819,11 @@ void dma_async_device_unregister(struct dma_device *device)
                WARN_ONCE(chan->client_count,
                          "%s called while %d clients hold a reference\n",
                          __func__, chan->client_count);
-               device_unregister(&chan->dev);
-               dma_chan_release(chan);
+               mutex_lock(&dma_list_mutex);
+               chan->dev->chan = NULL;
+               mutex_unlock(&dma_list_mutex);
+               device_unregister(&chan->dev->device);
        }
-
-       kref_put(&device->refcount, dma_async_device_cleanup);
-       wait_for_completion(&device->done);
 }
 EXPORT_SYMBOL(dma_async_device_unregister);
 
@@ -859,11 +848,14 @@ dma_async_memcpy_buf_to_buf(struct dma_chan *chan, void *dest,
        dma_addr_t dma_dest, dma_src;
        dma_cookie_t cookie;
        int cpu;
+       unsigned long flags;
 
        dma_src = dma_map_single(dev->dev, src, len, DMA_TO_DEVICE);
        dma_dest = dma_map_single(dev->dev, dest, len, DMA_FROM_DEVICE);
-       tx = dev->device_prep_dma_memcpy(chan, dma_dest, dma_src, len,
-                                        DMA_CTRL_ACK);
+       flags = DMA_CTRL_ACK |
+               DMA_COMPL_SRC_UNMAP_SINGLE |
+               DMA_COMPL_DEST_UNMAP_SINGLE;
+       tx = dev->device_prep_dma_memcpy(chan, dma_dest, dma_src, len, flags);
 
        if (!tx) {
                dma_unmap_single(dev->dev, dma_src, len, DMA_TO_DEVICE);
@@ -905,11 +897,12 @@ dma_async_memcpy_buf_to_pg(struct dma_chan *chan, struct page *page,
        dma_addr_t dma_dest, dma_src;
        dma_cookie_t cookie;
        int cpu;
+       unsigned long flags;
 
        dma_src = dma_map_single(dev->dev, kdata, len, DMA_TO_DEVICE);
        dma_dest = dma_map_page(dev->dev, page, offset, len, DMA_FROM_DEVICE);
-       tx = dev->device_prep_dma_memcpy(chan, dma_dest, dma_src, len,
-                                        DMA_CTRL_ACK);
+       flags = DMA_CTRL_ACK | DMA_COMPL_SRC_UNMAP_SINGLE;
+       tx = dev->device_prep_dma_memcpy(chan, dma_dest, dma_src, len, flags);
 
        if (!tx) {
                dma_unmap_single(dev->dev, dma_src, len, DMA_TO_DEVICE);
@@ -953,12 +946,13 @@ dma_async_memcpy_pg_to_pg(struct dma_chan *chan, struct page *dest_pg,
        dma_addr_t dma_dest, dma_src;
        dma_cookie_t cookie;
        int cpu;
+       unsigned long flags;
 
        dma_src = dma_map_page(dev->dev, src_pg, src_off, len, DMA_TO_DEVICE);
        dma_dest = dma_map_page(dev->dev, dest_pg, dest_off, len,
                                DMA_FROM_DEVICE);
-       tx = dev->device_prep_dma_memcpy(chan, dma_dest, dma_src, len,
-                                        DMA_CTRL_ACK);
+       flags = DMA_CTRL_ACK;
+       tx = dev->device_prep_dma_memcpy(chan, dma_dest, dma_src, len, flags);
 
        if (!tx) {
                dma_unmap_page(dev->dev, dma_src, len, DMA_TO_DEVICE);
@@ -983,54 +977,30 @@ void dma_async_tx_descriptor_init(struct dma_async_tx_descriptor *tx,
 {
        tx->chan = chan;
        spin_lock_init(&tx->lock);
+       INIT_LIST_HEAD(&tx->tx_list);
 }
 EXPORT_SYMBOL(dma_async_tx_descriptor_init);
 
 /* dma_wait_for_async_tx - spin wait for a transaction to complete
  * @tx: in-flight transaction to wait on
- *
- * This routine assumes that tx was obtained from a call to async_memcpy,
- * async_xor, async_memset, etc which ensures that tx is "in-flight" (prepped
- * and submitted).  Walking the parent chain is only meant to cover for DMA
- * drivers that do not implement the DMA_INTERRUPT capability and may race with
- * the driver's descriptor cleanup routine.
  */
 enum dma_status
 dma_wait_for_async_tx(struct dma_async_tx_descriptor *tx)
 {
-       enum dma_status status;
-       struct dma_async_tx_descriptor *iter;
-       struct dma_async_tx_descriptor *parent;
+       unsigned long dma_sync_wait_timeout = jiffies + msecs_to_jiffies(5000);
 
        if (!tx)
                return DMA_SUCCESS;
 
-       WARN_ONCE(tx->parent, "%s: speculatively walking dependency chain for"
-                 " %s\n", __func__, dev_name(&tx->chan->dev));
-
-       /* poll through the dependency chain, return when tx is complete */
-       do {
-               iter = tx;
-
-               /* find the root of the unsubmitted dependency chain */
-               do {
-                       parent = iter->parent;
-                       if (!parent)
-                               break;
-                       else
-                               iter = parent;
-               } while (parent);
-
-               /* there is a small window for ->parent == NULL and
-                * ->cookie == -EBUSY
-                */
-               while (iter->cookie == -EBUSY)
-                       cpu_relax();
-
-               status = dma_sync_wait(iter->chan, iter->cookie);
-       } while (status == DMA_IN_PROGRESS || (iter != tx));
-
-       return status;
+       while (tx->cookie == -EBUSY) {
+               if (time_after_eq(jiffies, dma_sync_wait_timeout)) {
+                       pr_err("%s timeout waiting for descriptor submission\n",
+                               __func__);
+                       return DMA_ERROR;
+               }
+               cpu_relax();
+       }
+       return dma_sync_wait(tx->chan, tx->cookie);
 }
 EXPORT_SYMBOL_GPL(dma_wait_for_async_tx);
 
@@ -1047,6 +1017,8 @@ void dma_run_dependencies(struct dma_async_tx_descriptor *tx)
        if (!dep)
                return;
 
+       /* we'll submit tx->next now, so clear the link */
+       tx->next = NULL;
        chan = dep->chan;
 
        /* keep submitting up until a channel switch is detected
@@ -1072,9 +1044,10 @@ EXPORT_SYMBOL_GPL(dma_run_dependencies);
 
 static int __init dma_bus_init(void)
 {
+       idr_init(&dma_idr);
        mutex_init(&dma_list_mutex);
        return class_register(&dma_devclass);
 }
-subsys_initcall(dma_bus_init);
+arch_initcall(dma_bus_init);