2019-10-01 07:45:21

by harshad shirwadkar

[permalink] [raw]
Subject: [PATCH v3 00/13] ext4: add fast commit support

This patch series adds support for fast commits which is a simplified
version of the scheme proposed by Park and Shin, in their paper,
"iJournaling: Fine-Grained Journaling for Improving the Latency of
Fsync System Call"[1]. The basic idea of fast commits is to make JBD2
give the client file system an opportunity to perform a faster
commit. Only if the file system cannot perform such a commit
operation, then JBD2 should fall back to traditional commits.

Because JBD2 operates at block granularity, for every file system
metadata update it commits all the changed blocks are written to the
journal at commit time. This is inefficient because updates to some
blocks that JBD2 commits are derivable from some other blocks. For
example, if a new extent is added to an inode, then corresponding
updates to the inode table, the block bitmap, the group descriptor and
the superblock can be derived based on just the extent information and
the corresponding inode information. So, if we take this relationship
between blocks into account and replay the journalled blocks smartly,
we could increase performance of file system commits significantly.

Fast commits introduced in this patch have two main contributions:

(1) Making JBD2 fast commit aware, so that clients of JBD2 can
implement fast commits

(2) Add support in ext4 to use JBD2's new interfaces and implement
fast commits

Testing
-------

e2fsprogs was updated to set fast commit feature flag and to ignore
fast commit blocks during e2fsck.

https://github.com/harshadjs/e2fsprogs.git

After applying all the patches in this series, following runs of
xfstests were performed:

- kvm-xfstest.sh -g log -c 4k
- kvm-xfstests.sh smoke

All the log tests were successful and smoke tests didn't introduce any
additional failures.

Performance Evaluation
----------------------

Ext4 file system performance was tested with and without fast commit
using fs_mark benchmark. Following was the command used:

Command: ./fs_mark -t 8 -n 1024 -s 65536 -w 4096 -d /mnt

Results:
Without Fast Commit: 1501.2 files/sec
With Fast commits: 3055 files/sec
~103% write performance improvement

Changes since V2:

- Added ability to support new file creation in fast commits. This
allows us to use fs_mark benchmark for performance testing

- Added support for asynchronous fast commits

- Many cleanups and bug fixes

- Re-organized the patch set, moved most of the new code to
ext4_jbd2.c instead of super.c

- Handling of review comments on previous patchset

Harshad Shirwadkar(13):
docs: Add fast commit documentation
ext4: add support for asynchronous fast commits
ext4: fast-commit recovery path changes
ext4: fast-commit commit path changes
ext4: fast-commit commit range tracking
ext4: track changed files for fast commit
ext4: add fields that are needed to track changed files
jbd2: fast-commit recovery path changes
jbd2: fast-commit commit path new APIs
jbd2: fast-commit commit path changes
jbd2: fast commit setup and enable
ext4: add handling for extended mount options
ext4: add fast commit support

Documentation/filesystems/ext4/journal.rst | 98 +-
Documentation/filesystems/journalling.rst | 22
fs/ext4/acl.c | 1
fs/ext4/balloc.c | 7
fs/ext4/ext4.h | 86 +
fs/ext4/ext4_jbd2.c | 902 +++++++++++++++++++
fs/ext4/ext4_jbd2.h | 98 ++
fs/ext4/extents.c | 43
fs/ext4/fsync.c | 7
fs/ext4/ialloc.c | 60 -
fs/ext4/inline.c | 14
fs/ext4/inode.c | 77 +
fs/ext4/ioctl.c | 9
fs/ext4/mballoc.c | 83 +
fs/ext4/mballoc.h | 2
fs/ext4/migrate.c | 1
fs/ext4/namei.c | 16
fs/ext4/super.c | 55 +
fs/ext4/xattr.c | 1
fs/jbd2/commit.c | 98 ++
fs/jbd2/journal.c | 343 ++++++-
fs/jbd2/recovery.c | 63 +
fs/jbd2/transaction.c | 3
include/linux/jbd2.h | 117 ++
include/trace/events/ext4.h | 61 +
include/trace/events/jbd2.h | 9
26 files changed, 2170 insertions(+), 106 deletions(-)
--
2.23.0.444.g18eeb5a265-goog


2019-10-01 07:45:22

by harshad shirwadkar

[permalink] [raw]
Subject: [PATCH v3 08/13] ext4: fast-commit commit range tracking

With this patch, we track logical range of file offsets that need to
be committed using fast commit. This allows us to find file extents
that need to be committed during the commit time.

Signed-off-by: Harshad Shirwadkar <[email protected]>
---
fs/ext4/ext4_jbd2.c | 34 ++++++++++++++++++++++++++++++++++
fs/ext4/ext4_jbd2.h | 2 ++
fs/ext4/inline.c | 4 +++-
fs/ext4/inode.c | 17 ++++++++++++++++-
4 files changed, 55 insertions(+), 2 deletions(-)

diff --git a/fs/ext4/ext4_jbd2.c b/fs/ext4/ext4_jbd2.c
index e70ad7a8e46e..0bb8de2139a5 100644
--- a/fs/ext4/ext4_jbd2.c
+++ b/fs/ext4/ext4_jbd2.c
@@ -405,6 +405,40 @@ void ext4_fc_del(struct inode *inode)
spin_unlock(&EXT4_SB(inode->i_sb)->s_fc_lock);
}

+void ext4_fc_update_commit_range(struct inode *inode, ext4_lblk_t start,
+ ext4_lblk_t end)
+{
+ struct ext4_inode_info *ei = EXT4_I(inode);
+ tid_t running_txn_tid = get_running_txn_tid(inode->i_sb);
+
+ if (!ext4_should_fast_commit(inode->i_sb))
+ return;
+
+ if (inode->i_ino < EXT4_FIRST_INO(inode->i_sb))
+ ext4_debug("Special inode %ld being modified\n", inode->i_ino);
+
+ if (!EXT4_SB(inode->i_sb)->s_fc_eligible)
+ return;
+
+ write_lock(&ei->i_fc.fc_lock);
+ if (ei->i_fc.fc_tid == running_txn_tid) {
+ ei->i_fc.fc_lblk_start = ei->i_fc.fc_lblk_start < start ?
+ ei->i_fc.fc_lblk_start : start;
+ ei->i_fc.fc_lblk_end = ei->i_fc.fc_lblk_end > end ?
+ ei->i_fc.fc_lblk_end : end;
+ write_unlock(&ei->i_fc.fc_lock);
+ return;
+ }
+
+ ext4_reset_inode_fc_info(&ei->i_fc);
+ ei->i_fc.fc_eligible = true;
+ ei->i_fc.fc_lblk_start = start;
+ ei->i_fc.fc_lblk_end = end;
+ ei->i_fc.fc_tid = running_txn_tid;
+ write_unlock(&ei->i_fc.fc_lock);
+
+}
+
void ext4_fc_mark_new(struct inode *inode)
{
struct ext4_inode_info *ei = EXT4_I(inode);
diff --git a/fs/ext4/ext4_jbd2.h b/fs/ext4/ext4_jbd2.h
index 65f20fbfb002..2cb7e7e1f025 100644
--- a/fs/ext4/ext4_jbd2.h
+++ b/fs/ext4/ext4_jbd2.h
@@ -501,6 +501,8 @@ ext4_fc_mark_ineligible(struct inode *inode)
spin_unlock(&sbi->s_fc_lock);
}

+void ext4_fc_update_commit_range(struct inode *inode, ext4_lblk_t start,
+ ext4_lblk_t end);

