New snapshot implementation. This implementation has shared storage and
unlimited number of snapshots.

The work is split to two modules:
dm-multisnapshot.ko - the general module
dm-store-mikulas.ko - the snapshot store

The modularity allows to load other snapshot stores.

Usage:
Create two logical volumes, one for origin and one for snapshots.
(assume /dev/mapper/vg1-lv1 for origin and /dev/mapper/vg1-lv2 for snapshot in
these examples)

Clear the first sector of the snapshot volume:
dd if=/dev/zero of=/dev/mapper/vg1-lv2 bs=4096 count=1

Table line arguments:
- origin device
- shared store device
- chunk size
- number of generic arguments
- generic arguments
	sync-snapshots --- synchronize snapshots according to the list
	preserve-on-error --- halt the origin on error in the snapshot store
- shared store type
- number of arguments for shared store type
- shared store arguments
	cache-threshold size --- a background write is started
	cache-limit size --- a limit for metadata cache size
if sync-snapshots was specified
	- number of snapshot ids
	- snapshot ids

Load the shared snapshot driver:
echo 0 `blockdev --getsize /dev/mapper/vg1-lv1` multisnapshot /dev/mapper/vg1-lv1 /dev/mapper/vg1-lv2 16 0 mikulas 0|dmsetup create ms
(16 is the chunk size in 512-byte sectors. You can place different number there)
This creates the origin store on /dev/mapper/ms. If the store was zeroed, it
creates new structure, otherwise it loads existing structure.

Once this is done, you should no longer access /dev/mapper/vg1-lv1 and
/dev/mapper/vg1-lv2 and only use /dev/mapper/ms.

Create new snapshot:
dmsetup message /dev/mapper/ms 0 create
dmsetup status /dev/mapper/ms
	(this will find out the newly created snapshot ID)
dmsetup suspend /dev/mapper/ms
dmsetup resume /dev/mapper/ms

Attach the snapshot:
echo 0 `blockdev --getsize /dev/mapper/vg1-lv1` multisnap-snap /dev/mapper/vg1-lv1 0|dmsetup create ms0
(that '0' is the snapshot id ... you can use different number)
This attaches the snapshot '0' on /dev/mapper/ms0

Delete the snapshot:
dmsetup message /dev/mapper/ms 0 delete 0
(the parameter after "delete" is the snapshot id)

See status:
dmsetup status prints these information about the multisnapshot device:
- number of arguments befor the snapshot id list (5)
- 0 on active storage, -error number on error (-ENOSPC, -EIO, etc.)
- the new snapshot number that will be created, "-" if there is none
- total number of chunks on the device
- total number of allocated chunks
- a number of chunks allocated for metadata
- a number of snapshots
- existing snapshot IDs

Unload it:
dmsetup remove ms
dmsetup remove ms0
... etc. (note, once you unload the origin, the snapshots become inaccessible
- the devices exist but they return -EIO on everything)


Signed-off-by: Mikulas Patocka <mpatocka@redhat.com>

---
 drivers/md/Kconfig                       |   19 
 drivers/md/Makefile                      |    8 
 drivers/md/dm-bufio.c                    |  969 ++++++++++++++++
 drivers/md/dm-bufio.h                    |   29 
 drivers/md/dm-multisnap-alloc.c          |  576 +++++++++
 drivers/md/dm-multisnap-blocks.c         |  198 +++
 drivers/md/dm-multisnap-btree.c          |  798 +++++++++++++
 drivers/md/dm-multisnap-commit.c         |  210 +++
 drivers/md/dm-multisnap-delete.c         |  134 ++
 drivers/md/dm-multisnap-freelist.c       |  236 +++
 drivers/md/dm-multisnap-io.c             |  191 +++
 drivers/md/dm-multisnap-mikulas-struct.h |  198 +++
 drivers/md/dm-multisnap-mikulas.c        |  667 +++++++++++
 drivers/md/dm-multisnap-mikulas.h        |  201 +++
 drivers/md/dm-multisnap-private.h        |  145 ++
 drivers/md/dm-multisnap-snaps.c          |  429 +++++++
 drivers/md/dm-multisnap.c                | 1871 +++++++++++++++++++++++++++++++
 drivers/md/dm-multisnap.h                |  153 ++
 18 files changed, 7032 insertions(+)

Index: linux-2.6.32/drivers/md/Kconfig
===================================================================
--- linux-2.6.32.orig/drivers/md/Kconfig
+++ linux-2.6.32/drivers/md/Kconfig
@@ -249,6 +249,25 @@ config DM_SNAPSHOT
        ---help---
          Allow volume managers to take writable snapshots of a device.
 
+config DM_MULTISNAPSHOT
+	tristate "Multisnapshot target"
+	depends on BLK_DEV_DM
+	---help---
+	  A new implementation of snapshots allowing sharing storage
+	  between several snapshots.
+
+	  A submenu allows to select a specific shared snapshot store
+	  driver.
+
+config DM_MULTISNAPSHOT_MIKULAS
+	tristate "Mikulas' snapshot store"
+	depends on DM_MULTISNAPSHOT
+	---help---
+	  Mikulas Patocka's snapshot store.
+
+	  A B+-tree-based log-structured storage allowing unlimited
+	  number of snapshots.
+
 config DM_MIRROR
        tristate "Mirror target"
        depends on BLK_DEV_DM
Index: linux-2.6.32/drivers/md/Makefile
===================================================================
--- linux-2.6.32.orig/drivers/md/Makefile
+++ linux-2.6.32/drivers/md/Makefile
@@ -7,6 +7,12 @@ dm-mod-y	+= dm.o dm-table.o dm-target.o 
 dm-multipath-y	+= dm-path-selector.o dm-mpath.o
 dm-snapshot-y	+= dm-snap.o dm-exception-store.o dm-snap-transient.o \
 		    dm-snap-persistent.o
+dm-multisnapshot-y += dm-multisnap.o
+dm-store-mikulas-y += dm-multisnap-mikulas.o dm-multisnap-alloc.o \
+		   dm-multisnap-blocks.o dm-multisnap-btree.o \
+		   dm-multisnap-commit.o dm-multisnap-delete.o \
+		   dm-multisnap-freelist.o dm-multisnap-io.o \
+		   dm-multisnap-snaps.o dm-bufio.o
 dm-mirror-y	+= dm-raid1.o
 dm-log-userspace-y \
 		+= dm-log-userspace-base.o dm-log-userspace-transfer.o
@@ -41,6 +47,8 @@ obj-$(CONFIG_DM_MULTIPATH)	+= dm-multipa
 obj-$(CONFIG_DM_MULTIPATH_QL)	+= dm-queue-length.o
 obj-$(CONFIG_DM_MULTIPATH_ST)	+= dm-service-time.o
 obj-$(CONFIG_DM_SNAPSHOT)	+= dm-snapshot.o
+obj-$(CONFIG_DM_MULTISNAPSHOT)	+= dm-multisnapshot.o
+obj-$(CONFIG_DM_MULTISNAPSHOT_MIKULAS) += dm-store-mikulas.o
 obj-$(CONFIG_DM_MIRROR)		+= dm-mirror.o dm-log.o dm-region-hash.o
 obj-$(CONFIG_DM_LOG_USERSPACE)	+= dm-log-userspace.o
 obj-$(CONFIG_DM_ZERO)		+= dm-zero.o
