[Top][All Lists]
[Date Prev][Date Next][Thread Prev][Thread Next][Date Index][Thread Index]
[Qemu-devel] [PATCH 15/26] FVD: add basic journal functionality
From: |
Chunqiang Tang |
Subject: |
[Qemu-devel] [PATCH 15/26] FVD: add basic journal functionality |
Date: |
Fri, 25 Feb 2011 17:37:55 -0500 |
This patch is part of the Fast Virtual Disk (FVD) proposal.
See http://wiki.qemu.org/Features/FVD.
This patch adds the basic journal functionality to FVD. The journal provides
several benefits. First, updating both the bitmap and the lookup table
requires only a single write to journal. Second, K concurrent updates to any
potions of the bitmap or the lookup table are converted to K sequential writes
in the journal, which can be merged into a single write by the host Linux
kernel. Third, it increases concurrency by avoiding locking the bitmap or the
lookup table. For example, updating one bit in the bitmap requires writing a
512-byte sector to the on-disk bitmap. This bitmap sector covers a total of
512*8*64K=256MB data, and any two writes to that same bitmap sector cannot
proceed concurrently. The journal solves this problem and eliminates
unnecessary locking.
Signed-off-by: Chunqiang Tang <address@hidden>
---
block.c | 2 +-
block/fvd-bitmap.c | 57 ++++
block/fvd-journal-buf.c | 34 ++
block/fvd-journal.c | 814 ++++++++++++++++++++++++++++++++++++++++++++++-
block/fvd-write.c | 1 +
block/fvd.c | 19 ++
6 files changed, 920 insertions(+), 7 deletions(-)
create mode 100644 block/fvd-journal-buf.c
diff --git a/block.c b/block.c
index f7d91a2..8b3083d 100644
--- a/block.c
+++ b/block.c
@@ -58,7 +58,7 @@ static int bdrv_read_em(BlockDriverState *bs, int64_t
sector_num,
static int bdrv_write_em(BlockDriverState *bs, int64_t sector_num,
const uint8_t *buf, int nb_sectors);
-static QTAILQ_HEAD(, BlockDriverState) bdrv_states =
+QTAILQ_HEAD(, BlockDriverState) bdrv_states =
QTAILQ_HEAD_INITIALIZER(bdrv_states);
static QLIST_HEAD(, BlockDriver) bdrv_drivers =
diff --git a/block/fvd-bitmap.c b/block/fvd-bitmap.c
index 30e4a4b..06d7912 100644
--- a/block/fvd-bitmap.c
+++ b/block/fvd-bitmap.c
@@ -66,6 +66,63 @@ static inline void update_fresh_bitmap(int64_t sector_num,
int nb_sectors,
}
}
+static void update_stale_bitmap(BDRVFvdState * s, int64_t sector_num,
+ int nb_sectors)
+{
+ if (sector_num >= s->base_img_sectors) {
+ return;
+ }
+
+ int64_t end = sector_num + nb_sectors;
+ if (end > s->base_img_sectors) {
+ end = s->base_img_sectors;
+ }
+
+ int64_t block_num = sector_num / s->block_size;
+ const int64_t block_end = (end - 1) / s->block_size;
+
+ for (; block_num <= block_end; block_num++) {
+ int64_t bitmap_byte_offset = block_num / 8;
+ uint8_t bitmap_bit_offset = block_num % 8;
+ uint8_t mask = (uint8_t) (0x01 << bitmap_bit_offset);
+ uint8_t b = s->stale_bitmap[bitmap_byte_offset];
+ if (!(b & mask)) {
+ ASSERT(s->stale_bitmap == s->fresh_bitmap ||
+ (s->fresh_bitmap[bitmap_byte_offset] & mask));
+ b |= mask;
+ s->stale_bitmap[bitmap_byte_offset] = b;
+ }
+ }
+}
+
+static void update_both_bitmaps(BDRVFvdState * s, int64_t sector_num,
+ int nb_sectors)
+{
+ if (sector_num >= s->base_img_sectors) {
+ return;
+ }
+
+ int64_t end = sector_num + nb_sectors;
+ if (end > s->base_img_sectors) {
+ end = s->base_img_sectors;
+ }
+
+ int64_t block_num = sector_num / s->block_size;
+ const int64_t block_end = (end - 1) / s->block_size;
+
+ for (; block_num <= block_end; block_num++) {
+ int64_t bitmap_byte_offset = block_num / 8;
+ uint8_t bitmap_bit_offset = block_num % 8;
+ uint8_t mask = (uint8_t) (0x01 << bitmap_bit_offset);
+ uint8_t b = s->fresh_bitmap[bitmap_byte_offset];
+ if (!(b & mask)) {
+ b |= mask;
+ s->fresh_bitmap[bitmap_byte_offset] =
+ s->stale_bitmap[bitmap_byte_offset] = b;
+ }
+ }
+}
+
static inline bool bitmap_show_sector_in_base_img(int64_t sector_num,
const BDRVFvdState * s,
int bitmap_offset,
diff --git a/block/fvd-journal-buf.c b/block/fvd-journal-buf.c
new file mode 100644
index 0000000..3efdd47
--- /dev/null
+++ b/block/fvd-journal-buf.c
@@ -0,0 +1,34 @@
+/*
+ * QEMU Fast Virtual Disk Format Metadata Journal
+ *
+ * Copyright IBM, Corp. 2010
+ *
+ * Authors:
+ * Chunqiang Tang <address@hidden>
+ *
+ * This work is licensed under the terms of the GNU GPL, version 2 or later.
+ * See the COPYING.LIB file in the top-level directory.
+ *
+ */
+
+/*=============================================================================
+ * There are two different ways of writing metadata changes to the journal:
+ * immediate write or buffered write. If cache=writethrough, metadata changes
+ * are written to the journal immediately. If cache!=writethrough, metadata
+ * changes are buffered in memory and later written to the journal either
+ * triggered by bdrv_aio_flush() or by a timeout. This module implements the
+ * case for cache!=writethrough.
+
*============================================================================*/
+
+static uint8_t * bjnl_alloc_journal_records_from_buf(BlockDriverState *bs,
+ bool update_bitmap,
+ size_t record_size)
+{
+ return NULL;
+}
+
+
+static void bjnl_clean_buf_timer_cb(BlockDriverState * bs)
+{
+ /* To be implemented. */
+}
diff --git a/block/fvd-journal.c b/block/fvd-journal.c
index 2edfc70..11796b0 100644
--- a/block/fvd-journal.c
+++ b/block/fvd-journal.c
@@ -11,28 +11,830 @@
*
*/
-#ifdef FVD_DEBUG
-static bool emulate_host_crash = true;
+/*=============================================================================
+ * A short description: this FVD module implements a journal for committing
+ * metadata changes. Each sector in the journal is self-contained so that
+ * updates are atomic. A sector may contain one or multiple journal records.
+ * There are two types of journal records:
+ * bitmap_update and table_update.
+ * Format of a bitmap_update record:
+ * BITMAP_JRECORD (uint32_t)
+ * num_dirty_sectors (uint32_t)
+ * dirty_sector_begin (int64_t)
+ * Format of a table_update record:
+ * TABLE_JRECORD (uint32_t)
+ * journal_epoch (uint64_t)
+ * num_dirty_table_entries (uint32_t)
+ * dirty_table_begin (uint32_t)
+ * table_entry_1 (uint32_t)
+ * table_entry_2 (uint32_t)
+ * ...
+ * If both the bitmap and the table need update, one sector contains a
+ * TABLE_JRECORD and a BITMAP_JRECORD, and these two records cover the same
+ * range of virtual disk data so that the corresponding parts of the bitmap
+ * and the table are always updated in one atomic operation.
+ *
+ * There are two different ways of writing metadata changes to the journal:
+ * immediate write or buffered write. If cache=writethrough, metadata changes
+ * are written to the journal immediately. If cache!=writethrough, metadata
+ * changes are buffered in memory and later written to the journal either
+ * triggered by bdrv_aio_flush() or by a timeout.
+ *
+ * Immediate journal write is implemented in this file, see
+ * ujnl_write_metadata_to_journal_now(). Buffered journal write is implemented
+ * in fvd-journal-buf.c, see bjnl_store_metadata_change_in_buffer() and
+
*============================================================================*/
+
+#define BITMAP_JRECORD ((uint32_t)0xEF2AB8ED)
+#define TABLE_JRECORD ((uint32_t)0xB4E6F7AC)
+#define EMPTY_JRECORD ((uint32_t)0xA5A5A5A5)
+#define BITMAP_JRECORD_SIZE (2 * sizeof(uint32_t) + sizeof(int64_t))
+#define TABLE_JRECORD_HDR_SIZE (3 * sizeof(uint32_t) + sizeof(uint64_t))
+#define TABLE_JRECORDS_PER_SECTOR \
+ ((512 - TABLE_JRECORD_HDR_SIZE)/sizeof(uint32_t))
+
+/* One BITMAP_JRECORD and this number of BITMAP_JRECORDs can fit
+ * in one journal sector. */
+#define MIXED_JRECORDS_PER_SECTOR ((512 - TABLE_JRECORD_HDR_SIZE - \
+ BITMAP_JRECORD_SIZE) / sizeof(uint32_t))
+
+#ifndef ENABLE_QDEBUG
+# define PRINT_TABLE_JRECORD(type) do{}while(0)
#else
-static bool emulate_host_crash = false;
+static void print_table_jrecord(uint32_t * type);
+# define PRINT_TABLE_JRECORD print_table_jrecord
#endif
+static int flush_metadata_to_disk (BlockDriverState * bs,
+ bool update_journal_epoch,
+ bool update_base_img_prefetched);
+static int64_t ujnl_allocate_journal_sectors(BlockDriverState * bs,
+ FvdAIOCB * acb, int nb_sectors);
+static void ujnl_write_metadata_to_journal_now(FvdAIOCB * acb,
+ bool update_bitmap, uint8_t *buf,
+ int64_t journal_sec, int nb_journal_sectors);
+static void bjnl_clean_buf_timer_cb(BlockDriverState * bs);
+static uint8_t * bjnl_alloc_journal_records_from_buf(BlockDriverState *bs,
+ bool update_bitmap, size_t record_size);
+
static inline int64_t calc_min_journal_size(int64_t table_entries)
{
- return 512;
+ return (table_entries + MIXED_JRECORDS_PER_SECTOR - 1)
+ / MIXED_JRECORDS_PER_SECTOR * 512;
}
static int init_journal(int read_only, BlockDriverState * bs,
FvdHeader * header)
{
- return -ENOTSUP;
+ BDRVFvdState *s = bs->opaque;
+ s->journal_size = header->journal_size / 512;
+ s->journal_offset = header->journal_offset / 512;
+ s->journal_epoch = header->stable_journal_epoch + 1;
+ s->next_journal_sector = 0;
+ s->use_bjnl = false;
+ QLIST_INIT(&s->ujnl.wait4_recycle);
+ s->ujnl.active_writes = 0;
+
+ if (s->journal_size <= 0) {
+ if (!s->table && !s->fresh_bitmap) {
+ return 0; /* No need to use the journal. */
+ }
+
+ if (!header->clean_shutdown) {
+ fprintf(stderr, "ERROR: the image may be corrupted because it was "
+ "not shut down gracefully last\ntime and it does not use "
+ "a journal. You may continue to use the image at your\n"
+ "own risk by manually resetting the clean_shutdown flag "
+ "in the image.\n\n");
+ s->dirty_image = true;
+ if (IN_QEMU_TOOL) {
+ return 0; /* Allow qemu tools to use the image. */
+ } else {
+ /* Do not allow boot the VM until the clean_shutdown flag is
+ * manually cleaned. */
+ return -EINVAL;
+ }
+ }
+
+ QDEBUG("Journal is disabled\n");
+ return 0;
+ }
+
+ if (!read_only && !IN_QEMU_TOOL && s->fvd_metadata->enable_write_cache
+ && header->journal_buf_size > 0) {
+ s->use_bjnl = true;
+ QTAILQ_INIT(&s->bjnl.queued_bufs);
+ s->bjnl.buf = NULL;
+ s->bjnl.def_buf_size = header->journal_buf_size;
+ s->bjnl.clean_buf_period = header->journal_clean_buf_period;
+ s->bjnl.buf_contains_bitmap_update = false;
+ s->bjnl.clean_buf_timer = qemu_new_timer(rt_clock,
+ (QEMUTimerCB *)bjnl_clean_buf_timer_cb,
bs);
+ s->bjnl.timer_scheduled = false;
+ }
+
+ if (header->clean_shutdown) {
+ QDEBUG("Journal is skipped as the VM was shut down gracefully "
+ "last time.\n");
+ return 0;
+ }
+
+ QDEBUG("Recover from the journal as the VM was not shut down gracefully "
+ "last time.\n");
+
+ uint8_t *journal = my_qemu_blockalign(s->fvd_metadata,
+ s->journal_size * 512);
+ int ret = bdrv_read(s->fvd_metadata, s->journal_offset,
+ journal, s->journal_size);
+ if (ret < 0) {
+ my_qemu_vfree(journal);
+ fprintf(stderr, "Failed to read the journal (%" PRId64 ") bytes\n",
+ s->journal_size * 512);
+ return -EIO;
+ }
+
+ /* Go through every journal sector. */
+ uint64_t max_epoch = 0;
+ uint8_t *sector = journal;
+ uint8_t *journal_end = journal + s->journal_size * 512;
+ uint64_t *chunk_epoch = NULL;
+
+ if (header->table_offset > 0) {
+ int table_entries = ROUND_UP(header->virtual_disk_size,
+ header->chunk_size) / header->chunk_size;
+ chunk_epoch = my_qemu_mallocz(sizeof(uint64_t) * table_entries);
+ }
+
+ while (sector < journal_end) {
+ uint32_t *type = (uint32_t *) sector; /* Journal record type. */
+ while ((uint8_t *) type < (sector + 512)) {
+ if (le32_to_cpu(*type) == BITMAP_JRECORD) {
+ uint32_t *nb_sectors = type + 1;
+ int64_t *sector_num = (int64_t *) (type + 2);
+ if (s->stale_bitmap) {
+ update_both_bitmaps(s, le64_to_cpu(*sector_num),
+ le32_to_cpu(*nb_sectors));
+ QDEBUG("JOURNAL: recover BITMAP_JRECORD sector_num=%"
+ PRId64 " nb_sectors=%u\n",
+ le64_to_cpu(*sector_num), le32_to_cpu(*nb_sectors));
+ }
+
+ /* First field of the next journal record. */
+ type = (uint32_t *) (sector_num + 1);
+ } else if (le32_to_cpu(*type) == TABLE_JRECORD) {
+ uint64_t *epoch = (uint64_t *) (type + 1);
+ uint32_t *count = (uint32_t *) (epoch + 1);
+ uint32_t *offset = count + 1;
+ uint32_t *content = offset + 1;
+ const uint32_t chunk = le32_to_cpu(*offset);
+ const uint64_t epo = le64_to_cpu(*epoch);
+ const uint32_t n = le32_to_cpu(*count);
+ uint32_t i;
+ QDEBUG("JOURNAL: recover TABLE_JRECORD epoch=%" PRIu64
+ " chunk_start=%u " "nb_chunks=%u\n", epo, chunk, n);
+ for (i = 0; i < n; i++) {
+ /* If a chunk can be mapped to different locations at
+ * different times, e.g., due to defragmentation
+ * activities that move chunks, the epoch number is used
+ * to identify the last effective mapping of a chunk. */
+ if (epo > header->stable_journal_epoch &&
+ epo > chunk_epoch[chunk + i]) {
+ chunk_epoch[chunk + i] = epo;
+ s->table[chunk + i] = content[i];
+
+ /* The dirty bit was not cleaned when the table entry
+ * was saved in the journal. */
+ CLEAN_DIRTY2(s->table[chunk + i]);
+ QDEBUG("\tAccept mapping chunk %u to %u\n",
+ chunk + i, READ_TABLE(content[i]));
+ } else {
+ QDEBUG("\tReject mapping chunk %u to %u\n",
+ chunk + i, READ_TABLE(content[i]));
+ }
+ }
+ type = content + n; /* First field of the next record. */
+ if (epo > max_epoch) {
+ max_epoch = epo;
+ }
+ } else {
+ /* End of valid records in this journal sector. */
+ ASSERT(le32_to_cpu(*type) == EMPTY_JRECORD);
+ break;
+ }
+ }
+
+ sector += 512;
+ }
+ my_qemu_vfree(journal);
+ if (chunk_epoch) {
+ my_qemu_free(chunk_epoch);
+ }
+
+ if (++max_epoch > s->journal_epoch) {
+ s->journal_epoch = max_epoch;
+ }
+ QDEBUG("JOURNAL: journal_epoch=%" PRIu64 "\n", s->journal_epoch);
+
+ if (!read_only) {
+ /* Write the recovered metadata. */
+ flush_metadata_to_disk(bs, true /*journal */ , false /*prefetch */ );
+ }
+
+ return 0;
}
-static void write_metadata_to_journal(struct FvdAIOCB *acb, bool update_bitmap)
+/*
+ * This function first flushes in-memory metadata to disk and then recycle the
+ * used journal sectors. It is possible to make this operation asynchronous so
+ * that the performance is better. However, the overall performance
+ * improvement may be limited since recycling the journal happens very
+ * infrequently and updating on-disk metadata finishes quickly because of the
+ * small size of the metadata.
+ */
+static int recycle_journal(BlockDriverState * bs)
{
+ BDRVFvdState *s = bs->opaque;
+ int ret;
+
+#ifdef ENABLE_QDEBUG
+ static int64_t recycle_count = 0;
+ QDEBUG("JOURNAL: start journal recycle %" PRId64 ".\n", recycle_count);
+ recycle_count++;
+ int64_t begin_time = qemu_get_clock(rt_clock);
+#endif
+
+ ret = flush_metadata_to_disk(bs, true /*journal */ , false /*prefetch */);
+ if (ret == 0) {
+ s->next_journal_sector = 0;
+ }
+
+#ifdef ENABLE_QDEBUG
+ int64_t end_time = qemu_get_clock(rt_clock);
+ QDEBUG("JOURNAL: journal recycle took %" PRId64 " ms.\n",
+ (end_time - begin_time));
+#endif
+
+ return ret;
+}
+
+static void write_metadata_to_journal_cb(void *opaque, int ret)
+{
+ FvdAIOCB *acb = (FvdAIOCB *) opaque;
+ BlockDriverState *bs = acb->common.bs;
+ BDRVFvdState *s = bs->opaque;
+
+ if (acb->cancel_in_progress) {
+ return;
+ }
+
+ if (ret == 0) {
+ QDEBUG("JOURNAL: acb%llu-%p write_metadata_to_journal_cb\n",
+ acb->uuid, acb);
+
+ if (s->table) {
+ /* Update the table. */
+ int i;
+ const uint32_t first_chunk = acb->sector_num / s->chunk_size;
+ const uint32_t last_chunk = (acb->sector_num + acb->nb_sectors - 1)
+ / s->chunk_size;
+ for (i = first_chunk; i <= last_chunk; i++) {
+ CLEAN_DIRTY2(s->table[i]);
+ }
+ }
+
+ if (s->stale_bitmap) {
+ /* If fresh_bitmap differs from stale_bitmap, fresh_bitmap has
+ * already been updated in write_data_cb() when invoking
+ * update_fresh_bitmap_and_check_stale_bitmap(). */
+ update_stale_bitmap(s, acb->sector_num, acb->nb_sectors);
+ }
+ } else {
+ QDEBUG("JOURNAL: acb%llu-%p write_metadata_to_journal_cb err
ret=%d\n",
+ acb->uuid, acb, ret);
+ }
+
+ /* Clean up. */
+ if (acb->type == OP_STORE_COMPACT) {
+ acb->common.cb(acb->common.opaque, ret);
+ if (acb->jcb.iov.iov_base != NULL) {
+ my_qemu_vfree(acb->jcb.iov.iov_base);
+ }
+ my_qemu_aio_release(acb);
+ } else {
+ ASSERT(acb->type == OP_WRITE);
+ finish_write(acb, ret);
+ }
+
+ if (!s->use_bjnl) {
+ ujnl_free_journal_sectors(bs);
+ }
+}
+
+static inline uint8_t * alloc_journal_records(FvdAIOCB *acb,
+ bool update_bitmap,
+ size_t buf_size,
+ int64_t *journal_sec)
+{
+ BlockDriverState *bs = acb->common.bs;
+ BDRVFvdState *s = bs->opaque;
+
+ if (s->use_bjnl) {
+ /* Use buffered journal update. */
+ return bjnl_alloc_journal_records_from_buf(bs, update_bitmap,
buf_size);
+ }
+
+ /* Allocate journal sectors for unbuffered journal update. */
+ size_t nb_sectors = (buf_size + 511) / 512;
+ *journal_sec = ujnl_allocate_journal_sectors(bs, acb, nb_sectors);
+ if (*journal_sec < 0) {
+ /* No journal sector is available now. It will be waken up later
+ * in ujnl_free_journal_sectors(). */
+ return NULL;
+ }
+
+ uint8_t *buf = my_qemu_blockalign(s->fvd_metadata, 512 * nb_sectors);
+ if (buf_size % 512 != 0) {
+ *((uint32_t*)(buf + buf_size)) = EMPTY_JRECORD; /* Mark buffer end. */
+ }
+ return buf;
+}
+
+static uint8_t * create_journal_records(FvdAIOCB * acb,
+ bool update_bitmap,
+ int64_t *p_journal_sec,
+ size_t *p_buf_size)
+{
+ BlockDriverState *bs = acb->common.bs;
+ BDRVFvdState *s = bs->opaque;
+ uint8_t *buf;
+ int64_t journal_sec = -1;
+ size_t buf_size;
+
+ if (update_bitmap && !s->table) {
+ /* Only update the bitmap. */
+ buf_size = BITMAP_JRECORD_SIZE;
+ buf = alloc_journal_records(acb, update_bitmap, buf_size,
&journal_sec);
+ if (!buf) {
+ return NULL; /* Wake up later in ujnl_free_journal_sectors(). */
+ }
+
+ uint32_t *type = (uint32_t *)buf; /*BITMAP_JRECORD*/
+ uint32_t *nb_sectors = type + 1;
+ int64_t *sector_num = (int64_t *) (type + 2);
+ *type = cpu_to_le32(BITMAP_JRECORD);
+ *nb_sectors = cpu_to_le32((uint32_t) acb->nb_sectors);
+ *sector_num = cpu_to_le64(acb->sector_num);
+ QDEBUG("JOURNAL: record BITMAP_JRECORD sector_num=%" PRId64
+ " nb_sectors=%u\n", acb->sector_num, acb->nb_sectors);
+
+ } else if (!update_bitmap) {
+ /* Only update the table. */
+
+ const int64_t first_chunk = acb->sector_num / s->chunk_size;
+ const int64_t last_chunk = (acb->sector_num + acb->nb_sectors - 1)
+ / s->chunk_size;
+ int num_chunks = last_chunk - first_chunk + 1;
+
+ /* Buf space for complete journal sectors. */
+ buf_size = (num_chunks / TABLE_JRECORDS_PER_SECTOR) * 512;
+
+ /* Buf space for the last partial journal sectors. */
+ int rc = num_chunks % TABLE_JRECORDS_PER_SECTOR;
+ if (rc > 0) {
+ buf_size += TABLE_JRECORD_HDR_SIZE + sizeof(uint32_t) * rc;
+ }
+
+ buf = alloc_journal_records(acb, update_bitmap, buf_size,
&journal_sec);
+ if (!buf) {
+ return NULL; /* Wake up later in ujnl_free_journal_sectors(). */
+ }
+
+ uint32_t *type = (uint32_t *)buf; /* TABLE_JRECORD */
+ int64_t chunk = first_chunk;
+
+ while (1) {
+ /* Start a new journal sector. */
+ uint64_t *epoch = (uint64_t *) (type + 1);
+ uint32_t *count = (uint32_t *) (epoch + 1);
+ uint32_t *offset = count + 1;
+ uint32_t *content = offset + 1;
+
+ *type = cpu_to_le32(TABLE_JRECORD);
+ *offset = cpu_to_le32(chunk);
+ *epoch = cpu_to_le64(s->journal_epoch);
+ s->journal_epoch++;
+ if (num_chunks <= TABLE_JRECORDS_PER_SECTOR) {
+ /* This is the last journal sector. */
+ *count = cpu_to_le32(num_chunks);
+ memcpy(content, &s->table[chunk],
+ sizeof(uint32_t) * num_chunks);
+ PRINT_TABLE_JRECORD(type);
+ break;
+ }
+
+ *count = cpu_to_le32(TABLE_JRECORDS_PER_SECTOR);
+ memcpy(content, &s->table[chunk],
+ sizeof(uint32_t) * TABLE_JRECORDS_PER_SECTOR);
+ chunk += TABLE_JRECORDS_PER_SECTOR;
+ num_chunks -= TABLE_JRECORDS_PER_SECTOR;
+ PRINT_TABLE_JRECORD(type);
+
+ /* Next TABLE_JRECORD field 1. */
+ type = content + TABLE_JRECORDS_PER_SECTOR;
+ }
+ } else {
+ /* Update both the table and the bitmap. It may use multiple journal
+ * sectors. Each sector is self-contained, including a TABLE_JRECORD
+ * and a BITMAP_JRECORD. The two records one the same sector cover the
+ * same range of virtual disk data. The purpose is to update the
+ * corresponding parts of the bitmap and the table in one atomic
+ * operation. */
+ const int64_t first_chunk = acb->sector_num / s->chunk_size;
+ const int64_t last_chunk = (acb->sector_num + acb->nb_sectors - 1)
+ / s->chunk_size;
+ int num_chunks = last_chunk - first_chunk + 1;
+
+ /* Buf space for complete journal sectors. */
+ buf_size = (num_chunks / MIXED_JRECORDS_PER_SECTOR) * 512;
+
+ /* Buf space for the last partial journal sectors. */
+ int rc = num_chunks % MIXED_JRECORDS_PER_SECTOR;
+ if (rc > 0) {
+ buf_size += BITMAP_JRECORD_SIZE + TABLE_JRECORD_HDR_SIZE
+ + sizeof(uint32_t) * rc;
+ }
+
+ buf = alloc_journal_records(acb, update_bitmap, buf_size,
&journal_sec);
+ if (!buf) {
+ return NULL; /* Wake up later in ujnl_free_journal_sectors(). */
+ }
+
+ uint32_t *type = (uint32_t *)buf; /*TABLE_JRECORD*/
+ int64_t chunk = first_chunk;
+ int64_t sector_num = acb->sector_num;
+ uint32_t nb_sectors;
+
+ /* Determine the number of data sectors whose bitmap change fits in
+ * the first journal sector. */
+ if (buf_size <= 512) {
+ nb_sectors = acb->nb_sectors; /* All fit in one journal sector. */
+ } else {
+ nb_sectors = (first_chunk + MIXED_JRECORDS_PER_SECTOR)
+ * s->chunk_size - acb->sector_num;
+ }
+
+ while (1) {
+ /* Start a new journal sector. */
+ uint64_t *epoch = (uint64_t *) (type + 1);
+ uint32_t *count = (uint32_t *) (epoch + 1);
+ uint32_t *offset = count + 1;
+ uint32_t *content = offset + 1;
+
+ *type = cpu_to_le32(TABLE_JRECORD);
+ *offset = cpu_to_le32(chunk);
+ *epoch = cpu_to_le64(s->journal_epoch);
+ s->journal_epoch++;
+
+ if (num_chunks <= MIXED_JRECORDS_PER_SECTOR) {
+ /* This is the last journal sector. */
+ *count = cpu_to_le32(num_chunks);
+ memcpy(content, &s->table[chunk],
+ sizeof(uint32_t) * num_chunks);
+ PRINT_TABLE_JRECORD(type);
+
+ /* A BITMAP_JRECORD follows a TABLE_JRECORD so that they are
+ * updated in one atomic operatoin. */
+ type = content + num_chunks; /* BITMAP_JRECORD. */
+ uint32_t *p_nb_sectors = type + 1;
+ int64_t *p_sector_num = (int64_t *) (type + 2);
+ *type = cpu_to_le32(BITMAP_JRECORD);
+ *p_nb_sectors = cpu_to_le32(nb_sectors);
+ *p_sector_num = cpu_to_le64(sector_num);
+ QDEBUG("JOURNAL: record BITMAP_JRECORD sector_num=%" PRId64
+ " nb_sectors=%u\n", sector_num, nb_sectors);
+ break;
+ }
+
+ *count = cpu_to_le32(MIXED_JRECORDS_PER_SECTOR);
+ memcpy(content, &s->table[chunk],
+ sizeof(uint32_t) * MIXED_JRECORDS_PER_SECTOR);
+ PRINT_TABLE_JRECORD(type);
+
+ /* A BITMAP_JRECORD follows a TABLE_JRECORD so that they are
+ * updated in one atomic operatoin. */
+ type = content + MIXED_JRECORDS_PER_SECTOR; /* BITMAP_JRECORD */
+ uint32_t *p_nb_sectors = type + 1;
+ int64_t *p_sector_num = (int64_t *) (type + 2);
+ *type = cpu_to_le32(BITMAP_JRECORD);
+ *p_nb_sectors = cpu_to_le32(nb_sectors);
+ *p_sector_num = cpu_to_le64(sector_num);
+ QDEBUG("JOURNAL: record BITMAP_JRECORD sector_num=%" PRId64
+ " nb_sectors=%u\n", sector_num, nb_sectors);
+
+ /* Prepare for the next journal sector. */
+ type = (uint32_t *) (p_sector_num + 1);
+ chunk += MIXED_JRECORDS_PER_SECTOR;
+ sector_num = chunk * s->chunk_size;
+ num_chunks -= MIXED_JRECORDS_PER_SECTOR;
+ if (num_chunks <= MIXED_JRECORDS_PER_SECTOR) {
+ /* Data sectors covered by the last journal sector. */
+ nb_sectors = (acb->sector_num + acb->nb_sectors)
+ - chunk * s->chunk_size;
+ } else {
+ nb_sectors = s->chunk_size * MIXED_JRECORDS_PER_SECTOR;
+ }
+ }
+ }
+
+ if (p_journal_sec) {
+ *p_journal_sec = journal_sec;
+ }
+ if (p_buf_size) {
+ *p_buf_size = buf_size;
+ }
+ return buf;
+}
+
+static void write_metadata_to_journal(FvdAIOCB * acb, bool update_bitmap)
+{
+ BlockDriverState *bs = acb->common.bs;
+ BDRVFvdState *s = bs->opaque;
+
+ ASSERT((s->table || s->fresh_bitmap)
+ && (!update_bitmap || (s->fresh_bitmap && acb->type == OP_WRITE))
+ && (update_bitmap || s->table)
+ && (acb->type == OP_WRITE || acb->type == OP_STORE_COMPACT));
+
+ if (acb->type == OP_WRITE) {
+ /* Save update_bitmap as it may be needed later if this request is
+ * queued.*/
+ acb->write.update_bitmap = update_bitmap;
+ }
+
+ if (s->metadata_err_prohibit_write) {
+ write_metadata_to_journal_cb(acb, -EIO); /* Fail the request now. */
+ return;
+ }
+
+ if (s->journal_size <= 0) {
+ write_metadata_to_journal_cb(acb, 0); /* Journal is disabled. */
+ return;
+ }
+
+ int64_t journal_sec = -1;
+ size_t buf_size = -1;
+ uint8_t *buf = create_journal_records(acb, update_bitmap,
+ &journal_sec, &buf_size);
+
+ /* Depending on the cache mode, either write metadata changes to journal
+ * immediately, or put it in buffer first. */
+ if (s->use_bjnl) {
+ /* Done for now. The buffer will be written to the journal later. */
+ write_metadata_to_journal_cb(acb, 0);
+ } else if (buf) {
+ int nb_sectors = (buf_size + 511) / 512;
+ ujnl_write_metadata_to_journal_now(acb, update_bitmap, buf,
+ journal_sec, nb_sectors);
+ }
+}
+
+static int flush_metadata_to_disk(BlockDriverState * bs,
+ bool update_journal_epoch,
+ bool update_base_img_prefetched)
+{
+ BDRVFvdState *s = bs->opaque;
+
+ if (bs->read_only) {
+ return 0;
+ }
+
+ /* Clean DIRTY_TABLE bit and write the table to disk. */
+ if (s->table) {
+ int i;
+ int table_entries = ROUND_UP(s->virtual_disk_size,
+ s->chunk_size * 512) / (s->chunk_size *
+ 512);
+ for (i = 0; i < table_entries; i++) {
+ CLEAN_DIRTY(s->table[i]);
+ }
+
+ int nb = (int)(s->table_size / 512);
+ QDEBUG("JOURNAL: flush table (%d sectors) to disk\n", nb);
+
+ if (bdrv_write(s->fvd_metadata, s->table_offset, (uint8_t *) s->table,
+ nb) < 0) {
+ goto fail;
+ }
+ }
+
+ /* Write fresh_bitmap to disk. */
+ if (s->fresh_bitmap) {
+ /* Ensure copy-on-read and prefetching data are stable. */
+ if (bdrv_flush(s->fvd_data)) {
+ goto fail;
+ }
+
+ if (s->fvd_data != s->fvd_metadata && s->table) {
+ /* Ensure table is stable before updating bitmap. */
+ if (bdrv_flush(s->fvd_metadata)) {
+ goto fail;
+ }
+ }
+
+ int nb = (int)(s->bitmap_size / 512);
+ QDEBUG("JOURNAL: flush bitmap (%d sectors) to disk\n", nb);
+
+ if (bdrv_write(s->fvd_metadata, s->bitmap_offset,
+ s->fresh_bitmap, nb) < 0) {
+ goto fail;
+ }
+ if (s->fresh_bitmap != s->stale_bitmap) {
+ memcpy(s->stale_bitmap, s->fresh_bitmap, s->bitmap_size);
+ }
+ }
+
+ if (update_journal_epoch || update_base_img_prefetched) {
+ /* Update the header. */
+ FvdHeader header;
+ if (read_fvd_header(s, &header)) {
+ goto fail;
+ }
+ if (update_base_img_prefetched) {
+ header.base_img_fully_prefetched = true;
+ }
+ if (update_journal_epoch) {
+ header.stable_journal_epoch = s->journal_epoch++;
+ }
+ if (update_fvd_header(s, &header)) {
+ goto fail;
+ }
+ }
+
+ /* Perform a final flush to ensure all metadata are stable. */
+ if (!bdrv_flush(s->fvd_metadata)) {
+ return 0;
+ }
+
+fail:
+ s->metadata_err_prohibit_write = true;
+ return -EIO;
+}
+
+#ifdef FVD_DEBUG
+static bool emulate_host_crash = true;
+#else
+static bool emulate_host_crash = false;
+#endif
+
+static void flush_metadata_to_disk_on_exit(BlockDriverState * bs)
+{
+ BDRVFvdState *s = bs->opaque;
+
+ if (bs->read_only || !s->fvd_metadata) {
+ return;
+ }
+
+ /* If (emulate_host_crash==true), do not flush metadata to disk
+ * so that it has to rely on journal for recovery. */
+ if (s->journal_size <= 0 || !emulate_host_crash) {
+ if (!flush_metadata_to_disk(bs, true, false) && !s->dirty_image) {
+ update_clean_shutdown_flag(s, true);
+ }
+ }
}
void fvd_emulate_host_crash(bool cond)
{
emulate_host_crash = cond;
}
+
+#ifdef ENABLE_QDEBUG
+static void print_table_jrecord(uint32_t * type)
+{
+ int32_t i;
+ uint64_t *p_epoch = (uint64_t *) (type + 1);
+ uint32_t *p_count = (uint32_t *) (p_epoch + 1);
+ uint32_t *p_offset = p_count + 1;
+ uint32_t *content = p_offset + 1;
+
+ uint64_t epoch = le64_to_cpu(*p_epoch);
+ uint32_t count = le32_to_cpu(*p_count);
+ uint32_t offset = le32_to_cpu(*p_offset);
+
+ QDEBUG("JOURNAL: record TABLE_JRECORD epoch=%" PRIu64
+ " chunk_start=%u " "nb_chunks=%u\n", epoch, offset, count);
+ for (i = 0; i < count; i++) {
+ QDEBUG("\tMap chunk %u to %u\n", offset + i, READ_TABLE(content[i]));
+ }
+}
+#endif
+
+/* Only used for unbuffered journal update. */
+static void ujnl_write_metadata_to_journal_now(FvdAIOCB * acb,
+ bool update_bitmap,
+ uint8_t *buf,
+ int64_t journal_sec,
+ int nb_journal_sectors)
+{
+ BlockDriverState *bs = acb->common.bs;
+ BDRVFvdState *s = bs->opaque;
+
+ QDEBUG("JOURNAL: acb%llu-%p write_metadata_to_journal journal_sec=%"
+ PRId64 " nb_journal_sectors=%d\n", acb->uuid, acb, journal_sec,
+ nb_journal_sectors);
+
+ acb->jcb.iov.iov_base = buf;
+ acb->jcb.iov.iov_len = 512 * nb_journal_sectors;
+ qemu_iovec_init_external(&acb->jcb.qiov, &acb->jcb.iov, 1);
+ acb->jcb.hd_acb = bdrv_aio_writev(s->fvd_metadata,
+ s->journal_offset + journal_sec,
+ &acb->jcb.qiov, nb_journal_sectors,
+ write_metadata_to_journal_cb, acb);
+ if (!acb->jcb.hd_acb) {
+ write_metadata_to_journal_cb(acb, -1);
+ }
+}
+
+/* Only used for unbuffered journal update. */
+static void ujnl_free_journal_sectors(BlockDriverState * bs)
+{
+ BDRVFvdState *s = bs->opaque;
+
+ if (s->journal_size <= 0) {
+ return;
+ }
+
+ s->ujnl.active_writes--;
+ ASSERT(s->ujnl.active_writes >= 0);
+
+ if (s->ujnl.active_writes > 0 || QLIST_EMPTY(&s->ujnl.wait4_recycle)) {
+ return;
+ }
+
+ /* Some requests are waiting for the journal to be recycled in order to
+ * get free journal sectors. */
+ recycle_journal(bs);
+
+ /* Restart requests in the ujnl.wait4_recycle list. First make a copy of
+ * the head and then empty the head. */
+ FvdAIOCB *acb = QLIST_FIRST(&s->ujnl.wait4_recycle);
+ QLIST_INIT(&s->ujnl.wait4_recycle);
+ FvdAIOCB *next;
+
+ /* Restart all dependent requests. Cannot use QLIST_FOREACH here, because
+ * the next link might not be the same any more after the callback. */
+ while (acb) {
+ next = acb->jcb.ujnl_next_wait4_recycle.le_next;
+ acb->jcb.ujnl_next_wait4_recycle.le_prev = NULL;
+ QDEBUG("WRITE: acb%llu-%p restart_write_metadata_to_journal "
+ "after recycle_journal\n", acb->uuid, acb);
+ if (acb->type == OP_WRITE) {
+ write_metadata_to_journal(acb, acb->write.update_bitmap);
+ } else {
+ write_metadata_to_journal(acb, false);
+ }
+ acb = next;
+ }
+}
+
+/* Only used for unbuffered journal update. */
+static int64_t ujnl_allocate_journal_sectors(BlockDriverState * bs,
+ FvdAIOCB * acb, int nb_sectors)
+{
+ BDRVFvdState *s = bs->opaque;
+ ASSERT(nb_sectors <= s->journal_size);
+
+ if (!QLIST_EMPTY(&s->ujnl.wait4_recycle)) {
+ /* Waiting for journal recycle to finish. */
+ ASSERT(s->ujnl.active_writes > 0);
+ QDEBUG("WRITE: acb%llu-%p wait4_journal_recycle active_writes=%d\n",
+ acb->uuid, acb, s->ujnl.active_writes);
+ QLIST_INSERT_HEAD(&s->ujnl.wait4_recycle, acb,
+ jcb.ujnl_next_wait4_recycle);
+ return -1;
+ }
+
+ int64_t journal_sec;
+ if (s->next_journal_sector + nb_sectors <= s->journal_size) {
+ journal_sec = s->next_journal_sector;
+ s->next_journal_sector += nb_sectors;
+ s->ujnl.active_writes++;
+ return journal_sec;
+ }
+
+ /* No free journal sector is available. Check if the journal can be
+ * recycled now. */
+ if (s->ujnl.active_writes == 0) {
+ recycle_journal(bs);
+ s->next_journal_sector = nb_sectors;
+ s->ujnl.active_writes = 1;
+ return 0; /* Use the first sector. */
+ }
+
+ /* Waiting for journal recycle to finish. It will be waken up later in
+ * ujnl_free_journal_sectors(). */
+ QLIST_INSERT_HEAD(&s->ujnl.wait4_recycle, acb,
jcb.ujnl_next_wait4_recycle);
+ QDEBUG("WRITE: acb%llu-%p wait4_journal_recycle active_writes=%d\n",
+ acb->uuid, acb, s->ujnl.active_writes);
+ return -1;
+}
diff --git a/block/fvd-write.c b/block/fvd-write.c
index f0580d4..623ec83 100644
--- a/block/fvd-write.c
+++ b/block/fvd-write.c
@@ -15,6 +15,7 @@ static void write_metadata_to_journal(struct FvdAIOCB *acb,
bool update_bitmap);
static int do_aio_write(struct FvdAIOCB *acb);
static void restart_dependent_writes(struct FvdAIOCB *acb);
static void free_write_resource(struct FvdAIOCB *acb);
+static void ujnl_free_journal_sectors(BlockDriverState * bs);
static inline BlockDriverAIOCB *store_data(int soft_write,
FvdAIOCB * parent_acb, BlockDriverState * bs,
int64_t sector_num, QEMUIOVector * orig_qiov, int nb_sectors,
diff --git a/block/fvd.c b/block/fvd.c
index 74845e7..2402a94 100644
--- a/block/fvd.c
+++ b/block/fvd.c
@@ -37,6 +37,7 @@
#include "block/fvd-store.c"
#include "block/fvd-load.c"
#include "block/fvd-journal.c"
+#include "block/fvd-journal-buf.c"
#include "block/fvd-prefetch.c"
#include "block/fvd-update.c"
@@ -65,3 +66,21 @@ static void bdrv_fvd_init(void)
}
block_init(bdrv_fvd_init);
+
+/* Since bdrv_close may not be properly invoked on a VM shutdown, we use a
+ * destructor to flush metadata to disk. This only affects performance and
+ * does not affect correctness. See Section 3.3.4 of the FVD-cow paper for
+ * the rationale. */
+extern QTAILQ_HEAD(, BlockDriverState) bdrv_states;
+ static void __attribute__ ((destructor)) flush_fvd_bitmap_to_disk(void)
+{
+ BlockDriverState *bs;
+ QTAILQ_FOREACH(bs, &bdrv_states, list) {
+ if (bs->drv == &bdrv_fvd) {
+ flush_metadata_to_disk_on_exit(bs);
+#ifdef FVD_DEBUG
+ dump_resource_summary(bs->opaque);
+#endif
+ }
+ }
+}
--
1.7.0.4
- [Qemu-devel] [PATCH 24/26] FVD: add impl of interface bdrv_has_zero_init(), (continued)
- [Qemu-devel] [PATCH 24/26] FVD: add impl of interface bdrv_has_zero_init(), Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 21/26] FVD: add impl of interface bdrv_close(), Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 14/26] FVD: add impl of loading data from compact image, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 26/26] FVD: add fully automated test-fvd.sh, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 23/26] FVD: add impl of interface bdrv_is_allocated(), Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 17/26] FVD: add impl of bdrv_flush() and bdrv_aio_flush(), Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 22/26] FVD: add impl of interface bdrv_update(), Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 13/26] FVD: add impl of storing data in compact image, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 19/26] FVD: add support for aio_cancel, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 25/26] FVD: add impl of interface bdrv_probe(), Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 15/26] FVD: add basic journal functionality,
Chunqiang Tang <=
- [Qemu-devel] [PATCH 18/26] FVD: add support for base image prefetching, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 03/26] FVD: add fully automated test-qcow2.sh, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 02/26] FVD: extend qemu-io to do fully automated testing, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 07/26] FVD: extend FVD header fvd.h to be more complete, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 12/26] FVD: add impl of interface bdrv_aio_readv(), Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 11/26] FVD: add impl of interface bdrv_aio_writev(), Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 06/26] FVD: skeleton of Fast Virtual Disk, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 05/26] FVD: add the 'qemu-img update' command, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 04/26] FVD: add fully automated test-vdi.sh, Chunqiang Tang, 2011/02/25
- [Qemu-devel] [PATCH 09/26] FVD: add impl of interface bdrv_create(), Chunqiang Tang, 2011/02/25