void ext4_fc_mark_new(struct inode *inode);
bool ext4_is_inode_fc_ineligible(struct inode *inode);
diff --git a/fs/ext4/inline.c b/fs/ext4/inline.c
index fbd561cba098..66b2c0e3f7e4 100644
--- a/fs/ext4/inline.c
+++ b/fs/ext4/inline.c
@@ -966,8 +966,10 @@ int ext4_da_write_inline_data_end(struct inode *inode, loff_t pos,
* But it's important to update i_size while still holding page lock:
* page writeout could otherwise come in and zero beyond i_size.
*/
- if (pos+copied > inode->i_size)
+ if (pos+copied > inode->i_size) {
+ ext4_fc_update_commit_range(inode, inode->i_size, pos + copied);
i_size_write(inode, pos+copied);
+ }
unlock_page(page);
put_page(page);

diff --git a/fs/ext4/inode.c b/fs/ext4/inode.c
index 6d2efbd9aba9..ea039e3e1a4d 100644
--- a/fs/ext4/inode.c
+++ b/fs/ext4/inode.c
@@ -1549,6 +1549,8 @@ static int ext4_journalled_write_end(struct file *file,
SetPageUptodate(page);
}
size_changed = ext4_update_inode_size(inode, pos + copied);
+ ext4_fc_update_commit_range(inode, pos, pos + copied);
+
ext4_set_inode_state(inode, EXT4_STATE_JDATA);
EXT4_I(inode)->i_datasync_tid = handle->h_transaction->t_tid;
unlock_page(page);
@@ -2610,8 +2612,12 @@ static int mpage_map_and_submit_extent(handle_t *handle,
i_size = i_size_read(inode);
if (disksize > i_size)
disksize = i_size;
- if (disksize > EXT4_I(inode)->i_disksize)
+ if (disksize > EXT4_I(inode)->i_disksize) {
+ ext4_fc_update_commit_range(inode,
+ EXT4_I(inode)->i_disksize,
+ disksize);
EXT4_I(inode)->i_disksize = disksize;
+ }
up_write(&EXT4_I(inode)->i_data_sem);
err2 = ext4_mark_inode_dirty(handle, inode);
ext4_fc_enqueue_inode(handle, inode);
@@ -3220,6 +3226,8 @@ static int ext4_da_write_end(struct file *file,
}
}

+ ext4_fc_update_commit_range(inode, pos, pos + copied);
+
if (write_mode != CONVERT_INLINE_DATA &&
ext4_test_inode_state(inode, EXT4_STATE_MAY_INLINE_DATA) &&
ext4_has_inline_data(inode))
@@ -3627,6 +3635,7 @@ static int ext4_iomap_end(struct inode *inode, loff_t offset, loff_t length,
goto orphan_del;
}

+ ext4_fc_update_commit_range(inode, offset, offset + written);
if (ext4_update_inode_size(inode, offset + written)) {
ext4_mark_inode_dirty(handle, inode);
ext4_fc_enqueue_inode(handle, inode);
@@ -3751,6 +3760,7 @@ static ssize_t ext4_direct_IO_write(struct kiocb *iocb, struct iov_iter *iter)
ext4_update_i_disksize(inode, inode->i_size);
ext4_journal_stop(handle);
}
+ ext4_fc_update_commit_range(inode, offset, offset + count);

BUG_ON(iocb->private == NULL);

@@ -3869,6 +3879,8 @@ static ssize_t ext4_direct_IO_write(struct kiocb *iocb, struct iov_iter *iter)
ext4_mark_inode_dirty(handle, inode);
ext4_fc_enqueue_inode(handle, inode);
}
+ ext4_fc_update_commit_range(inode, offset,
+ offset + end);
}
err = ext4_journal_stop(handle);
if (ret == 0)
@@ -5327,6 +5339,9 @@ static int ext4_do_update_inode(handle_t *handle,
cpu_to_le16(ei->i_file_acl >> 32);
raw_inode->i_file_acl_lo = cpu_to_le32(ei->i_file_acl);
if (ei->i_disksize != ext4_isize(inode->i_sb, raw_inode)) {
+ ext4_fc_update_commit_range(inode,
+ ext4_isize(inode->i_sb, raw_inode),
+ ei->i_disksize);
ext4_isize_set(raw_inode, ei->i_disksize);
need_datasync = 1;
}
--
2.23.0.444.g18eeb5a265-goog

2019-10-01 07:45:22

by harshad shirwadkar

[permalink] [raw]
Subject: [PATCH v3 04/13] jbd2: fast-commit commit path new APIs

This patch adds new helper APIs that ext4 needs for fast
commits. These new fast commit APIs are used by subsequent fast commit
patches to implement fast commits. Following new APIs are added:

/*
* Returns when either a full commit or a fast commit
* completes
*/
int jbd2_fc_complete_commit(journal_tc *journal, tid_t tid,
tid_t subtid)

/* Send all the data buffers related to an inode */
int journal_submit_inode_data(journal_t *journal,
struct jbd2_inode *jinode)

/* Map one fast commit buffer for use by the file system */
int jbd2_map_fc_buf(journal_t *journal, struct buffer_head **bh_out)

/* Wait on fast commit buffers to complete IO */
jbd2_wait_on_fc_bufs(journal_t *journal, int num_bufs)

/*
* Returns 1 if transaction identified by tid:subtid is already
* committed.
*/
int jbd2_commit_check(journal_t *journal, tid_t tid, tid_t subtid)

Signed-off-by: Harshad Shirwadkar <[email protected]>
---
fs/jbd2/commit.c | 32 +++++++++++++
fs/jbd2/journal.c | 110 +++++++++++++++++++++++++++++++++++++++++++
include/linux/jbd2.h | 8 ++++
3 files changed, 150 insertions(+)

diff --git a/fs/jbd2/commit.c b/fs/jbd2/commit.c
index 7db3e2b6336d..e85f51e1cc70 100644
--- a/fs/jbd2/commit.c
+++ b/fs/jbd2/commit.c
@@ -202,6 +202,38 @@ static int journal_submit_inode_data_buffers(struct address_space *mapping,
return ret;
}

+int jbd2_submit_inode_data(journal_t *journal, struct jbd2_inode *jinode)
+{
+ struct address_space *mapping;
+ loff_t dirty_start = jinode->i_dirty_start;
+ loff_t dirty_end = jinode->i_dirty_end;
+ int ret;
+
+ if (!jinode)
+ return 0;
+
+ if (!(jinode->i_flags & JI_WRITE_DATA))
+ return 0;
+
+ dirty_start = jinode->i_dirty_start;
+ dirty_end = jinode->i_dirty_end;
+
+ mapping = jinode->i_vfs_inode->i_mapping;
+ jinode->i_flags |= JI_COMMIT_RUNNING;
+
+ trace_jbd2_submit_inode_data(jinode->i_vfs_inode);
+ ret = journal_submit_inode_data_buffers(mapping, dirty_start,
+ dirty_end);
+
+ jinode->i_flags &= ~JI_COMMIT_RUNNING;
+ /* Protect JI_COMMIT_RUNNING flag */
+ smp_mb();
+ wake_up_bit(&jinode->i_flags, __JI_COMMIT_RUNNING);
+
+ return ret;
+}
+EXPORT_SYMBOL(jbd2_submit_inode_data);
+
/*
* Submit all the data buffers of inode associated with the transaction to
* disk.
diff --git a/fs/jbd2/journal.c b/fs/jbd2/journal.c
index 6853064605ff..14d549445418 100644
--- a/fs/jbd2/journal.c
+++ b/fs/jbd2/journal.c
@@ -781,6 +781,18 @@ int jbd2_log_wait_commit(journal_t *journal, tid_t tid)
return __jbd2_log_wait_commit(journal, tid, 0);
}

+int jbd2_commit_check(journal_t *journal, tid_t tid, tid_t subtid)
+{
+ if (journal->j_commit_sequence >= tid)
+ return 1;
+ if (!journal->j_running_transaction)
+ return 0;
+ if (journal->j_running_transaction->t_tid > tid)
+ return 1;
+ if (journal->j_running_transaction->t_subtid > subtid)
+ return 1;
+ return 0;
+}

/* Return 1 when transaction with given tid has already committed. */
int jbd2_transaction_committed(journal_t *journal, tid_t tid)
@@ -830,6 +842,33 @@ int jbd2_complete_transaction(journal_t *journal, tid_t tid)
}
EXPORT_SYMBOL(jbd2_complete_transaction);

+int jbd2_fc_complete_commit(journal_t *journal, tid_t tid, tid_t subtid)
+{
+ int need_to_wait = 1;
+
+ read_lock(&journal->j_state_lock);
+ if (journal->j_running_transaction &&
+ journal->j_running_transaction->t_tid == tid) {
+ /* Check if fast commit was already done */
+ if (tid_geq(journal->j_fc_sequence, subtid))
+ need_to_wait = 0;
+ if (journal->j_commit_request != tid) {
+ /* transaction not yet started, so request it */
+ read_unlock(&journal->j_state_lock);
+ jbd2_log_start_commit_fast(journal, tid);
+ goto wait_commit;
+ }
+ } else if (!(journal->j_committing_transaction &&
+ journal->j_committing_transaction->t_tid == tid))
+ need_to_wait = 0;
+ read_unlock(&journal->j_state_lock);
+ if (!need_to_wait)
+ return 0;
+wait_commit:
+ return __jbd2_log_wait_commit(journal, tid, subtid);
+}
+EXPORT_SYMBOL(jbd2_fc_complete_commit);
+
/*
* Log buffer allocation routines:
*/
@@ -850,6 +889,77 @@ int jbd2_journal_next_log_block(journal_t *journal, unsigned long long *retp)
return jbd2_journal_bmap(journal, blocknr, retp);
}

+int jbd2_map_fc_buf(journal_t *journal, struct buffer_head **bh_out)
+{
+ unsigned long long pblock;
+ unsigned long blocknr;
+ int ret = 0;
+ struct buffer_head *bh;
+ int fc_off;
+ journal_header_t *jhdr;
+
+ write_lock(&journal->j_state_lock);
+
+ if (journal->j_fc_off + journal->j_first_fc < journal->j_last_fc) {
+ fc_off = journal->j_fc_off;
+ blocknr = journal->j_first_fc + fc_off;
+ journal->j_fc_off++;
+ } else {
+ ret = -EINVAL;
+ }
+ write_unlock(&journal->j_state_lock);
+
+ if (ret)
+ return ret;
+
+ ret = jbd2_journal_bmap(journal, blocknr, &pblock);
+ if (ret)
+ return ret;
+
+ bh = __getblk(journal->j_dev, pblock, journal->j_blocksize);
+ if (!bh)
+ return -ENOMEM;
+
+ lock_buffer(bh);
+ jhdr = (journal_header_t *)bh->b_data;
+ jhdr->h_magic = cpu_to_be32(JBD2_MAGIC_NUMBER);
+ jhdr->h_blocktype = cpu_to_be32(JBD2_FC_BLOCK);
+ jhdr->h_sequence = cpu_to_be32(journal->j_running_transaction->t_tid);
+
+ set_buffer_uptodate(bh);
+ unlock_buffer(bh);
+ journal->j_fc_wbuf[fc_off] = bh;
+
+ *bh_out = bh;
+
+ return 0;
+}
+EXPORT_SYMBOL(jbd2_map_fc_buf);
+
+int jbd2_wait_on_fc_bufs(journal_t *journal, int num_blks)
+{
+ struct buffer_head *bh;
+ int i, j_fc_off;
+
+ read_lock(&journal->j_state_lock);
+ j_fc_off = journal->j_fc_off;
+ read_unlock(&journal->j_state_lock);
+
+ /*
+ * Wait in reverse order to minimize chances of us being woken up before
+ * all IOs have completed
+ */
+ for (i = j_fc_off - 1; i >= j_fc_off - num_blks; i--) {
+ bh = journal->j_fc_wbuf[i];
+ wait_on_buffer(bh);
+ if (unlikely(!buffer_uptodate(bh)))
+ return -EIO;
+ }
+
+ return 0;
+}
+EXPORT_SYMBOL(jbd2_wait_on_fc_bufs);
+
/*
* Conversion of logical to physical block numbers for the journal
*
diff --git a/include/linux/jbd2.h b/include/linux/jbd2.h
index 41315f648c0f..c6a2b82de4cf 100644
--- a/include/linux/jbd2.h
+++ b/include/linux/jbd2.h
@@ -124,6 +124,7 @@ typedef struct journal_s journal_t; /* Journal control structure */
#define JBD2_SUPERBLOCK_V1 3
#define JBD2_SUPERBLOCK_V2 4
#define JBD2_REVOKE_BLOCK 5
+#define JBD2_FC_BLOCK 6