Index: linux-2.6.32/drivers/md/dm-bufio.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-bufio.c
@@ -0,0 +1,969 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include <linux/device-mapper.h>
+#include <linux/dm-io.h>
+#include <linux/slab.h>
+#include <linux/vmalloc.h>
+
+#include "dm-bufio.h"
+
+/*
+ * dm_bufio_client_create --- create a buffered IO cache on a given device
+ * dm_bufio_client_destroy --- release a buffered IO cache
+ *
+ * dm_bufio_read --- read a given block from disk. Returns pointer to data.
+ *	Returns a pointer to dm_buffer that can be used to release the buffer
+ *	or to make it dirty.
+ * dm_bufio_new --- like dm_bufio_read, but don't read anything from the disk.
+ *	It is expected that the caller initializes the buffer and marks it
+ *	dirty.
+ * dm_bufio_release --- release a reference obtained with dm_bufio_read or
+ *	dm_bufio_new. The data pointer and dm_buffer pointer is no longer valid
+ *	after this call.
+ *
+ * WARNING: to avoid deadlocks, the thread can hold at most one buffer. Multiple
+ *	threads can hold each one buffer simultaneously.
+ *
+ * dm_bufio_mark_buffer_dirty --- mark a buffer dirty. It should be called after
+ *	the buffer is modified.
+ * dm_bufio_write_dirty_buffers --- write all dirty buffers. Guarantees that all
+ *	dirty buffers created prior to this call are on disk when this call
+ *	exits.
+ * dm_bufio_issue_flush --- send an empty write barrier to the device to flush
+ *	hardware disk cache.
+ *
+ * In case of memory pressure, the buffer may be written after
+ *	dm_bufio_mark_buffer_dirty, but before dm_bufio_write_dirty_buffers.
+ *	So, dm_bufio_write_dirty_buffers guarantees that the buffer is on-disk,
+ *	but the actual writing may occur earlier.
+ *
+ * dm_bufio_release_move --- like dm_bufio_release, and also move the buffer to
+ *	the new block. dm_bufio_write_dirty_buffers is needed to commit the new
+ *	block.
+ * dm_bufio_drop_buffers --- clear all buffers.
+ */
+
+/*
+ * Memory management policy:
+ *	When we're above threshold, start asynchronous writing of dirty buffers
+ *	and memory reclaiming --- but still allow new allocations.
+ *	When we're above limit, don't allocate any more space and synchronously
+ *	wait until existing buffers are freed.
+ *
+ * These default parameters can be overriden with parameters to
+ * dm_bufio_client_create.
+ */
+#define DM_BUFIO_THRESHOLD_MEMORY	(8 * 1048576)
+#define DM_BUFIO_LIMIT_MEMORY		(9 * 1048576)
+
+/*
+ * The number of bvec entries that are embedded directly in the buffer.
+ * If the chunk size is larger, dm-io is used to do the io.
+ */
+#define DM_BUFIO_INLINE_VECS		16
+
+/*
+ * Buffer hash
+ */
+#define DM_BUFIO_HASH_SIZE	(PAGE_SIZE / sizeof(struct hlist_head) / 2)
+#define DM_BUFIO_HASH(block)	((block) & (DM_BUFIO_HASH_SIZE - 1))
+
+/*
+ * Don't try to kmalloc blocks larger than this.
+ * For exaplanation, see dm_bufio_alloc_buffer_data below.
+ */
+#define DM_BUFIO_BLOCK_SIZE_KMALLOC_LIMIT	PAGE_SIZE
+
+/*
+ * Buffer state bits.
+ */
+#define B_READING	0
+#define B_WRITING	1
+#define B_DIRTY		2
+
+struct dm_bufio_client {
+	/*
+	 * Linking of buffers:
+	 *	all buffers are linked to cache_hash with their hash_list field.
+	 *	clean buffers that are not being written (B_WRITING not set)
+	 *		are linked to lru with their lru_list field.
+	 *	dirty and clean buffers that are being written are linked
+	 *		to dirty_lru with their	lru_list field. When the write
+	 *		finishes, the buffer cannot be immediatelly relinked
+	 *		(because we are in an interrupt context and relinking
+	 *		requires process context), so some clean-not-writing
+	 *		buffers	can be held on dirty_lru too. They are later
+	 *		added to
+	 *		lru in the process context.
+	 */
+	struct list_head lru;
+	struct list_head dirty_lru;
+	struct mutex lock;
+	struct block_device *bdev;
+	unsigned block_size;
+	unsigned char sectors_per_block_bits;
+	unsigned char pages_per_block_bits;
+
+	unsigned long n_buffers;
+	unsigned threshold_buffers;
+	unsigned limit_buffers;
+
+	struct dm_io_client *dm_io;
+
+	struct dm_buffer *reserved_buffer;
+	struct hlist_head cache_hash[DM_BUFIO_HASH_SIZE];
+	wait_queue_head_t free_buffer_wait;
+
+	int async_write_error;
+};
+
+/*
+ * A method, with wich the data is allocated:
+ * kmalloc(), __get_free_pages() or vmalloc().
+ * See the comment at dm_bufio_alloc_buffer_data.
+ */
+#define DATA_MODE_KMALLOC		1
+#define DATA_MODE_GET_FREE_PAGES	2
+#define DATA_MODE_VMALLOC		3
+
+struct dm_buffer {
+	struct hlist_node hash_list;
+	struct list_head lru_list;
+	sector_t block;
+	void *data;
+	char data_mode;		/* DATA_MODE_* */
+	unsigned hold_count;
+	int read_error;
+	int write_error;
+	unsigned long state;
+	struct dm_bufio_client *c;
+	struct bio bio;
+	struct bio_vec bio_vec[DM_BUFIO_INLINE_VECS];
+};
+
+/*
+ * Allocating buffer data.
+ *
+ * Small buffers are allocated with kmalloc, to use space optimally.
+ *
+ * Large buffers:
+ * We use get_free_pages or vmalloc, both have their advantages and
+ * disadvantages.
+ * __get_free_pages can randomly fail, if the memory is fragmented.
+ * __vmalloc won't randomly fail, but vmalloc space is limited (it may be
+ *	as low as 128M) --- so using it for caching is not appropriate.
+ * If the allocation may fail, we use __get_free_pages, memory fragmentation
+ *	won't have fatal effect here, it just causes flushes of some other
+ *	buffers and more I/O will be performed.
+ * If the allocation shouldn't fail, we use __vmalloc. This is only for
+ *	the initial reserve allocation, so there's no risk of wasting
+ *	all vmalloc space.
+ */
+
+static void *dm_bufio_alloc_buffer_data(struct dm_bufio_client *c, gfp_t gfp_mask, char *data_mode)
+{
+	if (c->block_size <= DM_BUFIO_BLOCK_SIZE_KMALLOC_LIMIT) {
+		*data_mode = DATA_MODE_KMALLOC;
+		return kmalloc(c->block_size, gfp_mask);
+	} else if (gfp_mask & __GFP_NORETRY) {
+		*data_mode = DATA_MODE_GET_FREE_PAGES;
+		return (void *)__get_free_pages(gfp_mask, c->pages_per_block_bits);
+	} else {
+		*data_mode = DATA_MODE_VMALLOC;
+		return __vmalloc(c->block_size, gfp_mask, PAGE_KERNEL);
+	}
+}
+
+/*
+ * Free buffer's data.
+ */
+
+static void dm_bufio_free_buffer_data(struct dm_bufio_client *c, void *data, char data_mode)
+{
+	switch (data_mode) {
+
+	case DATA_MODE_KMALLOC:
+		kfree(data);
+		break;
+	case DATA_MODE_GET_FREE_PAGES:
+		free_pages((unsigned long)data, c->pages_per_block_bits);
+		break;
+	case DATA_MODE_VMALLOC:
+		vfree(data);
+		break;
+	default:
+		printk(KERN_CRIT "dm_bufio_free_buffer_data: bad data mode: %d", data_mode);
+		BUG();
+
+	}
+}
+
+
+/*
+ * Allocate buffer and its data.
+ */
+
+static struct dm_buffer *alloc_buffer(struct dm_bufio_client *c, gfp_t gfp_mask)
+{
+	struct dm_buffer *b;
+	b = kmalloc(sizeof(struct dm_buffer), gfp_mask);
+	if (!b)
+		return NULL;
+	b->c = c;
+	b->data = dm_bufio_alloc_buffer_data(c, gfp_mask, &b->data_mode);
+	if (!b->data) {
+		kfree(b);
+		return NULL;
+	}
+	return b;
+}
+
+/*
+ * Free buffer and its data.
+ */
+
+static void free_buffer(struct dm_buffer *b)
+{
+	dm_bufio_free_buffer_data(b->c, b->data, b->data_mode);
+	kfree(b);
+}
+
+
+/*
+ * Link buffer to the hash list and clean or dirty queue.
+ */
+
+static void link_buffer(struct dm_buffer *b, sector_t block, int dirty)
+{
+	struct dm_bufio_client *c = b->c;
+	c->n_buffers++;
+	b->block = block;
+	list_add(&b->lru_list, dirty ? &c->dirty_lru : &c->lru);
+	hlist_add_head(&b->hash_list, &c->cache_hash[DM_BUFIO_HASH(block)]);
+}
+
+/*
+ * Unlink buffer from the hash list and dirty or clean queue.
+ */
+
+static void unlink_buffer(struct dm_buffer *b)
+{
+	BUG_ON(!b->c->n_buffers);
+	b->c->n_buffers--;
+	hlist_del(&b->hash_list);
+	list_del(&b->lru_list);
+}
+
+/*
+ * Place the buffer to the head of dirty or clean LRU queue.
+ */
+
+static void relink_lru(struct dm_buffer *b, int dirty)
+{
+	struct dm_bufio_client *c = b->c;
+	list_del(&b->lru_list);
+	list_add(&b->lru_list, dirty ? &c->dirty_lru : &c->lru);
+}
+
+/*
+ * This function is called when wait_on_bit is actually waiting.
+ * It unplugs the underlying block device, so that coalesced I/Os in
+ * the request queue are dispatched to the device.
+ */
+
+static int do_io_schedule(void *word)
+{
+	struct dm_buffer *b = container_of(word, struct dm_buffer, state);
+	struct dm_bufio_client *c = b->c;
+
+	blk_run_address_space(c->bdev->bd_inode->i_mapping);
+
+	io_schedule();
+
+	return 0;
+}
+
+static void write_dirty_buffer(struct dm_buffer *b);
+
+/*
+ * Wait until any activity on the buffer finishes.
+ * Possibly write the buffer if it is dirty.
+ * When this function finishes, there is no I/O running on the buffer
+ * and the buffer is not dirty.
+ */
+
+static void make_buffer_clean(struct dm_buffer *b)
+{
+	BUG_ON(b->hold_count);
+	if (likely(!b->state))	/* fast case */
+		return;
+	wait_on_bit(&b->state, B_READING, do_io_schedule, TASK_UNINTERRUPTIBLE);
+	write_dirty_buffer(b);
+	wait_on_bit(&b->state, B_WRITING, do_io_schedule, TASK_UNINTERRUPTIBLE);
+}
+
+/*
+ * Find some buffer that is not held by anybody, clean it, unlink it and
+ * return it.
+ * If "wait" is zero, try less harder and don't block.
+ */
+
+static struct dm_buffer *get_unclaimed_buffer(struct dm_bufio_client *c, int wait)
+{
+	struct dm_buffer *b;
+	list_for_each_entry_reverse(b, &c->lru, lru_list) {
+		cond_resched();
+		BUG_ON(test_bit(B_WRITING, &b->state));
+		BUG_ON(test_bit(B_DIRTY, &b->state));
+		if (!b->hold_count) {
+			if (!wait && unlikely(test_bit(B_READING, &b->state)))
+				continue;
+			make_buffer_clean(b);
+			unlink_buffer(b);
+			return b;
+		}
+	}
+	list_for_each_entry_reverse(b, &c->dirty_lru, lru_list) {
+		cond_resched();
+		BUG_ON(test_bit(B_READING, &b->state));
+		if (!b->hold_count) {
+			if (!wait && (unlikely(test_bit(B_DIRTY, &b->state)) ||
+				      unlikely(test_bit(B_WRITING, &b->state)))) {
+				if (!test_bit(B_WRITING, &b->state))
+					write_dirty_buffer(b);
+				continue;
+			}
+			make_buffer_clean(b);
+			unlink_buffer(b);
+			return b;
+		}
+	}
+	return NULL;
+}
+
+/*
+ * Wait until some other threads free some buffer or release hold count
+ * on some buffer.
+ *
+ * This function is entered with c->lock held, drops it and regains it before
+ * exiting.
+ */
+
+static void wait_for_free_buffer(struct dm_bufio_client *c)
+{
+	DECLARE_WAITQUEUE(wait, current);
+
+	add_wait_queue(&c->free_buffer_wait, &wait);
+	set_task_state(current, TASK_UNINTERRUPTIBLE);
+	mutex_unlock(&c->lock);
+
+	io_schedule();
+
+	set_task_state(current, TASK_RUNNING);
+	remove_wait_queue(&c->free_buffer_wait, &wait);
+
+	mutex_lock(&c->lock);
+}
+
+/*
+ * Allocate a new buffer. If the allocation is not possible, wait until some
+ * other thread frees a buffer.
+ *
+ * May drop the lock and regain it.
+ */
+
+static struct dm_buffer *alloc_buffer_wait(struct dm_bufio_client *c)
+{
+	struct dm_buffer *b;
+
+retry:
+	/*
+	 * dm-bufio is resistant to allocation failures (it just keeps
+	 * one buffer reserved in caes all the allocations fail).
+	 * So set flags to not try too hard:
+	 *	GFP_NOIO: don't recurse into the I/O layer
+	 *	__GFP_NOMEMALLOC: don't use emergency reserves
+	 *	__GFP_NORETRY: don't retry and rather return failure
+	 *	__GFP_NOWARN: don't print a warning in case of failure
+	 */
+	b = alloc_buffer(c, GFP_NOIO | __GFP_NOMEMALLOC | __GFP_NORETRY | __GFP_NOWARN);
+	if (b)
+		return b;
+
+	if (c->reserved_buffer) {
+		b = c->reserved_buffer;
+		c->reserved_buffer = NULL;
+		return b;
+	}
+
+	b = get_unclaimed_buffer(c, 1);
+	if (b)
+		return b;
+
+	wait_for_free_buffer(c);
+	goto retry;
+}
+
+/*
+ * Free a buffer and wake other threads waiting for free buffers.
+ */
+
+static void free_buffer_wake(struct dm_buffer *b)
+{
+	struct dm_bufio_client *c = b->c;
+
+	if (unlikely(!c->reserved_buffer))
+		c->reserved_buffer = b;
+	else
+		free_buffer(b);
+
+	wake_up(&c->free_buffer_wait);
+
+	cond_resched();
+}
+
+/*
+ * Check if we're over watermark.
+ * If we are over threshold_buffers, start freeing buffers.
+ * If we're over "limit_buffers", blocks until we get under the limit.
+ */
+
+static void check_watermark(struct dm_bufio_client *c)
+{
+	while (c->n_buffers > c->threshold_buffers) {
+		struct dm_buffer *b;
+		b = get_unclaimed_buffer(c, c->n_buffers > c->limit_buffers);
+		if (!b)
+			return;
+		free_buffer_wake(b);
+	}
+}
+
+static void dm_bufio_dmio_complete(unsigned long error, void *context);
+
+/*
+ * Submit I/O on the buffer.
+ *
+ * Bio interface is faster but it has some problems:
+ *	- the vector list is limited (increasing this limit increases
+ *		memory-consumption per buffer, so it is not viable)
+ *	- the memory must be direct-mapped, not vmallocated
+ *	- the I/O driver can spuriously reject requests if it thinks that
+ *		the requests are too big for the device or if they cross a
+ *		controller-defined memory boundary
+ *
+ * If the buffer is small enough (up to DM_BUFIO_INLINE_VECS pages) and
+ * it is not vmalloc()ated, try using the bio interface.
+ *
+ * If the buffer is big, if it is vmalloc()ated or if the underlying device
+ * rejects the bio because it is too large, use dmio layer to do the I/O.
+ * dmio layer splits the I/O to multiple requests, solving the above
+ * shorcomings.
+ */
+
+static void dm_bufio_submit_io(struct dm_buffer *b, int rw, sector_t block, bio_end_io_t *end_io)
+{
+	if (b->c->block_size <= DM_BUFIO_INLINE_VECS * PAGE_SIZE && b->data_mode != DATA_MODE_VMALLOC) {
+		char *ptr;
+		int len;
+		bio_init(&b->bio);
+		b->bio.bi_io_vec = b->bio_vec;
+		b->bio.bi_max_vecs = DM_BUFIO_INLINE_VECS;
+		b->bio.bi_sector = b->block << b->c->sectors_per_block_bits;
+		b->bio.bi_bdev = b->c->bdev;
+		b->bio.bi_end_io = end_io;
+
+		/*
+		 * we assume that if len >= PAGE_SIZE, ptr is page-aligned,
+		 * if len < PAGE_SIZE, the buffer doesn't cross page boundary.
+		 */
+		ptr = b->data;
+		len = b->c->block_size;
+		do {
+			if (!bio_add_page(&b->bio, virt_to_page(ptr), len < PAGE_SIZE ? len : PAGE_SIZE, virt_to_phys(ptr) & (PAGE_SIZE - 1))) {
+				BUG_ON(b->c->block_size <= PAGE_SIZE);
+				goto use_dmio;
+			}
+			len -= PAGE_SIZE;
+			ptr += PAGE_SIZE;
+		} while (len > 0);
+		submit_bio(rw, &b->bio);
+	} else
+use_dmio : {
+		int r;
+		struct dm_io_request io_req = {
+			.bi_rw = rw,
+			.notify.fn = dm_bufio_dmio_complete,
+			.notify.context = b,
+			.client = b->c->dm_io,
+		};
+		struct dm_io_region region = {
+			.bdev = b->c->bdev,
+			.sector = b->block << b->c->sectors_per_block_bits,
+			.count = b->c->block_size >> SECTOR_SHIFT,
+		};
+		if (b->data_mode != DATA_MODE_VMALLOC) {
+			io_req.mem.type = DM_IO_KMEM;
+			io_req.mem.ptr.addr = b->data;
+		} else {
+			io_req.mem.type = DM_IO_VMA;
+			io_req.mem.ptr.vma = b->data;
+		}
+		b->bio.bi_end_io = end_io;
+		r = dm_io(&io_req, 1, &region, NULL);
+		if (unlikely(r))
+			end_io(&b->bio, r);
+	}
+}
+
+/*
+ * dm-io completion routine. It just calls b->bio.bi_end_io, pretending
+ * that the request was handled directly with bio interface.
+ */
+
+static void dm_bufio_dmio_complete(unsigned long error, void *context)
+{
+	struct dm_buffer *b = context;
+	int err = 0;
+	if (unlikely(error != 0))
+		err = -EIO;
+	b->bio.bi_end_io(&b->bio, err);
+}
+
+/* Find a buffer in the hash. */
+
+static struct dm_buffer *dm_bufio_find(struct dm_bufio_client *c, sector_t block)
+{
+	struct dm_buffer *b;
+	struct hlist_node *hn;
+	hlist_for_each_entry(b, hn, &c->cache_hash[DM_BUFIO_HASH(block)], hash_list) {
+		cond_resched();
+		if (b->block == block)
+			return b;
+	}
+
+	return NULL;
+}
+
+static void read_endio(struct bio *bio, int error);
+
+/*
+ * A common routine for dm_bufio_new and dm_bufio_read.
+ * Operation of these function is very similar, except that dm_bufio_new
+ * doesn't read the buffer from the disk (assuming that the caller overwrites
+ * all the data and uses dm_bufio_mark_buffer_dirty to write new data back).
+ */
+
+static void *dm_bufio_new_read(struct dm_bufio_client *c, sector_t block, struct dm_buffer **bp, int read)
+{
+	struct dm_buffer *b, *new_b = NULL;
+
+	cond_resched();
+	mutex_lock(&c->lock);
+retry_search:
+	b = dm_bufio_find(c, block);
+	if (b) {
+		if (new_b)
+			free_buffer_wake(new_b);
+		b->hold_count++;
+		relink_lru(b, test_bit(B_DIRTY, &b->state) || test_bit(B_WRITING, &b->state));
+unlock_wait_ret:
+		mutex_unlock(&c->lock);
+wait_ret:
+		wait_on_bit(&b->state, B_READING, do_io_schedule, TASK_UNINTERRUPTIBLE);
+		if (b->read_error) {
+			int error = b->read_error;
+			dm_bufio_release(b);
+			return ERR_PTR(error);
+		}
+		*bp = b;
+		return b->data;
+	}
+	if (!new_b) {
+		new_b = alloc_buffer_wait(c);
+		goto retry_search;
+	}
+
+	check_watermark(c);
+
+	b = new_b;
+	b->hold_count = 1;
+	b->read_error = 0;
+	b->write_error = 0;
+	link_buffer(b, block, 0);
+
+	if (!read) {
+		b->state = 0;
+		goto unlock_wait_ret;
+	}
+
+	b->state = 1 << B_READING;
+
+	mutex_unlock(&c->lock);
+
+	dm_bufio_submit_io(b, READ, b->block, read_endio);
+
+	goto wait_ret;
+}
+
+/* Read the buffer and hold reference on it */
+
+void *dm_bufio_read(struct dm_bufio_client *c, sector_t block, struct dm_buffer **bp)
+{
+	return dm_bufio_new_read(c, block, bp, 1);
+}
+EXPORT_SYMBOL(dm_bufio_read);
+
+/* Get the buffer with possibly invalid data and hold reference on it */
+
+void *dm_bufio_new(struct dm_bufio_client *c, sector_t block, struct dm_buffer **bp)
+{
+	return dm_bufio_new_read(c, block, bp, 0);
+}
+EXPORT_SYMBOL(dm_bufio_new);
+
+/*
+ * The endio routine for reading: set the error, clear the bit and wake up
+ * anyone waiting on the buffer.
+ */
+
+static void read_endio(struct bio *bio, int error)
+{
+	struct dm_buffer *b = container_of(bio, struct dm_buffer, bio);
+	b->read_error = error;
+	BUG_ON(!test_bit(B_READING, &b->state));
+	smp_mb__before_clear_bit();
+	clear_bit(B_READING, &b->state);
+	smp_mb__after_clear_bit();
+	wake_up_bit(&b->state, B_READING);
+}
+
+/*
+ * Release the reference held on the buffer.
+ */
+
+void dm_bufio_release(struct dm_buffer *b)
+{
+	struct dm_bufio_client *c = b->c;
+	mutex_lock(&c->lock);
+	BUG_ON(!b->hold_count);
+	BUG_ON(test_bit(B_READING, &b->state));
+	b->hold_count--;
+	if (!b->hold_count) {
+		wake_up(&c->free_buffer_wait);
+		/*
+		 * If there were errors on the buffer, and the buffer is not
+		 * to be written, free the buffer. There is no point in caching
+		 * invalid buffer.
+		 */
+		if ((b->read_error || b->write_error) &&
+		    !test_bit(B_WRITING, &b->state) &&
+		    !test_bit(B_DIRTY, &b->state)) {
+			unlink_buffer(b);
+			free_buffer_wake(b);
+		}
+	}
+	mutex_unlock(&c->lock);
+}
+EXPORT_SYMBOL(dm_bufio_release);
+
+/*
+ * Mark that the data in the buffer were modified and the buffer needs to
+ * be written back.
+ */
+
+void dm_bufio_mark_buffer_dirty(struct dm_buffer *b)
+{
+	struct dm_bufio_client *c = b->c;
+
+	mutex_lock(&c->lock);
+
+	if (!test_and_set_bit(B_DIRTY, &b->state))
+		relink_lru(b, 1);
+
+	mutex_unlock(&c->lock);
+}
+EXPORT_SYMBOL(dm_bufio_mark_buffer_dirty);
+
+static void write_endio(struct bio *bio, int error);
+
+/*
+ * Initiate a write on a dirty buffer, but don't wait for it.
+ * If the buffer is not dirty, exit.
+ * If there some previous write going on, wait for it to finish (we can't
+ * have two writes on the same buffer simultaneously).
+ * Finally, submit our write and don't wait on it. We set B_WRITING indicating
+ * that there is a write in progress.
+ */
+
+static void write_dirty_buffer(struct dm_buffer *b)
+{
+	if (!test_bit(B_DIRTY, &b->state))
+		return;
+	clear_bit(B_DIRTY, &b->state);
+	wait_on_bit_lock(&b->state, B_WRITING, do_io_schedule, TASK_UNINTERRUPTIBLE);
+	dm_bufio_submit_io(b, WRITE, b->block, write_endio);
+}
+
+/*
+ * The endio routine for write.
+ * Set the error, clear B_WRITING bit and wake anyone who was waiting on it.
+ */
+
+static void write_endio(struct bio *bio, int error)
+{
+	struct dm_buffer *b = container_of(bio, struct dm_buffer, bio);
+	b->write_error = error;
+	if (unlikely(error)) {
+		struct dm_bufio_client *c = b->c;
+		cmpxchg(&c->async_write_error, 0, error);
+	}
+	BUG_ON(!test_bit(B_WRITING, &b->state));
+	smp_mb__before_clear_bit();
+	clear_bit(B_WRITING, &b->state);
+	smp_mb__after_clear_bit();
+	wake_up_bit(&b->state, B_WRITING);
+}
+
+/*
+ * Write all the dirty buffers asynchronously.
+ */
+
+static void write_dirty_buffers_async(struct dm_bufio_client *c)
+{
+	struct dm_buffer *b;
+	list_for_each_entry_reverse(b, &c->dirty_lru, lru_list) {
+		cond_resched();
+		BUG_ON(test_bit(B_READING, &b->state));
+		write_dirty_buffer(b);
+	}
+}
+
+/*
+ * Write all the dirty buffers synchronously.
+ * For performance, it is essential that the buffers are written asynchronously
+ * and simultaneously (so that the block layer can merge the writes) and then
+ * waited upon.
+ *
+ * Finally, we flush hardware disk cache.
+ */
+
+int dm_bufio_write_dirty_buffers(struct dm_bufio_client *c)
+{
+	int a, f;
+
+	struct dm_buffer *b, *tmp;
+	mutex_lock(&c->lock);
+	write_dirty_buffers_async(c);
+	mutex_unlock(&c->lock);
+	mutex_lock(&c->lock);
+	list_for_each_entry_safe_reverse(b, tmp, &c->dirty_lru, lru_list) {
+		cond_resched();
+		BUG_ON(test_bit(B_READING, &b->state));
+		if (test_bit(B_WRITING, &b->state)) {
+			b->hold_count++;
+			mutex_unlock(&c->lock);
+			wait_on_bit(&b->state, B_WRITING, do_io_schedule, TASK_UNINTERRUPTIBLE);
+			mutex_lock(&c->lock);
+			b->hold_count--;
+		}
+		if (!test_bit(B_DIRTY, &b->state) && !test_bit(B_WRITING, &b->state))
+			relink_lru(b, 0);
+	}
+	wake_up(&c->free_buffer_wait);
+	mutex_unlock(&c->lock);
+
+	a = xchg(&c->async_write_error, 0);
+	f = dm_bufio_issue_flush(c);
+	if (unlikely(a))
+		return a;
+	return f;
+}
+EXPORT_SYMBOL(dm_bufio_write_dirty_buffers);
+
+/*
+ * Use dm-io to send and empty barrier flush the device.
+ */
+
+int dm_bufio_issue_flush(struct dm_bufio_client *c)
+{
+	struct dm_io_request io_req = {
+		.bi_rw = WRITE_BARRIER,
+		.mem.type = DM_IO_KMEM,
+		.mem.ptr.bvec = NULL,
+		.client = c->dm_io,
+	};
+	struct dm_io_region io_reg = {
+		.bdev = c->bdev,
+		.sector = 0,
+		.count = 0,
+	};
+	return dm_io(&io_req, 1, &io_reg, NULL);
+}
+EXPORT_SYMBOL(dm_bufio_issue_flush);
+
+/*
+ * Release the buffer and copy it to the new location.
+ *
+ * We first delete any other buffer that may be at that new location.
+ *
+ * Then, we write the buffer to the original location if it was dirty.
+ *
+ * Then, if we are the only one who is holding the buffer, relink the buffer
+ * in the hash queue for the new location.
+ *
+ * If there was someone other holding the buffer, we write it to the new
+ * location but not relink it, because that other user needs to have the buffer
+ * at the same place.
+ */
+
+void dm_bufio_release_move(struct dm_buffer *b, sector_t new_block)
+{
+	struct dm_bufio_client *c = b->c;
+	struct dm_buffer *underlying;
+
+	mutex_lock(&c->lock);
+
+retry:
+	underlying = dm_bufio_find(c, new_block);
+	if (unlikely(underlying != NULL)) {
+		if (underlying->hold_count) {
+			wait_for_free_buffer(c);
+			goto retry;
+		}
+		make_buffer_clean(underlying);
+		unlink_buffer(underlying);
+		free_buffer_wake(underlying);
+	}
+
+	BUG_ON(!b->hold_count);
+	BUG_ON(test_bit(B_READING, &b->state));
+	write_dirty_buffer(b);
+	if (b->hold_count == 1) {
+		wait_on_bit(&b->state, B_WRITING, do_io_schedule, TASK_UNINTERRUPTIBLE);
+		set_bit(B_DIRTY, &b->state);
+		unlink_buffer(b);
+		link_buffer(b, new_block, 1);
+	} else {
+		wait_on_bit_lock(&b->state, B_WRITING, do_io_schedule, TASK_UNINTERRUPTIBLE);
+		dm_bufio_submit_io(b, WRITE, new_block, write_endio);
+		wait_on_bit(&b->state, B_WRITING, do_io_schedule, TASK_UNINTERRUPTIBLE);
+	}
+	mutex_unlock(&c->lock);
+	dm_bufio_release(b);
+}
+EXPORT_SYMBOL(dm_bufio_release_move);
+
+/*
+ * Free all the buffers (and possibly write them if they were dirty)
+ * It is required that the calling theread doesn't have any reference on
+ * any buffer.
+ */
+
+void dm_bufio_drop_buffers(struct dm_bufio_client *c)
+{
+	struct dm_buffer *b;
+	mutex_lock(&c->lock);
+	write_dirty_buffers_async(c);
+	while ((b = get_unclaimed_buffer(c, 1)))
+		free_buffer_wake(b);
+	BUG_ON(!list_empty(&c->lru));
+	BUG_ON(!list_empty(&c->dirty_lru));
+	mutex_unlock(&c->lock);
+}
+EXPORT_SYMBOL(dm_bufio_drop_buffers);
+
+/* Create the buffering interface */
+
+struct dm_bufio_client *dm_bufio_client_create(struct block_device *bdev, unsigned block_size, unsigned flags, __u64 cache_threshold, __u64 cache_limit)
+{
+	int r;
+	struct dm_bufio_client *c;
+	unsigned i;
+
+	BUG_ON(block_size < 1 << SECTOR_SHIFT || (block_size & (block_size - 1)));
+
+	c = kmalloc(sizeof(*c), GFP_KERNEL);
+	if (!c) {
+		r = -ENOMEM;
+		goto bad_client;
+	}
+
+	c->bdev = bdev;
+	c->block_size = block_size;
+	c->sectors_per_block_bits = ffs(block_size) - 1 - SECTOR_SHIFT;
+	c->pages_per_block_bits = ffs(block_size) - 1 >= PAGE_SHIFT ? ffs(block_size) - 1 - PAGE_SHIFT : 0;
+	INIT_LIST_HEAD(&c->lru);
+	INIT_LIST_HEAD(&c->dirty_lru);
+	for (i = 0; i < DM_BUFIO_HASH_SIZE; i++)
+		INIT_HLIST_HEAD(&c->cache_hash[i]);
+	mutex_init(&c->lock);
+	c->n_buffers = 0;
+
+	if (!cache_limit)
+		cache_limit = DM_BUFIO_LIMIT_MEMORY;
+	c->limit_buffers = cache_limit >> (c->sectors_per_block_bits + SECTOR_SHIFT);
+	if (!c->limit_buffers)
+		c->limit_buffers = 1;
+
+	if (!cache_threshold)
+		cache_threshold = DM_BUFIO_THRESHOLD_MEMORY;
+	if (cache_threshold > cache_limit)
+		cache_threshold = cache_limit;
+	c->threshold_buffers = cache_threshold >> (c->sectors_per_block_bits + SECTOR_SHIFT);
+	if (!c->threshold_buffers)
+		c->threshold_buffers = 1;
+
+	/*printk("%d %d\n", c->limit_buffers, c->threshold_buffers);*/
+
+	init_waitqueue_head(&c->free_buffer_wait);
+	c->async_write_error = 0;
+
+	/* Number of pages is not really hard limit, just a mempool size */
+	c->dm_io = dm_io_client_create((block_size + PAGE_SIZE - 1) / PAGE_SIZE);
+	if (IS_ERR(c->dm_io)) {
+		r = PTR_ERR(c->dm_io);
+		goto bad_dm_io;
+	}
+
+	c->reserved_buffer = alloc_buffer(c, GFP_KERNEL);
+	if (!c->reserved_buffer) {
+		r = -ENOMEM;
+		goto bad_buffer;
+	}
+
+	return c;
+
+bad_buffer:
+	dm_io_client_destroy(c->dm_io);
+bad_dm_io:
+	kfree(c);
+bad_client:
+	return ERR_PTR(r);
+}
+EXPORT_SYMBOL(dm_bufio_client_create);
+
+/*
+ * Free the buffering interface.
+ * It is required that there are no references on any buffers.
+ */
+
+void dm_bufio_client_destroy(struct dm_bufio_client *c)
+{
+	unsigned i;
+	dm_bufio_drop_buffers(c);
+	for (i = 0; i < DM_BUFIO_HASH_SIZE; i++)
+		BUG_ON(!hlist_empty(&c->cache_hash[i]));
+	BUG_ON(!c->reserved_buffer);
+	free_buffer(c->reserved_buffer);
+	BUG_ON(c->n_buffers != 0);
+	dm_io_client_destroy(c->dm_io);
+	kfree(c);
+}
+EXPORT_SYMBOL(dm_bufio_client_destroy);
Index: linux-2.6.32/drivers/md/dm-bufio.h
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-bufio.h
@@ -0,0 +1,29 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#ifndef DM_BUFIO_H
+#define DM_BUFIO_H
+
+struct dm_bufio_client;
+struct dm_buffer;
+
+void *dm_bufio_read(struct dm_bufio_client *c, sector_t block, struct dm_buffer **bp);
+void *dm_bufio_new(struct dm_bufio_client *c, sector_t block, struct dm_buffer **bp);
+void dm_bufio_release(struct dm_buffer *b);
+
+void dm_bufio_mark_buffer_dirty(struct dm_buffer *b);
+int dm_bufio_write_dirty_buffers(struct dm_bufio_client *c);
+int dm_bufio_issue_flush(struct dm_bufio_client *c);
+
+void dm_bufio_release_move(struct dm_buffer *b, sector_t new_block);
+
+struct dm_bufio_client *dm_bufio_client_create(struct block_device *bdev, unsigned block_size, unsigned flags, __u64 cache_threshold, __u64 cache_limit);
+void dm_bufio_client_destroy(struct dm_bufio_client *c);
+void dm_bufio_drop_buffers(struct dm_bufio_client *c);
+
+#endif
Index: linux-2.6.32/drivers/md/dm-multisnap-alloc.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-alloc.c
@@ -0,0 +1,576 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-mikulas.h"
+
+#define rshift_roundup(val, bits)	(((val) + ((chunk_t)1 << (bits)) - 1) >> (bits))
+
+#define BITS_PER_BYTE_SHIFT	3
+#define BYTES_PER_POINTER_SHIFT	3
+
+/*
+ * Initialize bitmaps, write them from the position "writing block".
+ */
+
+void dm_multisnap_create_bitmaps(struct dm_exception_store *s, chunk_t writing_block)
+{
+	int r;
+	struct dm_buffer *bp;
+	chunk_t direct_bitmap_blocks, total_bitmap_blocks, total_preallocated_blocks;
+	chunk_t lower_depth_block;
+	unsigned i, d;
+	chunk_t ii;
+
+	r = dm_multisnap_bitmap_depth(s->chunk_shift, s->dev_size);
+	if (r < 0) {
+		DMERR("dm_multisnap_create_bitmaps: device is too large");
+		dm_multisnap_set_error(s->dm, r);
+		return;
+	}
+	s->bitmap_depth = r;
+
+	direct_bitmap_blocks = rshift_roundup(s->dev_size, s->chunk_shift + BITS_PER_BYTE_SHIFT);
+
+	if (direct_bitmap_blocks > CB_BITMAP_IDX_MAX) {
+		DMERR("dm_multisnap_create_bitmaps: device is too large");
+		dm_multisnap_set_error(s->dm, -ERANGE);
+		return;
+	}
+
+	total_bitmap_blocks = 0;
+	for (i = 0; i <= s->bitmap_depth; i++) {
+		unsigned shift = (s->chunk_shift - BYTES_PER_POINTER_SHIFT) * i;
+		total_bitmap_blocks += rshift_roundup(direct_bitmap_blocks, shift);
+	}
+	total_preallocated_blocks = writing_block + total_bitmap_blocks;
+	for (ii = 0; ii < total_preallocated_blocks; ii++) {
+		if (dm_multisnap_is_commit_block(s, ii))
+			total_preallocated_blocks++;
+	}
+
+	if (total_preallocated_blocks >= s->dev_size) {
+		DMERR("dm_multisnap_create_bitmaps: device is too small");
+		dm_multisnap_set_error(s->dm, -ENOSPC);
+		return;
+	}
+
+/* Write direct bitmap blocks */
+
+	lower_depth_block = writing_block;
+	for (ii = 0; ii < direct_bitmap_blocks; ii++, writing_block++) {
+		void *bmp;
+		while (dm_multisnap_is_commit_block(s, writing_block))
+			writing_block++;
+		bmp = dm_bufio_new(s->bufio, writing_block, &bp);
+		if (IS_ERR(bmp)) {
+			DMERR("dm_multisnap_create_bitmaps: can't create direct bitmap block at %llx", (unsigned long long)writing_block);
+			dm_multisnap_set_error(s->dm, PTR_ERR(bmp));
+			return;
+		}
+		cond_resched();
+		memset(bmp, 0, s->chunk_size);
+		cond_resched();
+		for (i = 0; i < s->chunk_size << BITS_PER_BYTE_SHIFT; i++) {
+			chunk_t block_to_test = (ii << (s->chunk_shift + BITS_PER_BYTE_SHIFT)) | i;
+			if (block_to_test >= s->dev_size) {
+				generic___set_le_bit(i, bmp);
+			} else if (block_to_test < total_preallocated_blocks || dm_multisnap_is_commit_block(s, block_to_test)) {
+				generic___set_le_bit(i, bmp);
+				dm_multisnap_status_lock(s->dm);
+				s->total_allocated++;
+				dm_multisnap_status_unlock(s->dm);
+			}
+			cond_resched();
+		}
+		dm_bufio_mark_buffer_dirty(bp);
+		dm_bufio_release(bp);
+	}
+
+/* Write indirect bitmap blocks */
+
+	for (d = 1; d <= s->bitmap_depth; d++) {
+		chunk_t this_depth_block = writing_block;
+		for (ii = 0; ii < rshift_roundup(direct_bitmap_blocks, d * (s->chunk_shift - BYTES_PER_POINTER_SHIFT)); ii++, writing_block++) {
+			__u64 *bmp;
+			while (dm_multisnap_is_commit_block(s, writing_block))
+				writing_block++;
+			bmp = dm_bufio_new(s->bufio, writing_block, &bp);
+			if (IS_ERR(bmp)) {
+				DMERR("dm_multisnap_create_bitmaps: can't create indirect bitmap block at %llx", (unsigned long long)writing_block);
+				dm_multisnap_set_error(s->dm, PTR_ERR(bmp));
+				return;
+			}
+			for (i = 0; i < s->chunk_size >> BYTES_PER_POINTER_SHIFT; i++) {
+				if (((ii << d * (s->chunk_shift - BYTES_PER_POINTER_SHIFT)) | (i << (d - 1) * (s->chunk_shift - BYTES_PER_POINTER_SHIFT))) >= direct_bitmap_blocks) {
+					bmp[i] = cpu_to_le64(0);
+					continue;
+				}
+				while (dm_multisnap_is_commit_block(s, lower_depth_block))
+					lower_depth_block++;
+				bmp[i] = cpu_to_le64(lower_depth_block);
+				lower_depth_block++;
+			}
+			dm_bufio_mark_buffer_dirty(bp);
+			dm_bufio_release(bp);
+		}
+		lower_depth_block = this_depth_block;
+	}
+
+	s->bitmap_root = writing_block - 1;
+}
+
+static void dm_multisnap_add_bitmap(struct dm_exception_store *s);
+
+void dm_multisnap_extend_bitmaps(struct dm_exception_store *s, chunk_t new_size)
+{
+	while (s->dev_size < new_size) {
+		struct dm_buffer *bp;
+		void *bmp;
+		bitmap_t bitmap_no = s->dev_size >> (s->chunk_shift + BITS_PER_BYTE_SHIFT);
+		unsigned i = s->dev_size & ((1 << (s->chunk_shift + BITS_PER_BYTE_SHIFT)) - 1);
+		chunk_t c = s->dev_size;
+		if (!i) {
+			dm_multisnap_add_bitmap(s);
+			if (unlikely(dm_multisnap_has_error(s->dm)))
+				return;
+		}
+		bmp = dm_multisnap_map_bitmap(s, bitmap_no, &bp, NULL, NULL);
+		if (unlikely(!bmp))
+			return;
+		for (; i < s->chunk_size << BITS_PER_BYTE_SHIFT; i++, c++) {
+			if (unlikely(dm_multisnap_is_commit_block(s, c)))
+				generic___set_le_bit(i, bmp);
+			else
+				generic___clear_le_bit(i, bmp);
+		}
+		dm_bufio_mark_buffer_dirty(bp);
+		dm_bufio_release(bp);
+
+		s->dev_size = ((chunk_t)bitmap_no + 1) << (s->chunk_shift + BITS_PER_BYTE_SHIFT);
+		if (s->dev_size > new_size)
+			s->dev_size = new_size;
+	}
+}
+
+static void dm_multisnap_add_bitmap(struct dm_exception_store *s)
+{
+	struct path_element path[MAX_BITMAP_DEPTH];
+	struct dm_buffer *bp;
+	int d;
+	__u64 *bmpp;
+	unsigned i;
+	chunk_t c, bitmap_blk, new_blk;
+	bitmap_t bitmap_no = s->dev_size >> (s->chunk_shift + BITS_PER_BYTE_SHIFT);
+	void *bmp = dm_multisnap_alloc_make_block(s, &bitmap_blk, &bp);
+	if (!bmp)
+		return;
+	c = (chunk_t)bitmap_no << (s->chunk_shift + BITS_PER_BYTE_SHIFT);
+	for (i = 0; i < s->chunk_size << BITS_PER_BYTE_SHIFT; i++, c++) {
+		if (unlikely(dm_multisnap_is_commit_block(s, c)))
+			generic___set_le_bit(i, bmp);
+		else
+			generic___clear_le_bit(i, bmp);
+	}
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+
+	/* just get the path to the last block */
+	bmp = dm_multisnap_map_bitmap(s, bitmap_no - 1, &bp, NULL, path);
+	if (unlikely(!bmp))
+		return;
+	dm_bufio_release(bp);
+
+	for (d = s->bitmap_depth - 1; d >= 0; d--) {
+		if (path[d].idx + 1 < path[d].n_entries) {
+			__u64 *bmpp = dm_multisnap_read_block(s, path[d].block, &bp);
+			if (!bmpp)
+				return;
+			bmpp[path[d].idx + 1] = cpu_to_le64(bitmap_blk);
+			dm_bufio_mark_buffer_dirty(bp);
+			dm_bufio_release(bp);
+			return;
+		} else {
+			bmpp = dm_multisnap_alloc_make_block(s, &new_blk, &bp);
+			if (!bmpp)
+				return;
+			memset(bmpp, 0, s->chunk_size);
+			bmpp[0] = cpu_to_le64(bitmap_blk);
+			dm_bufio_mark_buffer_dirty(bp);
+			dm_bufio_release(bp);
+			bitmap_blk = new_blk;
+		}
+	}
+
+	/* make new root */
+	bmpp = dm_multisnap_alloc_make_block(s, &new_blk, &bp);
+	if (!bmpp)
+		return;
+	memset(bmpp, 0, s->chunk_size);
+	bmpp[0] = cpu_to_le64(s->bitmap_root);
+	bmpp[1] = cpu_to_le64(bitmap_blk);
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+	s->bitmap_root = new_blk;
+	s->bitmap_depth++;
+}
+
+/*
+ * Read a leaf bitmap node with index "bitmap".
+ * Return the pointer to the data, store the held buffer to bl.
+ * Return the block in block and path in path.
+ */
+
+void *dm_multisnap_map_bitmap(struct dm_exception_store *s, bitmap_t bitmap, struct dm_buffer **bp, chunk_t *block, struct path_element *path)
+{
+	__u64 *bmp;
+	unsigned idx;
+	unsigned d = s->bitmap_depth;
+	chunk_t blk = s->bitmap_root;
+	chunk_t parent = 0;
+
+	while (1) {
+		bmp = dm_multisnap_read_block(s, blk, bp);
+		if (unlikely(!bmp)) {
+			DMERR("dm_multisnap_map_bitmap: can't read bitmap at %llx (%llx), pointed to by %llx (%llx), depth %d/%d, index %llx",
+				(unsigned long long)blk,
+				(unsigned long long)dm_multisnap_remap_block(s, blk),
+				(unsigned long long)parent,
+				(unsigned long long)dm_multisnap_remap_block(s, parent),
+				s->bitmap_depth - d,
+				s->bitmap_depth,
+				(unsigned long long)bitmap);
+			return NULL;
+		}
+		if (!d) {
+			if (block)
+				*block = blk;
+			return bmp;
+		}
+
+		idx = (bitmap >> ((d - 1) * (s->chunk_shift - BYTES_PER_POINTER_SHIFT))) & ((s->chunk_size - 1) >> BYTES_PER_POINTER_SHIFT);
+
+		if (unlikely(path != NULL)) {
+			path[s->bitmap_depth - d].block = blk;
+			path[s->bitmap_depth - d].idx = idx;
+			path[s->bitmap_depth - d].n_entries = s->chunk_size >> BYTES_PER_POINTER_SHIFT;
+		}
+
+		parent = blk;
+		blk = le64_to_cpu(bmp[idx]);
+
+		dm_bufio_release(*bp);
+
+		d--;
+	}
+}
+
+/*
+ * Find a free bit from "start" to "end" (in bits).
+ * If wide_search is nonzero, search for the whole free byte first.
+ */
+
+static int find_bit(const void *bmp, unsigned start, unsigned end, int wide_search)
+{
+	const void *p;
+	unsigned bit;
+	if (unlikely(start >= end))
+		return -ENOSPC;
+	cond_resched();
+	if (likely(!generic_test_le_bit(start, bmp)))
+		return start;
+	if (likely(wide_search)) {
+		cond_resched();
+		p = memchr(bmp + (start >> 3), 0, (end >> 3) - (start >> 3));
+		cond_resched();
+		if (p) {
+			bit = (((const __u8 *)p - (const __u8 *)bmp) << 3) | (start & 7);
+			while (bit > start && !generic_test_le_bit(bit - 1, bmp))
+				bit--;
+			goto ret_bit;
+		}
+	}
+	bit = generic_find_next_zero_le_bit(bmp, end, start);
+	cond_resched();
+
+ret_bit:
+	if (unlikely(bit >= end))
+		return -ENOSPC;
+	return bit;
+}
+
+static unsigned bitmap_limit(struct dm_exception_store *s, bitmap_t bmp)
+{
+	if (bmp == (bitmap_t)(s->dev_size >> (s->chunk_shift + BITS_PER_BYTE_SHIFT)))
+		return (unsigned)s->dev_size & ((s->chunk_size << BITS_PER_BYTE_SHIFT) - 1);
+	return s->chunk_size << BITS_PER_BYTE_SHIFT;
+}
+
+int dm_multisnap_alloc_blocks(struct dm_exception_store *s, chunk_t *results, unsigned n_blocks, int flags)
+{
+	void *bmp;
+	struct dm_buffer *bp;
+	chunk_t block;
+	int wrap_around = 0;
+	int start_bit;
+	int wide_search;
+	int i;
+	bitmap_t bitmap_no;
+	int c;
+	int bit;
+	chunk_t to_free = 0;
+
+	bitmap_no = s->alloc_rover >> (s->chunk_shift + BITS_PER_BYTE_SHIFT);
+next_bitmap:
+	bmp = dm_multisnap_map_bitmap(s, bitmap_no, &bp, &block, NULL);
+	if (unlikely(!bmp))
+		return -1;
+
+	wide_search = 1;
+find_again:
+	start_bit = s->alloc_rover & ((s->chunk_size << BITS_PER_BYTE_SHIFT) - 1);
+
+	for (i = 0; i < n_blocks; i++) {
+find_another_bit:
+		bit = find_bit(bmp, start_bit, bitmap_limit(s, bitmap_no), wide_search);
+		if (unlikely(bit < 0)) {
+bit_find_failed:
+			if (wide_search) {
+				wide_search = 0;
+				goto find_again;
+			}
+			dm_bufio_release(bp);
+			s->alloc_rover = (chunk_t) ++bitmap_no << (s->chunk_shift + BITS_PER_BYTE_SHIFT);
+			if (unlikely(s->alloc_rover >= s->dev_size)) {
+				s->alloc_rover = 0;
+				bitmap_no = 0;
+				wrap_around++;
+				if (wrap_around >= 2) {
+					DMERR("snapshot overflow");
+					dm_multisnap_set_error(s->dm, -ENOSPC);
+					return -1;
+				}
+			}
+			goto next_bitmap;
+		}
+		results[i] = ((chunk_t)bitmap_no << (s->chunk_shift + BITS_PER_BYTE_SHIFT)) | bit;
+		start_bit = bit + 1;
+		dm_bufio_release(bp);
+
+		c = dm_multisnap_check_allocated_block(s, results[i]);
+		if (dm_multisnap_has_error(s->dm))
+			return -1;
+
+		bmp = dm_multisnap_read_block(s, block, &bp);
+		if (unlikely(!bmp))
+			return -1;
+
+		if (c)
+			goto find_another_bit;
+	}
+
+	if (flags & ALLOC_DRY)
+		goto bp_release_return;
+
+	if (!dm_multisnap_block_is_uncommitted(s, block)) {
+		chunk_t new_block;
+find_another_bit_for_bitmap:
+		bit = find_bit(bmp, start_bit, bitmap_limit(s, bitmap_no), wide_search);
+		if (unlikely(bit < 0))
+			goto bit_find_failed;
+
+		new_block = ((chunk_t)bitmap_no << (s->chunk_shift + BITS_PER_BYTE_SHIFT)) | bit;
+		start_bit = bit + 1;
+
+		dm_bufio_release(bp);
+		c = dm_multisnap_check_allocated_block(s, new_block);
+		if (dm_multisnap_has_error(s->dm))
+			return -1;
+
+		bmp = dm_multisnap_read_block(s, block, &bp);
+		if (unlikely(!bmp))
+			return -1;
+
+		if (c)
+			goto find_another_bit_for_bitmap;
+
+		/*
+		 * Warning: record the address of a block to free in a special
+		 * variable.
+		 *
+		 * If we freed it here, that could recurse back to
+		 * dm_multisnap_alloc_blocks and corrupt allocations. Free it
+		 * later when we are done with the allocation and all the
+		 * allocated blocks are marked in the bitmap.
+		 */
+		bmp = dm_multisnap_duplicate_block(s, block, new_block, bitmap_no, &bp, &to_free);
+		if (unlikely(!bmp))
+			return -1;
+
+		generic___set_le_bit(bit, bmp);
+		dm_multisnap_status_lock(s->dm);
+		s->total_allocated++;
+		dm_multisnap_status_unlock(s->dm);
+	}
+
+	for (i = 0; i < n_blocks; i++)
+		generic___set_le_bit(results[i] & ((s->chunk_size << BITS_PER_BYTE_SHIFT) - 1), bmp);
+	dm_multisnap_status_lock(s->dm);
+	s->total_allocated += n_blocks;
+	dm_multisnap_status_unlock(s->dm);
+
+	dm_bufio_mark_buffer_dirty(bp);
+
+bp_release_return:
+	dm_bufio_release(bp);
+
+	s->alloc_rover = (s->alloc_rover & ~(chunk_t)((s->chunk_size << BITS_PER_BYTE_SHIFT) - 1)) + start_bit;
+	if (unlikely(s->alloc_rover >= s->dev_size))
+		s->alloc_rover = 0;
+
+	if (unlikely(to_free != 0))
+		dm_multisnap_free_block(s, to_free, 0);
+
+	return 0;
+}
+
+void *dm_multisnap_alloc_duplicate_block(struct dm_exception_store *s, chunk_t block, struct dm_buffer **bp, void *ptr)
+{
+	int r;
+	chunk_t new_chunk;
+	void *data;
+
+	if (dm_multisnap_block_is_uncommitted(s, block))
+		return ptr;
+
+	dm_bufio_release(*bp);
+
+	r = dm_multisnap_alloc_blocks(s, &new_chunk, 1, 0);
+	if (r)
+		return NULL;
+
+	data = dm_multisnap_read_block(s, block, bp);
+	if (!data)
+		return NULL;
+
+	return dm_multisnap_duplicate_block(s, block, new_chunk, CB_BITMAP_IDX_NONE, bp, NULL);
+}
+
+void *dm_multisnap_alloc_make_block(struct dm_exception_store *s, chunk_t *result, struct dm_buffer **bp)
+{
+	int r = dm_multisnap_alloc_blocks(s, result, 1, 0);
+	if (unlikely(r < 0))
+		return NULL;
+
+	return dm_multisnap_make_block(s, *result, bp);
+}
+
+void dm_multisnap_free_blocks_immediate(struct dm_exception_store *s, chunk_t block, unsigned n_blocks)
+{
+	void *bmp;
+	struct dm_buffer *bp;
+
+	if (!n_blocks)
+		return;
+
+	if (unlikely(block + n_blocks > s->dev_size)) {
+		DMERR("dm_multisnap_free_block_immediate: freeing invalid blocks %llx, %x", (unsigned long long)block, n_blocks);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	if (block + n_blocks == s->alloc_rover)
+		s->alloc_rover = block;
+
+	do {
+		bitmap_t bitmap_no = block >> (s->chunk_shift + BITS_PER_BYTE_SHIFT);
+
+		bmp = dm_multisnap_map_bitmap(s, bitmap_no, &bp, NULL, NULL);
+		if (!bmp)
+			return;
+
+		do {
+			generic___clear_le_bit(block & ((s->chunk_size << BITS_PER_BYTE_SHIFT) - 1), bmp);
+			dm_multisnap_status_lock(s->dm);
+			s->total_allocated--;
+			dm_multisnap_status_unlock(s->dm);
+			n_blocks--;
+			block++;
+			cond_resched();
+		} while (n_blocks && (block & ((s->chunk_size << BITS_PER_BYTE_SHIFT) - 1)));
+
+		dm_bufio_mark_buffer_dirty(bp);
+		dm_bufio_release(bp);
+	} while (unlikely(n_blocks != 0));
+}
+
+
+void dm_multisnap_bitmap_finalize_tmp_remap(struct dm_exception_store *s, struct tmp_remap *tmp_remap)
+{
+	chunk_t block;
+	struct dm_buffer *bp;
+	__u64 *new_block;
+	struct path_element path[MAX_BITMAP_DEPTH];
+	int results_ptr;
+
+	chunk_t new_blockn;
+	int i;
+
+	/*
+	 * Preallocate twice the required amount of blocks, so that resolving
+	 * the next tmp_remap (created here, in dm_multisnap_alloc_blocks)
+	 * doesn't have to allocate anything.
+	 */
+	if (s->n_preallocated_blocks < s->bitmap_depth) {
+		if (unlikely(dm_multisnap_alloc_blocks(s, s->preallocated_blocks + s->n_preallocated_blocks, s->bitmap_depth * 2 - s->n_preallocated_blocks, 0) < 0))
+			return;
+		s->n_preallocated_blocks = s->bitmap_depth * 2;
+	}
+	results_ptr = 0;
+
+	new_block = dm_multisnap_map_bitmap(s, tmp_remap->bitmap_idx, &bp, &block, path);
+	if (unlikely(!new_block))
+		return;
+
+	dm_bufio_release(bp);
+
+	new_blockn = tmp_remap->new;
+	for (i = s->bitmap_depth - 1; i >= 0; i--) {
+		chunk_t block_to_free;
+		int remapped = 0;
+		__u64 *bmp = dm_multisnap_read_block(s, path[i].block, &bp);
+		if (unlikely(!bmp))
+			return;
+
+		if (!dm_multisnap_block_is_uncommitted(s, path[i].block)) {
+			remapped = 1;
+			dm_bufio_release_move(bp, s->preallocated_blocks[results_ptr]);
+			bmp = dm_multisnap_read_block(s, s->preallocated_blocks[results_ptr], &bp);
+			if (!bmp)
+				return;
+			/* !!! TODO: add to a list of newly allocated blocks */
+		}
+
+		block_to_free = le64_to_cpu(bmp[path[i].idx]);
+		bmp[path[i].idx] = cpu_to_le64(new_blockn);
+		dm_bufio_mark_buffer_dirty(bp);
+		dm_bufio_release(bp);
+
+		dm_multisnap_free_block(s, block_to_free, 0);
+
+		if (!remapped)
+			goto skip_it;
+		new_blockn = s->preallocated_blocks[results_ptr];
+		results_ptr++;
+	}
+
+	dm_multisnap_free_block(s, s->bitmap_root, 0);
+	s->bitmap_root = new_blockn;
+
+skip_it:
+	memmove(s->preallocated_blocks, s->preallocated_blocks + results_ptr, (s->n_preallocated_blocks -= results_ptr) * sizeof(chunk_t));
+}
Index: linux-2.6.32/drivers/md/dm-multisnap-blocks.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-blocks.c
@@ -0,0 +1,198 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-mikulas.h"
+
+/*
+ * Check that the block is valid.
+ */
+static int check_invalid(struct dm_exception_store *s, chunk_t block)
+{
+	if (unlikely(block >= s->dev_size) ||
+	    unlikely(block == SB_BLOCK) ||
+	    unlikely(dm_multisnap_is_commit_block(s, block))) {
+		DMERR("check_invalid: access to invalid part of the device: %llx, size %llx", (unsigned long long)block, (unsigned long long)s->dev_size);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return 1;
+	}
+	return 0;
+}
+
+static struct tmp_remap *find_tmp_remap(struct dm_exception_store *s, chunk_t block)
+{
+	struct tmp_remap *t;
+	struct hlist_node *hn;
+	unsigned hash = TMP_REMAP_HASH(block);
+	hlist_for_each_entry(t, hn, &s->tmp_remap[hash], hash_list) {
+		if (t->old == block)
+			return t;
+		cond_resched();
+	}
+	return NULL;
+}
+
+chunk_t dm_multisnap_remap_block(struct dm_exception_store *s, chunk_t block)
+{
+	struct tmp_remap *t;
+	t = find_tmp_remap(s, block);
+	if (t)
+		return t->new;
+	return block;
+}
+
+void *dm_multisnap_read_block(struct dm_exception_store *s, chunk_t block, struct dm_buffer **bp)
+{
+	void *buf;
+	cond_resched();
+
+	if (check_invalid(s, block))
+		return NULL;
+
+	block = dm_multisnap_remap_block(s, block);
+
+	if (check_invalid(s, block))
+		return NULL;
+
+	buf = dm_bufio_read(s->bufio, block, bp);
+	if (unlikely(IS_ERR(buf))) {
+		DMERR("dm_multisnap_read_block: error read chunk %llx", (unsigned long long)block);
+		dm_multisnap_set_error(s->dm, PTR_ERR(buf));
+		return NULL;
+	}
+	return buf;
+}
+
+int dm_multisnap_block_is_uncommitted(struct dm_exception_store *s, chunk_t chunk)
+{
+	struct tmp_remap *t;
+	check_invalid(s, chunk);
+	t = find_tmp_remap(s, chunk);
+	return t && t->uncommitted;
+}
+
+void *dm_multisnap_duplicate_block(struct dm_exception_store *s, chunk_t old_chunk, chunk_t new_chunk, bitmap_t bitmap_idx, struct dm_buffer **bp, chunk_t *to_free_ptr)
+{
+	chunk_t to_free_val;
+	void *buf;
+	struct tmp_remap *t;
+
+	if (unlikely(check_invalid(s, old_chunk)) ||
+	    unlikely(check_invalid(s, new_chunk)))
+		return NULL;
+
+	if (!to_free_ptr)
+		to_free_ptr = &to_free_val;
+	*to_free_ptr = 0;
+
+	t = find_tmp_remap(s, old_chunk);
+	if (t) {
+		if (unlikely(t->bitmap_idx != bitmap_idx)) {
+			DMERR("dm_multisnap_duplicate_block: bitmap_idx doesn't match, %X != %X", t->bitmap_idx, bitmap_idx);
+			dm_multisnap_set_error(s->dm, -EFSERROR);
+			return NULL;
+		}
+		*to_free_ptr = t->new;
+		t->new = new_chunk;
+	} else {
+		if (unlikely(list_empty(&s->free_tmp_remaps))) {
+			DMERR("dm_multisnap_duplicate_block: all remap blocks used");
+			dm_multisnap_set_error(s->dm, -EFSERROR);
+			return NULL;
+		}
+		t = list_first_entry(&s->free_tmp_remaps, struct tmp_remap, list);
+		t->new = new_chunk;
+		t->old = old_chunk;
+		t->bitmap_idx = bitmap_idx;
+		hlist_add_head(&t->hash_list, &s->tmp_remap[TMP_REMAP_HASH(old_chunk)]);
+		s->n_used_tmp_remaps++;
+	}
+	list_del(&t->list);
+	if (bitmap_idx == CB_BITMAP_IDX_NONE)
+		list_add_tail(&t->list, &s->used_bt_tmp_remaps);
+	else
+		list_add_tail(&t->list, &s->used_bitmap_tmp_remaps);
+	t->uncommitted = 1;
+	dm_bufio_release_move(*bp, new_chunk);
+
+	if (to_free_ptr == &to_free_val && to_free_val)
+		dm_multisnap_free_block(s, to_free_val, 0);
+
+	buf = dm_bufio_read(s->bufio, new_chunk, bp);
+	if (IS_ERR(buf)) {
+		DMERR("dm_multisnap_duplicate_block: error reading chunk %llx", (unsigned long long)new_chunk);
+		dm_multisnap_set_error(s->dm, PTR_ERR(buf));
+		return NULL;
+	}
+	return buf;
+}
+
+void dm_multisnap_free_tmp_remap(struct dm_exception_store *s, struct tmp_remap *t)
+{
+	list_del(&t->list);
+	hlist_del(&t->hash_list);
+	s->n_used_tmp_remaps--;
+	list_add(&t->list, &s->free_tmp_remaps);
+}
+
+void *dm_multisnap_make_block(struct dm_exception_store *s, chunk_t new_chunk, struct dm_buffer **bp)
+{
+	void *buf;
+
+	if (unlikely(check_invalid(s, new_chunk)))
+		return NULL;
+
+	/* !!! TODO: add it to the list of recently allocated blocks */
+
+	buf = dm_bufio_new(s->bufio, new_chunk, bp);
+	if (unlikely(IS_ERR(buf))) {
+		DMERR("dm_multisnap_make_block: error creating new block at chunk %llx", (unsigned long long)new_chunk);
+		dm_multisnap_set_error(s->dm, PTR_ERR(buf));
+		return NULL;
+	}
+	return buf;
+}
+
+void dm_multisnap_free_block_and_duplicates(struct dm_exception_store *s, chunk_t chunk)
+{
+	struct tmp_remap *t;
+
+	if (unlikely(check_invalid(s, chunk)))
+		return;
+
+	t = find_tmp_remap(s, chunk);
+	if (t) {
+		dm_multisnap_free_block(s, t->new, 0);
+		dm_multisnap_free_tmp_remap(s, t);
+	}
+	dm_multisnap_free_block(s, chunk, 0);
+}
+
+int dm_multisnap_is_commit_block(struct dm_exception_store *s, chunk_t block)
+{
+	if (unlikely(block < FIRST_CB_BLOCK))
+		return 0;
+	if (likely(!(s->cb_stride & (s->cb_stride - 1))))
+		return (block & (s->cb_stride - 1)) == (FIRST_CB_BLOCK & (s->cb_stride - 1));
+	else
+		return sector_div(block, s->cb_stride) == FIRST_CB_BLOCK % s->cb_stride;
+}
+
+void dm_multisnap_init_stop_cycles(stop_cycles_t *cy)
+{
+	(*cy)[1] = 0;
+}
+
+int dm_multisnap_stop_cycles(struct dm_exception_store *s, stop_cycles_t *cy, chunk_t key)
+{
+	if (unlikely((*cy)[0] == key) && unlikely((*cy)[1] != 0)) {
+		DMERR("dm_multisnap_stop_cycles: cycle detected at chunk %llx", (unsigned long long)key);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return -1;
+	}
+	return 0;
+}
Index: linux-2.6.32/drivers/md/dm-multisnap-btree.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-btree.c
@@ -0,0 +1,798 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-mikulas.h"
+
+/*
+ * Read one btree node and do basic consistency checks.
+ * Any btree access should be done with this function.
+ */
+
+static struct dm_multisnap_bt_node *dm_multisnap_read_btnode(struct dm_exception_store *s, int depth, chunk_t block, unsigned want_entries, struct dm_buffer **bp)
+{
+	struct dm_multisnap_bt_node *node;
+
+	BUG_ON((unsigned)depth >= s->bt_depth);
+
+	node = dm_multisnap_read_block(s, block, bp);
+	if (unlikely(!node))
+		return NULL;
+
+	if (unlikely(node->signature != BT_SIGNATURE)) {
+		dm_bufio_release(*bp);
+		DMERR("dm_multisnap_read_btnode: bad signature on btree node %llx", (unsigned long long)block);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return NULL;
+	}
+
+	if (unlikely((unsigned)(le32_to_cpu(node->n_entries) - 1) >= s->btree_entries) ||
+	    (want_entries && unlikely(le32_to_cpu(node->n_entries) != want_entries))) {
+		dm_bufio_release(*bp);
+		DMERR("dm_multisnap_read_btnode: bad number of entries in btree node %llx: %x, wanted %x", (unsigned long long)block, le32_to_cpu(node->n_entries), want_entries);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return NULL;
+	}
+
+	return node;
+}
+
+/*
+ * I know they hate inline, but I still maintain the point that it is
+ * appropriate here.
+ */
+
+static inline void write_orig_chunk(struct dm_multisnap_bt_entry *be, chunk_t n)
+{
+	write_48(be, orig_chunk, n);
+	if (sizeof(chunk_t) == 4 && unlikely(n > CHUNK_T_MAX))
+		be->orig_chunk2 = cpu_to_le16(0xffff);
+}
+
+/*
+ * Add an entry (key, new_chunk) at an appropriate index to the btree node.
+ * Move the existing entries
+ */
+
+static void add_at_idx(struct dm_multisnap_bt_node *node, unsigned index, struct bt_key *key, chunk_t new_chunk)
+{
+	memmove(&node->entries[index + 1], &node->entries[index], (le32_to_cpu(node->n_entries) - index) * sizeof(struct dm_multisnap_bt_entry));
+	write_orig_chunk(&node->entries[index], key->chunk);
+	write_48(&node->entries[index], new_chunk, new_chunk);
+	node->entries[index].snap_from = cpu_to_mikulas_snapid(key->snap_from);
+	node->entries[index].snap_to = cpu_to_mikulas_snapid(key->snap_to);
+	node->entries[index].flags = cpu_to_le32(0);
+	node->n_entries = cpu_to_le32(le32_to_cpu(node->n_entries) + 1);
+}
+
+/*
+ * Create an initial btree.
+ * (*writing_block) is updated to point after the btree.
+ */
+
+void dm_multisnap_create_btree(struct dm_exception_store *s, chunk_t *writing_block)
+{
+	struct dm_buffer *bp;
+	struct dm_multisnap_bt_node *node;
+	struct bt_key new_key;
+
+	while (dm_multisnap_is_commit_block(s, *writing_block))
+		(*writing_block)++;
+
+	if (*writing_block >= s->dev_size) {
+		DMERR("dm_multisnap_create_btree: device is too small");
+		dm_multisnap_set_error(s->dm, -ENOSPC);
+		return;
+	}
+
+	node = dm_bufio_new(s->bufio, *writing_block, &bp);
+	if (IS_ERR(node)) {
+		DMERR("dm_multisnap_create_btree: 't create direct bitmap block at %llx", (unsigned long long)*writing_block);
+		dm_multisnap_set_error(s->dm, PTR_ERR(node));
+		return;
+	}
+	memset(node, 0, s->chunk_size);
+	node->signature = BT_SIGNATURE;
+	node->n_entries = cpu_to_le32(0);
+
+	/*
+	 * A btree node must have at least one entry --- so create this empty
+	 * one
+	 */
+	new_key.snap_from = new_key.snap_to = SNAPID_T_LAST;
+	new_key.chunk = CHUNK_T_LAST;
+	add_at_idx(node, 0, &new_key, 0);
+
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+	s->bt_root = *writing_block;
+	s->bt_depth = 1;
+	(*writing_block)++;
+}
+
+/*
+ * Compare btree entry and a search key. Returns:
+ *	-1: the entry is lower than the key
+ *	1: the entry is higher than the key
+ *	0: the entry matches the key (both entry and key have ranges, a match
+ *		is returned when the ranges overlap)
+ */
+
+static int compare_key(struct dm_multisnap_bt_entry *e, struct bt_key *key)
+{
+	chunk_t orig_chunk = read_48(e, orig_chunk);
+	if (orig_chunk < key->chunk)
+		return -1;
+	if (orig_chunk > key->chunk)
+		return 1;
+
+	if (mikulas_snapid_to_cpu(e->snap_to) < key->snap_from)
+		return -1;
+	if (mikulas_snapid_to_cpu(e->snap_from) > key->snap_to)
+		return 1;
+
+	return 0;
+}
+
+/*
+ * Perform binary search on the btree node.
+ * Returns: 1 - found, 0 - not found
+ * 	*result - if found, then the first entry in the requested range
+ *		- if not found, then the first entry after the requested range
+ */
+
+static int binary_search(struct dm_multisnap_bt_node *node, struct bt_key *key, unsigned *result)
+{
+	int c;
+	int first = 0;
+	int last = le32_to_cpu(node->n_entries) - 1;
+
+	while (1) {
+		int middle = (first + last) >> 1;
+		struct dm_multisnap_bt_entry *e = &node->entries[middle];
+
+		c = compare_key(e, key);
+
+		if (first == last)
+			break;
+
+		if (c < 0)
+			first = middle + 1;
+		else
+			last = middle;
+
+		cond_resched();
+	}
+
+	*result = first;
+	return !c;
+}
+
+/*
+ * Find a given key in the btree.
+ *
+ * Returns: 1 - found, 0 - not found, -1 - error
+ *	In case of not error (0 or 1 is returned), the node and held buffer for
+ *	this node is returned (the buffer must be released with
+ *	dm_bufio_release). Also, path with s->bt_depth entries is returned.
+ */
+
+static int walk_btree(struct dm_exception_store *s, struct bt_key *key, struct dm_multisnap_bt_node **nodep, struct dm_buffer **bp, struct path_element path[MAX_BT_DEPTH])
+{
+#define		node (*nodep)
+	int r;
+	chunk_t block = s->bt_root;
+	unsigned d = 0;
+
+	/*
+	 * These four are purely to check tree consistency.
+	 * They could be commented out. But it's safer to leave them there.
+	 */
+	chunk_t want_last_chunk = CHUNK_T_LAST;
+	mikulas_snapid_t want_last_snapid = SNAPID_T_LAST;
+	chunk_t last_chunk;
+	mikulas_snapid_t last_snapid;
+
+	while (1) {
+		path[d].block = block;
+		node = dm_multisnap_read_btnode(s, d, block, 0, bp);
+		if (!node)
+			return -1;
+		path[d].n_entries = le32_to_cpu(node->n_entries);
+
+		/* Check consistency (can be commented out) */
+		last_chunk = read_48(&node->entries[path[d].n_entries - 1], orig_chunk);
+		last_snapid = mikulas_snapid_to_cpu(node->entries[path[d].n_entries - 1].snap_to);
+		if (unlikely(last_chunk != want_last_chunk) ||
+		    unlikely(last_snapid != want_last_snapid)) {
+#if 1
+			/* Convert old format into new format */
+			if (last_chunk == CHUNK_T_LAST && last_snapid == SNAPID_T_LAST && d != s->bt_depth - 1) {
+				write_orig_chunk(&node->entries[path[d].n_entries - 1], want_last_chunk);
+				node->entries[path[d].n_entries - 1].snap_from = node->entries[path[d].n_entries - 1].snap_to = cpu_to_mikulas_snapid(want_last_snapid);
+				dm_bufio_mark_buffer_dirty(*bp);
+			} else
+#endif
+			{
+				DMERR("walk_btree: invalid last entry in node %llx: last_chunk %llx, want_last_chunk %llx, last_snapid: %llx, want_last_snapid: %llx, searching for %llx, %llx-%llx",
+					(unsigned long long)last_chunk,
+					(unsigned long long)want_last_chunk,
+					(unsigned long long)last_snapid,
+					(unsigned long long)want_last_snapid,
+					(unsigned long long)block,
+					(unsigned long long)key->chunk,
+					(unsigned long long)key->snap_from,
+					(unsigned long long)key->snap_to);
+				dm_multisnap_set_error(s->dm, -EFSERROR);
+				return -1;
+			}
+		}
+
+		r = binary_search(node, key, &path[d].idx);
+
+		want_last_chunk = read_48(&node->entries[path[d].idx], orig_chunk);
+		want_last_snapid = mikulas_snapid_to_cpu(node->entries[path[d].idx].snap_to);
+
+		block = read_48(&node->entries[path[d].idx], new_chunk);
+		if (++d == s->bt_depth)
+			break;
+		dm_bufio_release(*bp);
+	}
+	if (unlikely(compare_key(&node->entries[path[s->bt_depth - 1].idx], key) < 0))
+		path[s->bt_depth - 1].idx++;
+	return r;
+#undef node
+}
+
+/*
+ * Find a given key in the btree.
+ *
+ * Returns: 1 - found, 0 - not found, -1 - error
+ *	In case the node is found, key contains updated key and result contains
+ *	the resulting chunk.
+ */
+
+int dm_multisnap_find_in_btree(struct dm_exception_store *s, struct bt_key *key, chunk_t *result)
+{
+	struct dm_multisnap_bt_node *node;
+	struct path_element path[MAX_BT_DEPTH];
+	struct dm_buffer *bp;
+
+	int r = walk_btree(s, key, &node, &bp, path);
+	if (unlikely(r < 0))
+		return r;
+
+	if (r) {
+		struct dm_multisnap_bt_entry *entry = &node->entries[path[s->bt_depth - 1].idx];
+		*result = read_48(entry, new_chunk);
+		key->chunk = read_48(entry, orig_chunk);
+		key->snap_from = mikulas_snapid_to_cpu(entry->snap_from);
+		key->snap_to = mikulas_snapid_to_cpu(entry->snap_to);
+	}
+	dm_bufio_release(bp);
+
+	return r;
+}
+
+/*
+ * Scan the btree sequentially.
+ * Start with the given key. Perform "call" on each leaf node. When call returns
+ * nonzero, terminate the scan and return the value returned from call.
+ * When the whole tree is scanned, return 0.
+ * On error, return -1.
+ */
+
+int dm_multisnap_list_btree(struct dm_exception_store *s, struct bt_key *key, int (*call)(struct dm_exception_store *, struct dm_multisnap_bt_node *, struct dm_multisnap_bt_entry *, void *), void *cookie)
+{
+	struct dm_multisnap_bt_node *node;
+	struct path_element path[MAX_BT_DEPTH];
+	struct dm_buffer *bp;
+	int depth;
+	int i;
+	int r;
+
+	r = walk_btree(s, key, &node, &bp, path);
+	if (unlikely(r < 0))
+		return r;
+
+list_next_node:
+	for (i = path[s->bt_depth - 1].idx; i < le32_to_cpu(node->n_entries); i++) {
+		cond_resched();
+		r = call(s, node, &node->entries[i], cookie);
+		if (unlikely(r)) {
+			dm_bufio_release(bp);
+			return r;
+		}
+	}
+	dm_bufio_release(bp);
+
+	for (depth = s->bt_depth - 2; depth >= 0; depth--) {
+		int idx;
+		node = dm_multisnap_read_btnode(s, depth, path[depth].block, path[depth].n_entries, &bp);
+		if (!node)
+			return -1;
+		idx = path[depth].idx + 1;
+		if (idx < path[depth].n_entries) {
+			r = compare_key(&node->entries[idx], key);
+			if (unlikely(r <= 0)) {
+				DMERR("dm_multisnap_list_btree: non-monotonic btree: node %llx, index %x", (unsigned long long)path[depth].block, idx);
+				dm_bufio_release(bp);
+				dm_multisnap_set_error(s->dm, -EFSERROR);
+				return 0;
+			}
+			path[depth].idx = idx;
+			do {
+				depth++;
+				path[depth].block = read_48(&node->entries[path[depth - 1].idx], new_chunk);
+				path[depth].idx = 0;
+				dm_bufio_release(bp);
+				node = dm_multisnap_read_btnode(s, depth, path[depth].block, 0, &bp);
+				if (!node)
+					return -1;
+				path[depth].n_entries = le32_to_cpu(node->n_entries);
+			} while (depth < s->bt_depth - 1);
+			goto list_next_node;
+		}
+		dm_bufio_release(bp);
+	}
+
+	return 0;
+}
+
+/*
+ * Add a key and chunk to the btree.
+ * The key must not overlap with any existing btree entry.
+ */
+
+void dm_multisnap_add_to_btree(struct dm_exception_store *s, struct bt_key *key, chunk_t new_chunk)
+{
+	struct dm_multisnap_bt_node *node;
+	struct dm_buffer *bp;
+	struct path_element path[MAX_BT_DEPTH];
+	int depth;
+
+	unsigned split_entries, split_index, split_offset, split_size;
+	struct bt_key new_key;
+	struct dm_multisnap_bt_entry *last_one;
+	chunk_t new_root;
+
+	int r = walk_btree(s, key, &node, &bp, path);
+
+	if (unlikely(r)) {
+		if (r > 0) {
+			dm_bufio_release(bp);
+			DMERR("dm_multisnap_add_to_btree: adding key that already exists: %llx, %llx-%llx", (unsigned long long)key->chunk, (unsigned long long)key->snap_from, (unsigned long long)key->snap_to);
+			dm_multisnap_set_error(s->dm, -EFSERROR);
+		}
+		return;
+	}
+
+	depth = s->bt_depth - 1;
+
+go_up:
+	node = dm_multisnap_alloc_duplicate_block(s, path[depth].block, &bp, node);
+	if (unlikely(!node))
+		return;
+
+	if (likely(le32_to_cpu(node->n_entries) < s->btree_entries)) {
+		add_at_idx(node, path[depth].idx, key, new_chunk);
+		dm_bufio_mark_buffer_dirty(bp);
+		dm_bufio_release(bp);
+		return;
+	}
+	cond_resched();
+	memcpy(s->tmp_chunk, node, s->chunk_size);
+	cond_resched();
+	add_at_idx(s->tmp_chunk, path[depth].idx, key, new_chunk);
+
+	split_entries = le32_to_cpu(((struct dm_multisnap_bt_node *)s->tmp_chunk)->n_entries);
+	split_index = split_entries / 2;
+	split_offset = sizeof(struct dm_multisnap_bt_node) + split_index * sizeof(struct dm_multisnap_bt_entry);
+	split_size = sizeof(struct dm_multisnap_bt_node) + split_entries * sizeof(struct dm_multisnap_bt_entry);
+	cond_resched();
+	memcpy(node, s->tmp_chunk, sizeof(struct dm_multisnap_bt_node));
+	cond_resched();
+	memcpy((char *)node + sizeof(struct dm_multisnap_bt_node), (char *)s->tmp_chunk + split_offset, split_size - split_offset);
+	cond_resched();
+	memset((char *)node + sizeof(struct dm_multisnap_bt_node) + split_size - split_offset, 0, s->chunk_size - (sizeof(struct dm_multisnap_bt_node) + split_size - split_offset));
+	cond_resched();
+	node->n_entries = cpu_to_le32(split_entries - split_index);
+
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+
+	node = dm_multisnap_alloc_make_block(s, &new_chunk, &bp);
+	if (unlikely(!node))
+		return;
+
+	cond_resched();
+	memcpy(node, s->tmp_chunk, split_offset);
+	cond_resched();
+	memset((char *)node + split_offset, 0, s->chunk_size - split_offset);
+	cond_resched();
+	node->n_entries = cpu_to_le32(split_index);
+
+	last_one = &node->entries[split_index - 1];
+	new_key.chunk = read_48(last_one, orig_chunk);
+	new_key.snap_from = mikulas_snapid_to_cpu(last_one->snap_to);
+	new_key.snap_to = mikulas_snapid_to_cpu(last_one->snap_to);
+
+	key = &new_key;
+
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+
+	if (depth--) {
+		node = dm_multisnap_read_btnode(s, depth, path[depth].block, path[depth].n_entries, &bp);
+		if (unlikely(!node))
+			return;
+		goto go_up;
+	}
+
+	if (s->bt_depth >= MAX_BT_DEPTH) {
+		DMERR("dm_multisnap_add_to_btree: max b+-tree depth reached");
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	node = dm_multisnap_alloc_make_block(s, &new_root, &bp);
+	if (unlikely(!node))
+		return;
+
+	cond_resched();
+	memset(node, 0, s->chunk_size);
+	cond_resched();
+	node->signature = BT_SIGNATURE;
+	node->n_entries = cpu_to_le32(0);
+	add_at_idx(node, 0, &new_key, new_chunk);
+	new_key.snap_from = new_key.snap_to = SNAPID_T_LAST;
+	new_key.chunk = CHUNK_T_LAST;
+	add_at_idx(node, 1, &new_key, path[0].block);
+
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+
+	s->bt_root = new_root;
+	s->bt_depth++;
+}
+
+/*
+ * Change the last entry from old_chunk/old_snapid to new_chunk/new_snapid.
+ * Start at a given depth and go upward to the root.
+ */
+
+static void dm_multisnap_fixup_backlimits(struct dm_exception_store *s, struct path_element path[MAX_BT_DEPTH], int depth, chunk_t old_chunk, mikulas_snapid_t old_snapid, chunk_t new_chunk, mikulas_snapid_t new_snapid)
+{
+	int idx;
+	struct dm_multisnap_bt_node *node;
+	struct dm_buffer *bp;
+
+	if (old_chunk == new_chunk && old_snapid == new_snapid)
+		return;
+
+	for (depth--; depth >= 0; depth--) {
+		node = dm_multisnap_read_btnode(s, depth, path[depth].block, path[depth].n_entries, &bp);
+		if (unlikely(!node))
+			return;
+
+		node = dm_multisnap_alloc_duplicate_block(s, path[depth].block, &bp, node);
+		if (unlikely(!node))
+			return;
+
+		idx = path[depth].idx;
+
+		if (unlikely(read_48(&node->entries[idx], orig_chunk) != old_chunk) ||
+		    unlikely(mikulas_snapid_to_cpu(node->entries[idx].snap_from) != old_snapid) ||
+		    unlikely(mikulas_snapid_to_cpu(node->entries[idx].snap_to) != old_snapid)) {
+			dm_bufio_release(bp);
+			DMERR("dm_multisnap_fixup_backlimits: btree limit does not match, block %llx, idx %x, orig_chunk %llx, snap_from %llx, snap_to %llx, want %llx, %llx",
+				(unsigned long long)path[depth].block,
+				idx,
+				(unsigned long long)read_48(&node->entries[idx], orig_chunk),
+				(unsigned long long)mikulas_snapid_to_cpu(node->entries[idx].snap_from),
+				(unsigned long long)mikulas_snapid_to_cpu(node->entries[idx].snap_to),
+				(unsigned long long)old_chunk,
+				(unsigned long long)old_snapid);
+			dm_multisnap_set_error(s->dm, -EFSERROR);
+			return;
+		}
+		write_48(&node->entries[idx], orig_chunk, new_chunk);
+		node->entries[idx].snap_from = node->entries[idx].snap_to = cpu_to_mikulas_snapid(new_snapid);
+
+		dm_bufio_mark_buffer_dirty(bp);
+		dm_bufio_release(bp);
+
+		if (path[depth].idx != path[depth].n_entries - 1)
+			return;
+	}
+	DMERR("dm_multisnap_fixup_backlimits: the last entry modified, %llx/%llx -> %llx/%llx",
+		(unsigned long long)old_chunk,
+		(unsigned long long)old_snapid,
+		(unsigned long long)new_chunk,
+		(unsigned long long)new_snapid);
+	dm_multisnap_set_error(s->dm, -EFSERROR);
+}
+
+/*
+ * Restrict the range of an existing btree entry.
+ * The key must have the same beginning or end as some existing entry (not both)
+ * The range of the key is excluded from the entry.
+ */
+
+void dm_multisnap_restrict_btree_entry(struct dm_exception_store *s, struct bt_key *key)
+{
+	struct dm_multisnap_bt_node *node;
+	struct path_element path[MAX_BT_DEPTH];
+	struct dm_buffer *bp;
+	int idx;
+	struct dm_multisnap_bt_entry *entry;
+	mikulas_snapid_t from, to, new_to;
+
+	int r = walk_btree(s, key, &node, &bp, path);
+	if (unlikely(r < 0))
+		return;
+
+	if (!r) {
+		dm_bufio_release(bp);
+		DMERR("dm_multisnap_restrict_btree_entry: unknown key: %llx, %llx-%llx", (unsigned long long)key->chunk, (unsigned long long)key->snap_from, (unsigned long long)key->snap_to);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	node = dm_multisnap_alloc_duplicate_block(s, path[s->bt_depth - 1].block, &bp, node);
+	if (unlikely(!node))
+		return;
+
+	idx = path[s->bt_depth - 1].idx;
+	entry = &node->entries[idx];
+	from = mikulas_snapid_to_cpu(entry->snap_from);
+	to = new_to = mikulas_snapid_to_cpu(entry->snap_to);
+	if (key->snap_from == from && key->snap_to < to)
+		entry->snap_from = cpu_to_mikulas_snapid(key->snap_to + 1);
+	else if (key->snap_from > from && key->snap_to == to)
+		new_to = entry->snap_to = cpu_to_mikulas_snapid(key->snap_from - 1);
+	else {
+		dm_bufio_release(bp);
+		DMERR("dm_multisnap_restrict_btree_entry: invali range to restruct: %llx, %llx-%llx %llx-%llx", (unsigned long long)key->chunk, (unsigned long long)from, (unsigned long long)to, (unsigned long long)key->snap_from, (unsigned long long)key->snap_to);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+
+	if (unlikely(idx == path[s->bt_depth - 1].n_entries - 1))
+		dm_multisnap_fixup_backlimits(s, path, s->bt_depth - 1, key->chunk, to, key->chunk, new_to);
+}
+
+/*
+ * Expand range of an existing btree entry.
+ * The key represents the whole new range (including the old and new part).
+ */
+
+void dm_multisnap_extend_btree_entry(struct dm_exception_store *s, struct bt_key *key)
+{
+	struct dm_multisnap_bt_node *node;
+	struct path_element path[MAX_BT_DEPTH];
+	struct dm_buffer *bp;
+	int idx;
+	struct dm_multisnap_bt_entry *entry;
+	mikulas_snapid_t from, to, new_to;
+
+	int r = walk_btree(s, key, &node, &bp, path);
+	if (unlikely(r < 0))
+		return;
+
+	if (!r) {
+		dm_bufio_release(bp);
+		DMERR("dm_multisnap_extend_btree_entry: unknown key: %llx, %llx-%llx", (unsigned long long)key->chunk, (unsigned long long)key->snap_from, (unsigned long long)key->snap_to);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	node = dm_multisnap_alloc_duplicate_block(s, path[s->bt_depth - 1].block, &bp, node);
+	if (unlikely(!node))
+		return;
+
+	idx = path[s->bt_depth - 1].idx;
+	entry = &node->entries[idx];
+	from = mikulas_snapid_to_cpu(entry->snap_from);
+	to = new_to = mikulas_snapid_to_cpu(entry->snap_to);
+	if (key->snap_from < from)
+		entry->snap_from = cpu_to_mikulas_snapid(key->snap_from);
+	if (key->snap_to > to)
+		new_to = entry->snap_to = cpu_to_mikulas_snapid(key->snap_to);
+
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+
+	if (unlikely(idx == path[s->bt_depth - 1].n_entries - 1))
+		dm_multisnap_fixup_backlimits(s, path, s->bt_depth - 1, key->chunk, to, key->chunk, new_to);
+}
+
+/*
+ * Delete an entry from the btree.
+ */
+
+void dm_multisnap_delete_from_btree(struct dm_exception_store *s, struct bt_key *key)
+{
+	struct dm_multisnap_bt_node *node;
+	struct path_element path[MAX_BT_DEPTH];
+	struct dm_buffer *bp;
+	int idx;
+	struct dm_multisnap_bt_entry *entry;
+	mikulas_snapid_t from, to;
+	int depth, n_entries;
+
+	struct dm_multisnap_bt_entry *last_one;
+	chunk_t last_one_chunk;
+	mikulas_snapid_t last_one_snap_to;
+
+	int r = walk_btree(s, key, &node, &bp, path);
+	if (unlikely(r < 0))
+		return;
+
+	if (unlikely(!r)) {
+		dm_bufio_release(bp);
+		DMERR("dm_multisnap_delete_from_btree: unknown key: %llx, %llx-%llx", (unsigned long long)key->chunk, (unsigned long long)key->snap_from, (unsigned long long)key->snap_to);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	depth = s->bt_depth - 1;
+
+	idx = path[depth].idx;
+	entry = &node->entries[idx];
+	from = mikulas_snapid_to_cpu(entry->snap_from);
+	to = mikulas_snapid_to_cpu(entry->snap_to);
+	if (unlikely(from != key->snap_from) || unlikely(to != key->snap_to)) {
+		dm_bufio_release(bp);
+		DMERR("dm_multisnap_restrict_btree: invali range to restruct: %llx, %llx-%llx %llx-%llx", (unsigned long long)key->chunk, (unsigned long long)from, (unsigned long long)to, (unsigned long long)key->snap_from, (unsigned long long)key->snap_to);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	while (unlikely((n_entries = le32_to_cpu(node->n_entries)) == 1)) {
+		dm_bufio_release(bp);
+		if (unlikely(!depth)) {
+			DMERR("dm_multisnap_restrict_btree: b-tree is empty");
+			dm_multisnap_set_error(s->dm, -EFSERROR);
+			return;
+		}
+		dm_multisnap_free_block_and_duplicates(s, path[depth].block);
+		depth--;
+		node = dm_multisnap_read_btnode(s, depth, path[depth].block, path[depth].n_entries, &bp);
+		if (!node)
+			return;
+	}
+
+	node = dm_multisnap_alloc_duplicate_block(s, path[depth].block, &bp, node);
+	if (unlikely(!node))
+		return;
+
+	idx = path[depth].idx;
+
+	/*{
+		int x;
+		printk("before:\n");
+		for (x = 0; x < n_entries; x++)
+			printk("%llx, %x-%x -> %llx\n", read_48(&node->entries[x], orig_chunk), mikulas_snapid_to_cpu(node->entries[x].snap_from), mikulas_snapid_to_cpu(node->entries[x].snap_to), read_48(&node->entries[x], new_chunk));
+	}*/
+
+	cond_resched();
+	memmove(node->entries + idx, node->entries + idx + 1, (n_entries - idx - 1) * sizeof(struct dm_multisnap_bt_entry));
+	cond_resched();
+	n_entries--;
+	memset(node->entries + n_entries, 0, sizeof(struct dm_multisnap_bt_entry));
+
+	node->n_entries = cpu_to_le32(n_entries);
+
+	/*{
+		int x;
+		printk("after:\n");
+		for (x = 0; x < n_entries; x++)
+			printk("%llx, %x-%x -> %llx\n", read_48(&node->entries[x], orig_chunk), mikulas_snapid_to_cpu(node->entries[x].snap_from), mikulas_snapid_to_cpu(node->entries[x].snap_to), read_48(&node->entries[x], new_chunk));
+	}*/
+
+	last_one = &node->entries[n_entries - 1];
+	last_one_chunk = read_48(last_one, orig_chunk);
+	last_one_snap_to = mikulas_snapid_to_cpu(last_one->snap_to);
+
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+
+	if (unlikely(idx == n_entries))
+		dm_multisnap_fixup_backlimits(s, path, depth, key->chunk, key->snap_to, last_one_chunk, last_one_snap_to);
+}
+
+/*
+ * Process btree tmp remaps.
+ * Find the whole path for tmp_remap and write the path as new entries, from
+ * the root.
+ */
+
+void dm_multisnap_bt_finalize_tmp_remap(struct dm_exception_store *s, struct tmp_remap *tmp_remap)
+{
+	struct dm_buffer *bp;
+	struct dm_multisnap_bt_node *node;
+	struct bt_key key;
+	struct path_element path[MAX_BT_DEPTH];
+	int results_ptr;
+
+	chunk_t new_blockn;
+	int r;
+	int i;
+
+	if (s->n_preallocated_blocks < s->bt_depth) {
+		if (dm_multisnap_alloc_blocks(s, s->preallocated_blocks + s->n_preallocated_blocks, s->bt_depth - s->n_preallocated_blocks, 0) < 0)
+			return;
+		s->n_preallocated_blocks = s->bt_depth;
+	}
+	results_ptr = 0;
+
+	/*
+	 * Read the key from this node --- we'll walk the btree according
+	 * to this key to find a path from the root.
+	 */
+	node = dm_multisnap_read_btnode(s, s->bt_depth - 1, tmp_remap->new, 0, &bp);
+	if (!node)
+		return;
+	key.chunk = read_48(&node->entries[0], orig_chunk);
+	key.snap_from = key.snap_to = mikulas_snapid_to_cpu(node->entries[0].snap_from);
+	dm_bufio_release(bp);
+
+	r = walk_btree(s, &key, &node, &bp, path);
+	if (r < 0)
+		return;
+
+	dm_bufio_release(bp);
+
+	for (i = s->bt_depth - 1; i >= 0; i--)
+		if (path[i].block == tmp_remap->old)
+			goto found;
+
+	DMERR("block %llx/%llx was not found in btree when searching for %llx/%llx", (unsigned long long)tmp_remap->old, (unsigned long long)tmp_remap->new, (unsigned long long)key.chunk, (unsigned long long)key.snap_from);
+	for (i = 0; i < s->bt_depth; i++)
+		DMERR("path[%d]: %llx/%x", i, (unsigned long long)path[i].block, path[i].idx);
+	dm_multisnap_set_error(s->dm, -EFSERROR);
+	return;
+
+found:
+
+	dm_multisnap_free_block(s, tmp_remap->old, 0);
+
+	new_blockn = tmp_remap->new;
+	for (i--; i >= 0; i--) {
+		int remapped = 0;
+		node = dm_multisnap_read_btnode(s, i, path[i].block, path[i].n_entries, &bp);
+		if (!node)
+			return;
+		if (!dm_multisnap_block_is_uncommitted(s, path[i].block)) {
+			remapped = 1;
+			dm_bufio_release_move(bp, s->preallocated_blocks[results_ptr]);
+			dm_multisnap_free_block_and_duplicates(s, path[i].block);
+			node = dm_multisnap_read_btnode(s, i, s->preallocated_blocks[results_ptr], path[i].n_entries, &bp);
+			if (!node)
+				return;
+			/* !!! TODO: add to a list of newly allocated blocks */
+		}
+		write_48(&node->entries[path[i].idx], new_chunk, new_blockn);
+		dm_bufio_mark_buffer_dirty(bp);
+		dm_bufio_release(bp);
+
+		if (!remapped)
+			goto skip_it;
+		new_blockn = s->preallocated_blocks[results_ptr];
+		results_ptr++;
+	}
+
+	s->bt_root = new_blockn;
+
+skip_it:
+	memmove(s->preallocated_blocks, s->preallocated_blocks + results_ptr, (s->n_preallocated_blocks -= results_ptr) * sizeof(chunk_t));
+}
+
Index: linux-2.6.32/drivers/md/dm-multisnap-commit.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-commit.c
@@ -0,0 +1,210 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-mikulas.h"
+
+static void dm_multisnap_finalize_tmp_remaps(struct dm_exception_store *s)
+{
+	struct tmp_remap *t;
+	int i;
+
+	while (s->n_used_tmp_remaps) {
+		if (dm_multisnap_has_error(s->dm))
+			return;
+		if (s->n_used_tmp_remaps < N_REMAPS - 1) {
+			/*
+			 * prefer btree remaps ...
+			 * if there are none, do bitmap remaps
+			 */
+			if (!list_empty(&s->used_bt_tmp_remaps)) {
+				t = container_of(s->used_bt_tmp_remaps.next, struct tmp_remap, list);
+				dm_multisnap_bt_finalize_tmp_remap(s, t);
+				dm_multisnap_free_tmp_remap(s, t);
+				continue;
+			}
+		}
+
+/* else: 0 or 1 free remaps : finalize bitmaps */
+		if (!list_empty(&s->used_bitmap_tmp_remaps)) {
+			t = container_of(s->used_bitmap_tmp_remaps.next, struct tmp_remap, list);
+			dm_multisnap_bitmap_finalize_tmp_remap(s, t);
+			dm_multisnap_free_tmp_remap(s, t);
+			continue;
+		} else {
+			DMERR("dm_multisnap_finalize_tmp_remaps: no bitmap tmp remaps, n_used_tmp_remaps %u", s->n_used_tmp_remaps);
+			dm_multisnap_set_error(s->dm, -EFSERROR);
+			return;
+		}
+	}
+
+	if (dm_multisnap_has_error(s->dm))
+		return;
+
+	for (i = s->n_preallocated_blocks - 1; i >= 0; i--)
+		dm_multisnap_free_blocks_immediate(s, s->preallocated_blocks[i], 1);
+	s->n_preallocated_blocks = 0;
+}
+
+void dm_multisnap_transaction_mark(struct dm_exception_store *s)
+{
+	/*
+	 * Accounting:
+	 * max number of modified/allocated blocks during btree add:
+	 *	s->bt_depth * 2 + 1
+	 * one additional entry for newly allocated data chunk
+	 * one additional entry for bitmap finalization
+	 */
+	if (unlikely(N_REMAPS - s->n_used_tmp_remaps < s->bt_depth * 2 + 3))
+		dm_multisnap_finalize_tmp_remaps(s);
+}
+
+void dm_multisnap_commit(struct dm_exception_store *s)
+{
+	struct tmp_remap *t;
+	chunk_t cb_addr;
+	chunk_t cb_div, cb_offset;
+	struct multisnap_commit_block *cb;
+	struct multisnap_superblock *sb;
+	unsigned idx;
+	struct dm_buffer *bp;
+	int r;
+
+	dm_multisnap_transaction_mark(s);
+
+	dm_multisnap_flush_freelist_before_commit(s);
+
+	if (dm_multisnap_has_error(s->dm)) {
+		struct multisnap_superblock *sb;
+
+		if (!dm_multisnap_drop_on_error(s->dm))
+			return;
+
+		sb = dm_bufio_read(s->bufio, SB_BLOCK, &bp);
+		if (IS_ERR(sb))
+			return;
+
+		if (!le32_to_cpu(sb->error)) {
+			sb->error = cpu_to_le32(dm_multisnap_has_error(s->dm));
+			dm_bufio_mark_buffer_dirty(bp);
+		}
+
+		dm_bufio_release(bp);
+		return;
+	}
+
+	list_for_each_entry(t, &s->used_bitmap_tmp_remaps, list)
+		t->uncommitted = 0;
+
+	list_for_each_entry(t, &s->used_bt_tmp_remaps, list)
+		t->uncommitted = 0;
+
+	r = dm_bufio_write_dirty_buffers(s->bufio);
+	if (unlikely(r < 0)) {
+		DMERR("dm_multisnap_commit: error writing data");
+		dm_multisnap_set_error(s->dm, r);
+		return;
+	}
+
+	cb_addr = s->alloc_rover;
+
+	if (cb_addr < FIRST_CB_BLOCK)
+		cb_addr = FIRST_CB_BLOCK;
+	cb_div = cb_addr - FIRST_CB_BLOCK;
+	cb_offset = sector_div(cb_div, s->cb_stride);
+	cb_addr += s->cb_stride - cb_offset;
+	if (cb_offset < s->cb_stride / 2 || cb_addr >= s->dev_size)
+		cb_addr -= s->cb_stride;
+
+	cb = dm_bufio_new(s->bufio, cb_addr, &bp);
+	if (IS_ERR(cb)) {
+		DMERR("dm_multisnap_commit: can't allocate new commit block at %llx", (unsigned long long)cb_addr);
+		dm_multisnap_set_error(s->dm, PTR_ERR(cb));
+		return;
+	}
+
+	s->commit_sequence++;
+
+	cb->signature = CB_SIGNATURE;
+	cb->snapshot_num = cpu_to_le32(s->snapshot_num);
+	cb->sequence = cpu_to_le64(s->commit_sequence);
+	write_48(cb, dev_size, s->dev_size);
+	write_48(cb, total_allocated, s->total_allocated);
+	write_48(cb, data_allocated, s->data_allocated);
+	write_48(cb, bitmap_root, s->bitmap_root);
+	write_48(cb, alloc_rover, s->alloc_rover);
+	write_48(cb, freelist, s->freelist_ptr);
+	write_48(cb, delete_rover, s->delete_rover_chunk);
+	write_48(cb, bt_root, s->bt_root);
+	cb->bt_depth = s->bt_depth;
+	cb->flags = s->flags;
+	memset(cb->pad, 0, sizeof cb->pad);
+	idx = 0;
+	list_for_each_entry(t, &s->used_bitmap_tmp_remaps, list) {
+		BUG_ON(idx >= N_REMAPS);
+		write_48(&cb->tmp_remap[idx], old, t->old);
+		write_48(&cb->tmp_remap[idx], new, t->new);
+		cb->tmp_remap[idx].bitmap_idx = cpu_to_le32(t->bitmap_idx);
+		idx++;
+	}
+	list_for_each_entry(t, &s->used_bt_tmp_remaps, list) {
+		BUG_ON(idx >= N_REMAPS);
+		write_48(&cb->tmp_remap[idx], old, t->old);
+		write_48(&cb->tmp_remap[idx], new, t->new);
+		cb->tmp_remap[idx].bitmap_idx = cpu_to_le32(t->bitmap_idx);
+		idx++;
+	}
+	for (; idx < N_REMAPS; idx++) {
+		write_48(&cb->tmp_remap[idx], old, 0);
+		write_48(&cb->tmp_remap[idx], new, 0);
+		cb->tmp_remap[idx].bitmap_idx = cpu_to_le32(0);
+	}
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+	r = dm_bufio_write_dirty_buffers(s->bufio);
+	if (unlikely(r < 0)) {
+		DMERR("dm_multisnap_commit: can't write commit block at %llx", (unsigned long long)cb_addr);
+		dm_multisnap_set_error(s->dm, r);
+		return;
+	}
+
+	if (likely(cb_addr == s->valid_commit_block) ||
+	    likely(cb_addr == s->valid_commit_block + s->cb_stride))
+		goto return_success;
+
+	sb = dm_bufio_read(s->bufio, SB_BLOCK, &bp);
+	if (IS_ERR(sb)) {
+		DMERR("dm_multisnap_commit: can't read super block");
+		dm_multisnap_set_error(s->dm, PTR_ERR(sb));
+		return;
+	}
+
+	if (unlikely(sb->signature != SB_SIGNATURE)) {
+		dm_bufio_release(bp);
+		DMERR("dm_multisnap_commit: invalid super block signature when committing");
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	sb->commit_block = cpu_to_le64(cb_addr);
+
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+	r = dm_bufio_write_dirty_buffers(s->bufio);
+	if (unlikely(r < 0)) {
+		DMERR("dm_multisnap_commit: can't write super block");
+		dm_multisnap_set_error(s->dm, r);
+		return;
+	}
+
+return_success:
+	s->valid_commit_block = cb_addr;
+
+	dm_multisnap_load_freelist(s);
+
+	return;
+}
Index: linux-2.6.32/drivers/md/dm-multisnap-delete.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-delete.c
@@ -0,0 +1,134 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-mikulas.h"
+
+/*
+ * Commit after this number of deleted entries.
+ * Too big number causes spurious overflows on nearly-full device.
+ * Too small number degrades delete performance.
+ */
+#define COMMIT_AFTER		128
+
+struct list_cookie {
+	struct bt_key key;
+	chunk_t new_chunk;
+};
+
+#define RET_END			1
+#define RET_DO_FREE		2
+#define RET_RESCHEDULE		3
+
+static int list_callback(struct dm_exception_store *s, struct dm_multisnap_bt_node *node, struct dm_multisnap_bt_entry *bt, void *cookie)
+{
+	struct list_cookie *lc = cookie;
+	mikulas_snapid_t found_from, found_to;
+
+	lc->key.chunk = read_48(bt, orig_chunk);
+	lc->key.snap_from = mikulas_snapid_to_cpu(bt->snap_from);
+	lc->key.snap_to = mikulas_snapid_to_cpu(bt->snap_to);
+
+	if (unlikely(lc->key.chunk > CHUNK_T_MAX))
+		return RET_END;
+
+	s->delete_rover_chunk = lc->key.chunk;
+	s->delete_rover_snapid = lc->key.snap_to + 1;
+	if (unlikely(!s->delete_rover_snapid))
+		s->delete_rover_chunk++;
+
+	if (!dm_multisnap_find_next_snapid_range(s, lc->key.snap_from, &found_from, &found_to) || found_from > lc->key.snap_to) {
+		/*
+		 * This range maps unused snapshots, delete it.
+		 * But we can't do it now, so submit it to the caller;
+		 */
+		lc->new_chunk = read_48(bt, new_chunk);
+		return RET_DO_FREE;
+	}
+
+	/*
+	 * If we are at a last entry in the btree node, drop the lock and
+	 * allow other requests to be processed.
+	 *
+	 * This avoids a starvation when there are no nodes to delete.
+	 */
+	if (bt == &node->entries[le32_to_cpu(node->n_entries) - 1])
+		return RET_RESCHEDULE;
+
+	return 0;
+}
+
+static void delete_step(struct dm_exception_store *s)
+{
+	struct bt_key key;
+	int r;
+	struct list_cookie lc;
+
+	key.chunk = s->delete_rover_chunk;
+	key.snap_from = s->delete_rover_snapid;
+	key.snap_to = s->delete_rover_snapid;
+
+	r = dm_multisnap_list_btree(s, &key, list_callback, &lc);
+
+	if (unlikely(r < 0))
+		return;
+
+	switch (r) {
+
+	case RET_END:
+		s->flags &= ~MULTISNAP_FLAG_DELETING;
+
+		/* If we finished the job and there is no pending I/O, commit */
+		if (dm_multisnap_can_commit(s->dm))
+			dm_multisnap_call_commit(s->dm);
+
+		return;
+	case RET_DO_FREE:
+		if (unlikely(dm_multisnap_has_error(s->dm)))
+			return;
+
+		/*printk("deleting: %lx (%llx-%llx) -> %lx\n", lc.key.chunk, lc.key.snap_from, lc.key.snap_to, lc.new_chunk);*/
+
+		dm_multisnap_delete_from_btree(s, &lc.key);
+
+		dm_multisnap_transaction_mark(s);
+
+		dm_multisnap_free_block(s, lc.new_chunk, FREELIST_DATA_FLAG);
+
+		/* fall through */
+	case RET_RESCHEDULE:
+		if (dm_multisnap_can_commit(s->dm)) {
+			if (++s->delete_commit_count >= COMMIT_AFTER) {
+				s->delete_commit_count = 0;
+				dm_multisnap_call_commit(s->dm);
+			}
+		}
+		return;
+	default:
+		printk(KERN_CRIT "delete_step: invalid return value %d", r);
+		BUG();
+
+	}
+}
+
+void dm_multisnap_background_delete(struct dm_exception_store *s, struct dm_multisnap_background_work *bw)
+{
+	if (unlikely(dm_multisnap_has_error(s->dm)))
+		return;
+
+	if (s->flags & MULTISNAP_FLAG_DELETING) {
+		delete_step(s);
+	} else if (s->flags & MULTISNAP_FLAG_PENDING_DELETE) {
+		s->flags &= ~MULTISNAP_FLAG_PENDING_DELETE;
+		s->flags |= MULTISNAP_FLAG_DELETING;
+		s->delete_rover_chunk = 0;
+		s->delete_rover_snapid = 0;
+	} else
+		return;
+
+	dm_multisnap_queue_work(s->dm, &s->delete_work);
+}
Index: linux-2.6.32/drivers/md/dm-multisnap-freelist.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-freelist.c
@@ -0,0 +1,236 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-mikulas.h"
+
+void dm_multisnap_init_freelist(struct dm_multisnap_freelist *fl, unsigned chunk_size)
+{
+	cond_resched();
+	memset(fl, 0, chunk_size);
+	cond_resched();
+	fl->signature = FL_SIGNATURE;
+	write_48(fl, backlink, 0);
+	fl->n_entries = cpu_to_le32(0);
+}
+
+static int add_to_freelist(struct dm_exception_store *s, chunk_t block, unsigned flags)
+{
+	int i;
+	struct dm_multisnap_freelist *fl = s->freelist;
+	for (i = le32_to_cpu(fl->n_entries) - 1; i >= 0; i--) {
+		chunk_t x = read_48(&fl->entries[i], block);
+		unsigned r = le16_to_cpu(fl->entries[i].run_length) & FREELIST_RL_MASK;
+		unsigned f = le16_to_cpu(fl->entries[i].run_length) & FREELIST_DATA_FLAG;
+		if (block >= x && block < x + r) {
+			DMERR("add_to_freelist: freeing already free block %llx (%llx - %x)", (unsigned long long)block, (unsigned long long)x, r);
+			dm_multisnap_set_error(s->dm, -EFSERROR);
+			return -1;
+		}
+		if (likely(r < FREELIST_RL_MASK) && likely(f == flags)) {
+			if (block == x - 1) {
+				write_48(&fl->entries[i], block, x - 1);
+				goto inc_length;
+			}
+			if (block == x + r) {
+inc_length:
+				fl->entries[i].run_length = cpu_to_le16((r + 1) | f);
+				return 1;
+			}
+		}
+		cond_resched();
+	}
+	i = le32_to_cpu(fl->n_entries);
+	if (i < dm_multisnap_freelist_entries(s->chunk_size)) {
+		fl->n_entries = cpu_to_le32(i + 1);
+		write_48(&fl->entries[i], block, block);
+		fl->entries[i].run_length = cpu_to_le16(1 | flags);
+		return 1;
+	}
+	return 0;
+}
+
+static struct dm_multisnap_freelist *read_freelist(struct dm_exception_store *s, chunk_t block, struct dm_buffer **bp)
+{
+	struct dm_multisnap_freelist *fl;
+	fl = dm_bufio_read(s->bufio, block, bp);
+	if (IS_ERR(fl)) {
+		DMERR("read_freelist: can't read freelist block %llx", (unsigned long long)block);
+		dm_multisnap_set_error(s->dm, PTR_ERR(fl));
+		return NULL;
+	}
+	if (fl->signature != FL_SIGNATURE) {
+		dm_bufio_release(*bp);
+		DMERR("read_freelist: bad signature freelist block %llx", (unsigned long long)block);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return NULL;
+	}
+	if (le32_to_cpu(fl->n_entries) > dm_multisnap_freelist_entries(s->chunk_size)) {
+		dm_bufio_release(*bp);
+		DMERR("read_freelist: bad number of entries in freelist block %llx", (unsigned long long)block);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return NULL;
+	}
+	return fl;
+}
+
+static void alloc_write_freelist(struct dm_exception_store *s)
+{
+	chunk_t new_block;
+	struct dm_multisnap_freelist *fl;
+	struct dm_buffer *bp;
+
+	if (dm_multisnap_alloc_blocks(s, &new_block, 1, ALLOC_DRY))
+		return;
+
+	fl = dm_bufio_new(s->bufio, new_block, &bp);
+	if (IS_ERR(fl)) {
+		DMERR("alloc_write_freelist: can't make new freelist block %llx", (unsigned long long)new_block);
+		dm_multisnap_set_error(s->dm, PTR_ERR(fl));
+		return;
+	}
+
+	memcpy(fl, s->freelist, s->chunk_size);
+
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+
+	dm_multisnap_init_freelist(s->freelist, s->chunk_size);
+	write_48(s->freelist, backlink, new_block);
+}
+
+void dm_multisnap_free_block(struct dm_exception_store *s, chunk_t block, unsigned flags)
+{
+	if (likely(add_to_freelist(s, block, flags)))
+		return;
+
+	alloc_write_freelist(s);
+	if (dm_multisnap_has_error(s->dm))
+		return;
+
+	if (add_to_freelist(s, block, flags))
+		return;
+
+	BUG();
+}
+
+static int check_against_freelist(struct dm_multisnap_freelist *fl, chunk_t block)
+{
+	int i;
+	for (i = le32_to_cpu(fl->n_entries) - 1; i >= 0; i--) {
+		chunk_t x = read_48(&fl->entries[i], block);
+		unsigned r = le16_to_cpu(fl->entries[i].run_length) & FREELIST_RL_MASK;
+		if (block - x >= 0 && unlikely(block - x < r))
+			return 1;
+		cond_resched();
+	}
+	return 0;
+}
+
+static int check_against_freelist_chain(struct dm_exception_store *s, chunk_t fl_block, chunk_t block)
+{
+	stop_cycles_t cy;
+	dm_multisnap_init_stop_cycles(&cy);
+
+	while (unlikely(fl_block != 0)) {
+		int c;
+		struct dm_buffer *bp;
+		struct dm_multisnap_freelist *fl;
+
+		if (dm_multisnap_stop_cycles(s, &cy, fl_block))
+			return -1;
+
+		if (unlikely(block == fl_block))
+			return 1;
+
+		fl = read_freelist(s, fl_block, &bp);
+		if (unlikely(!fl))
+			return -1;
+		c = check_against_freelist(fl, block);
+		fl_block = read_48(fl, backlink);
+		dm_bufio_release(bp);
+		if (unlikely(c))
+			return c;
+	}
+	return 0;
+}
+
+int dm_multisnap_check_allocated_block(struct dm_exception_store *s, chunk_t block)
+{
+	int c;
+
+	c = check_against_freelist(s->freelist, block);
+	if (unlikely(c))
+		return c;
+
+	c = check_against_freelist_chain(s, read_48(s->freelist, backlink), block);
+	if (unlikely(c))
+		return c;
+
+	c = check_against_freelist_chain(s, s->freelist_ptr, block);
+	if (unlikely(c))
+		return c;
+
+	return 0;
+}
+
+void dm_multisnap_flush_freelist_before_commit(struct dm_exception_store *s)
+{
+	alloc_write_freelist(s);
+
+	if (dm_multisnap_has_error(s->dm))
+		return;
+
+	s->freelist_ptr = read_48(s->freelist, backlink);
+}
+
+static void free_blocks_in_freelist(struct dm_exception_store *s, struct dm_multisnap_freelist *fl)
+{
+	int i;
+	for (i = le32_to_cpu(fl->n_entries) - 1; i >= 0; i--) {
+		chunk_t x = read_48(&fl->entries[i], block);
+		unsigned r = le16_to_cpu(fl->entries[i].run_length) & FREELIST_RL_MASK;
+		unsigned f = le16_to_cpu(fl->entries[i].run_length) & FREELIST_DATA_FLAG;
+		dm_multisnap_free_blocks_immediate(s, x, r);
+		if (likely(f & FREELIST_DATA_FLAG)) {
+			dm_multisnap_status_lock(s->dm);
+			s->data_allocated -= r;
+			dm_multisnap_status_unlock(s->dm);
+		}
+		cond_resched();
+	}
+}
+
+void dm_multisnap_load_freelist(struct dm_exception_store *s)
+{
+	chunk_t fl_block = s->freelist_ptr;
+
+	stop_cycles_t cy;
+	dm_multisnap_init_stop_cycles(&cy);
+
+	while (fl_block) {
+		struct dm_buffer *bp;
+		struct dm_multisnap_freelist *fl;
+
+		if (dm_multisnap_stop_cycles(s, &cy, fl_block))
+			break;
+
+		if (dm_multisnap_has_error(s->dm))
+			break;
+
+		fl = read_freelist(s, fl_block, &bp);
+		if (!fl)
+			break;
+		memcpy(s->freelist, fl, s->chunk_size);
+		dm_bufio_release(bp);
+
+		free_blocks_in_freelist(s, s->freelist);
+		fl_block = read_48(s->freelist, backlink);
+	}
+
+	dm_multisnap_init_freelist(s->freelist, s->chunk_size);
+}
Index: linux-2.6.32/drivers/md/dm-multisnap-io.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-io.c
@@ -0,0 +1,191 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-mikulas.h"
+
+/*
+ * This function will check if there is remapping for a given snapid/chunk.
+ * It returns 1 if remapping exists and is read-only (shared by other snapshots)
+ * and 2 if it exists and is read-write (not shared by anyone).
+ */
+
+int dm_multisnap_find_snapshot_chunk(struct dm_exception_store *s, snapid_t snapid, chunk_t chunk, int write, chunk_t *result)
+{
+	int r;
+	struct bt_key key;
+	mikulas_snapid_t from, to;
+	mikulas_snapid_t find_from, find_to;
+
+	key.chunk = chunk;
+	key.snap_from = snapid;
+	key.snap_to = snapid;
+	r = dm_multisnap_find_in_btree(s, &key, result);
+	if (!r) {
+		s->query_new_key.chunk = chunk;
+		s->query_new_key.snap_from = snapid;
+		s->query_new_key.snap_to = snapid;
+		s->query_active = 1;
+	}
+	if (r <= 0 || !write)
+		return r;
+
+	from = to = snapid;
+	if ((snapid & MIKULAS_SUBSNAPID_MASK) == MIKULAS_SUBSNAPID_MASK) {
+		from = snapid & ~MIKULAS_SUBSNAPID_MASK;
+		if (!dm_multisnap_find_next_snapid_range(s, snapid, &find_from, &find_to))
+			BUG();
+		if (from < find_from)
+			from = find_from;
+	}
+
+	/*
+	 * We are writing to a snapshot --- check if anything outside <from-to>
+	 * range exists, if it does, it needs to be copied.
+	 */
+
+	if (key.snap_from < from) {
+		if (likely(dm_multisnap_find_next_snapid_range(s, key.snap_from, &find_from, &find_to))) {
+			if (find_from < from) {
+				s->query_new_key.chunk = chunk;
+				s->query_new_key.snap_from = from;
+				s->query_new_key.snap_to = key.snap_to;
+				s->query_block_from = key.snap_from;
+				s->query_block_to = key.snap_to;
+				s->query_active = 2;
+				return 1;
+			}
+			if (unlikely(find_from > from))
+				BUG(); /* SNAPID not in our tree */
+		} else
+			BUG(); /* we're asking for a SNAPID not in our tree */
+	}
+	if (key.snap_to > to) {
+		if (likely(dm_multisnap_find_next_snapid_range(s, to + 1, &find_from, &find_to))) {
+			if (find_from <= key.snap_to) {
+				s->query_new_key.chunk = chunk;
+				s->query_new_key.snap_from = key.snap_from;
+				s->query_new_key.snap_to = to;
+				s->query_block_from = key.snap_from;
+				s->query_block_to = key.snap_to;
+				s->query_active = 2;
+				return 1;
+			}
+		}
+	}
+	return 2;
+}
+
+void dm_multisnap_reset_query(struct dm_exception_store *s)
+{
+	s->query_active = 0;
+
+	s->query_snapid = 0;
+}
+
+int dm_multisnap_query_next_remap(struct dm_exception_store *s, chunk_t chunk)
+{
+	int r;
+	chunk_t sink;
+	mikulas_snapid_t from, to;
+
+	s->query_active = 0;
+
+	while (dm_multisnap_find_next_snapid_range(s, s->query_snapid, &from, &to)) {
+		struct bt_key key;
+next_btree_search:
+		if (dm_multisnap_has_error(s->dm))
+			return -1;
+		key.chunk = chunk;
+		key.snap_from = from;
+		key.snap_to = to;
+		r = dm_multisnap_find_in_btree(s, &key, &sink);
+		if (unlikely(r < 0))
+			return -1;
+
+		if (!r) {
+			s->query_new_key.chunk = chunk;
+			s->query_new_key.snap_from = from;
+			s->query_new_key.snap_to = to;
+			s->query_active = 1;
+			return 1;
+		}
+
+		if (key.snap_from > from) {
+			s->query_new_key.chunk = chunk;
+			s->query_new_key.snap_from = from;
+			s->query_new_key.snap_to = key.snap_from - 1;
+			s->query_active = 1;
+			return 1;
+		}
+
+		if (key.snap_to < to) {
+			from = key.snap_to + 1;
+			goto next_btree_search;
+		}
+
+		s->query_snapid = to + 1;
+	}
+
+	return 0;
+}
+
+void dm_multisnap_add_next_remap(struct dm_exception_store *s, union chunk_descriptor *cd, chunk_t *new_chunk)
+{
+	int r;
+
+	BUG_ON(s->query_active != 1);
+	s->query_active = 0;
+
+	cd->range.from = s->query_new_key.snap_from;
+	cd->range.to = s->query_new_key.snap_to;
+
+	r = dm_multisnap_alloc_blocks(s, new_chunk, 1, 0);
+	if (unlikely(r < 0))
+		return;
+
+	dm_multisnap_status_lock(s->dm);
+	s->data_allocated++;
+	dm_multisnap_status_unlock(s->dm);
+
+	dm_multisnap_add_to_btree(s, &s->query_new_key, *new_chunk);
+	dm_multisnap_transaction_mark(s);
+}
+
+void dm_multisnap_make_chunk_writeable(struct dm_exception_store *s, union chunk_descriptor *cd, chunk_t *new_chunk)
+{
+	int r;
+
+	BUG_ON(s->query_active != 2);
+	s->query_active = 0;
+
+	cd->range.from = s->query_block_from;
+	cd->range.to = s->query_block_to;
+
+	r = dm_multisnap_alloc_blocks(s, new_chunk, 1, 0);
+	if (unlikely(r < 0))
+		return;
+
+	dm_multisnap_status_lock(s->dm);
+	s->data_allocated++;
+	dm_multisnap_status_unlock(s->dm);
+
+	dm_multisnap_restrict_btree_entry(s, &s->query_new_key);
+	dm_multisnap_transaction_mark(s);
+
+	if (unlikely(dm_multisnap_has_error(s->dm)))
+		return;
+
+	dm_multisnap_add_to_btree(s, &s->query_new_key, *new_chunk);
+	dm_multisnap_transaction_mark(s);
+}
+
+int dm_multisnap_check_conflict(struct dm_exception_store *s, union chunk_descriptor *cd, snapid_t snapid)
+{
+	return snapid >= cd->range.from && snapid <= cd->range.to;
+}
+
Index: linux-2.6.32/drivers/md/dm-multisnap-mikulas-struct.h
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-mikulas-struct.h
@@ -0,0 +1,198 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#ifndef DM_MULTISNAP_MIKULAS_STRUCT_H
+#define DM_MULTISNAP_MIKULAS_STRUCT_H
+
+/* on-disk structures */
+
+#include <linux/types.h>
+#include <asm/byteorder.h>
+
+#include "dm-multisnap.h"
+
+/*
+ * Encoding of snapshot numbers:
+ *
+ * If CONFIG_DM_MULTISNAPSHOT_MIKULAS_SNAP_OF_SNAP is not selected (normally it
+ * is), then mikulas_snapid_t is 32-bit sequential number. It continually grows.
+ *
+ * IF CONFIG_DM_MULTISNAPSHOT_MIKULAS_SNAP_OF_SNAP is selected (by default),
+ * then mikulas_snapid_t is 64-bit number. The high 32 bits are sequential
+ * snapshot number. With each new snapshot, it is incremented. The low 32 bits
+ * are subsnapshot number. Single snapshots (snapshots of the origin) have
+ * low 32 bits equal to all ones. Snapshots-of-snapshots have high 32 bits
+ * equal as their master snapshot and low 32 bits start with zero and is
+ * incremented with each new snapshot-of-snapshot.
+ *
+ * More levels (snapshots-of-snapshots-of-snapshots) are not allowed.
+ */
+
+#ifndef CONFIG_DM_MULTISNAPSHOT_MIKULAS_SNAP_OF_SNAP
+typedef __u32 mikulas_snapid_t;
+#define MIKULAS_SNAPID_STEP_BITS	0
+#define mikulas_snapid_to_cpu		le32_to_cpu
+#define cpu_to_mikulas_snapid		cpu_to_le32
+#else
+typedef __u64 mikulas_snapid_t;
+#define MIKULAS_SNAPID_STEP_BITS	32
+#define mikulas_snapid_to_cpu		le64_to_cpu
+#define cpu_to_mikulas_snapid		cpu_to_le64
+#endif
+
+#define MIKULAS_SUBSNAPID_MASK	(((mikulas_snapid_t)1 << MIKULAS_SNAPID_STEP_BITS) - 1)
+#define SNAPID_T_LAST		((mikulas_snapid_t)0xffffffffffffffffULL)
+#define SNAPID_T_MAX		((mikulas_snapid_t)0xfffffffffffffffeULL)
+
+#define CHUNK_BITS		48
+#define CHUNK_T_LAST		((chunk_t)(1LL << CHUNK_BITS) - 1)
+#define CHUNK_T_SNAP_PRESENT	((chunk_t)(1LL << CHUNK_BITS) - 1)
+#define CHUNK_T_MAX		((chunk_t)(1LL << CHUNK_BITS) - 2)
+
+#define CB_STRIDE_DEFAULT	1024
+
+#define SB_BLOCK	0
+
+#define SB_SIGNATURE	cpu_to_be32(0xF6015342)
+
+struct multisnap_superblock {
+	__u32 signature;
+	__u32 chunk_size;
+	__u32 cb_stride;
+	__s32 error;
+	__u64 commit_block;
+};
+
+
+#define FIRST_CB_BLOCK	1
+
+#define CB_SIGNATURE	cpu_to_be32(0xF6014342)
+
+struct commit_block_tmp_remap {
+	__u32 old1;
+	__u16 old2;
+	__u16 new2;
+	__u32 new1;
+	__u32 bitmap_idx;
+};
+
+#define CB_BITMAP_IDX_MAX	0xfffffffd
+#define CB_BITMAP_IDX_NONE	0xfffffffe
+#define CB_BITMAP_IDX_FREE	0xffffffff
+
+#define N_REMAPS	27
+
+struct multisnap_commit_block {
+	__u32 signature;
+	__u32 snapshot_num;
+	__u64 sequence;
+
+	__u32 dev_size1;
+	__u16 dev_size2;
+	__u16 total_allocated2;
+	__u32 total_allocated1;
+	__u32 data_allocated1;
+
+	__u16 data_allocated2;
+	__u16 bitmap_root2;
+	__u32 bitmap_root1;
+	__u32 alloc_rover1;
+	__u16 alloc_rover2;
+	__u16 freelist2;
+
+	__u32 freelist1;
+	__u32 delete_rover1;
+	__u16 delete_rover2;
+	__u16 bt_root2;
+	__u32 bt_root1;
+
+	__u8 bt_depth;
+	__u8 flags;
+	__u8 pad[14];
+
+	struct commit_block_tmp_remap tmp_remap[N_REMAPS];
+};
+
+#define MULTISNAP_FLAG_DELETING		0x01
+#define MULTISNAP_FLAG_PENDING_DELETE	0x02
+
+#define MAX_BITMAP_DEPTH		6
+
+static inline int dm_multisnap_bitmap_depth(unsigned chunk_shift, __u64 device_size)
+{
+	unsigned depth = 0;
+	__u64 entries = 8 << chunk_shift;
+	while (entries < device_size) {
+		depth++;
+		entries <<= chunk_shift - 3;
+		if (!entries)
+			return -ERANGE;
+	}
+
+	if (depth > MAX_BITMAP_DEPTH)
+		return -ERANGE;
+
+	return depth;
+}
+
+
+/* B+-tree entry. Sorted by orig_chunk and snap_from/to */
+
+#define MAX_BT_DEPTH		12
+
+struct dm_multisnap_bt_entry {
+	__u32 orig_chunk1;
+	__u16 orig_chunk2;
+	__u16 new_chunk2;
+	__u32 new_chunk1;
+	__u32 flags;
+	mikulas_snapid_t snap_from;
+	mikulas_snapid_t snap_to;
+};
+
+#define BT_SIGNATURE	cpu_to_be32(0xF6014254)
+
+struct dm_multisnap_bt_node {
+	__u32 signature;
+	__u32 n_entries;
+	struct dm_multisnap_bt_entry entries[0];
+};
+
+static inline unsigned dm_multisnap_btree_entries(unsigned chunk_size)
+{
+	return (chunk_size - sizeof(struct dm_multisnap_bt_node)) / sizeof(struct dm_multisnap_bt_entry);
+}
+
+
+/* Freelist */
+
+struct dm_multisnap_freelist_entry {
+	__u32 block1;
+	__u16 block2;
+	__u16 run_length;
+};
+
+#define FREELIST_RL_MASK	0x7fff
+#define FREELIST_DATA_FLAG	0x8000
+
+#define FL_SIGNATURE	cpu_to_be32(0xF601464C)
+
+struct dm_multisnap_freelist {
+	__u32 signature;
+	__u32 backlink1;
+	__u16 backlink2;
+	__u32 n_entries;
+	struct dm_multisnap_freelist_entry entries[0];
+};
+
+static inline unsigned dm_multisnap_freelist_entries(unsigned chunk_size)
+{
+	return (chunk_size - sizeof(struct dm_multisnap_freelist)) / sizeof(struct dm_multisnap_freelist);
+}
+
+#endif
Index: linux-2.6.32/drivers/md/dm-multisnap-mikulas.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-mikulas.c
@@ -0,0 +1,667 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-mikulas.h"
+
+static void init_commit_block(struct dm_exception_store *s)
+{
+	int i;
+
+	dm_multisnap_init_freelist(s->freelist, s->chunk_size);
+
+	s->snapshot_num = 0;
+	s->total_allocated = 0;
+	s->data_allocated = 0;
+	s->bitmap_root = 0;
+	s->alloc_rover = 0;
+	s->freelist_ptr = 0;
+	s->delete_rover_chunk = 0;
+	s->delete_rover_snapid = 0;
+	s->bt_root = 0;
+	s->bt_depth = 0;
+	s->flags = 0;
+
+	for (i = 0; i < TMP_REMAP_HASH_SIZE; i++)
+		INIT_HLIST_HEAD(&s->tmp_remap[i]);
+	s->n_used_tmp_remaps = 0;
+	INIT_LIST_HEAD(&s->used_bitmap_tmp_remaps);
+	INIT_LIST_HEAD(&s->used_bt_tmp_remaps);
+	INIT_LIST_HEAD(&s->free_tmp_remaps);
+
+	for (i = 0; i < N_REMAPS; i++) {
+		struct tmp_remap *t = &s->tmp_remap_store[i];
+		list_add(&t->list, &s->free_tmp_remaps);
+	}
+
+	s->dev_size = 0;
+	s->bitmap_depth = 0;
+	s->btree_entries = dm_multisnap_btree_entries(s->chunk_size);
+}
+
+static void load_commit_block(struct dm_exception_store *s)
+{
+	struct dm_buffer *bp;
+	struct multisnap_commit_block *cb;
+	__u64 dev_size;
+	int bitmap_depth;
+	unsigned i;
+
+	cb = dm_bufio_read(s->bufio, s->valid_commit_block, &bp);
+	if (IS_ERR(cb)) {
+		DMERR("load_commit_block: can't re-read commit block %llx", (unsigned long long)s->valid_commit_block);
+		dm_multisnap_set_error(s->dm, PTR_ERR(cb));
+		return;
+	}
+	if (cb->signature != CB_SIGNATURE) {
+		dm_bufio_release(bp);
+		DMERR("load_commit_block: bad signature when re-reading commit block %llx", (unsigned long long)s->valid_commit_block);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	init_commit_block(s);
+
+	dev_size = read_48(cb, dev_size);
+	s->snapshot_num = le32_to_cpu(cb->snapshot_num);
+	s->total_allocated = read_48(cb, total_allocated);
+	s->data_allocated = read_48(cb, data_allocated);
+	s->bitmap_root = read_48(cb, bitmap_root);
+	s->alloc_rover = read_48(cb, alloc_rover);
+	s->freelist_ptr = read_48(cb, freelist);
+	s->delete_rover_chunk = read_48(cb, delete_rover);
+	s->delete_rover_snapid = 0;
+	s->bt_root = read_48(cb, bt_root);
+	s->bt_depth = cb->bt_depth;
+	s->flags = cb->flags;
+
+	if (s->bt_depth > MAX_BT_DEPTH || !s->bt_depth) {
+		dm_bufio_release(bp);
+		DMERR("load_commit_block: invalid b+-tree depth in commit block %llx", (unsigned long long)s->valid_commit_block);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	INIT_LIST_HEAD(&s->free_tmp_remaps);
+	for (i = 0; i < N_REMAPS; i++) {
+		struct tmp_remap *t = &s->tmp_remap_store[i];
+		if (read_48(&cb->tmp_remap[i], old)) {
+			t->old = read_48(&cb->tmp_remap[i], old);
+			t->new = read_48(&cb->tmp_remap[i], new);
+			t->uncommitted = 0;
+			t->bitmap_idx = le32_to_cpu(cb->tmp_remap[i].bitmap_idx);
+			hlist_add_head(&t->hash_list, &s->tmp_remap[TMP_REMAP_HASH(t->old)]);
+			if (t->bitmap_idx == CB_BITMAP_IDX_NONE)
+				list_add(&t->list, &s->used_bt_tmp_remaps);
+			else
+				list_add(&t->list, &s->used_bitmap_tmp_remaps);
+			s->n_used_tmp_remaps++;
+		} else {
+			list_add(&t->list, &s->free_tmp_remaps);
+		}
+	}
+
+	dm_bufio_release(bp);
+
+	if ((chunk_t)(dev_size + s->cb_stride) < (chunk_t)dev_size) {
+		DMERR("load_commit_block: device is too large. Compile kernel with 64-bit sector numbers");
+		dm_multisnap_set_error(s->dm, -ERANGE);
+		return;
+	}
+	bitmap_depth = dm_multisnap_bitmap_depth(s->chunk_shift, dev_size);
+	if (bitmap_depth < 0) {
+		DMERR("load_commit_block: device is too large");
+		dm_multisnap_set_error(s->dm, bitmap_depth);
+		return;
+	}
+	s->dev_size = dev_size;
+	s->bitmap_depth = bitmap_depth;
+
+	dm_multisnap_load_freelist(s);
+}
+
+static void find_commit_block(struct dm_exception_store *s)
+{
+	struct dm_buffer *bp;
+	struct multisnap_commit_block *cb;
+	chunk_t cb_addr = s->sb_commit_block;
+	__u64 sequence;
+	__u64 dev_size;
+	s->valid_commit_block = 0;
+	s->commit_sequence = 0;
+
+try_next:
+	cb = dm_bufio_read(s->bufio, cb_addr, &bp);
+	if (IS_ERR(cb)) {
+		DMERR("find_commit_block: can't read commit block %llx", (unsigned long long)cb_addr);
+		dm_multisnap_set_error(s->dm, PTR_ERR(cb));
+		return;
+	}
+	if (cb->signature != CB_SIGNATURE) {
+		dm_bufio_release(bp);
+		DMERR("find_commit_block: bad signature on commit block %llx", (unsigned long long)cb_addr);
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+
+	sequence = le64_to_cpu(cb->sequence);
+	dev_size = read_48(cb, dev_size);
+
+	dm_bufio_release(bp);
+
+	if (sequence > s->commit_sequence) {
+		s->commit_sequence = sequence;
+		s->valid_commit_block = cb_addr;
+		if ((__u64)cb_addr + s->cb_stride < dev_size) {
+			cb_addr += s->cb_stride;
+			goto try_next;
+		}
+	}
+	if (!s->valid_commit_block) {
+		DMERR("find_commit_block: no valid commit block");
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		return;
+	}
+}
+
+static int get_size(struct dm_exception_store *s, chunk_t *size)
+{
+	__u64 dev_size;
+	dev_size = i_size_read(dm_multisnap_snapshot_bdev(s->dm)->bd_inode) >> s->chunk_shift;
+	*size = dev_size;
+	if ((chunk_t)(dev_size + s->cb_stride) < dev_size)
+		return -EFBIG;
+
+	return 0;
+}
+
+static void initialize_device(struct dm_exception_store *s)
+{
+	int r;
+	struct dm_buffer *bp;
+	struct multisnap_superblock *sb;
+	struct multisnap_commit_block *cb;
+	chunk_t cb_block;
+	chunk_t block_to_write;
+
+	s->cb_stride = CB_STRIDE_DEFAULT;
+
+	r = get_size(s, &s->dev_size);
+	if (r) {
+		DMERR("initialize_device: device is too large. Compile kernel with 64-bit sector numbers");
+		dm_multisnap_set_error(s->dm, r);
+		return;
+	}
+
+	s->total_allocated = 0;
+	s->data_allocated = 0;
+
+	block_to_write = SB_BLOCK + 1;
+
+/* Write btree */
+	dm_multisnap_create_btree(s, &block_to_write);
+	if (dm_multisnap_has_error(s->dm))
+		return;
+
+/* Write bitmaps */
+	dm_multisnap_create_bitmaps(s, block_to_write);
+	if (dm_multisnap_has_error(s->dm))
+		return;
+
+/* Write commit blocks */
+	if (FIRST_CB_BLOCK >= s->dev_size) {
+		DMERR("initialize_device: device is too small");
+		dm_multisnap_set_error(s->dm, -ENOSPC);
+		return;
+	}
+	for (cb_block = FIRST_CB_BLOCK; cb_block < s->dev_size; cb_block += s->cb_stride) {
+		cb = dm_bufio_new(s->bufio, cb_block, &bp);
+		if (IS_ERR(cb)) {
+			DMERR("initialize_device: can't allocate commit block at %llx", (unsigned long long)cb_block);
+			dm_multisnap_set_error(s->dm, PTR_ERR(cb));
+			return;
+		}
+		memset(cb, 0, s->chunk_size);
+		cb->signature = CB_SIGNATURE;
+		cb->sequence = cpu_to_le64(cb_block == FIRST_CB_BLOCK);
+		if (cb_block == FIRST_CB_BLOCK) {
+			cb->snapshot_num = cpu_to_le32(0);
+			write_48(cb, dev_size, s->dev_size);
+			write_48(cb, total_allocated, s->total_allocated);
+			write_48(cb, data_allocated, s->data_allocated);
+			write_48(cb, bitmap_root, s->bitmap_root);
+			write_48(cb, freelist, 0);
+			write_48(cb, delete_rover, 0);
+			write_48(cb, bt_root, s->bt_root);
+			cb->bt_depth = s->bt_depth;
+			cb->flags = 0;
+		}
+		dm_bufio_mark_buffer_dirty(bp);
+		dm_bufio_release(bp);
+	}
+	r = dm_bufio_write_dirty_buffers(s->bufio);
+	if (r) {
+		DMERR("initialize_device: write error when initializing device");
+		dm_multisnap_set_error(s->dm, r);
+		return;
+	}
+
+/* Write super block */
+	sb = dm_bufio_new(s->bufio, SB_BLOCK, &bp);
+	if (IS_ERR(sb)) {
+		DMERR("initialize_device: can't allocate super block");
+		dm_multisnap_set_error(s->dm, PTR_ERR(sb));
+		return;
+	}
+	memset(sb, 0, s->chunk_size);
+	sb->signature = SB_SIGNATURE;
+	sb->chunk_size = cpu_to_le32(s->chunk_size);
+	sb->cb_stride = cpu_to_le32(s->cb_stride);
+	sb->error = cpu_to_le32(0);
+	sb->commit_block = cpu_to_le64(FIRST_CB_BLOCK);
+	dm_bufio_mark_buffer_dirty(bp);
+	dm_bufio_release(bp);
+	r = dm_bufio_write_dirty_buffers(s->bufio);
+	if (r) {
+		DMERR("initialize_device: can't write super block");
+		dm_multisnap_set_error(s->dm, r);
+		return;
+	}
+}
+
+static void extend_exception_store(struct dm_exception_store *s, chunk_t new_size)
+{
+	struct dm_buffer *bp;
+	chunk_t cb_block;
+	struct multisnap_commit_block *cb;
+
+	/*printk("extending store: %Lx -> %Lx\n", (unsigned long long)s->dev_size, (unsigned long long)new_size);*/
+
+/* Write commit blocks */
+	for (cb_block = FIRST_CB_BLOCK; cb_block < new_size; cb_block += s->cb_stride) {
+		cond_resched();
+		if (cb_block < s->dev_size)
+			continue;
+		cb = dm_bufio_new(s->bufio, cb_block, &bp);
+		if (IS_ERR(cb)) {
+			DMERR("initialize_device: can't allocate commit block at %llx", (unsigned long long)cb_block);
+			dm_multisnap_set_error(s->dm, PTR_ERR(cb));
+			return;
+		}
+		memset(cb, 0, s->chunk_size);
+		cb->signature = CB_SIGNATURE;
+		cb->sequence = cpu_to_le64(0);
+		dm_bufio_mark_buffer_dirty(bp);
+		dm_bufio_release(bp);
+	}
+
+	dm_multisnap_extend_bitmaps(s, new_size);
+
+	s->valid_commit_block = (chunk_t)-1;
+
+	dm_multisnap_commit(s);
+}
+
+static int read_super(struct dm_exception_store *s, char **error)
+{
+	struct dm_buffer *bp;
+	struct multisnap_superblock *sb;
+	int initialized;
+	__s32 e;
+
+	init_commit_block(s);
+
+	initialized = 0;
+re_read:
+	sb = dm_bufio_read(s->bufio, SB_BLOCK, &bp);
+	if (IS_ERR(sb)) {
+		*error = "Could not read superblock";
+		return PTR_ERR(sb);
+	}
+
+	if (sb->signature != SB_SIGNATURE) {
+		int i;
+		if (initialized) {
+			*error = "Invalid signature after initialization";
+			return -EIO;
+		}
+		for (i = 0; i < 1 << SECTOR_SHIFT; i++) {
+			if (((char *)sb)[i]) {
+				dm_bufio_release(bp);
+				*error = "Uninitialized device";
+				return -ENXIO;
+			}
+		}
+		dm_bufio_release(bp);
+		initialize_device(s);
+		if (dm_multisnap_has_error(s->dm)) {
+			*error = "Can't initialize device";
+			return dm_multisnap_has_error(s->dm);
+		}
+		initialized = 1;
+		goto re_read;
+	}
+	if (le32_to_cpu(sb->chunk_size) != s->chunk_size) {
+		dm_bufio_release(bp);
+		*error = "Bad chunk size";
+		return -EINVAL;
+	}
+	s->cb_stride = le32_to_cpu(sb->cb_stride);
+	if (s->cb_stride <= 1) {
+		dm_bufio_release(bp);
+		*error = "Bad commit block stride in superblock";
+		return -EFSERROR;
+	}
+	s->sb_commit_block = le64_to_cpu(sb->commit_block);
+	e = le32_to_cpu(sb->error);
+	dm_bufio_release(bp);
+
+	find_commit_block(s);
+
+	if (dm_multisnap_has_error(s->dm)) {
+		if (dm_multisnap_drop_on_error(s->dm))
+			return 0;
+		*error = "Unable to find commit block";
+		return dm_multisnap_has_error(s->dm);
+	}
+
+	load_commit_block(s);
+
+	if (dm_multisnap_has_error(s->dm)) {
+		if (dm_multisnap_drop_on_error(s->dm))
+			return 0;
+		*error = "Unable to load commit block";
+		return dm_multisnap_has_error(s->dm);
+	}
+
+	if (e < 0) {
+		/* Don't read the B+-tree if there was an error */
+		DMERR("read_super: activating invalidated snapshot store, error %d", e);
+		dm_multisnap_set_error(s->dm, e);
+		return 0;
+	}
+
+	dm_multisnap_read_snapshots(s);
+	if (dm_multisnap_has_error(s->dm)) {
+		if (dm_multisnap_drop_on_error(s->dm))
+			return 0;
+		*error = "Could not read snapshot list";
+		return dm_multisnap_has_error(s->dm);
+	}
+
+	return 0;
+}
+
+static void dm_multisnap_mikulas_lock_acquired(struct dm_exception_store *s, int flags)
+{
+	int r;
+	chunk_t new_size;
+
+	if (!dm_multisnap_can_commit(s->dm))
+		return;
+
+	r = get_size(s, &new_size);
+	if (unlikely(r))
+		return;
+
+	if (unlikely(new_size != s->dev_size)) {
+		if (unlikely(new_size < s->dev_size)) {
+			DMERR("dm_multisnap_mikulas_lock_acquired: device shrinked");
+			dm_multisnap_set_error(s->dm, -EINVAL);
+			return;
+		}
+		extend_exception_store(s, new_size);
+	}
+}
+
+/*#define PRINT_BTREE*/
+
+#ifdef PRINT_BTREE
+static int print_btree_callback(struct dm_exception_store *s, struct dm_multisnap_bt_entry *bt, void *cookie)
+{
+	printk(KERN_DEBUG "entry: %llx, %x-%x -> %llx\n", (unsigned long long)read_48(bt, orig_chunk), mikulas_snapid_to_cpu(bt->snap_from), mikulas_snapid_to_cpu(bt->snap_to), (unsigned long long)read_48(bt, new_chunk));
+	return 0;
+}
+
+static void print_btree(struct dm_exception_store *s)
+{
+	struct bt_key key = { 0, 0, 0 };
+	int r = dm_multisnap_list_btree(s, &key, print_btree_callback, NULL);
+	printk(KERN_DEBUG "list ended: %d\n", r);
+}
+#endif
+
+/*#define PRINT_BITMAPS*/
+
+#ifdef PRINT_BITMAPS
+static void print_bitmaps(struct dm_exception_store *s)
+{
+	chunk_t c;
+	printk(KERN_DEBUG "allocated:");
+	for (c = 0; c < s->dev_size; c += s->chunk_size * 8) {
+		struct dm_buffer *bp;
+		unsigned i;
+		void *bmp = dm_multisnap_map_bitmap(s, c >> (s->chunk_shift + 3), &bp, NULL, NULL);
+		if (!bmp)
+			continue;
+		for (i = 0; i < s->chunk_size * 8; i++)
+			if (generic_test_le_bit(i, bmp)) {
+				chunk_t block = c + i;
+				if (!dm_multisnap_is_commit_block(s, block))
+					printk(" %llx", (unsigned long long)block);
+				cond_resched();
+			}
+		}
+
+		dm_bufio_release(bp);
+	}
+	printk("\n");
+}
+#endif
+
+static int dm_multisnap_mikulas_init(struct dm_multisnap *dm, struct dm_exception_store **sp, unsigned argc, char **argv, char **error)
+{
+	int r;
+	struct dm_exception_store *s;
+
+	s = kzalloc(sizeof(struct dm_exception_store), GFP_KERNEL);
+	if (!s) {
+		*error = "Could not allocate private area";
+		r = -ENOMEM;
+		goto bad_private;
+	}
+	*sp = s;
+
+	s->dm = dm;
+	s->chunk_size = dm_multisnap_chunk_size(dm);
+	s->chunk_shift = ffs(s->chunk_size) - 1;
+
+	s->active_snapshots = RB_ROOT;
+	s->n_preallocated_blocks = 0;
+	s->query_active = 0;
+
+	s->delete_work.work = dm_multisnap_background_delete;
+	s->delete_work.queued = 0;
+	s->delete_commit_count = 0;
+
+	s->cache_threshold = 0;
+	s->cache_limit = 0;
+
+	while (argc) {
+		char *string;
+		r = dm_multisnap_get_string(&argv, &argc, &string, error);
+		if (r)
+			goto bad_arguments;
+		if (!strcasecmp(string, "cache-threshold")) {
+			r = dm_multisnap_get_uint64(&argv, &argc, &s->cache_threshold, error);
+			if (r)
+				goto bad_arguments;
+		} else if (!strcasecmp(string, "cache-limit")) {
+			r = dm_multisnap_get_uint64(&argv, &argc, &s->cache_limit, error);
+			if (r)
+				goto bad_arguments;
+		} else {
+			*error = "Unknown parameter";
+			r = -EINVAL;
+			goto bad_arguments;
+		}
+	}
+
+
+	s->tmp_chunk = vmalloc(s->chunk_size + sizeof(struct dm_multisnap_bt_entry));
+	if (!s->tmp_chunk) {
+		*error = "Can't allocate temporary chunk";
+		r = -ENOMEM;
+		goto bad_tmp_chunk;
+	}
+
+	s->freelist = vmalloc(s->chunk_size);
+	if (!s->freelist) {
+		*error = "Can't allocate freelist";
+		r = -ENOMEM;
+		goto bad_freelist;
+	}
+
+	s->bufio = dm_bufio_client_create(dm_multisnap_snapshot_bdev(s->dm), s->chunk_size, 0, s->cache_threshold, s->cache_limit);
+	if (IS_ERR(s->bufio)) {
+		*error = "Can't create bufio client";
+		r = PTR_ERR(s->bufio);
+		goto bad_bufio;
+	}
+
+	r = read_super(s, error);
+	if (r)
+		goto bad_super;
+
+	if (s->flags & (MULTISNAP_FLAG_DELETING | MULTISNAP_FLAG_PENDING_DELETE))
+		dm_multisnap_queue_work(s->dm, &s->delete_work);
+
+#ifdef PRINT_BTREE
+	print_btree(s);
+#endif
+#ifdef PRINT_BITMAPS
+	print_bitmaps(s);
+#endif
+
+	return 0;
+
+bad_super:
+	dm_bufio_client_destroy(s->bufio);
+bad_bufio:
+	vfree(s->freelist);
+bad_freelist:
+	vfree(s->tmp_chunk);
+bad_tmp_chunk:
+bad_arguments:
+	kfree(s);
+bad_private:
+	return r;
+}
+
+static void dm_multisnap_mikulas_exit(struct dm_exception_store *s)
+{
+	int i;
+
+	dm_multisnap_cancel_work(s->dm, &s->delete_work);
+
+	i = 0;
+	while (!list_empty(&s->used_bitmap_tmp_remaps)) {
+		struct tmp_remap *t = list_first_entry(&s->used_bitmap_tmp_remaps, struct tmp_remap, list);
+		list_del(&t->list);
+		hlist_del(&t->hash_list);
+		i++;
+	}
+
+	while (!list_empty(&s->used_bt_tmp_remaps)) {
+		struct tmp_remap *t = list_first_entry(&s->used_bt_tmp_remaps, struct tmp_remap, list);
+		list_del(&t->list);
+		hlist_del(&t->hash_list);
+		i++;
+	}
+
+	BUG_ON(i != s->n_used_tmp_remaps);
+	while (!list_empty(&s->free_tmp_remaps)) {
+		struct tmp_remap *t = list_first_entry(&s->free_tmp_remaps, struct tmp_remap, list);
+		list_del(&t->list);
+		i++;
+	}
+	BUG_ON(i != N_REMAPS);
+
+	for (i = 0; i < TMP_REMAP_HASH_SIZE; i++)
+		BUG_ON(!hlist_empty(&s->tmp_remap[i]));
+
+	dm_bufio_client_destroy(s->bufio);
+	vfree(s->freelist);
+	vfree(s->tmp_chunk);
+	kfree(s);
+}
+
+static void dm_multisnap_status_table(struct dm_exception_store *s, char *result, unsigned maxlen)
+{
+	int npar = 0;
+	if (s->cache_threshold)
+		npar += 2;
+	if (s->cache_limit)
+		npar += 2;
+
+	snprintf(result, maxlen, " %d", npar);
+	dm_multisnap_adjust_string(&result, &maxlen);
+
+	if (s->cache_threshold) {
+		snprintf(result, maxlen, " cache-threshold %llu", (unsigned long long)s->cache_threshold);
+		dm_multisnap_adjust_string(&result, &maxlen);
+	}
+	if (s->cache_limit) {
+		snprintf(result, maxlen, " cache-limit %llu", (unsigned long long)s->cache_limit);
+		dm_multisnap_adjust_string(&result, &maxlen);
+	}
+}
+
+struct dm_multisnap_exception_store dm_multisnap_mikulas_store = {
+	.name			= "mikulas",
+	.module			= THIS_MODULE,
+	.init_exception_store	= dm_multisnap_mikulas_init,
+	.exit_exception_store	= dm_multisnap_mikulas_exit,
+	.store_lock_acquired	= dm_multisnap_mikulas_lock_acquired,
+#ifdef CONFIG_DM_MULTISNAPSHOT_MIKULAS_SNAP_OF_SNAP
+	.print_snapid		= dm_multisnap_print_snapid,
+	.read_snapid		= dm_multisnap_read_snapid,
+#endif
+	.status_table		= dm_multisnap_status_table,
+	.get_space		= dm_multisnap_get_space,
+	.allocate_snapid	= dm_multisnap_allocate_snapid,
+	.create_snapshot	= dm_multisnap_create_snapshot,
+	.delete_snapshot	= dm_multisnap_delete_snapshot,
+	.get_next_snapid	= dm_multisnap_get_next_snapid,
+	.compare_snapids_for_create = dm_multisnap_compare_snapids_for_create,
+	.find_snapshot_chunk	= dm_multisnap_find_snapshot_chunk,
+	.reset_query		= dm_multisnap_reset_query,
+	.query_next_remap	= dm_multisnap_query_next_remap,
+	.add_next_remap		= dm_multisnap_add_next_remap,
+	.make_chunk_writeable	= dm_multisnap_make_chunk_writeable,
+	.check_conflict		= dm_multisnap_check_conflict,
+	.commit			= dm_multisnap_commit,
+};
+
+static int __init dm_multisnapshot_mikulas_module_init(void)
+{
+	BUG_ON(sizeof(struct multisnap_commit_block) != 512);
+	return dm_multisnap_register_exception_store(&dm_multisnap_mikulas_store);
+}
+
+static void __exit dm_multisnapshot_mikulas_module_exit(void)
+{
+	dm_multisnap_unregister_exception_store(&dm_multisnap_mikulas_store);
+}
+
+module_init(dm_multisnapshot_mikulas_module_init);
+module_exit(dm_multisnapshot_mikulas_module_exit);
+
+MODULE_DESCRIPTION(DM_NAME " multisnapshot Mikulas' exceptions store");
+MODULE_AUTHOR("Mikulas Patocka");
+MODULE_LICENSE("GPL");
+
Index: linux-2.6.32/drivers/md/dm-multisnap-mikulas.h
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-mikulas.h
@@ -0,0 +1,201 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#ifndef DM_MULTISNAP_MIKULAS_H
+#define DM_MULTISNAP_MIKULAS_H
+
+/*
+ * This can be optionally undefined to get 32-bit snapshot numbers.
+ * Breaks on-disk format compatibility.
+ */
+#define CONFIG_DM_MULTISNAPSHOT_MIKULAS_SNAP_OF_SNAP
+
+#include "dm-multisnap.h"
+#include "dm-multisnap-mikulas-struct.h"
+
+#include "dm-bufio.h"
+
+#include <linux/vmalloc.h>
+
+typedef __u32 bitmap_t;
+
+#define read_48(struc, entry)		(le32_to_cpu((struc)->entry##1) | ((chunk_t)le16_to_cpu((struc)->entry##2) << 31 << 1))
+#define write_48(struc, entry, val)	do { (struc)->entry##1 = cpu_to_le32(val); (struc)->entry##2 = cpu_to_le16((chunk_t)(val) >> 31 >> 1); } while (0)
+
+#define TMP_REMAP_HASH_SIZE	256
+#define TMP_REMAP_HASH(c)	((c) & (TMP_REMAP_HASH_SIZE - 1))
+
+struct tmp_remap {
+	/* List entry for tmp_remap */
+	struct hlist_node hash_list;
+	/* List entry for used_tmp_remaps/free_tmp_remaps */
+	struct list_head list;
+	chunk_t old;
+	chunk_t new;
+	bitmap_t bitmap_idx;
+	int uncommitted;
+};
+
+struct bt_key {
+	chunk_t chunk;
+	mikulas_snapid_t snap_from;
+	mikulas_snapid_t snap_to;
+};
+
+struct path_element {
+	chunk_t block;
+	unsigned idx;
+	unsigned n_entries;
+};
+
+struct dm_exception_store {
+	struct dm_multisnap *dm;
+	struct dm_bufio_client *bufio;
+
+	chunk_t dev_size;
+	unsigned chunk_size;
+	unsigned char chunk_shift;
+	unsigned char bitmap_depth;
+	unsigned btree_entries;
+	__u8 bt_depth;
+	__u8 flags;
+	__u32 snapshot_num;
+	unsigned cb_stride;
+
+	chunk_t bitmap_root;
+	chunk_t alloc_rover;
+	chunk_t bt_root;
+	chunk_t sb_commit_block;
+	chunk_t valid_commit_block;
+	chunk_t delete_rover_chunk;
+	mikulas_snapid_t delete_rover_snapid;
+
+	chunk_t total_allocated;
+	chunk_t data_allocated;
+
+	__u64 commit_sequence;
+
+	void *tmp_chunk;
+
+	struct rb_root active_snapshots;
+
+	/* Used during query/add remap */
+	chunk_t query_snapid;
+	struct bt_key query_new_key;
+	unsigned char query_active;
+	chunk_t query_block_from;
+	chunk_t query_block_to;
+
+	/* List heads for struct tmp_remap->list */
+	unsigned n_used_tmp_remaps;
+	struct list_head used_bitmap_tmp_remaps;
+	struct list_head used_bt_tmp_remaps;
+	struct list_head free_tmp_remaps;
+	/* List head for struct tmp_remap->hash_list */
+	struct hlist_head tmp_remap[TMP_REMAP_HASH_SIZE];
+	struct tmp_remap tmp_remap_store[N_REMAPS];
+
+	unsigned n_preallocated_blocks;
+	chunk_t preallocated_blocks[MAX_BITMAP_DEPTH * 2];
+
+	struct dm_multisnap_freelist *freelist;
+	chunk_t freelist_ptr;
+
+	struct dm_multisnap_background_work delete_work;
+	unsigned delete_commit_count;
+
+	__u64 cache_threshold;
+	__u64 cache_limit;
+};
+
+/* dm-multisnap-alloc.c */
+
+void dm_multisnap_create_bitmaps(struct dm_exception_store *s, chunk_t start);
+void dm_multisnap_extend_bitmaps(struct dm_exception_store *s, chunk_t new_size);
+void *dm_multisnap_map_bitmap(struct dm_exception_store *s, bitmap_t bitmap, struct dm_buffer **bp, chunk_t *block, struct path_element *path);
+int dm_multisnap_alloc_blocks(struct dm_exception_store *s, chunk_t *results, unsigned n_blocks, int flags);
+#define ALLOC_DRY	1
+void *dm_multisnap_alloc_duplicate_block(struct dm_exception_store *s, chunk_t block, struct dm_buffer **bp, void *ptr);
+void *dm_multisnap_alloc_make_block(struct dm_exception_store *s, chunk_t *result, struct dm_buffer **bp);
+void dm_multisnap_free_blocks_immediate(struct dm_exception_store *s, chunk_t block, unsigned n_blocks);
+void dm_multisnap_bitmap_finalize_tmp_remap(struct dm_exception_store *s, struct tmp_remap *tmp_remap);
+
+/* dm-multisnap-blocks.c */
+
+chunk_t dm_multisnap_remap_block(struct dm_exception_store *s, chunk_t block);
+void *dm_multisnap_read_block(struct dm_exception_store *s, chunk_t block, struct dm_buffer **bp);
+int dm_multisnap_block_is_uncommitted(struct dm_exception_store *s, chunk_t block);
+void *dm_multisnap_duplicate_block(struct dm_exception_store *s, chunk_t old_chunk, chunk_t new_chunk, bitmap_t bitmap_idx, struct dm_buffer **bp, chunk_t *to_free);
+void dm_multisnap_free_tmp_remap(struct dm_exception_store *s, struct tmp_remap *t);
+void *dm_multisnap_make_block(struct dm_exception_store *s, chunk_t new_chunk, struct dm_buffer **bp);
+void dm_multisnap_free_block_and_duplicates(struct dm_exception_store *s, chunk_t chunk);
+
+int dm_multisnap_is_commit_block(struct dm_exception_store *s, chunk_t block);
+
+typedef chunk_t stop_cycles_t[2];
+
+void dm_multisnap_init_stop_cycles(stop_cycles_t *cy);
+int dm_multisnap_stop_cycles(struct dm_exception_store *s, stop_cycles_t *cy, chunk_t key);
+
+/* dm-multisnap-btree.c */
+
+void dm_multisnap_create_btree(struct dm_exception_store *s, chunk_t *start);
+int dm_multisnap_find_in_btree(struct dm_exception_store *s, struct bt_key *key, chunk_t *result);
+void dm_multisnap_add_to_btree(struct dm_exception_store *s, struct bt_key *key, chunk_t new_chunk);
+void dm_multisnap_restrict_btree_entry(struct dm_exception_store *s, struct bt_key *key);
+void dm_multisnap_extend_btree_entry(struct dm_exception_store *s, struct bt_key *key);
+void dm_multisnap_delete_from_btree(struct dm_exception_store *s, struct bt_key *key);
+void dm_multisnap_bt_finalize_tmp_remap(struct dm_exception_store *s, struct tmp_remap *tmp_remap);
+int dm_multisnap_list_btree(struct dm_exception_store *s, struct bt_key *key, int (*call)(struct dm_exception_store *, struct dm_multisnap_bt_node *, struct dm_multisnap_bt_entry *, void *), void *cookie);
+
+/* dm-multisnap-commit.c */
+
+void dm_multisnap_transaction_mark(struct dm_exception_store *s);
+void dm_multisnap_commit(struct dm_exception_store *s);
+
+/* dm-multisnap-delete.c */
+
+void dm_multisnap_background_delete(struct dm_exception_store *s, struct dm_multisnap_background_work *bw);
+
+/* dm-multisnap-freelist.c */
+
+void dm_multisnap_init_freelist(struct dm_multisnap_freelist *fl, unsigned chunk_size);
+void dm_multisnap_free_block(struct dm_exception_store *s, chunk_t block, unsigned flags);
+int dm_multisnap_check_allocated_block(struct dm_exception_store *s, chunk_t block);
+void dm_multisnap_flush_freelist_before_commit(struct dm_exception_store *s);
+void dm_multisnap_load_freelist(struct dm_exception_store *s);
+
+/* dm-multisnap-io.c */
+
+int dm_multisnap_find_snapshot_chunk(struct dm_exception_store *s, snapid_t snapid, chunk_t chunk, int write, chunk_t *result);
+void dm_multisnap_reset_query(struct dm_exception_store *s);
+int dm_multisnap_query_next_remap(struct dm_exception_store *s, chunk_t chunk);
+void dm_multisnap_add_next_remap(struct dm_exception_store *s, union chunk_descriptor *cd, chunk_t *new_chunk);
+void dm_multisnap_make_chunk_writeable(struct dm_exception_store *s, union chunk_descriptor *cd, chunk_t *new_chunk);
+int dm_multisnap_check_conflict(struct dm_exception_store *s, union chunk_descriptor *cd, snapid_t snapid);
+
+/* dm-multisnap-snaps.c */
+
+snapid_t dm_multisnap_get_next_snapid(struct dm_exception_store *s, snapid_t snapid);
+int dm_multisnap_compare_snapids_for_create(const void *p1, const void *p2);
+int dm_multisnap_find_next_snapid_range(struct dm_exception_store *s, snapid_t snapid, snapid_t *from, snapid_t *to);
+
+void dm_multisnap_destroy_snapshot_tree(struct dm_exception_store *s);
+void dm_multisnap_read_snapshots(struct dm_exception_store *s);
+int dm_multisnap_allocate_snapid(struct dm_exception_store *s, snapid_t *snapid);
+int dm_multisnap_create_snapshot(struct dm_exception_store *s, snapid_t snapid);
+int dm_multisnap_delete_snapshot(struct dm_exception_store *s, snapid_t snapid);
+
+void dm_multisnap_get_space(struct dm_exception_store *s, unsigned long long *chunks_total, unsigned long long *chunks_allocated, unsigned long long *chunks_metadata_allocated);
+
+#ifdef CONFIG_DM_MULTISNAPSHOT_MIKULAS_SNAP_OF_SNAP
+void dm_multisnap_print_snapid(struct dm_exception_store *s, char *string, unsigned maxlen, snapid_t snapid);
+int dm_multisnap_read_snapid(struct dm_exception_store *s, char *string, snapid_t *snapid, char **error);
+#endif
+
+#endif
Index: linux-2.6.32/drivers/md/dm-multisnap-private.h
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-private.h
@@ -0,0 +1,145 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#ifndef DM_MULTISNAP_PRIVATE_H
+#define DM_MULTISNAP_PRIVATE_H
+
+#include "dm-multisnap.h"
+
+/*
+ * Private structures for dm-multisnap.c.
+ * This file should not be included by exception store drivers.
+ */
+
+#include <linux/dm-kcopyd.h>
+
+#define PENDING_HASH_SIZE	256
+#define PENDING_HASH(c)		((c) & (PENDING_HASH_SIZE - 1))
+#define PENDING_MEMPOOL_SIZE	256
+
+#define MULTISNAP_KCOPYD_PAGES	(((1UL << 20) >> PAGE_SHIFT) ? : 1)
+
+#define MAX_CHUNKS_TO_REMAP	DM_KCOPYD_MAX_REGIONS
+
+#define DM_TRACKED_CHUNK_HASH_SIZE	16
+#define DM_TRACKED_CHUNK_HASH(x)	((unsigned long)(x) & (DM_TRACKED_CHUNK_HASH_SIZE - 1))
+#define DM_TRACKED_CHUNK_POOL_SIZE	256
+
+struct dm_multisnap {
+	struct dm_exception_store *p;
+	struct dm_multisnap_exception_store *store;
+
+	struct dm_dev *origin;
+	struct dm_dev *snapshot;
+
+	int error;
+
+	unsigned chunk_size;
+	unsigned char chunk_shift;
+
+	unsigned char flags;
+#define DM_MULTISNAP_SYNC_SNAPSHOTS		1
+#define DM_MULTISNAP_PRESERVE_ON_ERROR		2
+
+	sector_t origin_sectors;
+
+	struct mutex master_lock;
+	struct mutex status_lock;
+	struct workqueue_struct *wq;
+	struct work_struct work;
+	struct bio_list bios; /* protected with dm_multisnap_bio_list_lock */
+	struct list_head background_works;
+
+	/* All snapshot IOs */
+	mempool_t *tracked_chunk_pool;
+
+	/* these two are protected with dm_multisnap_bio_list_lock */
+	long n_tracked_ios;
+	struct hlist_head tracked_chunk_hash[DM_TRACKED_CHUNK_HASH_SIZE];
+
+	mempool_t *pending_pool;
+
+	struct dm_kcopyd_client *kcopyd;
+
+	/*
+	 * The following two variables do a trick to avoid the need for
+	 * atomic operations.
+	 *
+	 * kcopyd_jobs_submitted_count is incremented each time a job is
+	 * submitted to kcopyd. master_lock protects it.
+	 *
+	 * kcopyd_jobs_finished_count is incremented each time a kcopyd
+	 * callback is called. The callback is single-threaded, so it needs
+	 * no protection.
+	 *
+	 * Both kcopyd_jobs_submitted_count and kcopyd_jobs_finished_count
+	 * can be updated simultaneously. But none of these variables is
+	 * updated multiple times concurrently.
+	 *
+	 * When these two are equal, there are no jobs in flight. When they
+	 * are equal and master_lock is held, we know that there are no jobs
+	 * in flight and no new can be submitted --- i.e. we can commit.
+	 */
+	unsigned long kcopyd_jobs_submitted_count;
+	unsigned long kcopyd_jobs_finished_count;
+
+	/* This may only be accessed from kcopyd callback, it has no locking */
+	struct list_head pes_waiting_for_commit;
+
+	/* Increased each time a commit happens */
+	unsigned commit_sequence;
+
+	/* List head for struct dm_multisnap_pending_exception->hash_list */
+	struct hlist_head pending_hash[PENDING_HASH_SIZE];
+
+	char pending_mempool_allocation_failed;
+
+	/* The new snapshot id to be created */
+	char new_snapid_valid;
+	snapid_t new_snapid;
+
+	/* List head for struct dm_multisnap_snap->list_snaps */
+	struct list_head all_snaps;
+
+	/* List entry for all_multisnapshots */
+	struct list_head list_all;
+};
+
+struct dm_multisnap_snap {
+	struct dm_multisnap *s;
+	snapid_t snapid;
+	/* List entry for struct dm_multisnap->list_all */
+	struct list_head list_snaps;
+	char origin_name[16];
+	char snapid_string[1];
+};
+
+struct dm_multisnap_tracked_chunk {
+	struct hlist_node node;
+	chunk_t chunk;
+	unsigned long bio_rw;
+	struct dm_multisnap *s;
+};
+
+struct dm_multisnap_pending_exception {
+	/* List entry for struct dm_multisnap->pending_hash */
+	struct hlist_node hash_list;
+
+	struct dm_multisnap *s;
+	struct bio_list bios;
+
+	chunk_t chunk;
+
+	int n_descs;
+	union chunk_descriptor desc[MAX_CHUNKS_TO_REMAP];
+
+	/* List entry for struct dm_multisnap->pes_waiting_for_commit */
+	struct list_head list;
+};
+
+#endif
Index: linux-2.6.32/drivers/md/dm-multisnap-snaps.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap-snaps.c
@@ -0,0 +1,429 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-mikulas.h"
+
+struct snapshot_range {
+	struct rb_node node;
+	mikulas_snapid_t from;
+	mikulas_snapid_t to;
+};
+
+static struct snapshot_range *rb_find_insert_snapshot(struct dm_exception_store *s, mikulas_snapid_t from, mikulas_snapid_t to, int add)
+{
+	struct snapshot_range *new;
+	struct snapshot_range *found = NULL;
+	struct rb_node **p = &s->active_snapshots.rb_node;
+	struct rb_node *parent = NULL;
+	while (*p) {
+		parent = *p;
+#define rn	rb_entry(parent, struct snapshot_range, node)
+		if (to < rn->from) {
+go_left:
+			p = &rn->node.rb_left;
+		} else if (from > rn->to) {
+			p = &rn->node.rb_right;
+		} else {
+			if (!add) {
+				found = rn;
+		/* If there is range query, we need to find the leftmost node */
+				if (from < rn->from)
+					goto go_left;
+				break;
+			} else {
+				dm_multisnap_set_error(s->dm, -EFSERROR);
+				DMERR("rb_insert_snapshot: inserting overlapping entry: (%llx,%llx) overlaps (%llx,%llx)", (unsigned long long)from, (unsigned long long)to, (unsigned long long)rn->from, (unsigned long long)rn->to);
+				return NULL;
+			}
+		}
+#undef rn
+	}
+	if (!add)
+		return found;
+
+	dm_multisnap_status_assert_locked(s->dm);
+
+	new = kmalloc(sizeof(struct snapshot_range), GFP_KERNEL);
+	if (!new) {
+		DMERR("rb_insert_snapshot: can't allocate memory for snapshot descriptor");
+		dm_multisnap_set_error(s->dm, -ENOMEM);
+		return NULL;
+	}
+
+	new->from = from;
+	new->to = to;
+
+	rb_link_node(&new->node, parent, p);
+	rb_insert_color(&new->node, &s->active_snapshots);
+
+	return new;
+}
+
+static struct snapshot_range *rb_find_snapshot(struct dm_exception_store *s, mikulas_snapid_t from, mikulas_snapid_t to)
+{
+	return rb_find_insert_snapshot(s, from, to, 0);
+}
+
+static int rb_insert_snapshot_unlocked(struct dm_exception_store *s, mikulas_snapid_t from, mikulas_snapid_t to)
+{
+	struct snapshot_range *rn;
+	rn = rb_find_insert_snapshot(s, from, to, 1);
+	if (!rn)
+		return -1;
+	return 0;
+}
+
+static int rb_insert_snapshot(struct dm_exception_store *s, mikulas_snapid_t from, mikulas_snapid_t to)
+{
+	int r;
+	dm_multisnap_status_lock(s->dm);
+	r = rb_insert_snapshot_unlocked(s, from, to);
+	dm_multisnap_status_unlock(s->dm);
+	return r;
+}
+
+static int rb_extend_range(struct dm_exception_store *s, mikulas_snapid_t from, mikulas_snapid_t to)
+{
+	struct snapshot_range *rn;
+	rn = rb_find_insert_snapshot(s, from, from, 0);
+	if (!rn) {
+		DMERR("rb_extend_range: snapshot %llx not found", (unsigned long long)from);
+		return -1;
+	}
+	if (rn->to != from) {
+		DMERR("rb_extend_range: bad attempt to extend range: %llx >= %llx", (unsigned long long)rn->to, (unsigned long long)from);
+		return -1;
+	}
+	dm_multisnap_status_lock(s->dm);
+	rn->to = to;
+	dm_multisnap_status_unlock(s->dm);
+	return 0;
+}
+
+static int rb_delete_range(struct dm_exception_store *s, mikulas_snapid_t from, mikulas_snapid_t to)
+{
+	struct snapshot_range *sr = rb_find_snapshot(s, from, from);
+
+	if (!sr || sr->to < to) {
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		DMERR("rb_delete_range: deleting non-existing snapid %llx-%llx", (unsigned long long)from, (unsigned long long)to);
+		return -1;
+	}
+
+	dm_multisnap_status_lock(s->dm);
+	if (sr->from < from) {
+		mikulas_snapid_t orig_to = sr->to;
+		sr->to = from - 1;
+		if (orig_to > to) {
+			if (rb_insert_snapshot_unlocked(s, to + 1, orig_to)) {
+				sr->to = orig_to;
+				dm_multisnap_status_unlock(s->dm);
+				return -1;
+			}
+		}
+	} else {
+		if (sr->to > to) {
+			sr->from = to + 1;
+		} else {
+			rb_erase(&sr->node, &s->active_snapshots);
+			kfree(sr);
+		}
+	}
+	dm_multisnap_status_unlock(s->dm);
+	return 0;
+}
+
+snapid_t dm_multisnap_get_next_snapid(struct dm_exception_store *s, snapid_t snapid)
+{
+	struct snapshot_range *rn;
+
+	rn = rb_find_snapshot(s, snapid, SNAPID_T_MAX);
+	if (!rn)
+		return SNAPID_T_ORIGIN;
+	if (rn->from > snapid)
+		snapid = rn->from;
+	if (rn->to >= (snapid | MIKULAS_SUBSNAPID_MASK))
+		return snapid | MIKULAS_SUBSNAPID_MASK;
+	return snapid;
+}
+
+int dm_multisnap_find_next_snapid_range(struct dm_exception_store *s, snapid_t snapid, snapid_t *from, snapid_t *to)
+{
+	struct snapshot_range *rn;
+	rn = rb_find_snapshot(s, snapid, SNAPID_T_MAX);
+	if (!rn)
+		return 0;
+	*from = rn->from;
+	*to = rn->to;
+	return 1;
+}
+
+void dm_multisnap_destroy_snapshot_tree(struct dm_exception_store *s)
+{
+	struct rb_node *root;
+	while ((root = s->active_snapshots.rb_node)) {
+#define rn	rb_entry(root, struct snapshot_range, node)
+		rb_erase(root, &s->active_snapshots);
+		kfree(rn);
+#undef rn
+	}
+}
+
+void dm_multisnap_read_snapshots(struct dm_exception_store *s)
+{
+	struct bt_key snap_key;
+	chunk_t ignore;
+	int r;
+
+	dm_multisnap_destroy_snapshot_tree(s);
+
+	snap_key.snap_from = 0;
+find_next:
+	snap_key.snap_to = SNAPID_T_MAX;
+	snap_key.chunk = CHUNK_T_SNAP_PRESENT;
+
+	r = dm_multisnap_find_in_btree(s, &snap_key, &ignore);
+
+	if (unlikely(r < 0))
+		return;
+
+	if (r) {
+		/*printk("inserting snapid %llx-%llx\n", (unsigned long long)snap_key.snap_from, (unsigned long long)snap_key.snap_to);*/
+		if (unlikely(snap_key.snap_to > SNAPID_T_MAX)) {
+			dm_multisnap_set_error(s->dm, -EFSERROR);
+			DMERR("dm_multisnap_read_snapshots: invalid snapshot id");
+			return;
+		}
+		r = rb_insert_snapshot(s, snap_key.snap_from, snap_key.snap_to);
+		if (unlikely(r < 0))
+			return;
+		snap_key.snap_from = snap_key.snap_to + 1;
+		goto find_next;
+	}
+}
+
+int dm_multisnap_allocate_snapid(struct dm_exception_store *s, snapid_t *snapid)
+{
+	*snapid = ((mikulas_snapid_t)s->snapshot_num << MIKULAS_SNAPID_STEP_BITS) | MIKULAS_SUBSNAPID_MASK;
+	/*printk("allocating: %x, %llx\n", s->snapshot_num, (unsigned long long)*snapid);*/
+	if (s->snapshot_num == 0xffffffff || *snapid > SNAPID_T_MAX) {
+		DMERR("dm_multisnap_allocate_snapid: 2^32 snapshot limit reached");
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+int dm_multisnap_create_snapshot(struct dm_exception_store *s, snapid_t snapid)
+{
+	int r;
+	struct bt_key snap_key;
+
+	if ((snapid & MIKULAS_SUBSNAPID_MASK) != MIKULAS_SUBSNAPID_MASK) {
+		DMERR("dm_multisnap_create_snapshot: snapshots of snapshots not yet supported");
+		return -EOPNOTSUPP;
+	}
+
+	if ((snapid >> MIKULAS_SNAPID_STEP_BITS) < s->snapshot_num || snapid > SNAPID_T_MAX) {
+		DMERR("dm_multisnap_create_snapshot: invalid snapshot id %llx (allowed range %llx - %llx)", (unsigned long long)snapid, (unsigned long long)s->snapshot_num, (unsigned long long)SNAPID_T_MAX);
+		return -EINVAL;
+	}
+	if (dm_multisnap_snapshot_exists(s->dm, snapid)) {
+		DMERR("dm_multisnap_create_snapshot: snapshot with id %llx already exists", (unsigned long long)snapid);
+		return -EINVAL;
+	}
+
+	if (snapid > MIKULAS_SUBSNAPID_MASK && dm_multisnap_snapshot_exists(s->dm, snapid - MIKULAS_SUBSNAPID_MASK - 1)) {
+		/* Extend existing key range */
+
+		r = rb_extend_range(s, snapid - MIKULAS_SUBSNAPID_MASK - 1, snapid);
+
+		if (r < 0)
+			return dm_multisnap_has_error(s->dm);
+
+		snap_key.chunk = CHUNK_T_SNAP_PRESENT;
+		snap_key.snap_from = snapid - MIKULAS_SUBSNAPID_MASK - 1;
+		snap_key.snap_to = snapid;
+		dm_multisnap_extend_btree_entry(s, &snap_key);
+	} else {
+		/* Add new entry */
+
+		r = rb_insert_snapshot(s, snapid - MIKULAS_SUBSNAPID_MASK, snapid);
+		if (r < 0)
+			return dm_multisnap_has_error(s->dm);
+
+		snap_key.chunk = CHUNK_T_SNAP_PRESENT;
+		snap_key.snap_from = snapid - MIKULAS_SUBSNAPID_MASK;
+		snap_key.snap_to = snapid;
+		dm_multisnap_add_to_btree(s, &snap_key, 0);
+	}
+	if (dm_multisnap_has_error(s->dm))
+		return dm_multisnap_has_error(s->dm);
+
+	/*printk("multisnapshot: created snapshot with id %llu\n", (unsigned long long)snapid);*/
+
+	s->snapshot_num = (snapid >> MIKULAS_SNAPID_STEP_BITS) + 1;
+
+	dm_multisnap_transaction_mark(s);
+	dm_multisnap_commit(s);
+
+	return 0;
+}
+
+int dm_multisnap_delete_snapshot(struct dm_exception_store *s, snapid_t snapid)
+{
+	int r;
+	struct bt_key snap_key;
+	mikulas_snapid_t from, to;
+	chunk_t ignore;
+	struct snapshot_range *sr = rb_find_snapshot(s, snapid, snapid);
+
+	if (!sr) {
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		DMERR("dm_multisnap_delete_snapshot: snapshot id %llx not found in rb-tree", (unsigned long long)snapid);
+	}
+
+	from = to = snapid;
+	if ((snapid & MIKULAS_SUBSNAPID_MASK) == MIKULAS_SUBSNAPID_MASK) {
+		from = snapid & ~MIKULAS_SUBSNAPID_MASK;
+		if (from < sr->from)
+			from = sr->from;
+	}
+
+	r = rb_delete_range(s, from, to);
+	if (r < 0)
+		return dm_multisnap_has_error(s->dm);
+
+	snap_key.chunk = CHUNK_T_SNAP_PRESENT;
+	snap_key.snap_from = from;
+	snap_key.snap_to = from;
+
+	r = dm_multisnap_find_in_btree(s, &snap_key, &ignore);
+	if (r <= 0) {
+		if (!r) {
+			dm_multisnap_set_error(s->dm, -EFSERROR);
+			DMERR("dm_multisnap_delete_snapshot: snapshot id %llx not found in b-tree", (unsigned long long)from);
+		}
+		return dm_multisnap_has_error(s->dm);
+	}
+	if (snap_key.snap_to < to) {
+		dm_multisnap_set_error(s->dm, -EFSERROR);
+		DMERR("dm_multisnap_delete_snapshot: snapshot id %llx-%llx not found in b-tree", (unsigned long long)from, (unsigned long long)to);
+		return dm_multisnap_has_error(s->dm);
+	}
+
+	if (snap_key.snap_from < from) {
+		snap_key.snap_from = from;
+		dm_multisnap_restrict_btree_entry(s, &snap_key);
+
+		dm_multisnap_transaction_mark(s);
+
+		if (dm_multisnap_has_error(s->dm))
+			return dm_multisnap_has_error(s->dm);
+
+		if (snap_key.snap_to > to) {
+			snap_key.snap_from = to + 1;
+			dm_multisnap_add_to_btree(s, &snap_key, 0);
+		}
+	} else {
+		if (snap_key.snap_to > to) {
+			snap_key.snap_to = to;
+			dm_multisnap_restrict_btree_entry(s, &snap_key);
+		} else {
+			dm_multisnap_delete_from_btree(s, &snap_key);
+		}
+	}
+
+	dm_multisnap_transaction_mark(s);
+
+	s->flags |= MULTISNAP_FLAG_PENDING_DELETE;
+	dm_multisnap_queue_work(s->dm, &s->delete_work);
+
+	dm_multisnap_commit(s);
+
+	return 0;
+}
+
+int dm_multisnap_compare_snapids_for_create(const void *p1, const void *p2)
+{
+	mikulas_snapid_t s1 = *(const snapid_t *)p1;
+	mikulas_snapid_t s2 = *(const snapid_t *)p2;
+	mikulas_snapid_t ms1 = s1 >> MIKULAS_SNAPID_STEP_BITS;
+	mikulas_snapid_t ms2 = s2 >> MIKULAS_SNAPID_STEP_BITS;
+	if (ms1 < ms2)
+		return -1;
+	if (ms1 > ms2)
+		return 1;
+	s1 &= MIKULAS_SUBSNAPID_MASK;
+	s2 &= MIKULAS_SUBSNAPID_MASK;
+	if (s1 == MIKULAS_SUBSNAPID_MASK && s2 != MIKULAS_SUBSNAPID_MASK)
+		return -1;
+	if (s1 != MIKULAS_SUBSNAPID_MASK && s2 == MIKULAS_SUBSNAPID_MASK)
+		return 1;
+	if (s1 < s2)
+		return -1;
+	if (s1 > s2)
+		return 1;
+	return 0;
+}
+
+void dm_multisnap_get_space(struct dm_exception_store *s, unsigned long long *chunks_total, unsigned long long *chunks_allocated, unsigned long long *chunks_metadata_allocated)
+{
+	dm_multisnap_status_assert_locked(s->dm);
+	*chunks_total = s->dev_size;
+	*chunks_allocated = s->total_allocated;
+	*chunks_metadata_allocated = s->total_allocated - s->data_allocated;
+}
+
+#ifdef CONFIG_DM_MULTISNAPSHOT_MIKULAS_SNAP_OF_SNAP
+
+void dm_multisnap_print_snapid(struct dm_exception_store *s, char *string, unsigned maxlen, snapid_t snapid)
+{
+	unsigned master = snapid >> MIKULAS_SNAPID_STEP_BITS;
+	unsigned subsnap = snapid & MIKULAS_SUBSNAPID_MASK;
+	if (subsnap == MIKULAS_SUBSNAPID_MASK)
+		snprintf(string, maxlen, "%u", master);
+	else
+		snprintf(string, maxlen, "%u.%u", master, subsnap);
+}
+
+int dm_multisnap_read_snapid(struct dm_exception_store *s, char *string, snapid_t *snapid, char **error)
+{
+	unsigned long master;
+	unsigned long subsnap;
+	if (!string[0]) {
+err:
+		*error = "Invalid snapshot id";
+		return -EINVAL;
+	}
+
+	master = simple_strtoul(string, &string, 10);
+
+	if (!string[0])
+		subsnap = MIKULAS_SUBSNAPID_MASK;
+	else {
+		if (string[0] != '.' || !string[1])
+			goto err;
+		string++;
+		subsnap = simple_strtoul(string, &string, 10);
+		if (string[0])
+			goto err;
+		if (subsnap >= MIKULAS_SUBSNAPID_MASK) {
+bad_number:
+			*error = "Number out of range";
+			return -EINVAL;
+		}
+	}
+
+	if (master >= SNAPID_T_MAX >> MIKULAS_SNAPID_STEP_BITS)
+		goto bad_number;
+
+	*snapid = (mikulas_snapid_t)master << MIKULAS_SNAPID_STEP_BITS | subsnap;
+	return 0;
+}
+
+#endif
Index: linux-2.6.32/drivers/md/dm-multisnap.c
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap.c
@@ -0,0 +1,1871 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#include "dm-multisnap-private.h"
+
+#include <linux/delay.h>
+#include <linux/vmalloc.h>
+#include <linux/sort.h>
+
+static void dm_multisnap_process_bios(struct dm_multisnap *s);
+
+static void dm_multisnap_lock(struct dm_multisnap *s)
+{
+	mutex_lock(&s->master_lock);
+	if (s->p && s->store->store_lock_acquired)
+		s->store->store_lock_acquired(s->p, 0);
+}
+
+static void dm_multisnap_unlock(struct dm_multisnap *s)
+{
+	mutex_unlock(&s->master_lock);
+}
+
+static int dm_multisnap_lock_contended(struct dm_multisnap *s)
+{
+	return !list_empty(&s->master_lock.wait_list);
+}
+
+static void dm_multisnap_assert_locked(struct dm_multisnap *s)
+{
+	BUG_ON(!mutex_is_locked(&s->master_lock));
+}
+
+void dm_multisnap_status_lock(struct dm_multisnap *s)
+{
+	mutex_lock(&s->status_lock);
+}
+EXPORT_SYMBOL(dm_multisnap_status_lock);
+
+void dm_multisnap_status_unlock(struct dm_multisnap *s)
+{
+	mutex_unlock(&s->status_lock);
+}
+EXPORT_SYMBOL(dm_multisnap_status_unlock);
+
+void dm_multisnap_status_assert_locked(struct dm_multisnap *s)
+{
+	BUG_ON(!mutex_is_locked(&s->status_lock));
+}
+EXPORT_SYMBOL(dm_multisnap_status_assert_locked);
+
+struct block_device *dm_multisnap_snapshot_bdev(struct dm_multisnap *s)
+{
+	return s->snapshot->bdev;
+}
+EXPORT_SYMBOL(dm_multisnap_snapshot_bdev);
+
+unsigned dm_multisnap_chunk_size(struct dm_multisnap *s)
+{
+	return s->chunk_size;
+}
+EXPORT_SYMBOL(dm_multisnap_chunk_size);
+
+void dm_multisnap_set_error(struct dm_multisnap *s, int error)
+{
+	if (!s->error)
+		s->error = error;
+	dump_stack();
+}
+EXPORT_SYMBOL(dm_multisnap_set_error);
+
+int dm_multisnap_has_error(struct dm_multisnap *s)
+{
+	return s->error;
+}
+EXPORT_SYMBOL(dm_multisnap_has_error);
+
+int dm_multisnap_drop_on_error(struct dm_multisnap *s)
+{
+	return !(s->flags & DM_MULTISNAP_PRESERVE_ON_ERROR);
+}
+EXPORT_SYMBOL(dm_multisnap_drop_on_error);
+
+static DEFINE_MUTEX(all_multisnapshots_lock);
+static LIST_HEAD(all_multisnapshots);
+
+static chunk_t sector_to_chunk(struct dm_multisnap *s, sector_t sector)
+{
+	return sector >> (s->chunk_shift - SECTOR_SHIFT);
+}
+
+static sector_t chunk_to_sector(struct dm_multisnap *s, chunk_t chunk)
+{
+	return chunk << (s->chunk_shift - SECTOR_SHIFT);
+}
+
+int dm_multisnap_snapshot_exists(struct dm_multisnap *s, snapid_t snapid)
+{
+	return snapid == s->store->get_next_snapid(s->p, snapid);
+}
+EXPORT_SYMBOL(dm_multisnap_snapshot_exists);
+
+/*
+ * Any reading/writing of snapids in table/status/message must go
+ * through this functions, so that snapid format for userspace can
+ * be overriden.
+ */
+
+static void print_snapid(struct dm_multisnap *s, char *string, unsigned maxlen, snapid_t snapid)
+{
+	if (s->store->print_snapid)
+		s->store->print_snapid(s->p, string, maxlen, snapid);
+	else
+		snprintf(string, maxlen, "%llu", (unsigned long long)snapid);
+}
+
+static int read_snapid(struct dm_multisnap *s, char *string, snapid_t *snapid, char **error)
+{
+	if (s->store->read_snapid)
+		return s->store->read_snapid(s->p, string, snapid, error);
+	else {
+		int r;
+
+		char *argv_array[1] = { string };
+		char **argv = argv_array;
+		unsigned argc = 1;
+		__u64 uint64;
+
+		r = dm_multisnap_get_uint64(&argv, &argc, &uint64, error);
+		if (r)
+			return r;
+
+		*snapid = uint64;
+		return 0;
+	}
+}
+
+/* --- bio list --- */
+
+static DEFINE_SPINLOCK(dm_multisnap_bio_list_lock);
+
+static void wakeup_kmultisnapd(struct dm_multisnap *s)
+{
+	queue_work(s->wq, &s->work);
+}
+
+static void dm_multisnap_enqueue_bio_unlocked(struct dm_multisnap *s, struct bio *bio)
+{
+	bio_list_add(&s->bios, bio);
+}
+
+static void dm_multisnap_enqueue_bio(struct dm_multisnap *s, struct bio *bio)
+{
+	spin_lock_irq(&dm_multisnap_bio_list_lock);
+	dm_multisnap_enqueue_bio_unlocked(s, bio);
+	spin_unlock_irq(&dm_multisnap_bio_list_lock);
+}
+
+static void dm_multisnap_enqueue_bio_list(struct dm_multisnap *s, struct bio_list *bl)
+{
+	struct bio *bio;
+	while ((bio = bio_list_pop(bl)))
+		dm_multisnap_enqueue_bio(s, bio);
+}
+
+/* Reduce the size of the bio */
+
+static void bio_trim(struct bio *bio, unsigned size)
+{
+	unsigned i;
+	bio->bi_size = size;
+	for (i = 0; i < bio->bi_vcnt; i++) {
+		if (size <= bio->bi_io_vec[i].bv_len) {
+			bio->bi_io_vec[i].bv_len = size;
+			bio->bi_vcnt = i + 1;
+			bio->bi_flags &= ~(1 << BIO_SEG_VALID);
+			return;
+		}
+		size -= bio->bi_io_vec[i].bv_len;
+	}
+	BUG();
+}
+
+/* --- encode 64-bit snapids in bio */
+
+static snapid_t bio_get_snapid(struct bio *bio)
+{
+	return ((__u64)bio->bi_seg_front_size << 32) | bio->bi_seg_back_size;
+}
+
+static void bio_put_snapid(struct bio *bio, snapid_t snapid)
+{
+	bio->bi_seg_front_size = (__u64)snapid >> 32;
+	bio->bi_seg_back_size = snapid;
+}
+
+/* --- tracked chnuks --- */
+
+static struct kmem_cache *tracked_chunk_cache;
+
+static int chunk_is_tracked(struct dm_multisnap *s, chunk_t chunk)
+{
+	struct dm_multisnap_tracked_chunk *c;
+	struct hlist_node *hn;
+
+	spin_lock_irq(&dm_multisnap_bio_list_lock);
+
+	hlist_for_each_entry(c, hn,
+	    &s->tracked_chunk_hash[DM_TRACKED_CHUNK_HASH(chunk)], node) {
+		if (likely(c->chunk == chunk)) {
+			spin_unlock_irq(&dm_multisnap_bio_list_lock);
+			return 1;
+		}
+	}
+
+	spin_unlock_irq(&dm_multisnap_bio_list_lock);
+
+	return 0;
+}
+
+/* --- pending exception cache --- */
+
+static struct kmem_cache *pending_exception_cache;
+
+#define GFP_PENDING_EXCEPTION	GFP_NOIO
+
+static void pending_exception_ctor(void *pe_)
+{
+	struct dm_multisnap_pending_exception *pe = pe_;
+	bio_list_init(&pe->bios);
+}
+
+static struct dm_multisnap_pending_exception *dm_multisnap_alloc_pending_exception(struct dm_multisnap *s, chunk_t chunk)
+{
+	struct dm_multisnap_pending_exception *pe;
+	/*
+	 * Warning, we don't want to wait. Because we are holding master_lock
+	 * and taking this lock is needed to complete the exception.
+	 *
+	 * If an allocation failure happens, we must go up, drop the lock,
+	 * try dummy mempool allocation and go here again.
+	 */
+	pe = mempool_alloc(s->pending_pool, GFP_PENDING_EXCEPTION & ~__GFP_WAIT);
+	if (unlikely(!pe))
+		return NULL;
+
+	pe->s = s;
+	pe->chunk = chunk;
+	hlist_add_head(&pe->hash_list, &s->pending_hash[PENDING_HASH(chunk)]);
+	return pe;
+}
+
+static void dm_multisnap_free_pending_exception(struct dm_multisnap_pending_exception *pe)
+{
+	hlist_del(&pe->hash_list);
+	mempool_free(pe, pe->s->pending_pool);
+}
+
+static void dm_multisnap_wait_for_pending_exception(struct dm_multisnap *s)
+{
+	/*
+	 * Wait until there is something in the mempool. Free it immediatelly.
+	 */
+	struct dm_multisnap_pending_exception *pe;
+
+	pe = mempool_alloc(s->pending_pool, GFP_PENDING_EXCEPTION | __GFP_WAIT);
+	mempool_free(pe, s->pending_pool);
+}
+
+/*
+ * Check if the chunk+snapid conflicts with any pending exception.
+ *
+ * If it does, queue the bio on the pending exception.
+ */
+
+static int check_pending_io(struct dm_multisnap *s, struct bio *bio, chunk_t chunk, snapid_t snapid)
+{
+	struct dm_multisnap_pending_exception *pe;
+	struct hlist_node *hn;
+	hlist_for_each_entry(pe, hn, &s->pending_hash[PENDING_HASH(chunk)], hash_list) {
+		if (pe->chunk == chunk) {
+			int i;
+			if (snapid == SNAPID_T_ORIGIN)
+				goto conflict;
+			for (i = 0; i < pe->n_descs; i++) {
+				if (s->store->check_conflict(s->p, &pe->desc[i], snapid))
+					goto conflict;
+			}
+		}
+		cond_resched();
+	}
+	return 0;
+
+conflict:
+	bio_list_add(&pe->bios, bio);
+	return 1;
+}
+
+/* --- commit --- */
+
+/*
+ * Test if commit can be performed. If these two variables are not equal,
+ * there are some pending kcopyd jobs and we must not commit.
+ */
+
+int dm_multisnap_can_commit(struct dm_multisnap *s)
+{
+	return s->kcopyd_jobs_submitted_count == s->kcopyd_jobs_finished_count;
+}
+EXPORT_SYMBOL(dm_multisnap_can_commit);
+
+/*
+ * Call exception store commit method.
+ * This can be called only if dm_multisnap_can_commit returned true;
+ * master_lock must be locked.
+ */
+
+void dm_multisnap_call_commit(struct dm_multisnap *s)
+{
+	s->store->commit(s->p);
+	s->commit_sequence++;
+}
+EXPORT_SYMBOL(dm_multisnap_call_commit);
+
+/*
+ * Force commit at this point. It is guaranteed that commit happened when
+ * this function exits.
+ * master_lock must be unlocked.
+ *
+ * If the commit cannot be performed immediatelly (because there are pending
+ * chunks being copied), the function drops the lock and polls. It won't
+ * livelock --- either it will be possible to do the commit or someone
+ * have done the commit already (commit_sequence changed).
+ *
+ * The polling is justified because this function is only called when deleting
+ * a snapshot or when suspending the origin with postsuspend. These functions
+ * are not performance-critical, thus 1ms delay won't cause a performance
+ * problem.
+ */
+
+static int dm_multisnap_force_commit(struct dm_multisnap *s)
+{
+	int err;
+	unsigned commit_sequence;
+
+	dm_multisnap_lock(s);
+
+	commit_sequence = s->commit_sequence;
+
+	while (!dm_multisnap_can_commit(s)) {
+		dm_multisnap_unlock(s);
+		msleep(1);
+		dm_multisnap_lock(s);
+		if (s->commit_sequence != commit_sequence)
+			goto unlock_ret;
+	}
+
+	dm_multisnap_call_commit(s);
+
+unlock_ret:
+	err = dm_multisnap_has_error(s);
+	dm_multisnap_unlock(s);
+
+	return err;
+}
+
+/* --- kcopyd callback --- */
+
+static void remap_callback(int read_err, unsigned long write_err, void *pe_)
+{
+	struct dm_multisnap_pending_exception *pe = pe_;
+	struct dm_multisnap *s = pe->s;
+
+	if (unlikely((read_err | write_err) != 0)) {
+		DMERR("remap_callback: kcopyd I/O error: %d, %lx", read_err, write_err);
+		dm_multisnap_set_error(s, -EIO);
+	}
+
+	list_add_tail(&pe->list, &s->pes_waiting_for_commit);
+
+	s->kcopyd_jobs_finished_count++;
+
+	/* If there are more jobs pending, don't commit */
+	if (!dm_multisnap_can_commit(s))
+		return;
+
+	dm_multisnap_lock(s);
+
+	/* Recheck after the loc was taken */
+	if (unlikely(!dm_multisnap_can_commit(s))) {
+		/* Not yet ... kmultisnapd has just added something */
+		dm_multisnap_unlock(s);
+		return;
+	}
+
+	/* We need to commit stuff */
+
+	dm_multisnap_call_commit(s);
+
+	do {
+		pe = container_of(s->pes_waiting_for_commit.next, struct dm_multisnap_pending_exception, list);
+
+		/*
+		 * When we are about to free the pending exception, we must
+		 * wait for all reads to the apropriate chunk to
+		 * finish.
+		 *
+		 * This prevents the following race condition:
+		 * - someone reads the chunk in the snapshot with no exception
+		 * - that read is remapped directly to the origin, the read
+		 *	is delayed for some reason
+		 * - someone other writes to the origin, this triggers realloc
+		 * - the realloc finishes
+		 * - the write is dispatched to the origin
+		 * - the read submitted first is dispatched and reads modified
+		 *	data
+		 *
+		 * This race is very improbable (non-shared snapshots had this
+		 * race too and it hasn't ever been reported seen, except in
+		 * artifically simulated cases). So we use active waiting with
+		 * msleep(1).
+		 */
+
+		while (chunk_is_tracked(s, pe->chunk))
+			msleep(1);
+
+		list_del(&pe->list);
+		dm_multisnap_enqueue_bio_list(s, &pe->bios);
+		dm_multisnap_free_pending_exception(pe);
+	} while (!list_empty(&s->pes_waiting_for_commit));
+
+	/*
+	 * Process the bios that we have just added to the queue.
+	 * It's faster to process them now than to hand them over to
+	 * kmultisnapd.
+	 */
+	dm_multisnap_process_bios(s);
+
+	dm_multisnap_unlock(s);
+
+	blk_unplug(bdev_get_queue(s->origin->bdev));
+	blk_unplug(bdev_get_queue(s->snapshot->bdev));
+}
+
+static void dispatch_kcopyd(struct dm_multisnap *s, struct dm_multisnap_pending_exception *pe, int from_snapshot, chunk_t chunk, struct bio *bio, struct dm_io_region *dests, unsigned n_dests)
+{
+	unsigned i;
+	struct dm_io_region src;
+
+	pe->n_descs = n_dests;
+
+	bio_list_add(&pe->bios, bio);
+
+	src.bdev = likely(!from_snapshot) ? s->origin->bdev : s->snapshot->bdev;
+	src.sector = chunk_to_sector(s, chunk);
+	src.count = s->chunk_size >> SECTOR_SHIFT;
+
+	if (likely(!from_snapshot) && unlikely(src.sector + src.count > s->origin_sectors)) {
+		if (src.sector >= s->origin_sectors)
+			src.count = 0;
+		else
+			src.count = s->origin_sectors - src.sector;
+
+		for (i = 0; i < pe->n_descs; i++)
+			dests[i].count = src.count;
+	}
+
+	s->kcopyd_jobs_submitted_count++;
+
+	dm_kcopyd_copy(s->kcopyd, &src, n_dests, dests, 0, remap_callback, pe);
+}
+
+/* --- bio processing --- */
+
+/*
+ * Process bio on the origin.
+ * Reads and barriers never go here, they are dispatched directly.
+ */
+
+static void do_origin_write(struct dm_multisnap *s, struct bio *bio)
+{
+	int r;
+	unsigned i;
+	chunk_t chunk, new_chunk;
+	struct dm_multisnap_pending_exception *pe;
+	struct dm_io_region dests[MAX_CHUNKS_TO_REMAP];
+
+	/* reads are processed directly in multisnap_origin_map */
+	BUG_ON(bio_rw(bio) != WRITE);
+
+	if (bio->bi_sector + (bio->bi_size >> SECTOR_SHIFT) > s->origin_sectors) {
+		DMERR("do_origin_write: access out of device, flags %lx, sector %llx, size %x, origin sectors %llx", bio->bi_flags, (unsigned long long)bio->bi_sector, bio->bi_size, (unsigned long long)s->origin_sectors);
+		bio_endio(bio, -EIO);
+		return;
+	}
+
+	if (unlikely(dm_multisnap_has_error(s)))
+		goto err_endio;
+
+	s->store->reset_query(s->p);
+
+	chunk = sector_to_chunk(s, bio->bi_sector);
+
+	r = s->store->query_next_remap(s->p, chunk);
+	if (unlikely(r < 0))
+		goto err_endio;
+
+	if (likely(!r)) {
+		/* There is nothing to remap */
+
+		if (unlikely(check_pending_io(s, bio, chunk, SNAPID_T_ORIGIN)))
+			return;
+dispatch_write:
+		bio->bi_bdev = s->origin->bdev;
+		generic_make_request(bio);
+		return;
+	}
+
+	pe = dm_multisnap_alloc_pending_exception(s, chunk);
+	if (unlikely(!pe)) {
+		s->pending_mempool_allocation_failed = 1;
+		dm_multisnap_enqueue_bio(s, bio);
+		return;
+	}
+
+	i = 0;
+	goto midcycle;
+	for (; i < MAX_CHUNKS_TO_REMAP; i++) {
+		r = s->store->query_next_remap(s->p, chunk);
+		if (unlikely(r < 0))
+			goto free_err_endio;
+		if (likely(!r))
+			break;
+
+midcycle:
+		s->store->add_next_remap(s->p, &pe->desc[i], &new_chunk);
+		if (unlikely(dm_multisnap_has_error(s)))
+			goto free_err_endio;
+
+		dests[i].bdev = s->snapshot->bdev;
+		dests[i].sector = chunk_to_sector(s, new_chunk);
+		dests[i].count = s->chunk_size >> SECTOR_SHIFT;
+	}
+
+	dispatch_kcopyd(s, pe, 0, chunk, bio, dests, i);
+	return;
+
+free_err_endio:
+	dm_multisnap_free_pending_exception(pe);
+err_endio:
+	r = -EIO;
+	if (!(s->flags & DM_MULTISNAP_PRESERVE_ON_ERROR))
+		goto dispatch_write;
+
+	bio_endio(bio, r);
+	return;
+}
+
+/*
+ * Process bio on the snapshot.
+ * Barriers never go here, they are dispatched directly.
+ */
+
+static void do_snapshot_io(struct dm_multisnap *s, struct bio *bio, snapid_t id)
+{
+	chunk_t chunk, result, copy_from;
+	int r;
+	struct dm_multisnap_pending_exception *pe;
+	struct dm_io_region dest;
+
+	if (unlikely(bio_rw(bio) == WRITE) && unlikely(!s->store->make_chunk_writeable))
+		goto err_endio;
+
+	if (unlikely(dm_multisnap_has_error(s)))
+		goto err_endio;
+
+	chunk = sector_to_chunk(s, bio->bi_sector);
+	r = s->store->find_snapshot_chunk(s->p, id, chunk, bio_rw(bio) == WRITE, &result);
+	if (unlikely(r < 0))
+		goto err_endio;
+
+	if (!r) {
+
+		/* Not found in the snapshot */
+
+		if (likely(bio_rw(bio) != WRITE)) {
+			union map_info *map_context;
+			struct dm_multisnap_tracked_chunk *c;
+
+			if (unlikely(bio->bi_sector + (bio->bi_size >> SECTOR_SHIFT) > s->origin_sectors)) {
+				zero_fill_bio(bio);
+				if (bio->bi_sector >= s->origin_sectors) {
+					bio_endio(bio, 0);
+					return;
+				}
+				bio_trim(bio, (s->origin_sectors - bio->bi_sector) << SECTOR_SHIFT);
+			}
+
+			/*
+			 * Redirect reads to the origin.
+			 * Record the bio in the hash of tracked bios.
+			 * This prevents read-vs-realloc race.
+			 *
+			 * An important requirement is that when any bio is
+			 * added to tracked_chunk_hash, the bio must be finished
+			 * and removed from the hash without taking master_lock.
+			 *
+			 * So we add it immediatelly before submitting the bio
+			 * with generic_make_request.
+			 */
+
+			bio->bi_bdev = s->origin->bdev;
+
+			map_context = dm_get_mapinfo(bio);
+			BUG_ON(!map_context);
+			c = map_context->ptr;
+
+			spin_lock_irq(&dm_multisnap_bio_list_lock);
+			BUG_ON(!hlist_unhashed(&c->node));
+			hlist_add_head(&c->node, &s->tracked_chunk_hash[DM_TRACKED_CHUNK_HASH(c->chunk)]);
+			spin_unlock_irq(&dm_multisnap_bio_list_lock);
+		} else {
+			pe = dm_multisnap_alloc_pending_exception(s, chunk);
+			if (unlikely(!pe))
+				goto failed_pe_allocation;
+
+			s->store->add_next_remap(s->p, &pe->desc[0], &result);
+			if (unlikely(dm_multisnap_has_error(s)))
+				goto free_err_endio;
+
+			dest.bdev = s->snapshot->bdev;
+			dest.sector = chunk_to_sector(s, result);
+			dest.count = s->chunk_size >> SECTOR_SHIFT;
+
+			dispatch_kcopyd(s, pe, 0, chunk, bio, &dest, 1);
+			return;
+		}
+	} else {
+
+		/* Found in the snapshot */
+
+		if (unlikely(check_pending_io(s, bio, chunk, id)))
+			return;
+
+		if (unlikely(bio_rw(bio) == WRITE) && r == 1) {
+			copy_from = result;
+
+			pe = dm_multisnap_alloc_pending_exception(s, chunk);
+			if (unlikely(!pe))
+				goto failed_pe_allocation;
+
+			s->store->make_chunk_writeable(s->p, &pe->desc[0], &result);
+			if (unlikely(dm_multisnap_has_error(s)))
+				goto free_err_endio;
+
+			dest.bdev = s->snapshot->bdev;
+			dest.sector = chunk_to_sector(s, result);
+			dest.count = s->chunk_size >> SECTOR_SHIFT;
+
+			dispatch_kcopyd(s, pe, 1, copy_from, bio, &dest, 1);
+			return;
+		}
+
+		bio->bi_bdev = s->snapshot->bdev;
+		bio->bi_sector &= (s->chunk_size >> SECTOR_SHIFT) - 1;
+		bio->bi_sector |= chunk_to_sector(s, result);
+	}
+	generic_make_request(bio);
+	return;
+
+free_err_endio:
+	dm_multisnap_free_pending_exception(pe);
+err_endio:
+	r = -EIO;
+	bio_endio(bio, r);
+	return;
+
+failed_pe_allocation:
+	s->pending_mempool_allocation_failed = 1;
+	dm_multisnap_enqueue_bio(s, bio);
+	return;
+}
+
+/*
+ * The main routine used to process everything in the thread.
+ * It must be called with master_lock held.
+ * It is usually called from the worker thread, but can also be called
+ * from other places (for example kcopyd callback), assuming that the caller
+ * holds master_lock.
+ */
+
+static void dm_multisnap_process_bios(struct dm_multisnap *s)
+{
+	struct bio *bio;
+	snapid_t snapid;
+
+again:
+	cond_resched();
+
+	if (!list_empty(&s->background_works)) {
+		struct dm_multisnap_background_work *bw = list_entry(s->background_works.next, struct dm_multisnap_background_work, list);
+		list_del(&bw->list);
+		bw->queued = 0;
+		bw->work(s->p, bw);
+
+		cond_resched();
+	}
+
+	spin_lock_irq(&dm_multisnap_bio_list_lock);
+	bio = bio_list_pop(&s->bios);
+	spin_unlock_irq(&dm_multisnap_bio_list_lock);
+
+	if (unlikely(!bio))
+		return;
+
+	snapid = bio_get_snapid(bio);
+	if (snapid == SNAPID_T_ORIGIN)
+		do_origin_write(s, bio);
+	else
+		do_snapshot_io(s, bio, snapid);
+
+	if (!bio_list_empty(&s->bios) || !list_empty(&s->background_works)) {
+		if (likely(!bio_list_empty(&s->bios)) &&
+		    likely(!s->pending_mempool_allocation_failed) &&
+		    likely(!dm_multisnap_lock_contended(s)))
+			goto again;
+		wakeup_kmultisnapd(s);
+	}
+}
+
+/*
+ * Background-job routines exported for exception store drivers.
+ *
+ * Jobs queued with these routines will be executed on background, with the
+ * master lock held.
+ */
+
+void dm_multisnap_queue_work(struct dm_multisnap *s, struct dm_multisnap_background_work *bw)
+{
+	dm_multisnap_assert_locked(s);
+
+	if (bw->queued) {
+		BUG_ON(bw->queued != 1);
+		return;
+	}
+
+	bw->queued = 1;
+	list_add(&bw->list, &s->background_works);
+	wakeup_kmultisnapd(s);
+}
+EXPORT_SYMBOL(dm_multisnap_queue_work);
+
+void dm_multisnap_cancel_work(struct dm_multisnap *s, struct dm_multisnap_background_work *bw)
+{
+	dm_multisnap_assert_locked(s);
+
+	if (!bw->queued)
+		return;
+
+	bw->queued = 0;
+	list_del(&bw->list);
+}
+EXPORT_SYMBOL(dm_multisnap_cancel_work);
+
+/*
+ * The main work thread.
+ */
+
+static void dm_multisnap_work(struct work_struct *work)
+{
+	struct dm_multisnap *s = container_of(work, struct dm_multisnap, work);
+
+	dm_multisnap_lock(s);
+	dm_multisnap_process_bios(s);
+	dm_multisnap_unlock(s);
+
+	/*
+	 * If there was some mempool allocation failure, we must fail, outside
+	 * the lock, until there is some free memory.
+	 * If this branch is taken, the work is already queued again, so it
+	 * reexecutes after finding some memory.
+	 */
+	if (unlikely(s->pending_mempool_allocation_failed)) {
+		s->pending_mempool_allocation_failed = 0;
+		dm_multisnap_wait_for_pending_exception(s);
+	}
+
+	blk_unplug(bdev_get_queue(s->origin->bdev));
+	blk_unplug(bdev_get_queue(s->snapshot->bdev));
+}
+
+static struct dm_multisnap *find_multisnapshot(struct block_device *origin)
+{
+	struct dm_multisnap *s;
+	list_for_each_entry(s, &all_multisnapshots, list_all)
+		if (s->origin->bdev == origin)
+			return s;
+	return NULL;
+}
+
+/* --- exception stores --- */
+
+static DEFINE_MUTEX(exception_stores_lock);
+static LIST_HEAD(all_exception_stores);
+
+static struct dm_multisnap_exception_store *dm_multisnap_find_exception_store(const char *name)
+{
+	struct dm_multisnap_exception_store *store;
+
+	list_for_each_entry(store, &all_exception_stores, list)
+		if (!strcmp(store->name, name))
+			return store;
+
+	return NULL;
+}
+
+static int dm_multisnap_exception_store_active(struct dm_multisnap_exception_store *find)
+{
+	struct dm_multisnap_exception_store *store;
+
+	list_for_each_entry(store, &all_exception_stores, list)
+		if (store == find)
+			return 1;
+
+	return 0;
+}
+
+int dm_multisnap_register_exception_store(struct dm_multisnap_exception_store *store)
+{
+	mutex_lock(&exception_stores_lock);
+
+	BUG_ON(dm_multisnap_exception_store_active(store));
+
+	if (dm_multisnap_find_exception_store(store->name)) {
+		mutex_unlock(&exception_stores_lock);
+		return -EEXIST;
+	}
+	list_add(&store->list, &all_exception_stores);
+
+	mutex_unlock(&exception_stores_lock);
+
+	return 0;
+}
+EXPORT_SYMBOL(dm_multisnap_register_exception_store);
+
+void dm_multisnap_unregister_exception_store(struct dm_multisnap_exception_store *store)
+{
+	mutex_lock(&exception_stores_lock);
+
+	BUG_ON(!dm_multisnap_exception_store_active(store));
+	list_del(&store->list);
+
+	mutex_unlock(&exception_stores_lock);
+}
+EXPORT_SYMBOL(dm_multisnap_unregister_exception_store);
+
+static struct dm_multisnap_exception_store *dm_multisnap_get_exception_store(const char *name)
+{
+	struct dm_multisnap_exception_store *store;
+
+	mutex_lock(&exception_stores_lock);
+
+	store = dm_multisnap_find_exception_store(name);
+	if (store) {
+		if (!try_module_get(store->module))
+			store = NULL;
+	}
+
+	mutex_unlock(&exception_stores_lock);
+
+	return store;
+}
+
+static void dm_multisnap_put_exception_store(struct dm_multisnap_exception_store *store)
+{
+	mutex_lock(&exception_stores_lock);
+
+	BUG_ON(!dm_multisnap_exception_store_active(store));
+	module_put(store->module);
+
+	mutex_unlock(&exception_stores_lock);
+}
+
+/* --- argument parser --- */
+
+int dm_multisnap_get_string(char ***argv, unsigned *argc, char **string, char **error)
+{
+	if (!*argc) {
+		*error = "Not enough arguments";
+		return -EINVAL;
+	}
+	*string = *(*argv)++;
+	(*argc)--;
+	return 0;
+}
+EXPORT_SYMBOL(dm_multisnap_get_string);
+
+int dm_multisnap_get_uint64(char ***argv, unsigned *argc, __u64 *uint64, char **error)
+{
+	char *string;
+	int r = dm_multisnap_get_string(argv, argc, &string, error);
+	if (r)
+		return r;
+	if (!*string) {
+invalid_number:
+		*error = "Invalid number";
+		return -EINVAL;
+	}
+	*uint64 = simple_strtoull(string, &string, 10);
+	if (*string)
+		goto invalid_number;
+	return 0;
+}
+EXPORT_SYMBOL(dm_multisnap_get_uint64);
+
+int dm_multisnap_get_uint(char ***argv, unsigned *argc, unsigned *uint, char **error)
+{
+	__u64 uint64;
+	int r = dm_multisnap_get_uint64(argv, argc, &uint64, error);
+	if (r)
+		return r;
+	*uint = uint64;
+	if (uint64 != *uint) {
+		*error = "Number out of range";
+		return -ERANGE;
+	}
+	return 0;
+}
+EXPORT_SYMBOL(dm_multisnap_get_uint);
+
+int dm_multisnap_get_argcount(char ***argv, unsigned *argc, unsigned *uint, char **error)
+{
+	int r = dm_multisnap_get_uint(argv, argc, uint, error);
+	if (r)
+		return r;
+	if (*uint > *argc) {
+		*error = "Not enough arguments";
+		return -EINVAL;
+	}
+	return 0;
+}
+EXPORT_SYMBOL(dm_multisnap_get_argcount);
+
+void dm_multisnap_adjust_string(char **result, unsigned *maxlen)
+{
+	unsigned len = strlen(*result);
+	*result += len;
+	*maxlen -= len;
+}
+EXPORT_SYMBOL(dm_multisnap_adjust_string);
+
+/* --- target methods --- */
+
+static int compare_snapids(const void *p1, const void *p2)
+{
+	snapid_t s1 = *(const snapid_t *)p1;
+	snapid_t s2 = *(const snapid_t *)p2;
+	if (s1 < s2)
+		return -1;
+	if (s1 > s2)
+		return 1;
+	return 0;
+}
+
+/* --- constructor & destructor --- */
+
+static int multisnap_origin_ctr(struct dm_target *ti, unsigned argc, char **argv)
+{
+	int r;
+	int i;
+	char *origin_path;
+	char *snapshot_path;
+	unsigned chunk_size;
+	unsigned generic_args;
+	char *store_name;
+	unsigned store_args;
+	unsigned num_snapshots;
+
+	struct dm_multisnap *s, *ss;
+
+	mutex_lock(&all_multisnapshots_lock);
+
+	r = dm_multisnap_get_string(&argv, &argc, &origin_path, &ti->error);
+	if (r)
+		goto bad_arguments;
+	r = dm_multisnap_get_string(&argv, &argc, &snapshot_path, &ti->error);
+	if (r)
+		goto bad_arguments;
+	r = dm_multisnap_get_uint(&argv, &argc, &chunk_size, &ti->error);
+	if (r)
+		goto bad_arguments;
+
+	s = kmalloc(sizeof(struct dm_multisnap), GFP_KERNEL);
+	if (!s) {
+		ti->error = "Can't allocate multisnapshot structure";
+		r = -ENOMEM;
+		goto bad_s;
+	}
+
+	ti->private = s;
+
+	s->p = NULL;
+	s->error = 0;
+	s->flags = 0;
+	mutex_init(&s->master_lock);
+	mutex_init(&s->status_lock);
+	INIT_WORK(&s->work, dm_multisnap_work);
+	bio_list_init(&s->bios);
+	INIT_LIST_HEAD(&s->background_works);
+	s->kcopyd_jobs_submitted_count = 0;
+	s->kcopyd_jobs_finished_count = 0;
+	INIT_LIST_HEAD(&s->pes_waiting_for_commit);
+	s->commit_sequence = 0;
+	for (i = 0; i < PENDING_HASH_SIZE; i++)
+		INIT_HLIST_HEAD(&s->pending_hash[i]);
+	s->pending_mempool_allocation_failed = 0;
+	s->new_snapid_valid = 0;
+	INIT_LIST_HEAD(&s->all_snaps);
+
+	r = dm_multisnap_get_argcount(&argv, &argc, &generic_args, &ti->error);
+	if (r)
+		goto bad_arguments;
+	while (generic_args--) {
+		char *arg;
+		r = dm_multisnap_get_string(&argv, &argc, &arg, &ti->error);
+		if (r)
+			goto bad_generic_arguments;
+
+	/* Synchronize snapshot list against a list given in the target table */
+		if (!strcasecmp(arg, "sync-snapshots"))
+			s->flags |= DM_MULTISNAP_SYNC_SNAPSHOTS;
+	/* Don't drop the snapshot store on error, rather stop the origin */
+		else if (!strcasecmp(arg, "preserve-on-error"))
+			s->flags |= DM_MULTISNAP_PRESERVE_ON_ERROR;
+		else {
+			r = -EINVAL;
+			ti->error = "Invalid argument";
+			goto bad_generic_arguments;
+		}
+	}
+
+	r = dm_get_device(ti, origin_path, 0, 0, FMODE_READ | FMODE_WRITE, &s->origin);
+	if (r) {
+		ti->error = "Could not get origin device";
+		goto bad_origin;
+	}
+	s->origin_sectors = i_size_read(s->origin->bdev->bd_inode) >> SECTOR_SHIFT;
+
+	r = dm_get_device(ti, snapshot_path, 0, 0, FMODE_READ | FMODE_WRITE, &s->snapshot);
+	if (r) {
+		ti->error = "Could not get snapshot device";
+		goto bad_snapshot;
+	}
+
+	/*
+	 * Prevent multiple load over the same devices.
+	 *
+	 * Currently, multisnapshot target is loaded just once, there is no
+	 * place where it would be reloaded (even lvchange --refresh doesn't
+	 * do it), so there is no need to handle loading the target multiple
+	 * times for the same devices and "handover" of the exception store.
+	 *
+	 * As a safeguard to protect against possible data corruption from
+	 * userspace misbehavior, we check that there is no other target loaded
+	 * that has the origin or the snapshot store on the same devices.
+	 */
+
+	list_for_each_entry(ss, &all_multisnapshots, list_all)
+		if (ss->origin->bdev == s->origin->bdev ||
+		    ss->snapshot->bdev == s->snapshot->bdev) {
+			ti->error = "Another multisnapshot with the same devices";
+			r = -EINVAL;
+			goto bad_conflicting_snapshot;
+		}
+
+	/* Validate the chunk size */
+
+	if (chunk_size > INT_MAX / 512) {
+		ti->error = "Chunk size is too high";
+		r = -EINVAL;
+		goto bad_chunk_size;
+	}
+	if (!is_power_of_2(chunk_size)) {
+		ti->error = "Chunk size is not power of two";
+		r = -EINVAL;
+		goto bad_chunk_size;
+	}
+	chunk_size *= 512;
+	if (chunk_size < bdev_logical_block_size(s->origin->bdev) ||
+	    chunk_size < bdev_logical_block_size(s->snapshot->bdev)) {
+		ti->error = "Chunk size is smaller than device block size";
+		r = -EINVAL;
+		goto bad_chunk_size;
+	}
+	s->chunk_size = chunk_size;
+	s->chunk_shift = ffs(chunk_size) - 1;
+
+	s->pending_pool = mempool_create_slab_pool(PENDING_MEMPOOL_SIZE, pending_exception_cache);
+	if (!s->pending_pool) {
+		ti->error = "Could not allocate mempool for pending exceptions";
+		r = -ENOMEM;
+		goto bad_pending_pool;
+	}
+
+	s->tracked_chunk_pool = mempool_create_slab_pool(DM_TRACKED_CHUNK_POOL_SIZE, tracked_chunk_cache);
+	if (!s->tracked_chunk_pool) {
+		ti->error = "Could not allocate tracked_chunk mempool for tracking reads";
+		goto bad_tracked_chunk_pool;
+	}
+	s->n_tracked_ios = 0;
+	for (i = 0; i < DM_TRACKED_CHUNK_HASH_SIZE; i++)
+		INIT_HLIST_HEAD(&s->tracked_chunk_hash[i]);
+
+	r = dm_kcopyd_client_create(MULTISNAP_KCOPYD_PAGES, &s->kcopyd);
+	if (r) {
+		ti->error = "Could not create kcopyd client";
+		goto bad_kcopyd;
+	}
+
+	r = dm_multisnap_get_string(&argv, &argc, &store_name, &ti->error);
+	if (r)
+		goto bad_store;
+
+	r = dm_multisnap_get_argcount(&argv, &argc, &store_args, &ti->error);
+	if (r)
+		goto bad_store;
+
+	s->store = dm_multisnap_get_exception_store(store_name);
+	if (!s->store) {
+		request_module("dm-store-%s", store_name);
+		s->store = dm_multisnap_get_exception_store(store_name);
+		if (!s->store) {
+			ti->error = "Can't get exception store type";
+			r = -ENOENT;
+			goto bad_store;
+		}
+	}
+
+	s->wq = create_singlethread_workqueue("kmultisnapd");
+	if (!s->wq) {
+		ti->error = "Could not create kernel thread";
+		r = -ENOMEM;
+		goto bad_thread;
+	}
+
+	dm_multisnap_lock(s);
+	r = s->store->init_exception_store(s, &s->p, store_args, argv, &ti->error);
+	if (r) {
+		s->p = NULL;
+		goto exception_store_error;
+	}
+
+	ti->split_io = s->chunk_size >> SECTOR_SHIFT;
+	ti->num_flush_requests = 1;
+
+	argv += store_args;
+	argc -= store_args;
+
+	/*
+	 * Synchronize snapshot IDs according to the table line:
+	 *	allocate IDs that are specified on the table line
+	 *	free IDs that are not specified on the table line
+	 */
+	if (s->flags & DM_MULTISNAP_SYNC_SNAPSHOTS) {
+		snapid_t sn, n, *snapids;
+		r = dm_multisnap_get_argcount(&argv, &argc, &num_snapshots, &ti->error);
+		if (r)
+			goto error_syncing_snapshots;
+		snapids = vmalloc(sizeof(snapid_t) * (num_snapshots + 1));
+		if (!snapids && num_snapshots) {
+			ti->error = "Could not allocate snapids array";
+			goto bad_kcopyd;
+		}
+		for (n = 0; n < num_snapshots; n++) {
+			char *string;
+			r = dm_multisnap_get_string(&argv, &argc, &string, &ti->error);
+			if (r) {
+				vfree(snapids);
+				goto error_syncing_snapshots;
+			}
+			r = read_snapid(s, string, &snapids[n], &ti->error);
+			if (r) {
+				vfree(snapids);
+				goto error_syncing_snapshots;
+			}
+		}
+		snapids[num_snapshots] = SNAPID_T_ORIGIN;
+
+		/* Delete the snapshots that shouldn't be there */
+		sort(snapids, num_snapshots, sizeof(snapid_t), compare_snapids, NULL);
+		sn = s->store->get_next_snapid(s->p, 0);
+		for (n = 0; n <= num_snapshots; n++) {
+			while (sn < snapids[n]) {
+				if (!dm_multisnap_has_error(s)) {
+					r = s->store->delete_snapshot(s->p, sn);
+					if (r && s->flags & DM_MULTISNAP_PRESERVE_ON_ERROR) {
+						ti->error = "Can't delete snapshot";
+						vfree(snapids);
+						goto error_syncing_snapshots;
+					}
+				}
+				sn = s->store->get_next_snapid(s->p, sn + 1);
+				if (sn == SNAPID_T_ORIGIN)
+					goto delete_done;
+			}
+			if (sn == snapids[n]) {
+				sn = s->store->get_next_snapid(s->p, sn + 1);
+				if (sn == SNAPID_T_ORIGIN)
+					goto delete_done;
+			}
+		}
+delete_done:
+
+		/* Create the snapshots that should be there */
+		if (s->store->compare_snapids_for_create)
+			sort(snapids, num_snapshots, sizeof(snapid_t), s->store->compare_snapids_for_create, NULL);
+		for (n = 0; n <= num_snapshots; n++) {
+			if (!dm_multisnap_snapshot_exists(s, snapids[n])) {
+				if (!dm_multisnap_has_error(s)) {
+					r = s->store->create_snapshot(s->p, snapids[n]);
+					if (r && s->flags & DM_MULTISNAP_PRESERVE_ON_ERROR) {
+						ti->error = "Can't create snapshot";
+						vfree(snapids);
+						goto error_syncing_snapshots;
+					}
+				}
+			}
+		}
+		vfree(snapids);
+	}
+
+	dm_multisnap_unlock(s);
+
+	list_add(&s->list_all, &all_multisnapshots);
+
+	mutex_unlock(&all_multisnapshots_lock);
+	return 0;
+
+error_syncing_snapshots:
+	s->store->exit_exception_store(s->p);
+	s->p = NULL;
+exception_store_error:
+	dm_multisnap_unlock(s);
+	destroy_workqueue(s->wq);
+bad_thread:
+	dm_multisnap_put_exception_store(s->store);
+bad_store:
+	dm_kcopyd_client_destroy(s->kcopyd);
+bad_kcopyd:
+	mempool_destroy(s->tracked_chunk_pool);
+bad_tracked_chunk_pool:
+	mempool_destroy(s->pending_pool);
+bad_pending_pool:
+bad_conflicting_snapshot:
+bad_chunk_size:
+	dm_put_device(ti, s->snapshot);
+bad_snapshot:
+	dm_put_device(ti, s->origin);
+bad_origin:
+bad_generic_arguments:
+	kfree(s);
+bad_s:
+bad_arguments:
+	mutex_unlock(&all_multisnapshots_lock);
+	return r;
+}
+
+static void multisnap_origin_dtr(struct dm_target *ti)
+{
+	struct dm_multisnap *s = ti->private;
+	struct dm_multisnap_snap *sn;
+	unsigned i;
+
+	mutex_lock(&all_multisnapshots_lock);
+
+	/* Make sure that any more IOs won't be submitted by snapshot targets */
+	list_for_each_entry(sn, &s->all_snaps, list_snaps) {
+		spin_lock_irq(&dm_multisnap_bio_list_lock);
+		sn->s = NULL;
+		spin_unlock_irq(&dm_multisnap_bio_list_lock);
+	}
+	list_del(&s->all_snaps);
+
+	/*
+	 * This code is called in the destructor, it is not performance
+	 * sensitive and thus we use polling with active waiting (msleep(1)).
+	 *
+	 * A possible 1ms delay on device destruction won't cause any trouble
+	 * and this polling is simpler and less bug-prone than using wait
+	 * queues.
+	 */
+poll_for_ios:
+	/* Wait for IOs on the snapshot */
+	spin_lock_irq(&dm_multisnap_bio_list_lock);
+	if (s->n_tracked_ios) {
+		spin_unlock_irq(&dm_multisnap_bio_list_lock);
+		msleep(1);
+		goto poll_for_ios;
+	}
+	spin_unlock_irq(&dm_multisnap_bio_list_lock);
+
+	/* Bug-check that there are really no IOs */
+	BUG_ON(!bio_list_empty(&s->bios));
+	for (i = 0; i < DM_TRACKED_CHUNK_HASH_SIZE; i++)
+		BUG_ON(!hlist_empty(&s->tracked_chunk_hash[i]));
+
+	/* Wait for pending reallocations */
+	dm_multisnap_lock(s);
+	for (i = 0; i < PENDING_HASH_SIZE; i++)
+		if (!hlist_empty(&s->pending_hash[i])) {
+			dm_multisnap_unlock(s);
+			msleep(1);
+			goto poll_for_ios;
+		}
+	dm_multisnap_unlock(s);
+
+	flush_workqueue(s->wq);
+
+	dm_multisnap_lock(s);
+	dm_multisnap_call_commit(s);
+	s->store->exit_exception_store(s->p);
+	s->p = NULL;
+	list_del(&s->list_all);
+	dm_multisnap_unlock(s);
+
+	destroy_workqueue(s->wq);
+	kfree(s->p);
+	dm_kcopyd_client_destroy(s->kcopyd);
+	mempool_destroy(s->tracked_chunk_pool);
+	mempool_destroy(s->pending_pool);
+	dm_put_device(ti, s->snapshot);
+	dm_put_device(ti, s->origin);
+	dm_multisnap_put_exception_store(s->store);
+
+	kfree(s);
+
+	mutex_unlock(&all_multisnapshots_lock);
+}
+
+static int multisnap_origin_map(struct dm_target *ti, struct bio *bio, union map_info *map_context)
+{
+	struct dm_multisnap *s = ti->private;
+
+	/*
+	 * Do the most common case quickly: reads and write barriers are
+	 * dispatched to the origin device directly.
+	 */
+	if (likely(bio_rw(bio) != WRITE) || unlikely(bio_empty_barrier(bio))) {
+		bio->bi_bdev = s->origin->bdev;
+		return DM_MAPIO_REMAPPED;
+	}
+
+	bio_put_snapid(bio, SNAPID_T_ORIGIN);
+
+	dm_multisnap_enqueue_bio(s, bio);
+	wakeup_kmultisnapd(s);
+
+	return DM_MAPIO_SUBMITTED;
+}
+
+static int multisnap_origin_message(struct dm_target *ti, unsigned argc, char **argv)
+{
+	struct dm_multisnap *s = ti->private;
+	int r;
+
+	mutex_lock(&all_multisnapshots_lock);
+	dm_multisnap_lock(s);
+
+	if (argc == 1 && !strcasecmp(argv[0], "create")) {
+		/*
+		 * Prepare snapshot creation.
+		 *
+		 * We allocate a snapid, and return it in the status.
+		 *
+		 * The snapshot is really created in postsuspend method (to
+		 * make sure that possibly mounted filesystem is quiescent and
+		 * the snapshot will be consistent).
+		 */
+		r = dm_multisnap_has_error(s);
+		if (r)
+			goto unlock_ret;
+
+
+		dm_multisnap_status_lock(s);
+		s->new_snapid_valid = 0;
+		dm_multisnap_status_unlock(s);
+
+		r = s->store->allocate_snapid(s->p, &s->new_snapid);
+		if (r)
+			goto unlock_ret;
+
+		dm_multisnap_status_lock(s);
+		s->new_snapid_valid = 1;
+		dm_multisnap_status_unlock(s);
+
+		r = dm_multisnap_has_error(s);
+		goto unlock_ret;
+	}
+	if (argc == 2 && !strcasecmp(argv[0], "delete")) {
+		/*
+		 * Delete a snapshot.
+		 */
+		char *error;
+		snapid_t snapid;
+		struct dm_multisnap_snap *sn;
+		struct bio *bio, *next;
+
+		r = read_snapid(s, argv[1], &snapid, &error);
+		if (r) {
+			DMWARN("invalid snapshot id: %s", error);
+			goto unlock_ret;
+		}
+
+		if (!s->store->delete_snapshot) {
+			DMERR("snapshot store doesn't support delete");
+			r = -EOPNOTSUPP;
+			goto unlock_ret;
+		}
+
+		r = dm_multisnap_has_error(s);
+		if (r)
+			goto unlock_ret;
+
+		/* Kick off possibly attached snapshot */
+		list_for_each_entry(sn, &s->all_snaps, list_snaps) {
+			if (sn->snapid == snapid) {
+				spin_lock_irq(&dm_multisnap_bio_list_lock);
+				sn->s = NULL;
+				spin_unlock_irq(&dm_multisnap_bio_list_lock);
+			}
+		}
+
+		/* Terminate bios queued for this snapshot so far */
+		spin_lock_irq(&dm_multisnap_bio_list_lock);
+		bio = bio_list_get(&s->bios);
+		spin_unlock_irq(&dm_multisnap_bio_list_lock);
+		for (; bio; bio = next) {
+			next = bio->bi_next;
+			bio->bi_next = NULL;
+			if (bio_get_snapid(bio) == snapid)
+				bio_endio(bio, -EIO);
+			else
+				dm_multisnap_enqueue_bio(s, bio);
+		}
+
+		if (!dm_multisnap_snapshot_exists(s, snapid)) {
+			DMWARN("snapshot with this id doesn't exists.");
+			r = -EINVAL;
+			goto unlock_ret;
+		}
+
+		r = s->store->delete_snapshot(s->p, snapid);
+		if (r)
+			goto unlock_ret;
+
+		dm_multisnap_unlock(s);
+
+		r = dm_multisnap_force_commit(s);
+
+		goto unlock2_ret;
+	}
+
+	DMWARN("unrecognised message received.");
+	r = -EINVAL;
+
+unlock_ret:
+	dm_multisnap_unlock(s);
+unlock2_ret:
+	mutex_unlock(&all_multisnapshots_lock);
+
+	return r;
+}
+
+/* Print used snapshot IDs into a supplied string */
+
+static void print_snapshot_ids(struct dm_multisnap *s, char *result, unsigned maxlen)
+{
+	snapid_t nsnap = 0;
+	snapid_t sn = 0;
+	while ((sn = s->store->get_next_snapid(s->p, sn)) != SNAPID_T_ORIGIN)
+		sn++, nsnap++;
+	snprintf(result, maxlen, " %llu", (unsigned long long)nsnap);
+	dm_multisnap_adjust_string(&result, &maxlen);
+	sn = 0;
+	while ((sn = s->store->get_next_snapid(s->p, sn)) != SNAPID_T_ORIGIN) {
+		snprintf(result, maxlen, " ");
+		dm_multisnap_adjust_string(&result, &maxlen);
+		print_snapid(s, result, maxlen, sn);
+		dm_multisnap_adjust_string(&result, &maxlen);
+		sn++;
+	}
+}
+
+static int multisnap_origin_status(struct dm_target *ti, status_type_t type, char *result, unsigned maxlen)
+{
+	struct dm_multisnap *s = ti->private;
+
+	/*
+	 * Use a special status lock, so that this code can execute even
+	 * when the underlying device is suspended and there is no possibility
+	 * to optain the master lock.
+	 */
+	dm_multisnap_status_lock(s);
+
+	switch (type) {
+		case STATUSTYPE_INFO: {
+			unsigned long long total, alloc, meta;
+			snprintf(result, maxlen, "5 %d ", dm_multisnap_has_error(s));
+			dm_multisnap_adjust_string(&result, &maxlen);
+			if (s->new_snapid_valid)
+				print_snapid(s, result, maxlen, s->new_snapid);
+			else
+				snprintf(result, maxlen, "-");
+			dm_multisnap_adjust_string(&result, &maxlen);
+			if (s->store->get_space)
+				s->store->get_space(s->p, &total, &alloc, &meta);
+			else
+				total = alloc = meta = 0;
+			total <<= s->chunk_shift - SECTOR_SHIFT;
+			alloc <<= s->chunk_shift - SECTOR_SHIFT;
+			meta <<= s->chunk_shift - SECTOR_SHIFT;
+			snprintf(result, maxlen, " %llu %llu %llu", total, alloc, meta);
+			dm_multisnap_adjust_string(&result, &maxlen);
+			print_snapshot_ids(s, result, maxlen);
+			dm_multisnap_adjust_string(&result, &maxlen);
+			break;
+		}
+		case STATUSTYPE_TABLE: {
+			unsigned ngen = 0;
+			if (s->flags & DM_MULTISNAP_SYNC_SNAPSHOTS)
+				ngen++;
+			if (s->flags & DM_MULTISNAP_PRESERVE_ON_ERROR)
+				ngen++;
+			snprintf(result, maxlen, "%s %s %u %u%s%s %s",
+				s->origin->name,
+				s->snapshot->name,
+				s->chunk_size / 512,
+				ngen,
+				s->flags & DM_MULTISNAP_SYNC_SNAPSHOTS ?
+					" sync-snapshots" : "",
+				s->flags & DM_MULTISNAP_PRESERVE_ON_ERROR ?
+					" preserve-on-error" : "",
+				s->store->name);
+			dm_multisnap_adjust_string(&result, &maxlen);
+			if (s->store->status_table)
+				s->store->status_table(s->p, result, maxlen);
+			else
+				snprintf(result, maxlen, " 0");
+			dm_multisnap_adjust_string(&result, &maxlen);
+			if (s->flags & DM_MULTISNAP_SYNC_SNAPSHOTS) {
+				print_snapshot_ids(s, result, maxlen);
+				dm_multisnap_adjust_string(&result, &maxlen);
+			}
+			break;
+		}
+	}
+
+	dm_multisnap_status_unlock(s);
+
+	/* If there's no space left in the buffer, ask for larger size */
+	return maxlen <= 1;
+}
+
+/*
+ * In postsuspend, we optionally create a snapshot that we prepared with
+ * a message.
+ */
+
+static void multisnap_origin_postsuspend(struct dm_target *ti)
+{
+	struct dm_multisnap *s = ti->private;
+
+	dm_multisnap_lock(s);
+
+	if (s->new_snapid_valid && !dm_multisnap_has_error(s)) {
+		/*
+		 * No way to return the error code, but it is recorded
+		 * in s->error anyway.
+		 */
+		s->store->create_snapshot(s->p, s->new_snapid);
+		s->new_snapid_valid = 0;
+	}
+
+	dm_multisnap_unlock(s);
+
+	dm_multisnap_force_commit(s);
+}
+
+static int multisnap_snap_ctr(struct dm_target *ti, unsigned argc, char **argv)
+{
+	int r;
+	char *origin_path;
+	char *snapid_str;
+	snapid_t snapid;
+	int doesnt_exist;
+
+	struct dm_dev *origin;
+
+	struct dm_multisnap *s;
+	struct dm_multisnap_snap *sn;
+
+	r = dm_multisnap_get_string(&argv, &argc, &origin_path, &ti->error);
+	if (r)
+		goto bad_arguments;
+	r = dm_multisnap_get_string(&argv, &argc, &snapid_str, &ti->error);
+	if (r)
+		goto bad_arguments;
+	r = dm_get_device(ti, origin_path, 0, 0, FMODE_READ | FMODE_WRITE, &origin);
+	if (r) {
+		ti->error = "Could not get origin device";
+		goto bad_origin;
+	}
+	mutex_lock(&all_multisnapshots_lock);
+	s = find_multisnapshot(origin->bdev);
+	if (!s) {
+		r = -ENXIO;
+		ti->error = "Origin target not loaded";
+		goto origin_not_loaded;
+	}
+
+	dm_multisnap_lock(s);
+
+	r = read_snapid(s, snapid_str, &snapid, &ti->error);
+	if (r) {
+		dm_multisnap_unlock(s);
+		goto snapid_doesnt_exist;
+	}
+
+	doesnt_exist = 0;
+	if (!dm_multisnap_snapshot_exists(s, snapid)) {
+		if (dm_multisnap_has_error(s) && dm_multisnap_drop_on_error(s)) {
+			/*
+			 * If there was an error, we don't know which snapshot
+			 * IDs are available. So we must accept it. But we
+			 * abort all accesses to this snapshot with an error.
+			 */
+			 doesnt_exist = 1;
+		} else {
+			dm_multisnap_unlock(s);
+			r = -ENOENT;
+			ti->error = "Snapshot with this id doesn't exist";
+			goto snapid_doesnt_exist;
+		}
+	}
+	dm_multisnap_unlock(s);
+
+	sn = kmalloc(sizeof(*sn) + strlen(snapid_str), GFP_KERNEL);
+	if (!sn) {
+		ti->error = "Could not allocate multisnapshot_snap structure";
+		r = -ENOMEM;
+		goto cant_allocate;
+	}
+	sn->s = doesnt_exist ? NULL : s;
+	sn->snapid = snapid;
+	list_add(&sn->list_snaps, &s->all_snaps);
+	strlcpy(sn->origin_name, origin->name, sizeof sn->origin_name);
+	strcpy(sn->snapid_string, snapid_str);
+
+	mutex_unlock(&all_multisnapshots_lock);
+
+	dm_put_device(ti, origin);
+
+	ti->private = sn;
+	ti->split_io = s->chunk_size >> SECTOR_SHIFT;
+	ti->num_flush_requests = 1;
+
+	return 0;
+
+cant_allocate:
+snapid_doesnt_exist:
+origin_not_loaded:
+	dm_put_device(ti, origin);
+	mutex_unlock(&all_multisnapshots_lock);
+bad_origin:
+bad_arguments:
+	return r;
+}
+
+static void multisnap_snap_dtr(struct dm_target *ti)
+{
+	struct dm_multisnap_snap *sn = ti->private;
+
+	mutex_lock(&all_multisnapshots_lock);
+
+	list_del(&sn->list_snaps);
+	kfree(sn);
+
+	mutex_unlock(&all_multisnapshots_lock);
+}
+
+/*
+ * Each snapshot I/O is counted in n_tracked_ios in the origin and
+ * has struct dm_multisnap_tracked_chunk allocated.
+ * dm_multisnap_tracked_chunk->node can be optionally linked into origin's hash
+ * of tracked I/Os.
+ */
+
+static int multisnap_snap_map(struct dm_target *ti, struct bio *bio, union map_info *map_context)
+{
+	struct dm_multisnap_snap *sn = ti->private;
+	struct dm_multisnap *s;
+	struct dm_multisnap_tracked_chunk *c;
+
+	bio_put_snapid(bio, sn->snapid);
+
+	spin_lock_irq(&dm_multisnap_bio_list_lock);
+	s = sn->s;
+	if (unlikely(!s)) {
+		spin_unlock_irq(&dm_multisnap_bio_list_lock);
+		return -EIO;
+	}
+		/*
+		 * make sure that the origin is not unloaded under us while
+		 * we drop the lock
+		 */
+	s->n_tracked_ios++;
+
+	c = mempool_alloc(s->tracked_chunk_pool, GFP_ATOMIC);
+	if (unlikely(!c)) {
+		spin_unlock_irq(&dm_multisnap_bio_list_lock);
+		c = mempool_alloc(s->tracked_chunk_pool, GFP_NOIO);
+		spin_lock_irq(&dm_multisnap_bio_list_lock);
+	}
+	c->s = s;
+	c->chunk = sector_to_chunk(s, bio->bi_sector);
+	c->bio_rw = bio_rw(bio);
+	INIT_HLIST_NODE(&c->node);
+	map_context->ptr = c;
+
+	if (unlikely(bio_empty_barrier(bio))) {
+		bio->bi_bdev = s->snapshot->bdev;
+		spin_unlock_irq(&dm_multisnap_bio_list_lock);
+		return DM_MAPIO_REMAPPED;
+	}
+
+	dm_multisnap_enqueue_bio_unlocked(s, bio);
+	spin_unlock_irq(&dm_multisnap_bio_list_lock);
+
+	wakeup_kmultisnapd(s);
+
+	return DM_MAPIO_SUBMITTED;
+}
+
+static int multisnap_snap_end_io(struct dm_target *ti, struct bio *bio, int error, union map_info *map_context)
+{
+	struct dm_multisnap_tracked_chunk *c = map_context->ptr;
+	struct dm_multisnap *s = c->s;
+	unsigned long flags;
+
+	spin_lock_irqsave(&dm_multisnap_bio_list_lock, flags);
+
+	s->n_tracked_ios--;
+	if (!hlist_unhashed(&c->node))
+		hlist_del(&c->node);
+	mempool_free(c, s->tracked_chunk_pool);
+
+	spin_unlock_irqrestore(&dm_multisnap_bio_list_lock, flags);
+
+	return 0;
+}
+
+static int multisnap_snap_status(struct dm_target *ti, status_type_t type, char *result, unsigned maxlen)
+{
+	struct dm_multisnap_snap *sn = ti->private;
+
+	switch (type) {
+
+	case STATUSTYPE_INFO:
+		/* there is no status */
+		result[0] = 0;
+		dm_multisnap_adjust_string(&result, &maxlen);
+		break;
+	case STATUSTYPE_TABLE:
+		snprintf(result, maxlen, "%s %s", sn->origin_name, sn->snapid_string);
+		dm_multisnap_adjust_string(&result, &maxlen);
+		break;
+	}
+
+	/* If there's no space left in the buffer, ask for larger size */
+	return maxlen <= 1;
+}
+
+static struct target_type multisnap_origin_target = {
+	.name		= "multisnapshot",
+	.version 	= {1, 0, 0},
+	.module		= THIS_MODULE,
+	.ctr		= multisnap_origin_ctr,
+	.dtr		= multisnap_origin_dtr,
+	.map		= multisnap_origin_map,
+	.message 	= multisnap_origin_message,
+	.status		= multisnap_origin_status,
+	.postsuspend	= multisnap_origin_postsuspend,
+};
+
+static struct target_type multisnap_snap_target = {
+	.name		= "multisnap-snap",
+	.version 	= {1, 0, 0},
+	.module		= THIS_MODULE,
+	.ctr		= multisnap_snap_ctr,
+	.dtr		= multisnap_snap_dtr,
+	.map		= multisnap_snap_map,
+	.end_io		= multisnap_snap_end_io,
+	.status		= multisnap_snap_status,
+};
+
+static int __init dm_multisnapshot_init(void)
+{
+	int r;
+
+	pending_exception_cache = kmem_cache_create(
+			"dm_multisnap_pending_exception",
+			sizeof(struct dm_multisnap_pending_exception),
+			__alignof__(struct dm_multisnap_pending_exception),
+			0,
+			pending_exception_ctor);
+	if (!pending_exception_cache) {
+		DMERR("Couldn't create exception cache.");
+		r = -ENOMEM;
+		goto bad_exception_cache;
+	}
+	tracked_chunk_cache = KMEM_CACHE(dm_multisnap_tracked_chunk, 0);
+	if (!tracked_chunk_cache) {
+		DMERR("Couldn't create cache to track chunks in use.");
+		r = -ENOMEM;
+		goto bad_tracked_chunk_cache;
+	}
+
+	r = dm_register_target(&multisnap_origin_target);
+	if (r < 0) {
+		DMERR("multisnap_origin_target target register failed %d", r);
+		goto bad_multisnap_origin_target;
+	}
+
+	r = dm_register_target(&multisnap_snap_target);
+	if (r < 0) {
+		DMERR("multisnap_snap_target target register failed %d", r);
+		goto bad_multisnap_snap_target;
+	}
+
+	return 0;
+
+bad_multisnap_snap_target:
+	dm_unregister_target(&multisnap_origin_target);
+bad_multisnap_origin_target:
+	kmem_cache_destroy(tracked_chunk_cache);
+bad_tracked_chunk_cache:
+	kmem_cache_destroy(pending_exception_cache);
+bad_exception_cache:
+	return r;
+}
+
+static void __exit dm_multisnapshot_exit(void)
+{
+	dm_unregister_target(&multisnap_origin_target);
+	dm_unregister_target(&multisnap_snap_target);
+	kmem_cache_destroy(tracked_chunk_cache);
+	kmem_cache_destroy(pending_exception_cache);
+}
+
+/* Module hooks */
+module_init(dm_multisnapshot_init);
+module_exit(dm_multisnapshot_exit);
+
+MODULE_DESCRIPTION(DM_NAME " multisnapshot target");
+MODULE_AUTHOR("Mikulas Patocka");
+MODULE_LICENSE("GPL");
Index: linux-2.6.32/drivers/md/dm-multisnap.h
===================================================================
--- /dev/null
+++ linux-2.6.32/drivers/md/dm-multisnap.h
@@ -0,0 +1,153 @@
+/*
+ * Copyright (C) 2009 Red Hat Czech, s.r.o.
+ *
+ * Mikulas Patocka <mpatocka@redhat.com>
+ *
+ * This file is released under the GPL.
+ */
+
+#ifndef DM_MULTISNAP_H
+#define DM_MULTISNAP_H
+
+/*
+ * This file defines the interface between generic driver (dm-multisnap.c)
+ * and exception store drivers.
+ */
+
+#include <linux/device-mapper.h>
+#include <linux/list.h>
+
+#define EFSERROR	EPERM
+
+#define DM_MSG_PREFIX "multisnapshot"
+
+#define SNAPID_T_ORIGIN		0xffffffffffffffffULL
+
+typedef sector_t chunk_t;
+typedef __u64 snapid_t;
+
+struct dm_multisnap;		/* private to dm-multisnap.c */
+struct dm_exception_store;	/* private to the exception store driver */
+
+struct dm_multisnap_background_work {
+	struct list_head list;
+	void (*work)(struct dm_exception_store *, struct dm_multisnap_background_work *);
+	int queued;
+};
+
+union chunk_descriptor {
+	__u64 bitmask;
+	struct {
+		snapid_t from;
+		snapid_t to;
+	} range;
+};
+
+struct dm_multisnap_exception_store {
+	struct list_head list;
+	struct module *module;
+	const char *name;
+
+	/* < 0 - error */
+	int (*init_exception_store)(struct dm_multisnap *dm, struct dm_exception_store **s, unsigned argc, char **argv, char **error);
+
+	void (*exit_exception_store)(struct dm_exception_store *s);
+
+	void (*store_lock_acquired)(struct dm_exception_store *s, int flags);
+
+	/* These two can override format of snapids in the table. Can be NULL */
+	void (*print_snapid)(struct dm_exception_store *s, char *string, unsigned maxlen, snapid_t snapid);
+	int (*read_snapid)(struct dm_exception_store *s, char *string, snapid_t *snapid, char **error);
+
+	/* return the exception-store specific table arguments */
+	void (*status_table)(struct dm_exception_store *s, char *result, unsigned maxlen);
+
+	/* return the space */
+	void (*get_space)(struct dm_exception_store *s, unsigned long long *chunks_total, unsigned long long *chunks_allocated, unsigned long long *chunks_metadata_allocated);
+
+	/* < 0 - error */
+	int (*allocate_snapid)(struct dm_exception_store *s, snapid_t *snapid);
+
+	/* < 0 - error */
+	int (*create_snapshot)(struct dm_exception_store *s, snapid_t snapid);
+
+	/* < 0 - error (may be NULL if not supported) */
+	int (*delete_snapshot)(struct dm_exception_store *s, snapid_t snapid);
+
+	/*
+	 * Get the first snapid at or after snapid in its argument.
+	 * If there are no more snapids, return SNAPID_T_ORIGIN.
+	 */
+	snapid_t (*get_next_snapid)(struct dm_exception_store *s, snapid_t snapid);
+
+	/*
+	 * qsort()-compatible function to order snapshots for creation.
+	 * may be NULL if standard ordering should be used.
+	 */
+	int (*compare_snapids_for_create)(const void *p1, const void *p2);
+
+	/* 0 - not found, 1 - found (read-only), 2 - found (writeable), < 0 - error */
+	int (*find_snapshot_chunk)(struct dm_exception_store *s, snapid_t id, chunk_t chunk, int write, chunk_t *result);
+
+	/*
+	 * Chunk interface between exception store and generic code.
+	 * Allowed sequences:
+	 *
+	 * - first call reset_query
+	 *   then repeatedly query next exception to make with query_next_remap
+	 *   and add it to btree with add_next_remap. This can be repeated until
+	 *   query_next_remap indicates that it has nothing more or until all 8
+	 *   kcopyd slots are filled.
+	 *
+	 * - call find_snapshot_chunk, if it returns 0, you can call
+	 *   add_next_remap to add the chunk to the btree.
+	 *
+	 * - call find_snapshot_chunk, if it returns 1 (shared chunk), call
+	 *   make_chunk_writeable to relocate that chunk.
+	 */
+
+	void (*reset_query)(struct dm_exception_store *s);
+	int (*query_next_remap)(struct dm_exception_store *s, chunk_t chunk);
+	void (*add_next_remap)(struct dm_exception_store *s, union chunk_descriptor *cd, chunk_t *new_chunk);
+
+	/* may be NULL if writeable snapshots are not supported */
+	void (*make_chunk_writeable)(struct dm_exception_store *s, union chunk_descriptor *cd, chunk_t *new_chunk);
+	int (*check_conflict)(struct dm_exception_store *s, union chunk_descriptor *cd, snapid_t snapid);
+
+	void (*commit)(struct dm_exception_store *s);
+};
+
+/* dm-multisnap.c */
+
+/* Access generic information about the snapshot */
+struct block_device *dm_multisnap_snapshot_bdev(struct dm_multisnap *s);
+unsigned dm_multisnap_chunk_size(struct dm_multisnap *s);
+void dm_multisnap_set_error(struct dm_multisnap *s, int error);
+int dm_multisnap_has_error(struct dm_multisnap *s);
+int dm_multisnap_drop_on_error(struct dm_multisnap *s);
+int dm_multisnap_snapshot_exists(struct dm_multisnap *s, snapid_t snapid);
+
+void dm_multisnap_status_lock(struct dm_multisnap *s);
+void dm_multisnap_status_unlock(struct dm_multisnap *s);
+void dm_multisnap_status_assert_locked(struct dm_multisnap *s);
+
+/* Commit. dm_multisnap_call_commit can be called only if dm_multisnap_can_commit returns true */
+int dm_multisnap_can_commit(struct dm_multisnap *s);
+void dm_multisnap_call_commit(struct dm_multisnap *s);
+
+/* Delayed work for delete/merge */
+void dm_multisnap_queue_work(struct dm_multisnap *s, struct dm_multisnap_background_work *bw);
+void dm_multisnap_cancel_work(struct dm_multisnap *s, struct dm_multisnap_background_work *bw);
+
+/* Parsing command line */
+int dm_multisnap_get_string(char ***argv, unsigned *argc, char **string, char **error);
+int dm_multisnap_get_uint64(char ***argv, unsigned *argc, __u64 *uint64, char **error);
+int dm_multisnap_get_uint(char ***argv, unsigned *argc, unsigned *uint, char **error);
+int dm_multisnap_get_argcount(char ***argv, unsigned *argc, unsigned *uint, char **error);
+void dm_multisnap_adjust_string(char **result, unsigned *maxlen);
+
+/* Register/unregister the exception store driver */
+int dm_multisnap_register_exception_store(struct dm_multisnap_exception_store *store);
+void dm_multisnap_unregister_exception_store(struct dm_multisnap_exception_store *store);
+
+#endif