/*
* Standard header for all descriptor blocks:
@@ -1579,6 +1580,7 @@ int jbd2_transaction_committed(journal_t *journal, tid_t tid);
int jbd2_complete_transaction(journal_t *journal, tid_t tid);
int jbd2_log_do_checkpoint(journal_t *journal);
int jbd2_trans_will_send_data_barrier(journal_t *journal, tid_t tid);
+int jbd2_fc_complete_commit(journal_t *journal, tid_t tid, tid_t subtid);

void __jbd2_log_wait_for_space(journal_t *journal);
extern void __jbd2_journal_drop_transaction(journal_t *, transaction_t *);
@@ -1729,6 +1731,12 @@ static inline tid_t jbd2_get_latest_transaction(journal_t *journal)
return tid;
}

+/* Fast commit related APIs */
+int jbd2_map_fc_buf(journal_t *journal, struct buffer_head **bh_out);
+int jbd2_wait_on_fc_bufs(journal_t *journal, int num_blks);
+int jbd2_submit_inode_data(journal_t *journal, struct jbd2_inode *jinode);
+int jbd2_commit_check(journal_t *journal, tid_t tid, tid_t subtid);
+
#ifdef __KERNEL__

#define buffer_trace_init(bh) do {} while (0)
--
2.23.0.444.g18eeb5a265-goog

2019-10-01 07:45:22

by harshad shirwadkar

[permalink] [raw]
Subject: [PATCH v3 05/13] jbd2: fast-commit recovery path changes

This patch adds fast-commit recovery path changes for JBD2. If we find
a fast commit block that is valid in our recovery phase call file
system specific routine to handle that block.

We also clear the fast commit flag in jbd2_mark_journal_empty() which
is called after successful recovery as well successful
checkpointing. This allows JBD2 journal to be compatible with older
versions when there are no fast commit blocks.

Signed-off-by: Harshad Shirwadkar <[email protected]>
---
fs/jbd2/journal.c | 12 +++++++++
fs/jbd2/recovery.c | 63 +++++++++++++++++++++++++++++++++++++++++---
include/linux/jbd2.h | 13 +++++++++
3 files changed, 84 insertions(+), 4 deletions(-)

diff --git a/fs/jbd2/journal.c b/fs/jbd2/journal.c
index 14d549445418..e0684212384d 100644
--- a/fs/jbd2/journal.c
+++ b/fs/jbd2/journal.c
@@ -1635,6 +1635,7 @@ int jbd2_journal_update_sb_log_tail(journal_t *journal, tid_t tail_tid,
static void jbd2_mark_journal_empty(journal_t *journal, int write_op)
{
journal_superblock_t *sb = journal->j_superblock;
+ bool had_fast_commit = false;

BUG_ON(!mutex_is_locked(&journal->j_checkpoint_mutex));
lock_buffer(journal->j_sb_buffer);
@@ -1648,9 +1649,20 @@ static void jbd2_mark_journal_empty(journal_t *journal, int write_op)

sb->s_sequence = cpu_to_be32(journal->j_tail_sequence);
sb->s_start = cpu_to_be32(0);
+ if (jbd2_has_feature_fast_commit(journal)) {
+ /*
+ * When journal is clean, no need to commit fast commit flag and
+ * make file system incompatible with older kernels.
+ */
+ jbd2_clear_feature_fast_commit(journal);
+ had_fast_commit = true;
+ }

jbd2_write_superblock(journal, write_op);

+ if (had_fast_commit)
+ jbd2_set_feature_fast_commit(journal);
+
/* Log is no longer empty */
write_lock(&journal->j_state_lock);
journal->j_flags |= JBD2_FLUSHED;
diff --git a/fs/jbd2/recovery.c b/fs/jbd2/recovery.c
index a4967b27ffb6..c1f4c94ed375 100644
--- a/fs/jbd2/recovery.c
+++ b/fs/jbd2/recovery.c
@@ -35,7 +35,6 @@ struct recovery_info
int nr_revoke_hits;
};

-enum passtype {PASS_SCAN, PASS_REVOKE, PASS_REPLAY};
static int do_one_pass(journal_t *journal,
struct recovery_info *info, enum passtype pass);
static int scan_revoke_records(journal_t *, struct buffer_head *,
@@ -225,8 +224,12 @@ static int count_tags(journal_t *journal, struct buffer_head *bh)
/* Make sure we wrap around the log correctly! */
#define wrap(journal, var) \
do { \
- if (var >= (journal)->j_last) \
- var -= ((journal)->j_last - (journal)->j_first); \
+ unsigned long _wrap_last = \
+ jbd2_has_feature_fast_commit(journal) ? \
+ (journal)->j_last_fc : (journal)->j_last; \
+ \
+ if (var >= _wrap_last) \
+ var -= (_wrap_last - (journal)->j_first); \
} while (0)

/**
@@ -413,6 +416,51 @@ static int jbd2_block_tag_csum_verify(journal_t *j, journal_block_tag_t *tag,
return tag->t_checksum == cpu_to_be16(csum32);
}

+static int fc_do_one_pass(journal_t *journal,
+ struct recovery_info *info, enum passtype pass)
+{
+ unsigned int expected_commit_id = info->end_transaction;
+ unsigned long next_fc_block;
+ struct buffer_head *bh;
+ unsigned int seq;
+ journal_header_t *jhdr;
+ int err = 0;
+
+ next_fc_block = journal->j_first_fc;
+
+ while (next_fc_block <= journal->j_last_fc) {
+ jbd_debug(3, "Fast commit replay: next block %lld",
+ next_fc_block);
+ err = jread(&bh, journal, next_fc_block);
+ if (err)
+ break;
+
+ jhdr = (journal_header_t *)bh->b_data;
+ seq = be32_to_cpu(jhdr->h_sequence);
+ if (be32_to_cpu(jhdr->h_magic) != JBD2_MAGIC_NUMBER ||
+ seq != expected_commit_id) {
+ break;
+ }
+ jbd_debug(3, "Processing fast commit blk with seq %d",
+ seq);
+ if (journal->j_fc_replay_callback) {
+ err = journal->j_fc_replay_callback(
+ journal, bh, pass,
+ next_fc_block -
+ journal->j_first_fc);
+ if (err)
+ break;
+ }
+ next_fc_block++;
+ }
+
+ if (err)
+ jbd_debug(3, "Fast commit replay failed, err = %d\n", err);
+
+ return err;
+}
+
+
static int do_one_pass(journal_t *journal,
struct recovery_info *info, enum passtype pass)
{
@@ -470,7 +518,7 @@ static int do_one_pass(journal_t *journal,
break;

jbd_debug(2, "Scanning for sequence ID %u at %lu/%lu\n",
- next_commit_ID, next_log_block, journal->j_last);
+ next_commit_ID, next_log_block, journal->j_last_fc);

/* Skip over each chunk of the transaction looking
* either the next descriptor block or the final commit
@@ -768,6 +816,8 @@ static int do_one_pass(journal_t *journal,
if (err)
goto failed;
continue;
+ case JBD2_FC_BLOCK:
+ continue;

default:
jbd_debug(3, "Unrecognised magic %d, end of scan.\n",
@@ -799,6 +849,11 @@ static int do_one_pass(journal_t *journal,
success = -EIO;
}
}
+
+
+ if (jbd2_has_feature_fast_commit(journal) && pass != PASS_REVOKE)
+ fc_do_one_pass(journal, info, pass);
+
if (block_error && success == 0)
success = -EIO;
return success;
diff --git a/include/linux/jbd2.h b/include/linux/jbd2.h
index c6a2b82de4cf..312103fc9581 100644
--- a/include/linux/jbd2.h
+++ b/include/linux/jbd2.h
@@ -762,6 +762,8 @@ jbd2_time_diff(unsigned long start, unsigned long end)

#define JBD2_NR_BATCH 64

+enum passtype {PASS_SCAN, PASS_REVOKE, PASS_REPLAY};
+
/**
* struct journal_s - The journal_s type is the concrete type associated with
* journal_t.
@@ -1243,6 +1245,17 @@ struct journal_s
* after every commit operation.
*/
void (*j_fc_cleanup_callback)(struct journal_s *journal);
+
+ /*
+ * @j_fc_replay_callback:
+ *
+ * File-system specific function that performs replay of a fast
+ * commit. JBD2 calls this function for each fast commit block found in
+ * the journal.
+ */
+ int (*j_fc_replay_callback)(struct journal_s *journal,
+ struct buffer_head *bh,
+ enum passtype pass, int off);
};

#define jbd2_might_wait_for_commit(j) \
--
2.23.0.444.g18eeb5a265-goog

2019-10-01 07:45:22

by harshad shirwadkar

[permalink] [raw]
Subject: [PATCH v3 11/13] ext4: add support for asynchronous fast commits

Until this patch, fast commits could only be invoked by jbd2 thread.
This patch allows file system to perform fast commit in an async manner
without involving jbd2 thread. This makes fast commits even faster as
it gets rid of the time spent in context switching to jbd2 thread. In
order to avoid race between jbd2 thread and async fast commits, we add
new jbd2 APIs that allow file systems to indicate their intent of
performing an async fast commit.

Signed-off-by: Harshad Shirwadkar <[email protected]>
---
fs/ext4/ext4.h | 3 ++
fs/ext4/ext4_jbd2.c | 74 +++++++++++++++++++++++++++++++++++++++++++
fs/ext4/fsync.c | 7 ++--
fs/jbd2/commit.c | 11 +++++++
fs/jbd2/journal.c | 59 ++++++++++++++++++++++++++++++++++
fs/jbd2/transaction.c | 2 ++
include/linux/jbd2.h | 10 ++++++
7 files changed, 164 insertions(+), 2 deletions(-)

diff --git a/fs/ext4/ext4.h b/fs/ext4/ext4.h
index cd5b567d8ca8..a8a481c5ffa4 100644
--- a/fs/ext4/ext4.h
+++ b/fs/ext4/ext4.h
@@ -2716,6 +2716,9 @@ extern int ext4_group_extend(struct super_block *sb,
extern int ext4_resize_fs(struct super_block *sb, ext4_fsblk_t n_blocks_count);

/* super.c */
+int ext4_fc_async_commit(journal_t *journal, tid_t commit_tid,
+ tid_t commit_subtid, struct inode *inode,
+ struct dentry *dentry);
extern struct buffer_head *ext4_sb_bread(struct super_block *sb,
sector_t block, int op_flags);
extern int ext4_seq_options_show(struct seq_file *seq, void *offset);
diff --git a/fs/ext4/ext4_jbd2.c b/fs/ext4/ext4_jbd2.c
index 12d6e70bf676..cf796268322b 100644
--- a/fs/ext4/ext4_jbd2.c
+++ b/fs/ext4/ext4_jbd2.c
@@ -1144,6 +1144,80 @@ static int ext4_journal_fc_replay_cb(journal_t *journal, struct buffer_head *bh,
return ret;
}

+int ext4_fc_async_commit(journal_t *journal, tid_t commit_tid,
+ tid_t commit_subtid, struct inode *inode,
+ struct dentry *dentry)
+{
+ struct ext4_inode_info *ei = EXT4_I(inode);
+ struct super_block *sb = inode->i_sb;
+ struct buffer_head *bh;
+ int ret;
+
+ if (!ext4_should_fast_commit(sb))
+ return jbd2_complete_transaction(journal, commit_tid);
+
+ read_lock(&ei->i_fc.fc_lock);
+ if (ei->i_fc.fc_tid != commit_tid) {
+ read_unlock(&ei->i_fc.fc_lock);
+ return 0;
+ }
+ read_unlock(&ei->i_fc.fc_lock);
+
+ if (ext4_is_inode_fc_ineligible(inode))
+ return jbd2_complete_transaction(journal, commit_tid);
+
+ if (jbd2_commit_check(journal, commit_tid, commit_subtid))
+ return 0;
+
+ ret = jbd2_start_async_fc(journal, commit_tid);
+ if (ret)
+ return jbd2_fc_complete_commit(journal, commit_tid,
+ commit_subtid);
+
+ trace_ext4_journal_fc_commit_cb_start(sb);
+
+ ret = jbd2_submit_inode_data(journal, ei->jinode);
+ if (ret)
+ goto out;
+
+ ret = jbd2_map_fc_buf(journal, &bh);
+ if (ret) {
+ jbd2_stop_async_fc(journal, commit_tid);
+ trace_ext4_journal_fc_commit_cb_stop(sb, 0, "map_fc_buf");
+ return jbd2_complete_transaction(journal, commit_tid);
+
+ }
+
+ ret = ext4_fc_write_inode(journal, bh, inode, commit_tid,
+ commit_subtid, 1, dentry);
+
+ if (ret < 0) {
+ brelse(bh);
+ jbd2_stop_async_fc(journal, commit_tid);
+ trace_ext4_journal_fc_commit_cb_stop(sb, 0, "fc_write_inode");
+ return jbd2_complete_transaction(journal, commit_tid);
+ }
+ lock_buffer(bh);
+ clear_buffer_dirty(bh);
+ set_buffer_uptodate(bh);
+ bh->b_end_io = ext4_end_buffer_io_sync;
+ submit_bh(REQ_OP_WRITE, REQ_SYNC, bh);
+
+ jbd2_stop_async_fc(journal, commit_tid);
+ wait_on_buffer(bh);
+ if (unlikely(!buffer_uptodate(bh))) {
+ trace_ext4_journal_fc_commit_cb_stop(sb, 0, "IO");
+ return -EIO;
+ }
+
+out:
+ trace_ext4_journal_fc_commit_cb_stop(sb,
+ ret < 0 ? 0 : ret,
+ ret >= 0 ? "success" : "fail");
+ wake_up(&journal->j_wait_async_fc);
+ return ret;
+}
+
void ext4_init_fast_commit(struct super_block *sb, journal_t *journal)
{
if (ext4_should_fast_commit(sb)) {
diff --git a/fs/ext4/fsync.c b/fs/ext4/fsync.c
index 5508baa11bb6..5bbfc55e1756 100644
--- a/fs/ext4/fsync.c
+++ b/fs/ext4/fsync.c
@@ -98,7 +98,7 @@ int ext4_sync_file(struct file *file, loff_t start, loff_t end, int datasync)
struct ext4_inode_info *ei = EXT4_I(inode);
journal_t *journal = EXT4_SB(inode->i_sb)->s_journal;
int ret = 0, err;
- tid_t commit_tid;
+ tid_t commit_tid, commit_subtid;
bool needs_barrier = false;

if (unlikely(ext4_forced_shutdown(EXT4_SB(inode->i_sb))))
@@ -148,10 +148,13 @@ int ext4_sync_file(struct file *file, loff_t start, loff_t end, int datasync)
}

commit_tid = datasync ? ei->i_datasync_tid : ei->i_sync_tid;
+ commit_subtid = datasync ? ei->i_datasync_subtid : ei->i_sync_subtid;
+
if (journal->j_flags & JBD2_BARRIER &&
!jbd2_trans_will_send_data_barrier(journal, commit_tid))
needs_barrier = true;
- ret = jbd2_complete_transaction(journal, commit_tid);
+ ret = ext4_fc_async_commit(journal, commit_tid, commit_subtid,
+ inode, file->f_path.dentry);
if (needs_barrier) {
issue_flush:
err = blkdev_issue_flush(inode->i_sb->s_bdev, GFP_KERNEL, NULL);
diff --git a/fs/jbd2/commit.c b/fs/jbd2/commit.c
index e85f51e1cc70..18cb70fa2421 100644
--- a/fs/jbd2/commit.c
+++ b/fs/jbd2/commit.c
@@ -452,6 +452,17 @@ void jbd2_journal_commit_transaction(journal_t *journal, bool *fc)

write_lock(&journal->j_state_lock);
full_commit = journal->j_do_full_commit;
+ journal->j_running_transaction->t_async_fc_allowed = false;
+ while (journal->j_running_transaction->t_async_fc_ongoing) {
+ DEFINE_WAIT(wait);
+
+ prepare_to_wait(&journal->j_wait_async_fc, &wait,
+ TASK_UNINTERRUPTIBLE);
+ write_unlock(&journal->j_state_lock);
+ schedule();
+ write_lock(&journal->j_state_lock);
+ finish_wait(&journal->j_wait_async_fc, &wait);
+ }
write_unlock(&journal->j_state_lock);

/* Let file-system try its own fast commit */
diff --git a/fs/jbd2/journal.c b/fs/jbd2/journal.c
index e0684212384d..81daa2cff67f 100644
--- a/fs/jbd2/journal.c
+++ b/fs/jbd2/journal.c
@@ -794,6 +794,64 @@ int jbd2_commit_check(journal_t *journal, tid_t tid, tid_t subtid)
return 0;
}

+int jbd2_start_async_fc(journal_t *journal, tid_t tid)
+{
+ transaction_t *txn;
+ int ret = -EINVAL;
+
+ if (!journal->j_running_transaction)
+ return ret;
+
+ if (journal->j_running_transaction->t_tid != tid)
+ return ret;
+
+ txn = journal->j_running_transaction;
+ write_lock(&journal->j_state_lock);
+ while (txn->t_state == T_RUNNING) {
+ DEFINE_WAIT(wait);
+
+ if (txn->t_async_fc_allowed) {
+ if (!txn->t_async_fc_ongoing) {
+ txn->t_async_fc_ongoing = true;
+ ret = 0;
+ break;
+ }
+ prepare_to_wait(&journal->j_wait_async_fc,
+ &wait, TASK_UNINTERRUPTIBLE);
+ write_unlock(&journal->j_state_lock);
+ schedule();
+ write_lock(&journal->j_state_lock);
+ finish_wait(&journal->j_wait_async_fc, &wait);
+ } else {
+ ret = -ECANCELED;
+ break;
+ }
+ }
+ write_unlock(&journal->j_state_lock);
+
+ return ret;
+}
+
+int jbd2_stop_async_fc(journal_t *journal, tid_t tid)
+{
+ transaction_t *txn;
+
+ if (!journal->j_running_transaction)
+ return -EINVAL;
+
+ if (journal->j_running_transaction->t_tid != tid)
+ return -EINVAL;
+
+ txn = journal->j_running_transaction;
+ write_lock(&journal->j_state_lock);
+ J_ASSERT(txn->t_state == T_RUNNING);
+ txn->t_async_fc_ongoing = false;
+ txn->t_subtid++;
+ write_unlock(&journal->j_state_lock);
+ return 0;
+
+}
+
/* Return 1 when transaction with given tid has already committed. */
int jbd2_transaction_committed(journal_t *journal, tid_t tid)
{
@@ -1308,6 +1366,7 @@ static journal_t *journal_init_common(struct block_device *bdev,
init_waitqueue_head(&journal->j_wait_commit);
init_waitqueue_head(&journal->j_wait_updates);
init_waitqueue_head(&journal->j_wait_reserved);
+ init_waitqueue_head(&journal->j_wait_async_fc);
mutex_init(&journal->j_barrier);
mutex_init(&journal->j_checkpoint_mutex);
spin_lock_init(&journal->j_revoke_lock);
diff --git a/fs/jbd2/transaction.c b/fs/jbd2/transaction.c
index ce7f03cfd90b..f17f813b5610 100644
--- a/fs/jbd2/transaction.c
+++ b/fs/jbd2/transaction.c
@@ -103,6 +103,8 @@ static void jbd2_get_transaction(journal_t *journal,
transaction->t_max_wait = 0;
transaction->t_start = jiffies;
transaction->t_requested = 0;
+ transaction->t_async_fc_allowed = true;
+ transaction->t_async_fc_ongoing = false;
}

/*
diff --git a/include/linux/jbd2.h b/include/linux/jbd2.h
index 312103fc9581..5610f16de919 100644
--- a/include/linux/jbd2.h
+++ b/include/linux/jbd2.h
@@ -604,6 +604,7 @@ struct transaction_s
T_FINISHED
} t_state;

+ bool t_async_fc_allowed, t_async_fc_ongoing;
/*
* Where in the log does this transaction's commit start? [no locking]
*/
@@ -869,6 +870,13 @@ struct journal_s
*/
wait_queue_head_t j_wait_reserved;

+ /**
+ * @j_wait_async_fc:
+ *
+ * Wait queue to wait for completion of async fast commits.
+ */
+ wait_queue_head_t j_wait_async_fc;
+
/**
* @j_checkpoint_mutex:
*
@@ -1594,6 +1602,8 @@ int jbd2_complete_transaction(journal_t *journal, tid_t tid);
int jbd2_log_do_checkpoint(journal_t *journal);
int jbd2_trans_will_send_data_barrier(journal_t *journal, tid_t tid);
int jbd2_fc_complete_commit(journal_t *journal, tid_t tid, tid_t subtid);
+int jbd2_start_async_fc(journal_t *journal, tid_t tid);
+int jbd2_stop_async_fc(journal_t *journal, tid_t tid);

void __jbd2_log_wait_for_space(journal_t *journal);
extern void __jbd2_journal_drop_transaction(journal_t *, transaction_t *);
--
2.23.0.444.g18eeb5a265-goog

2019-10-04 19:14:26

by Theodore Ts'o

[permalink] [raw]
Subject: Re: [PATCH v3 00/13] ext4: add fast commit support

On Tue, Oct 01, 2019 at 12:40:49AM -0700, Harshad Shirwadkar wrote:
>
> Testing
> -------
>
> e2fsprogs was updated to set fast commit feature flag and to ignore
> fast commit blocks during e2fsck.
>
> https://github.com/harshadjs/e2fsprogs.git
>
> After applying all the patches in this series, following runs of
> xfstests were performed:
>
> - kvm-xfstest.sh -g log -c 4k
> - kvm-xfstests.sh smoke
>
> All the log tests were successful and smoke tests didn't introduce any
> additional failures.

You should probably also try running the shutdown tests, and
eventually, run all of the auto group. I've added a fast_commit group
to {kvm,gce}-xfstests, although to use it a modified e2fsprogs which
understands the fast_commit feature. I can make kvm-xfstests and
gce-xfstests image using an e2fsprogs package from debian/experimental
which has fast_commit enabled.

When I tried running all of the auto group tests, the following
failure was found in generic/047 (which is a shutdown group test).

- Ted

BEGIN TEST fast_commit (1 test): Ext4 4k block w/fast_commit Fri Oct 4 13:44:45 EDT 2019
DEVICE: /dev/vdd
EXT_MKFS_OPTIONS: -I 256 -O fast_commit,64bit
EXT_MOUNT_OPTIONS: -o block_validity
FSTYP -- ext4
PLATFORM -- Linux/x86_64 kvm-xfstests 5.3.0-rc4-xfstests-00012-gedca88337ca9 #1202 SMP Thu Oct 3 17:27:50 EDT 2019
MKFS_OPTIONS -- -q -I 256 -O fast_commit,64bit /dev/vdc
MOUNT_OPTIONS -- -o acl,user_xattr -o block_validity /dev/vdc /vdc

generic/047 [13:44:46][ 24.671344] run fstests generic/047 at 2019-10-04 13:44:46
[ 24.951140] EXT4-fs (vdc): shut down requested (1)
[ 24.952280] Aborting journal on device vdc-8.
[ 28.012724] EXT4-fs (vdc): shut down requested (2)
[ 28.013639] Aborting journal on device vdc-8.
[ 28.014486]
[ 28.014845] ============================================
[ 28.015996] WARNING: possible recursive locking detected
[ 28.017072] 5.3.0-rc4-xfstests-00012-gedca88337ca9 #1202 Not tainted
[ 28.018374] --------------------------------------------
[ 28.019693] jbd2/vdc-8/1476 is trying to acquire lock:
[ 28.020635] 000000005ce13aef (&(&sbi->s_fc_lock)->rlock){+.+.}, at: ext4_journal_fc_cleanup_cb+0x2f/0xa0
[ 28.022387]
[ 28.022387] but task is already holding lock:
[ 28.023414] 000000005ce13aef (&(&sbi->s_fc_lock)->rlock){+.+.}, at: ext4_journal_fc_commit_cb+0x83/0xa90
[ 28.025237]
[ 28.025237] other info that might help us debug this:
[ 28.026350] Possible unsafe locking scenario:
[ 28.026350]
[ 28.027336] CPU0
[ 28.027758] ----
[ 28.028240] lock(&(&sbi->s_fc_lock)->rlock);
[ 28.029105] lock(&(&sbi->s_fc_lock)->rlock);
[ 28.029937]
[ 28.029937] *** DEADLOCK ***
[ 28.029937]
[ 28.031154] May be due to missing lock nesting notation
[ 28.031154]
[ 28.032780] 1 lock held by jbd2/vdc-8/1476:
[ 28.033760] #0: 000000005ce13aef (&(&sbi->s_fc_lock)->rlock){+.+.}, at: ext4_journal_fc_commit_cb+0x83/0xa90
[ 28.035436]
[ 28.035436] stack backtrace:
[ 28.036197] CPU: 1 PID: 1476 Comm: jbd2/vdc-8 Not tainted 5.3.0-rc4-xfstests-00012-gedca88337ca9 #1202
[ 28.037868] Hardware name: QEMU Standard PC (i440FX + PIIX, 1996), BIOS 1.12.0-1 04/01/2014
[ 28.039289] Call Trace:
[ 28.039772] dump_stack+0x67/0x90
[ 28.040427] validate_chain.cold+0x1be/0x21b
[ 28.041305] __lock_acquire+0x447/0x7c0
[ 28.042069] lock_acquire+0x9a/0x180
[ 28.042738] ? ext4_journal_fc_cleanup_cb+0x2f/0xa0
[ 28.043663] _raw_spin_lock+0x31/0x80
[ 28.044346] ? ext4_journal_fc_cleanup_cb+0x2f/0xa0
[ 28.045264] ext4_journal_fc_cleanup_cb+0x2f/0xa0
[ 28.046154] jbd2_journal_commit_transaction+0x243/0x24bb
[ 28.047156] ? sched_clock_cpu+0xc/0xc0
[ 28.048099] ? lock_timer_base+0x10/0x80
[ 28.048935] ? kvm_sched_clock_read+0x14/0x30
[ 28.050022] ? sched_clock+0x5/0x10
[ 28.050853] ? sched_clock_cpu+0xc/0xc0
[ 28.051793] ? kjournald2+0x143/0x3f0
[ 28.052606] kjournald2+0x143/0x3f0
[ 28.053311] ? __wake_up_common_lock+0xc0/0xc0
[ 28.054935] kthread+0x108/0x140
[ 28.055975] ? __jbd2_debug+0x50/0x50
[ 28.057105] ? __kthread_create_on_node+0x1a0/0x1a0
[ 28.058346] ret_from_fork+0x3a/0x50

2019-10-04 20:16:36

by harshad shirwadkar

[permalink] [raw]
Subject: Re: [PATCH v3 00/13] ext4: add fast commit support

Thanks for that, I fixed this deadlock, I'll run all the tests that
you mentioned.

On Fri, Oct 4, 2019 at 12:12 PM Theodore Y. Ts'o <[email protected]> wrote:
>
> On Tue, Oct 01, 2019 at 12:40:49AM -0700, Harshad Shirwadkar wrote:
> >
> > Testing
> > -------
> >
> > e2fsprogs was updated to set fast commit feature flag and to ignore
> > fast commit blocks during e2fsck.
> >
> > https://github.com/harshadjs/e2fsprogs.git
> >
> > After applying all the patches in this series, following runs of
> > xfstests were performed:
> >
> > - kvm-xfstest.sh -g log -c 4k
> > - kvm-xfstests.sh smoke
> >
> > All the log tests were successful and smoke tests didn't introduce any
> > additional failures.
>
> You should probably also try running the shutdown tests, and
> eventually, run all of the auto group. I've added a fast_commit group
> to {kvm,gce}-xfstests, although to use it a modified e2fsprogs which
> understands the fast_commit feature. I can make kvm-xfstests and
> gce-xfstests image using an e2fsprogs package from debian/experimental
> which has fast_commit enabled.
>
> When I tried running all of the auto group tests, the following
> failure was found in generic/047 (which is a shutdown group test).
>
> - Ted
>
> BEGIN TEST fast_commit (1 test): Ext4 4k block w/fast_commit Fri Oct 4 13:44:45 EDT 2019
> DEVICE: /dev/vdd
> EXT_MKFS_OPTIONS: -I 256 -O fast_commit,64bit
> EXT_MOUNT_OPTIONS: -o block_validity
> FSTYP -- ext4
> PLATFORM -- Linux/x86_64 kvm-xfstests 5.3.0-rc4-xfstests-00012-gedca88337ca9 #1202 SMP Thu Oct 3 17:27:50 EDT 2019
> MKFS_OPTIONS -- -q -I 256 -O fast_commit,64bit /dev/vdc
> MOUNT_OPTIONS -- -o acl,user_xattr -o block_validity /dev/vdc /vdc
>
> generic/047 [13:44:46][ 24.671344] run fstests generic/047 at 2019-10-04 13:44:46
> [ 24.951140] EXT4-fs (vdc): shut down requested (1)
> [ 24.952280] Aborting journal on device vdc-8.
> [ 28.012724] EXT4-fs (vdc): shut down requested (2)
> [ 28.013639] Aborting journal on device vdc-8.
> [ 28.014486]
> [ 28.014845] ============================================
> [ 28.015996] WARNING: possible recursive locking detected
> [ 28.017072] 5.3.0-rc4-xfstests-00012-gedca88337ca9 #1202 Not tainted
> [ 28.018374] --------------------------------------------
> [ 28.019693] jbd2/vdc-8/1476 is trying to acquire lock:
> [ 28.020635] 000000005ce13aef (&(&sbi->s_fc_lock)->rlock){+.+.}, at: ext4_journal_fc_cleanup_cb+0x2f/0xa0
> [ 28.022387]
> [ 28.022387] but task is already holding lock:
> [ 28.023414] 000000005ce13aef (&(&sbi->s_fc_lock)->rlock){+.+.}, at: ext4_journal_fc_commit_cb+0x83/0xa90
> [ 28.025237]
> [ 28.025237] other info that might help us debug this:
> [ 28.026350] Possible unsafe locking scenario:
> [ 28.026350]
> [ 28.027336] CPU0
> [ 28.027758] ----
> [ 28.028240] lock(&(&sbi->s_fc_lock)->rlock);
> [ 28.029105] lock(&(&sbi->s_fc_lock)->rlock);
> [ 28.029937]
> [ 28.029937] *** DEADLOCK ***
> [ 28.029937]
> [ 28.031154] May be due to missing lock nesting notation
> [ 28.031154]
> [ 28.032780] 1 lock held by jbd2/vdc-8/1476:
> [ 28.033760] #0: 000000005ce13aef (&(&sbi->s_fc_lock)->rlock){+.+.}, at: ext4_journal_fc_commit_cb+0x83/0xa90
> [ 28.035436]
> [ 28.035436] stack backtrace:
> [ 28.036197] CPU: 1 PID: 1476 Comm: jbd2/vdc-8 Not tainted 5.3.0-rc4-xfstests-00012-gedca88337ca9 #1202
> [ 28.037868] Hardware name: QEMU Standard PC (i440FX + PIIX, 1996), BIOS 1.12.0-1 04/01/2014
> [ 28.039289] Call Trace:
> [ 28.039772] dump_stack+0x67/0x90
> [ 28.040427] validate_chain.cold+0x1be/0x21b
> [ 28.041305] __lock_acquire+0x447/0x7c0
> [ 28.042069] lock_acquire+0x9a/0x180
> [ 28.042738] ? ext4_journal_fc_cleanup_cb+0x2f/0xa0
> [ 28.043663] _raw_spin_lock+0x31/0x80
> [ 28.044346] ? ext4_journal_fc_cleanup_cb+0x2f/0xa0
> [ 28.045264] ext4_journal_fc_cleanup_cb+0x2f/0xa0
> [ 28.046154] jbd2_journal_commit_transaction+0x243/0x24bb
> [ 28.047156] ? sched_clock_cpu+0xc/0xc0
> [ 28.048099] ? lock_timer_base+0x10/0x80
> [ 28.048935] ? kvm_sched_clock_read+0x14/0x30
> [ 28.050022] ? sched_clock+0x5/0x10
> [ 28.050853] ? sched_clock_cpu+0xc/0xc0
> [ 28.051793] ? kjournald2+0x143/0x3f0
> [ 28.052606] kjournald2+0x143/0x3f0
> [ 28.053311] ? __wake_up_common_lock+0xc0/0xc0
> [ 28.054935] kthread+0x108/0x140
> [ 28.055975] ? __jbd2_debug+0x50/0x50
> [ 28.057105] ? __kthread_create_on_node+0x1a0/0x1a0
> [ 28.058346] ret_from_fork+0x3a/0x50

2019-10-17 12:48:41

by Theodore Ts'o

[permalink] [raw]
Subject: Re: [PATCH v3 04/13] jbd2: fast-commit commit path new APIs

On Tue, Oct 01, 2019 at 12:40:53AM -0700, Harshad Shirwadkar wrote:
> This patch adds new helper APIs that ext4 needs for fast
> commits. These new fast commit APIs are used by subsequent fast commit
> patches to implement fast commits. Following new APIs are added:
>
> /*
> * Returns when either a full commit or a fast commit
> * completes
> */
> int jbd2_fc_complete_commit(journal_tc *journal, tid_t tid,
> tid_t subtid)
>
> /* Send all the data buffers related to an inode */
> int journal_submit_inode_data(journal_t *journal,
> struct jbd2_inode *jinode)
>
> /* Map one fast commit buffer for use by the file system */
> int jbd2_map_fc_buf(journal_t *journal, struct buffer_head **bh_out)
>
> /* Wait on fast commit buffers to complete IO */
> jbd2_wait_on_fc_bufs(journal_t *journal, int num_bufs)
>
> /*
> * Returns 1 if transaction identified by tid:subtid is already
> * committed.
> */
> int jbd2_commit_check(journal_t *journal, tid_t tid, tid_t subtid)

Please move these commits into the code, before each function. This
documentation is going to be useful long after the patch gets merged,
and people will be looking for them in the source code, and not
necessarily in the commit description.

>
> diff --git a/fs/jbd2/commit.c b/fs/jbd2/commit.c
> index 7db3e2b6336d..e85f51e1cc70 100644
> --- a/fs/jbd2/commit.c
> +++ b/fs/jbd2/commit.c
> @@ -202,6 +202,38 @@ static int journal_submit_inode_data_buffers(struct address_space *mapping,
> return ret;
> }
>
> +int jbd2_submit_inode_data(journal_t *journal, struct jbd2_inode *jinode)

This code was pulled out of journal_submit_data_buffers(), but given
how it was called, there were locking assumptions that were broken as
a result.

> +{
> + struct address_space *mapping;
> + loff_t dirty_start = jinode->i_dirty_start;
> + loff_t dirty_end = jinode->i_dirty_end;
> + int ret;
> +
> + if (!jinode)
> + return 0;
> +
> + if (!(jinode->i_flags & JI_WRITE_DATA))
> + return 0;

Originally in journal_submit_data_buffers() we were holding onto
j_list_lock, and that's needed to safely reference jinode->i_flags

> +
> + dirty_start = jinode->i_dirty_start;
> + dirty_end = jinode->i_dirty_end;
> +
> + mapping = jinode->i_vfs_inode->i_mapping;
> + jinode->i_flags |= JI_COMMIT_RUNNING;

Originally there was a spin_uinlock(&journal->j_list_lock) here. And
that's important since there was a memory barrier there which we
needed in order to make sure other CPU's would see the
JI_COMMIT_RUNNING flag.

It's not clear we need to worry about this, if this is only going to
be used in the async fast commit context. This is another example of
how trying to do the fast commit in the userspace (or nfs server's)
process context is much simpler, since the the JI_COMMIT_RUNNING flag
is needed to make sure there isn't a race with the inode getting
evicted and jbd2_journal_release_jbd_inode.

And if we're calling this function from ext4_jbd2.c, where the inode's
ref count is elevated and there is no risk of the inode getting
evicted from memory, then this particular race is not a problem, and
so messing with JI_COMMIT_RUNNING and the call to wake_up_bit is all
not necessary.

By the way, this function only submits the data to be written out. It
does not wait for the writeout to be completed. For that, you need
the equivalent of journal_finish_inode_data_buffers(), and I don't see
that equivalent functionality in the fast commit code path?

- Ted

2019-10-17 12:48:54

by Theodore Ts'o

[permalink] [raw]
Subject: Re: [PATCH v3 05/13] jbd2: fast-commit recovery path changes

On Tue, Oct 01, 2019 at 12:40:54AM -0700, Harshad Shirwadkar wrote:
> diff --git a/fs/jbd2/journal.c b/fs/jbd2/journal.c
> index 14d549445418..e0684212384d 100644
> --- a/fs/jbd2/journal.c
> +++ b/fs/jbd2/journal.c
>
> jbd2_write_superblock(journal, write_op);
>
> + if (had_fast_commit)
> + jbd2_set_feature_fast_commit(journal);
> +

Why the logic with had_fast_commit and (re-)setting the fast commit
feature flag?

This ties back to how we handle the logic around setting the fast
commit flag if requested by the file system....

> @@ -768,6 +816,8 @@ static int do_one_pass(journal_t *journal,
> if (err)
> goto failed;
> continue;
> + case JBD2_FC_BLOCK:
> + continue;

Why should a Fast Commit block ever show up in the primary part of the
journal? It should never happen, right?

In which case, we should probably at least issue a warning, and not
just skip the block.

- Ted

2019-10-17 14:04:47

by Theodore Ts'o

[permalink] [raw]
Subject: Re: [PATCH v3 08/13] ext4: fast-commit commit range tracking

On Tue, Oct 01, 2019 at 12:40:57AM -0700, Harshad Shirwadkar wrote:
> With this patch, we track logical range of file offsets that need to
> be committed using fast commit. This allows us to find file extents
> that need to be committed during the commit time.

We don't actually need to track when data is modified in the page
cache, which is what this commit is actually doing. We only need to
track newly allocated blocks, at granularity of the logical block
number.

That's because we only need to force out newly allocated blocks to
make sure we don't reveal stale data when we are in data=ordered mode.
And it also follows that we don't need to track logical block ranges
and submit inode data in data=writeback or data=journalled mode.

In the case where the user has actually called fsync() on the the
inode, we do a data integrity writeback in ext4_sync_file, and that's
independent on the fast commit code.

But if the file is being modified using buffered writes, or if an
already allocated block is changed, and the file has *not* been
changed, we don't need to write out those blocks on a fast commit.
For example, in the case where we are the fast commit is being
initiated via ext4_nfs_commit_metadata() -> ext4_write_inode(), we
only care about submitting data for the newly allocated blocks. And
that's what we want to track here.

Hence, all of the callers of ext4_fc_update_commit_range() here are in
the wrong place. (Also, they are calling ext4_fc_update_commit_range
with byte offsets, when the function is expecting logical block
numbers, but that really matter, since the existing call sites need to
be all removed and replaced with new ones in ext4_map_blocks().

- Ted

2019-10-22 00:52:12

by harshad shirwadkar

[permalink] [raw]
Subject: Re: [PATCH v3 05/13] jbd2: fast-commit recovery path changes

On Wed, Oct 16, 2019 at 10:30 AM Theodore Y. Ts'o <[email protected]> wrote:
>
> On Tue, Oct 01, 2019 at 12:40:54AM -0700, Harshad Shirwadkar wrote:
> > diff --git a/fs/jbd2/journal.c b/fs/jbd2/journal.c
> > index 14d549445418..e0684212384d 100644
> > --- a/fs/jbd2/journal.c
> > +++ b/fs/jbd2/journal.c
> >
> > jbd2_write_superblock(journal, write_op);
> >
> > + if (had_fast_commit)
> > + jbd2_set_feature_fast_commit(journal);
> > +
>
> Why the logic with had_fast_commit and (re-)setting the fast commit
> feature flag?
>
> This ties back to how we handle the logic around setting the fast
> commit flag if requested by the file system....

Fast commit feature flag serves 2 purposes: 1) If the flag is turned
on in on-disk superblock, it means that the superblock contains fast
commit blocks that should be replayed. 2) If the flag is turned on in
the in-memory representation of the superblock, it serves as an
indicator for the rest of the JBD2 code that fast commit feature is
enabled. Based on that flag, for example, the journal thread decides
to try fast commits. In this particular case, since the journal is
empty we don't want to commit fast commit feature flag on-disk but we
want to retain that flag in in-memory structure.

>
> > @@ -768,6 +816,8 @@ static int do_one_pass(journal_t *journal,
> > if (err)
> > goto failed;
> > continue;
> > + case JBD2_FC_BLOCK:
> > + continue;
>
> Why should a Fast Commit block ever show up in the primary part of the
> journal? It should never happen, right?
That's right, I'll fix this in next version.
>
> In which case, we should probably at least issue a warning, and not
> just skip the block.
>
> - Ted

2019-10-25 19:24:39

by Xiaoguang Wang

[permalink] [raw]
Subject: Re: [PATCH v3 11/13] ext4: add support for asynchronous fast commits

hi,

> Until this patch, fast commits could only be invoked by jbd2 thread.
> This patch allows file system to perform fast commit in an async manner
> without involving jbd2 thread. This makes fast commits even faster as
> it gets rid of the time spent in context switching to jbd2 thread. In
> order to avoid race between jbd2 thread and async fast commits, we add
> new jbd2 APIs that allow file systems to indicate their intent of
> performing an async fast commit.
>
> Signed-off-by: Harshad Shirwadkar <[email protected]>
> ---
> fs/ext4/ext4.h | 3 ++
> fs/ext4/ext4_jbd2.c | 74 +++++++++++++++++++++++++++++++++++++++++++
> fs/ext4/fsync.c | 7 ++--
> fs/jbd2/commit.c | 11 +++++++
> fs/jbd2/journal.c | 59 ++++++++++++++++++++++++++++++++++
> fs/jbd2/transaction.c | 2 ++
> include/linux/jbd2.h | 10 ++++++
> 7 files changed, 164 insertions(+), 2 deletions(-)
>
> diff --git a/fs/ext4/ext4.h b/fs/ext4/ext4.h
> index cd5b567d8ca8..a8a481c5ffa4 100644
> --- a/fs/ext4/ext4.h
> +++ b/fs/ext4/ext4.h
> @@ -2716,6 +2716,9 @@ extern int ext4_group_extend(struct super_block *sb,
> extern int ext4_resize_fs(struct super_block *sb, ext4_fsblk_t n_blocks_count);
>
> /* super.c */
> +int ext4_fc_async_commit(journal_t *journal, tid_t commit_tid,
> + tid_t commit_subtid, struct inode *inode,
> + struct dentry *dentry);
> extern struct buffer_head *ext4_sb_bread(struct super_block *sb,
> sector_t block, int op_flags);
> extern int ext4_seq_options_show(struct seq_file *seq, void *offset);
> diff --git a/fs/ext4/ext4_jbd2.c b/fs/ext4/ext4_jbd2.c
> index 12d6e70bf676..cf796268322b 100644
> --- a/fs/ext4/ext4_jbd2.c
> +++ b/fs/ext4/ext4_jbd2.c
> @@ -1144,6 +1144,80 @@ static int ext4_journal_fc_replay_cb(journal_t *journal, struct buffer_head *bh,
> return ret;
> }
>
> +int ext4_fc_async_commit(journal_t *journal, tid_t commit_tid,
> + tid_t commit_subtid, struct inode *inode,
> + struct dentry *dentry)
> +{
> + struct ext4_inode_info *ei = EXT4_I(inode);
> + struct super_block *sb = inode->i_sb;
> + struct buffer_head *bh;
> + int ret;
> +
> + if (!ext4_should_fast_commit(sb))
> + return jbd2_complete_transaction(journal, commit_tid);
> +
> + read_lock(&ei->i_fc.fc_lock);
> + if (ei->i_fc.fc_tid != commit_tid) {
> + read_unlock(&ei->i_fc.fc_lock);
> + return 0;
> + }
> + read_unlock(&ei->i_fc.fc_lock);
> +
> + if (ext4_is_inode_fc_ineligible(inode))
> + return jbd2_complete_transaction(journal, commit_tid);
> +
> + if (jbd2_commit_check(journal, commit_tid, commit_subtid))
> + return 0;
> +
> + ret = jbd2_start_async_fc(journal, commit_tid);
> + if (ret)
> + return jbd2_fc_complete_commit(journal, commit_tid,
> + commit_subtid);
> +
> + trace_ext4_journal_fc_commit_cb_start(sb);
> +
> + ret = jbd2_submit_inode_data(journal, ei->jinode);
> + if (ret)
> + goto out;
> +
> + ret = jbd2_map_fc_buf(journal, &bh);
> + if (ret) {
> + jbd2_stop_async_fc(journal, commit_tid);
> + trace_ext4_journal_fc_commit_cb_stop(sb, 0, "map_fc_buf");
> + return jbd2_complete_transaction(journal, commit_tid);
> +
> + }
> +
> + ret = ext4_fc_write_inode(journal, bh, inode, commit_tid,
> + commit_subtid, 1, dentry);
> +
> + if (ret < 0) {
> + brelse(bh);
> + jbd2_stop_async_fc(journal, commit_tid);
> + trace_ext4_journal_fc_commit_cb_stop(sb, 0, "fc_write_inode");
> + return jbd2_complete_transaction(journal, commit_tid);
> + }
> + lock_buffer(bh);
> + clear_buffer_dirty(bh);
> + set_buffer_uptodate(bh);
> + bh->b_end_io = ext4_end_buffer_io_sync;
> + submit_bh(REQ_OP_WRITE, REQ_SYNC, bh);
> +
> + jbd2_stop_async_fc(journal, commit_tid);
> + wait_on_buffer(bh);
> + if (unlikely(!buffer_uptodate(bh))) {
> + trace_ext4_journal_fc_commit_cb_stop(sb, 0, "IO");
> + return -EIO;
> + }
> +
> +out:
> + trace_ext4_journal_fc_commit_cb_stop(sb,
> + ret < 0 ? 0 : ret,
> + ret >= 0 ? "success" : "fail");
> + wake_up(&journal->j_wait_async_fc);
> + return ret;
> +}
> +
> void ext4_init_fast_commit(struct super_block *sb, journal_t *journal)
> {
> if (ext4_should_fast_commit(sb)) {
> diff --git a/fs/ext4/fsync.c b/fs/ext4/fsync.c
> index 5508baa11bb6..5bbfc55e1756 100644
> --- a/fs/ext4/fsync.c
> +++ b/fs/ext4/fsync.c
> @@ -98,7 +98,7 @@ int ext4_sync_file(struct file *file, loff_t start, loff_t end, int datasync)
> struct ext4_inode_info *ei = EXT4_I(inode);
> journal_t *journal = EXT4_SB(inode->i_sb)->s_journal;
> int ret = 0, err;
> - tid_t commit_tid;
> + tid_t commit_tid, commit_subtid;
> bool needs_barrier = false;
>
> if (unlikely(ext4_forced_shutdown(EXT4_SB(inode->i_sb))))
> @@ -148,10 +148,13 @@ int ext4_sync_file(struct file *file, loff_t start, loff_t end, int datasync)
> }
>
> commit_tid = datasync ? ei->i_datasync_tid : ei->i_sync_tid;
> + commit_subtid = datasync ? ei->i_datasync_subtid : ei->i_sync_subtid;
> +
> if (journal->j_flags & JBD2_BARRIER &&
> !jbd2_trans_will_send_data_barrier(journal, commit_tid))
> needs_barrier = true;
> - ret = jbd2_complete_transaction(journal, commit_tid);
> + ret = ext4_fc_async_commit(journal, commit_tid, commit_subtid,
> + inode, file->f_path.dentry);
> if (needs_barrier) {
> issue_flush:
> err = blkdev_issue_flush(inode->i_sb->s_bdev, GFP_KERNEL, NULL);
> diff --git a/fs/jbd2/commit.c b/fs/jbd2/commit.c
> index e85f51e1cc70..18cb70fa2421 100644
> --- a/fs/jbd2/commit.c
> +++ b/fs/jbd2/commit.c
> @@ -452,6 +452,17 @@ void jbd2_journal_commit_transaction(journal_t *journal, bool *fc)
>
> write_lock(&journal->j_state_lock);
> full_commit = journal->j_do_full_commit;
> + journal->j_running_transaction->t_async_fc_allowed = false;
> + while (journal->j_running_transaction->t_async_fc_ongoing) {
> + DEFINE_WAIT(wait);
> +
> + prepare_to_wait(&journal->j_wait_async_fc, &wait,
> + TASK_UNINTERRUPTIBLE);
> + write_unlock(&journal->j_state_lock);
> + schedule();
> + write_lock(&journal->j_state_lock);
> + finish_wait(&journal->j_wait_async_fc, &wait);
> + }
> write_unlock(&journal->j_state_lock);
>
> /* Let file-system try its own fast commit */
> diff --git a/fs/jbd2/journal.c b/fs/jbd2/journal.c
> index e0684212384d..81daa2cff67f 100644
> --- a/fs/jbd2/journal.c
> +++ b/fs/jbd2/journal.c
> @@ -794,6 +794,64 @@ int jbd2_commit_check(journal_t *journal, tid_t tid, tid_t subtid)
> return 0;
> }
>
> +int jbd2_start_async_fc(journal_t *journal, tid_t tid)
> +{
> + transaction_t *txn;
> + int ret = -EINVAL;
> +
> + if (!journal->j_running_transaction)
> + return ret;
> +
> + if (journal->j_running_transaction->t_tid != tid)
> + return ret;
> +
> + txn = journal->j_running_transaction;
> + write_lock(&journal->j_state_lock);
> + while (txn->t_state == T_RUNNING) {
> + DEFINE_WAIT(wait);
> +
> + if (txn->t_async_fc_allowed) {
> + if (!txn->t_async_fc_ongoing) {
> + txn->t_async_fc_ongoing = true;
> + ret = 0;
> + break;
> + }
> + prepare_to_wait(&journal->j_wait_async_fc,
> + &wait, TASK_UNINTERRUPTIBLE);
> + write_unlock(&journal->j_state_lock);
> + schedule();
> + write_lock(&journal->j_state_lock);
> + finish_wait(&journal->j_wait_async_fc, &wait);
It seems that above code logic will prevent concurrent fsync operations using fast
commit feature?

Regards,
Xiaoguang Wang

> + } else {
> + ret = -ECANCELED;
> + break;
> + }
> + }
> + write_unlock(&journal->j_state_lock);
> +
> + return ret;
> +}
> +
> +int jbd2_stop_async_fc(journal_t *journal, tid_t tid)
> +{
> + transaction_t *txn;
> +
> + if (!journal->j_running_transaction)
> + return -EINVAL;
> +
> + if (journal->j_running_transaction->t_tid != tid)
> + return -EINVAL;
> +
> + txn = journal->j_running_transaction;
> + write_lock(&journal->j_state_lock);
> + J_ASSERT(txn->t_state == T_RUNNING);
> + txn->t_async_fc_ongoing = false;
> + txn->t_subtid++;
> + write_unlock(&journal->j_state_lock);
> + return 0;
> +
> +}
> +
> /* Return 1 when transaction with given tid has already committed. */
> int jbd2_transaction_committed(journal_t *journal, tid_t tid)
> {
> @@ -1308,6 +1366,7 @@ static journal_t *journal_init_common(struct block_device *bdev,
> init_waitqueue_head(&journal->j_wait_commit);
> init_waitqueue_head(&journal->j_wait_updates);
> init_waitqueue_head(&journal->j_wait_reserved);
> + init_waitqueue_head(&journal->j_wait_async_fc);
> mutex_init(&journal->j_barrier);
> mutex_init(&journal->j_checkpoint_mutex);
> spin_lock_init(&journal->j_revoke_lock);
> diff --git a/fs/jbd2/transaction.c b/fs/jbd2/transaction.c
> index ce7f03cfd90b..f17f813b5610 100644
> --- a/fs/jbd2/transaction.c
> +++ b/fs/jbd2/transaction.c
> @@ -103,6 +103,8 @@ static void jbd2_get_transaction(journal_t *journal,
> transaction->t_max_wait = 0;
> transaction->t_start = jiffies;
> transaction->t_requested = 0;
> + transaction->t_async_fc_allowed = true;
> + transaction->t_async_fc_ongoing = false;
> }
>
> /*
> diff --git a/include/linux/jbd2.h b/include/linux/jbd2.h
> index 312103fc9581..5610f16de919 100644
> --- a/include/linux/jbd2.h
> +++ b/include/linux/jbd2.h
> @@ -604,6 +604,7 @@ struct transaction_s
> T_FINISHED
> } t_state;
>
> + bool t_async_fc_allowed, t_async_fc_ongoing;
> /*
> * Where in the log does this transaction's commit start? [no locking]
> */
> @@ -869,6 +870,13 @@ struct journal_s
> */
> wait_queue_head_t j_wait_reserved;
>
> + /**
> + * @j_wait_async_fc:
> + *
> + * Wait queue to wait for completion of async fast commits.
> + */
> + wait_queue_head_t j_wait_async_fc;
> +
> /**
> * @j_checkpoint_mutex:
> *
> @@ -1594,6 +1602,8 @@ int jbd2_complete_transaction(journal_t *journal, tid_t tid);
> int jbd2_log_do_checkpoint(journal_t *journal);
> int jbd2_trans_will_send_data_barrier(journal_t *journal, tid_t tid);
> int jbd2_fc_complete_commit(journal_t *journal, tid_t tid, tid_t subtid);
> +int jbd2_start_async_fc(journal_t *journal, tid_t tid);
> +int jbd2_stop_async_fc(journal_t *journal, tid_t tid);
>
> void __jbd2_log_wait_for_space(journal_t *journal);
> extern void __jbd2_journal_drop_transaction(journal_t *, transaction_t *);
>

2019-10-30 05:14:02

by harshad shirwadkar

[permalink] [raw]
Subject: Re: [PATCH v3 08/13] ext4: fast-commit commit range tracking

Thanks for this, I'll remove these calls and add calls in ext4_map_blocks.

On Wed, Oct 16, 2019 at 2:36 PM Theodore Y. Ts'o <[email protected]> wrote:
>
> On Tue, Oct 01, 2019 at 12:40:57AM -0700, Harshad Shirwadkar wrote:
> > With this patch, we track logical range of file offsets that need to
> > be committed using fast commit. This allows us to find file extents
> > that need to be committed during the commit time.
>
> We don't actually need to track when data is modified in the page
> cache, which is what this commit is actually doing. We only need to
> track newly allocated blocks, at granularity of the logical block
> number.
>
> That's because we only need to force out newly allocated blocks to
> make sure we don't reveal stale data when we are in data=ordered mode.
> And it also follows that we don't need to track logical block ranges
> and submit inode data in data=writeback or data=journalled mode.
>
> In the case where the user has actually called fsync() on the the
> inode, we do a data integrity writeback in ext4_sync_file, and that's
> independent on the fast commit code.
>
> But if the file is being modified using buffered writes, or if an
> already allocated block is changed, and the file has *not* been
> changed, we don't need to write out those blocks on a fast commit.
> For example, in the case where we are the fast commit is being
> initiated via ext4_nfs_commit_metadata() -> ext4_write_inode(), we
> only care about submitting data for the newly allocated blocks. And
> that's what we want to track here.
>
> Hence, all of the callers of ext4_fc_update_commit_range() here are in
> the wrong place. (Also, they are calling ext4_fc_update_commit_range
> with byte offsets, when the function is expecting logical block
Thanks for pointing that out. My code as of now works with logical
file offsets instead of logical block offsets. So I should have used
file offset type instead of logical block type for arguments of
ext4_fc_update_commit_range. But it makes sense to just use logical
block offsets everywhere. I'll fix this in next version.

> numbers, but that really matter, since the existing call sites need to
> be all removed and replaced with new ones in ext4_map_blocks().
>
> - Ted