aboutsummaryrefslogtreecommitdiffstats
path: root/fs/jbd2
diff options
context:
space:
mode:
Diffstat (limited to 'fs/jbd2')
-rw-r--r--fs/jbd2/checkpoint.c216
-rw-r--r--fs/jbd2/commit.c211
-rw-r--r--fs/jbd2/journal.c617
-rw-r--r--fs/jbd2/recovery.c248
-rw-r--r--fs/jbd2/revoke.c8
-rw-r--r--fs/jbd2/transaction.c285
6 files changed, 1171 insertions, 414 deletions
diff --git a/fs/jbd2/checkpoint.c b/fs/jbd2/checkpoint.c
index 96bf33986d03..51bd38da21cd 100644
--- a/fs/jbd2/checkpoint.c
+++ b/fs/jbd2/checkpoint.c
@@ -80,23 +80,15 @@ static inline void __buffer_relink_io(struct journal_head *jh)
}
/*
- * Try to release a checkpointed buffer from its transaction.
- * Returns 1 if we released it and 2 if we also released the
- * whole transaction.
+ * Check a checkpoint buffer could be release or not.
*
* Requires j_list_lock
*/
-static int __try_to_free_cp_buf(struct journal_head *jh)
+static inline bool __cp_buffer_busy(struct journal_head *jh)
{
- int ret = 0;
struct buffer_head *bh = jh2bh(jh);
- if (jh->b_transaction == NULL && !buffer_locked(bh) &&
- !buffer_dirty(bh) && !buffer_write_io_error(bh)) {
- JBUFFER_TRACE(jh, "remove from checkpoint list");
- ret = __jbd2_journal_remove_checkpoint(jh) + 1;
- }
- return ret;
+ return (jh->b_transaction || buffer_locked(bh) || buffer_dirty(bh));
}
/*
@@ -106,6 +98,8 @@ static int __try_to_free_cp_buf(struct journal_head *jh)
* for a checkpoint to free up some space in the log.
*/
void __jbd2_log_wait_for_space(journal_t *journal)
+__acquires(&journal->j_state_lock)
+__releases(&journal->j_state_lock)
{
int nblocks, space_left;
/* assert_spin_locked(&journal->j_state_lock); */
@@ -209,7 +203,7 @@ int jbd2_log_do_checkpoint(journal_t *journal)
tid_t this_tid;
int result, batch_count = 0;
- jbd_debug(1, "Start checkpoint\n");
+ jbd2_debug(1, "Start checkpoint\n");
/*
* First thing: if there are any transactions in the log which
@@ -218,7 +212,7 @@ int jbd2_log_do_checkpoint(journal_t *journal)
*/
result = jbd2_cleanup_journal_tail(journal);
trace_jbd2_checkpoint(journal, result);
- jbd_debug(1, "cleanup_journal_tail returned %d\n", result);
+ jbd2_debug(1, "cleanup_journal_tail returned %d\n", result);
if (result <= 0)
return result;
@@ -226,7 +220,6 @@ int jbd2_log_do_checkpoint(journal_t *journal)
* OK, we need to start writing disk blocks. Take one transaction
* and write it.
*/
- result = 0;
spin_lock(&journal->j_list_lock);
if (!journal->j_checkpoint_transactions)
goto out;
@@ -293,8 +286,6 @@ restart:
goto restart;
}
if (!buffer_dirty(bh)) {
- if (unlikely(buffer_write_io_error(bh)) && !result)
- result = -EIO;
BUFFER_TRACE(bh, "remove from checkpoint");
if (__jbd2_journal_remove_checkpoint(jh))
/* The transaction was released; we're done */
@@ -354,8 +345,6 @@ restart2:
spin_lock(&journal->j_list_lock);
goto restart2;
}
- if (unlikely(buffer_write_io_error(bh)) && !result)
- result = -EIO;
/*
* Now in whatever state the buffer currently is, we
@@ -367,10 +356,7 @@ restart2:
}
out:
spin_unlock(&journal->j_list_lock);
- if (result < 0)
- jbd2_journal_abort(journal, result);
- else
- result = jbd2_cleanup_journal_tail(journal);
+ result = jbd2_cleanup_journal_tail(journal);
return (result < 0) ? result : 0;
}
@@ -414,7 +400,7 @@ int jbd2_cleanup_journal_tail(journal_t *journal)
* jbd2_cleanup_journal_tail() doesn't get called all that often.
*/
if (journal->j_flags & JBD2_BARRIER)
- blkdev_issue_flush(journal->j_fs_dev, GFP_NOFS, NULL);
+ blkdev_issue_flush(journal->j_fs_dev);
return __jbd2_update_log_tail(journal, first_tid, blocknr);
}
@@ -435,7 +421,6 @@ static int journal_clean_one_cp_list(struct journal_head *jh, bool destroy)
{
struct journal_head *last_jh;
struct journal_head *next_jh = jh;
- int ret;
if (!jh)
return 0;
@@ -444,13 +429,11 @@ static int journal_clean_one_cp_list(struct journal_head *jh, bool destroy)
do {
jh = next_jh;
next_jh = jh->b_cpnext;
- if (!destroy)
- ret = __try_to_free_cp_buf(jh);
- else
- ret = __jbd2_journal_remove_checkpoint(jh) + 1;
- if (!ret)
+
+ if (!destroy && __cp_buffer_busy(jh))
return 0;
- if (ret == 2)
+
+ if (__jbd2_journal_remove_checkpoint(jh))
return 1;
/*
* This function only frees up some memory
@@ -466,6 +449,137 @@ static int journal_clean_one_cp_list(struct journal_head *jh, bool destroy)
}
/*
+ * journal_shrink_one_cp_list
+ *
+ * Find 'nr_to_scan' written-back checkpoint buffers in the given list
+ * and try to release them. If the whole transaction is released, set
+ * the 'released' parameter. Return the number of released checkpointed
+ * buffers.
+ *
+ * Called with j_list_lock held.
+ */
+static unsigned long journal_shrink_one_cp_list(struct journal_head *jh,
+ unsigned long *nr_to_scan,
+ bool *released)
+{
+ struct journal_head *last_jh;
+ struct journal_head *next_jh = jh;
+ unsigned long nr_freed = 0;
+ int ret;
+
+ if (!jh || *nr_to_scan == 0)
+ return 0;
+
+ last_jh = jh->b_cpprev;
+ do {
+ jh = next_jh;
+ next_jh = jh->b_cpnext;
+
+ (*nr_to_scan)--;
+ if (__cp_buffer_busy(jh))
+ continue;
+
+ nr_freed++;
+ ret = __jbd2_journal_remove_checkpoint(jh);
+ if (ret) {
+ *released = true;
+ break;
+ }
+
+ if (need_resched())
+ break;
+ } while (jh != last_jh && *nr_to_scan);
+
+ return nr_freed;
+}
+
+/*
+ * jbd2_journal_shrink_checkpoint_list
+ *
+ * Find 'nr_to_scan' written-back checkpoint buffers in the journal
+ * and try to release them. Return the number of released checkpointed
+ * buffers.
+ *
+ * Called with j_list_lock held.
+ */
+unsigned long jbd2_journal_shrink_checkpoint_list(journal_t *journal,
+ unsigned long *nr_to_scan)
+{
+ transaction_t *transaction, *last_transaction, *next_transaction;
+ bool released;
+ tid_t first_tid = 0, last_tid = 0, next_tid = 0;
+ tid_t tid = 0;
+ unsigned long nr_freed = 0;
+ unsigned long nr_scanned = *nr_to_scan;
+
+again:
+ spin_lock(&journal->j_list_lock);
+ if (!journal->j_checkpoint_transactions) {
+ spin_unlock(&journal->j_list_lock);
+ goto out;
+ }
+
+ /*
+ * Get next shrink transaction, resume previous scan or start
+ * over again. If some others do checkpoint and drop transaction
+ * from the checkpoint list, we ignore saved j_shrink_transaction
+ * and start over unconditionally.
+ */
+ if (journal->j_shrink_transaction)
+ transaction = journal->j_shrink_transaction;
+ else
+ transaction = journal->j_checkpoint_transactions;
+
+ if (!first_tid)
+ first_tid = transaction->t_tid;
+ last_transaction = journal->j_checkpoint_transactions->t_cpprev;
+ next_transaction = transaction;
+ last_tid = last_transaction->t_tid;
+ do {
+ transaction = next_transaction;
+ next_transaction = transaction->t_cpnext;
+ tid = transaction->t_tid;
+ released = false;
+
+ nr_freed += journal_shrink_one_cp_list(transaction->t_checkpoint_list,
+ nr_to_scan, &released);
+ if (*nr_to_scan == 0)
+ break;
+ if (need_resched() || spin_needbreak(&journal->j_list_lock))
+ break;
+ if (released)
+ continue;
+
+ nr_freed += journal_shrink_one_cp_list(transaction->t_checkpoint_io_list,
+ nr_to_scan, &released);
+ if (*nr_to_scan == 0)
+ break;
+ if (need_resched() || spin_needbreak(&journal->j_list_lock))
+ break;
+ } while (transaction != last_transaction);
+
+ if (transaction != last_transaction) {
+ journal->j_shrink_transaction = next_transaction;
+ next_tid = next_transaction->t_tid;
+ } else {
+ journal->j_shrink_transaction = NULL;
+ next_tid = 0;
+ }
+
+ spin_unlock(&journal->j_list_lock);
+ cond_resched();
+
+ if (*nr_to_scan && next_tid)
+ goto again;
+out:
+ nr_scanned -= *nr_to_scan;
+ trace_jbd2_shrink_checkpoint_list(journal, first_tid, tid, last_tid,
+ nr_freed, nr_scanned, next_tid);
+
+ return nr_freed;
+}
+
+/*
* journal_clean_checkpoint_list
*
* Find all the written-back checkpoint buffers in the journal and release them.
@@ -562,24 +676,37 @@ int __jbd2_journal_remove_checkpoint(struct journal_head *jh)
struct transaction_chp_stats_s *stats;
transaction_t *transaction;
journal_t *journal;
- int ret = 0;
+ struct buffer_head *bh = jh2bh(jh);
JBUFFER_TRACE(jh, "entry");
- if ((transaction = jh->b_cp_transaction) == NULL) {
+ transaction = jh->b_cp_transaction;
+ if (!transaction) {
JBUFFER_TRACE(jh, "not on transaction");
- goto out;
+ return 0;
}
journal = transaction->t_journal;
JBUFFER_TRACE(jh, "removing from transaction");
+
+ /*
+ * If we have failed to write the buffer out to disk, the filesystem
+ * may become inconsistent. We cannot abort the journal here since
+ * we hold j_list_lock and we have to be careful about races with
+ * jbd2_journal_destroy(). So mark the writeback IO error in the
+ * journal here and we abort the journal later from a better context.
+ */
+ if (buffer_write_io_error(bh))
+ set_bit(JBD2_CHECKPOINT_IO_ERROR, &journal->j_atomic_flags);
+
__buffer_unlink(jh);
jh->b_cp_transaction = NULL;
+ percpu_counter_dec(&journal->j_checkpoint_jh_count);
jbd2_journal_put_journal_head(jh);
- if (transaction->t_checkpoint_list != NULL ||
- transaction->t_checkpoint_io_list != NULL)
- goto out;
+ /* Is this transaction empty? */
+ if (transaction->t_checkpoint_list || transaction->t_checkpoint_io_list)
+ return 0;
/*
* There is one special case to worry about: if we have just pulled the
@@ -591,10 +718,12 @@ int __jbd2_journal_remove_checkpoint(struct journal_head *jh)
* See the comment at the end of jbd2_journal_commit_transaction().
*/
if (transaction->t_state != T_FINISHED)
- goto out;
+ return 0;
- /* OK, that was the last buffer for the transaction: we can now
- safely remove this transaction from the log */
+ /*
+ * OK, that was the last buffer for the transaction, we can now
+ * safely remove this transaction from the log.
+ */
stats = &transaction->t_chp_stats;
if (stats->cs_chp_time)
stats->cs_chp_time = jbd2_time_diff(stats->cs_chp_time,
@@ -604,9 +733,7 @@ int __jbd2_journal_remove_checkpoint(struct journal_head *jh)
__jbd2_journal_drop_transaction(journal, transaction);
jbd2_journal_free_transaction(transaction);
- ret = 1;
-out:
- return ret;
+ return 1;
}
/*
@@ -637,6 +764,7 @@ void __jbd2_journal_insert_checkpoint(struct journal_head *jh,
jh->b_cpnext->b_cpprev = jh;
}
transaction->t_checkpoint_list = jh;
+ percpu_counter_inc(&transaction->t_journal->j_checkpoint_jh_count);
}
/*
@@ -652,6 +780,8 @@ void __jbd2_journal_insert_checkpoint(struct journal_head *jh,
void __jbd2_journal_drop_transaction(journal_t *journal, transaction_t *transaction)
{
assert_spin_locked(&journal->j_list_lock);
+
+ journal->j_shrink_transaction = NULL;
if (transaction->t_cpnext) {
transaction->t_cpnext->t_cpprev = transaction->t_cpprev;
transaction->t_cpprev->t_cpnext = transaction->t_cpnext;
@@ -674,5 +804,5 @@ void __jbd2_journal_drop_transaction(journal_t *journal, transaction_t *transact
trace_jbd2_drop_transaction(journal, transaction);
- jbd_debug(1, "Dropping transaction %d, all done\n", transaction->t_tid);
+ jbd2_debug(1, "Dropping transaction %d, all done\n", transaction->t_tid);
}
diff --git a/fs/jbd2/commit.c b/fs/jbd2/commit.c
index 27373f5792a4..885a7a6cc53e 100644
--- a/fs/jbd2/commit.c
+++ b/fs/jbd2/commit.c
@@ -62,6 +62,7 @@ static void journal_end_buffer_io_sync(struct buffer_head *bh, int uptodate)
*/
static void release_buffer_page(struct buffer_head *bh)
{
+ struct folio *folio;
struct page *page;
if (buffer_dirty(bh))
@@ -71,18 +72,19 @@ static void release_buffer_page(struct buffer_head *bh)
page = bh->b_page;
if (!page)
goto nope;
- if (page->mapping)
+ folio = page_folio(page);
+ if (folio->mapping)
goto nope;
/* OK, it's a truncated page */
- if (!trylock_page(page))
+ if (!folio_trylock(folio))
goto nope;
- get_page(page);
+ folio_get(folio);
__brelse(bh);
- try_to_free_buffers(page);
- unlock_page(page);
- put_page(page);
+ try_to_free_buffers(folio);
+ folio_unlock(folio);
+ folio_put(folio);
return;
nope:
@@ -120,8 +122,8 @@ static int journal_submit_commit_record(journal_t *journal,
{
struct commit_header *tmp;
struct buffer_head *bh;
- int ret;
struct timespec64 now;
+ blk_opf_t write_flags = REQ_OP_WRITE | REQ_SYNC;
*cbh = NULL;
@@ -153,13 +155,11 @@ static int journal_submit_commit_record(journal_t *journal,
if (journal->j_flags & JBD2_BARRIER &&
!jbd2_has_feature_async_commit(journal))
- ret = submit_bh(REQ_OP_WRITE,
- REQ_SYNC | REQ_PREFLUSH | REQ_FUA, bh);
- else
- ret = submit_bh(REQ_OP_WRITE, REQ_SYNC, bh);
+ write_flags |= REQ_PREFLUSH | REQ_FUA;
+ submit_bh(write_flags, bh);
*cbh = bh;
- return ret;
+ return 0;
}
/*
@@ -187,20 +187,48 @@ static int journal_wait_on_commit_record(journal_t *journal,
* use writepages() because with delayed allocation we may be doing
* block allocation in writepages().
*/
-static int journal_submit_inode_data_buffers(struct address_space *mapping,
- loff_t dirty_start, loff_t dirty_end)
+int jbd2_journal_submit_inode_data_buffers(struct jbd2_inode *jinode)
{
- int ret;
+ struct address_space *mapping = jinode->i_vfs_inode->i_mapping;
struct writeback_control wbc = {
.sync_mode = WB_SYNC_ALL,
.nr_to_write = mapping->nrpages * 2,
- .range_start = dirty_start,
- .range_end = dirty_end,
+ .range_start = jinode->i_dirty_start,
+ .range_end = jinode->i_dirty_end,
};
- ret = generic_writepages(mapping, &wbc);
- return ret;
+ /*
+ * submit the inode data buffers. We use writepage
+ * instead of writepages. Because writepages can do
+ * block allocation with delalloc. We need to write
+ * only allocated blocks here.
+ */
+ return generic_writepages(mapping, &wbc);
+}
+
+/* Send all the data buffers related to an inode */
+int jbd2_submit_inode_data(struct jbd2_inode *jinode)
+{
+
+ if (!jinode || !(jinode->i_flags & JI_WRITE_DATA))
+ return 0;
+
+ trace_jbd2_submit_inode_data(jinode->i_vfs_inode);
+ return jbd2_journal_submit_inode_data_buffers(jinode);
+
+}
+EXPORT_SYMBOL(jbd2_submit_inode_data);
+
+int jbd2_wait_inode_data(journal_t *journal, struct jbd2_inode *jinode)
+{
+ if (!jinode || !(jinode->i_flags & JI_WAIT_DATA) ||
+ !jinode->i_vfs_inode || !jinode->i_vfs_inode->i_mapping)
+ return 0;
+ return filemap_fdatawait_range_keep_errors(
+ jinode->i_vfs_inode->i_mapping, jinode->i_dirty_start,
+ jinode->i_dirty_end);
}
+EXPORT_SYMBOL(jbd2_wait_inode_data);
/*
* Submit all the data buffers of inode associated with the transaction to
@@ -215,29 +243,20 @@ static int journal_submit_data_buffers(journal_t *journal,
{
struct jbd2_inode *jinode;
int err, ret = 0;
- struct address_space *mapping;
spin_lock(&journal->j_list_lock);
list_for_each_entry(jinode, &commit_transaction->t_inode_list, i_list) {
- loff_t dirty_start = jinode->i_dirty_start;
- loff_t dirty_end = jinode->i_dirty_end;
-
if (!(jinode->i_flags & JI_WRITE_DATA))
continue;
- mapping = jinode->i_vfs_inode->i_mapping;
jinode->i_flags |= JI_COMMIT_RUNNING;
spin_unlock(&journal->j_list_lock);
- /*
- * submit the inode data buffers. We use writepage
- * instead of writepages. Because writepages can do
- * block allocation with delalloc. We need to write
- * only allocated blocks here.
- */
+ /* submit the inode data buffers. */
trace_jbd2_submit_inode_data(jinode->i_vfs_inode);
- err = journal_submit_inode_data_buffers(mapping, dirty_start,
- dirty_end);
- if (!ret)
- ret = err;
+ if (journal->j_submit_inode_data_buffers) {
+ err = journal->j_submit_inode_data_buffers(jinode);
+ if (!ret)
+ ret = err;
+ }
spin_lock(&journal->j_list_lock);
J_ASSERT(jinode->i_transaction == commit_transaction);
jinode->i_flags &= ~JI_COMMIT_RUNNING;
@@ -248,6 +267,15 @@ static int journal_submit_data_buffers(journal_t *journal,
return ret;
}
+int jbd2_journal_finish_inode_data_buffers(struct jbd2_inode *jinode)
+{
+ struct address_space *mapping = jinode->i_vfs_inode->i_mapping;
+
+ return filemap_fdatawait_range_keep_errors(mapping,
+ jinode->i_dirty_start,
+ jinode->i_dirty_end);
+}
+
/*
* Wait for data submitted for writeout, refile inodes to proper
* transaction if needed.
@@ -262,18 +290,16 @@ static int journal_finish_inode_data_buffers(journal_t *journal,
/* For locking, see the comment in journal_submit_data_buffers() */
spin_lock(&journal->j_list_lock);
list_for_each_entry(jinode, &commit_transaction->t_inode_list, i_list) {
- loff_t dirty_start = jinode->i_dirty_start;
- loff_t dirty_end = jinode->i_dirty_end;
-
if (!(jinode->i_flags & JI_WAIT_DATA))
continue;
jinode->i_flags |= JI_COMMIT_RUNNING;
spin_unlock(&journal->j_list_lock);
- err = filemap_fdatawait_range_keep_errors(
- jinode->i_vfs_inode->i_mapping, dirty_start,
- dirty_end);
- if (!ret)
- ret = err;
+ /* wait for the inode data buffers writeout. */
+ if (journal->j_finish_inode_data_buffers) {
+ err = journal->j_finish_inode_data_buffers(jinode);
+ if (!ret)
+ ret = err;
+ }
spin_lock(&journal->j_list_lock);
jinode->i_flags &= ~JI_COMMIT_RUNNING;
smp_mb();
@@ -393,7 +419,7 @@ void jbd2_journal_commit_transaction(journal_t *journal)
/* Do we need to erase the effects of a prior jbd2_journal_flush? */
if (journal->j_flags & JBD2_FLUSHED) {
- jbd_debug(3, "super block updated\n");
+ jbd2_debug(3, "super block updated\n");
mutex_lock_io(&journal->j_checkpoint_mutex);
/*
* We hold j_checkpoint_mutex so tail cannot change under us.
@@ -407,19 +433,43 @@ void jbd2_journal_commit_transaction(journal_t *journal)
REQ_SYNC);
mutex_unlock(&journal->j_checkpoint_mutex);
} else {
- jbd_debug(3, "superblock not updated\n");
+ jbd2_debug(3, "superblock not updated\n");
}
J_ASSERT(journal->j_running_transaction != NULL);
J_ASSERT(journal->j_committing_transaction == NULL);
+ write_lock(&journal->j_state_lock);
+ journal->j_flags |= JBD2_FULL_COMMIT_ONGOING;
+ while (journal->j_flags & JBD2_FAST_COMMIT_ONGOING) {
+ DEFINE_WAIT(wait);
+
+ prepare_to_wait(&journal->j_fc_wait, &wait,
+ TASK_UNINTERRUPTIBLE);
+ write_unlock(&journal->j_state_lock);
+ schedule();
+ write_lock(&journal->j_state_lock);
+ finish_wait(&journal->j_fc_wait, &wait);
+ /*
+ * TODO: by blocking fast commits here, we are increasing
+ * fsync() latency slightly. Strictly speaking, we don't need
+ * to block fast commits until the transaction enters T_FLUSH
+ * state. So an optimization is possible where we block new fast
+ * commits here and wait for existing ones to complete
+ * just before we enter T_FLUSH. That way, the existing fast
+ * commits and this full commit can proceed parallely.
+ */
+ }
+ write_unlock(&journal->j_state_lock);
+
commit_transaction = journal->j_running_transaction;
trace_jbd2_start_commit(journal, commit_transaction);
- jbd_debug(1, "JBD2: starting commit of transaction %d\n",
+ jbd2_debug(1, "JBD2: starting commit of transaction %d\n",
commit_transaction->t_tid);
write_lock(&journal->j_state_lock);
+ journal->j_fc_off = 0;
J_ASSERT(commit_transaction->t_state == T_RUNNING);
commit_transaction->t_state = T_LOCKED;
@@ -434,24 +484,10 @@ void jbd2_journal_commit_transaction(journal_t *journal)
stats.run.rs_running = jbd2_time_diff(commit_transaction->t_start,
stats.run.rs_locked);
- spin_lock(&commit_transaction->t_handle_lock);
- while (atomic_read(&commit_transaction->t_updates)) {
- DEFINE_WAIT(wait);
+ // waits for any t_updates to finish
+ jbd2_journal_wait_updates(journal);
- prepare_to_wait(&journal->j_wait_updates, &wait,
- TASK_UNINTERRUPTIBLE);
- if (atomic_read(&commit_transaction->t_updates)) {
- spin_unlock(&commit_transaction->t_handle_lock);
- write_unlock(&journal->j_state_lock);
- schedule();
- write_lock(&journal->j_state_lock);
- spin_lock(&commit_transaction->t_handle_lock);
- }
- finish_wait(&journal->j_wait_updates, &wait);
- }
- spin_unlock(&commit_transaction->t_handle_lock);
commit_transaction->t_state = T_SWITCH;
- write_unlock(&journal->j_state_lock);
J_ASSERT (atomic_read(&commit_transaction->t_outstanding_credits) <=
journal->j_max_transaction_buffers);
@@ -471,6 +507,8 @@ void jbd2_journal_commit_transaction(journal_t *journal)
* has reserved. This is consistent with the existing behaviour
* that multiple jbd2_journal_get_write_access() calls to the same
* buffer are perfectly permissible.
+ * We use journal->j_state_lock here to serialize processing of
+ * t_reserved_list with eviction of buffers from journal_unmap_buffer().
*/
while (commit_transaction->t_reserved_list) {
jh = commit_transaction->t_reserved_list;
@@ -490,6 +528,7 @@ void jbd2_journal_commit_transaction(journal_t *journal)
jbd2_journal_refile_buffer(journal, jh);
}
+ write_unlock(&journal->j_state_lock);
/*
* Now try to drop any written-back buffers from the journal's
* checkpoint lists. We do this *before* commit because it potentially
@@ -499,7 +538,7 @@ void jbd2_journal_commit_transaction(journal_t *journal)
__jbd2_journal_clean_checkpoint_list(journal, false);
spin_unlock(&journal->j_list_lock);
- jbd_debug(3, "JBD2: commit phase 1\n");
+ jbd2_debug(3, "JBD2: commit phase 1\n");
/*
* Clear revoked flag to reflect there is no revoked buffers
@@ -512,13 +551,13 @@ void jbd2_journal_commit_transaction(journal_t *journal)
*/
jbd2_journal_switch_revoke_table(journal);
+ write_lock(&journal->j_state_lock);
/*
* Reserved credits cannot be claimed anymore, free them
*/
atomic_sub(atomic_read(&journal->j_reserved_credits),
&commit_transaction->t_outstanding_credits);
- write_lock(&journal->j_state_lock);
trace_jbd2_commit_flushing(journal, commit_transaction);
stats.run.rs_flushing = jiffies;
stats.run.rs_locked = jbd2_time_diff(stats.run.rs_locked,
@@ -529,10 +568,10 @@ void jbd2_journal_commit_transaction(journal_t *journal)
journal->j_running_transaction = NULL;
start_time = ktime_get();
commit_transaction->t_log_start = journal->j_head;
- wake_up(&journal->j_wait_transaction_locked);
+ wake_up_all(&journal->j_wait_transaction_locked);
write_unlock(&journal->j_state_lock);
- jbd_debug(3, "JBD2: commit phase 2a\n");
+ jbd2_debug(3, "JBD2: commit phase 2a\n");
/*
* Now start flushing things to disk, in the order they appear
@@ -545,7 +584,7 @@ void jbd2_journal_commit_transaction(journal_t *journal)
blk_start_plug(&plug);
jbd2_journal_write_revoke_records(commit_transaction, &log_bufs);
- jbd_debug(3, "JBD2: commit phase 2b\n");
+ jbd2_debug(3, "JBD2: commit phase 2b\n");
/*
* Way to go: we have now written out all of the data for a
@@ -601,7 +640,7 @@ void jbd2_journal_commit_transaction(journal_t *journal)
if (!descriptor) {
J_ASSERT (bufs == 0);
- jbd_debug(4, "JBD2: get descriptor\n");
+ jbd2_debug(4, "JBD2: get descriptor\n");
descriptor = jbd2_journal_get_descriptor_buffer(
commit_transaction,
@@ -611,7 +650,7 @@ void jbd2_journal_commit_transaction(journal_t *journal)
continue;
}
- jbd_debug(4, "JBD2: got buffer %llu (%p)\n",
+ jbd2_debug(4, "JBD2: got buffer %llu (%p)\n",
(unsigned long long)descriptor->b_blocknr,
descriptor->b_data);
tagp = &descriptor->b_data[sizeof(journal_header_t)];
@@ -696,7 +735,7 @@ void jbd2_journal_commit_transaction(journal_t *journal)
commit_transaction->t_buffers == NULL ||
space_left < tag_bytes + 16 + csum_size) {
- jbd_debug(4, "JBD2: Submit %d IOs\n", bufs);
+ jbd2_debug(4, "JBD2: Submit %d IOs\n", bufs);
/* Write an end-of-descriptor marker before
submitting the IOs. "tag" still points to
@@ -722,7 +761,7 @@ start_journal_io:
clear_buffer_dirty(bh);
set_buffer_uptodate(bh);
bh->b_end_io = journal_end_buffer_io_sync;
- submit_bh(REQ_OP_WRITE, REQ_SYNC, bh);
+ submit_bh(REQ_OP_WRITE | REQ_SYNC, bh);
}
cond_resched();
@@ -760,14 +799,14 @@ start_journal_io:
if (first_block < journal->j_tail)
freed += journal->j_last - journal->j_first;
/* Update tail only if we free significant amount of space */
- if (freed < journal->j_maxlen / 4)
+ if (freed < jbd2_journal_get_max_txn_bufs(journal))
update_tail = 0;
}
J_ASSERT(commit_transaction->t_state == T_COMMIT);
commit_transaction->t_state = T_COMMIT_DFLUSH;
write_unlock(&journal->j_state_lock);
- /*
+ /*
* If the journal is not located on the file system device,
* then we must flush the file system device before we issue
* the commit record
@@ -775,7 +814,7 @@ start_journal_io:
if (commit_transaction->t_need_data_flush &&
(journal->j_fs_dev != journal->j_dev) &&
(journal->j_flags & JBD2_BARRIER))
- blkdev_issue_flush(journal->j_fs_dev, GFP_NOFS, NULL);
+ blkdev_issue_flush(journal->j_fs_dev);
/* Done it all: now write the commit record asynchronously. */
if (jbd2_has_feature_async_commit(journal)) {
@@ -798,7 +837,7 @@ start_journal_io:
so we incur less scheduling load.
*/
- jbd_debug(3, "JBD2: commit phase 3\n");
+ jbd2_debug(3, "JBD2: commit phase 3\n");
while (!list_empty(&io_bufs)) {
struct buffer_head *bh = list_entry(io_bufs.prev,
@@ -841,7 +880,7 @@ start_journal_io:
J_ASSERT (commit_transaction->t_shadow_list == NULL);
- jbd_debug(3, "JBD2: commit phase 4\n");
+ jbd2_debug(3, "JBD2: commit phase 4\n");
/* Here we wait for the revoke record and descriptor record buffers */
while (!list_empty(&log_bufs)) {
@@ -865,7 +904,7 @@ start_journal_io:
if (err)
jbd2_journal_abort(journal, err);
- jbd_debug(3, "JBD2: commit phase 5\n");
+ jbd2_debug(3, "JBD2: commit phase 5\n");
write_lock(&journal->j_state_lock);
J_ASSERT(commit_transaction->t_state == T_COMMIT_DFLUSH);
commit_transaction->t_state = T_COMMIT_JFLUSH;
@@ -882,7 +921,7 @@ start_journal_io:
stats.run.rs_blocks_logged++;
if (jbd2_has_feature_async_commit(journal) &&
journal->j_flags & JBD2_BARRIER) {
- blkdev_issue_flush(journal->j_dev, GFP_NOFS, NULL);
+ blkdev_issue_flush(journal->j_dev);
}
if (err)
@@ -904,7 +943,7 @@ start_journal_io:
transaction can be removed from any checkpoint list it was on
before. */
- jbd_debug(3, "JBD2: commit phase 6\n");
+ jbd2_debug(3, "JBD2: commit phase 6\n");
J_ASSERT(list_empty(&commit_transaction->t_inode_list));
J_ASSERT(commit_transaction->t_buffers == NULL);
@@ -997,9 +1036,10 @@ restart_loop:
* journalled data) we need to unmap buffer and clear
* more bits. We also need to be careful about the check
* because the data page mapping can get cleared under
- * out hands, which alse need not to clear more bits
- * because the page and buffers will be freed and can
- * never be reused once we are done with them.
+ * our hands. Note that if mapping == NULL, we don't
+ * need to make buffer unmapped because the page is
+ * already detached from the mapping and buffers cannot
+ * get reused.
*/
mapping = READ_ONCE(bh->b_page->mapping);
if (mapping && !sb_is_blkdev_sb(mapping->host->i_sb)) {
@@ -1080,7 +1120,7 @@ restart_loop:
/* Done with this transaction! */
- jbd_debug(3, "JBD2: commit phase 7\n");
+ jbd2_debug(3, "JBD2: commit phase 7\n");
J_ASSERT(commit_transaction->t_state == T_COMMIT_JFLUSH);
@@ -1118,12 +1158,16 @@ restart_loop:
if (journal->j_commit_callback)
journal->j_commit_callback(journal, commit_transaction);
+ if (journal->j_fc_cleanup_callback)
+ journal->j_fc_cleanup_callback(journal, 1, commit_transaction->t_tid);
trace_jbd2_end_commit(journal, commit_transaction);
- jbd_debug(1, "JBD2: commit %d complete, head %d\n",
+ jbd2_debug(1, "JBD2: commit %d complete, head %d\n",
journal->j_commit_sequence, journal->j_tail_sequence);
write_lock(&journal->j_state_lock);
+ journal->j_flags &= ~JBD2_FULL_COMMIT_ONGOING;
+ journal->j_flags &= ~JBD2_FAST_COMMIT_ONGOING;
spin_lock(&journal->j_list_lock);
commit_transaction->t_state = T_FINISHED;
/* Check if the transaction can be dropped now that we are finished */
@@ -1135,6 +1179,7 @@ restart_loop:
spin_unlock(&journal->j_list_lock);
write_unlock(&journal->j_state_lock);
wake_up(&journal->j_wait_done_commit);
+ wake_up(&journal->j_fc_wait);
/*
* Calculate overall stats
diff --git a/fs/jbd2/journal.c b/fs/jbd2/journal.c
index a49d0e670ddf..2696f43e7239 100644
--- a/fs/jbd2/journal.c
+++ b/fs/jbd2/journal.c
@@ -49,8 +49,7 @@
#include <asm/page.h>
#ifdef CONFIG_JBD2_DEBUG
-ushort jbd2_journal_enable_debug __read_mostly;
-EXPORT_SYMBOL(jbd2_journal_enable_debug);
+static ushort jbd2_journal_enable_debug __read_mostly;
module_param_named(jbd2_debug, jbd2_journal_enable_debug, ushort, 0644);
MODULE_PARM_DESC(jbd2_debug, "Debugging level for jbd2");
@@ -81,16 +80,17 @@ EXPORT_SYMBOL(jbd2_journal_errno);
EXPORT_SYMBOL(jbd2_journal_ack_err);
EXPORT_SYMBOL(jbd2_journal_clear_err);
EXPORT_SYMBOL(jbd2_log_wait_commit);
-EXPORT_SYMBOL(jbd2_log_start_commit);
EXPORT_SYMBOL(jbd2_journal_start_commit);
EXPORT_SYMBOL(jbd2_journal_force_commit_nested);
EXPORT_SYMBOL(jbd2_journal_wipe);
EXPORT_SYMBOL(jbd2_journal_blocks_per_page);
-EXPORT_SYMBOL(jbd2_journal_invalidatepage);
+EXPORT_SYMBOL(jbd2_journal_invalidate_folio);
EXPORT_SYMBOL(jbd2_journal_try_to_free_buffers);
EXPORT_SYMBOL(jbd2_journal_force_commit);
EXPORT_SYMBOL(jbd2_journal_inode_ranged_write);
EXPORT_SYMBOL(jbd2_journal_inode_ranged_wait);
+EXPORT_SYMBOL(jbd2_journal_submit_inode_data_buffers);
+EXPORT_SYMBOL(jbd2_journal_finish_inode_data_buffers);
EXPORT_SYMBOL(jbd2_journal_init_jbd_inode);
EXPORT_SYMBOL(jbd2_journal_release_jbd_inode);
EXPORT_SYMBOL(jbd2_journal_begin_ordered_truncate);
@@ -113,7 +113,6 @@ void __jbd2_debug(int level, const char *file, const char *func,
printk(KERN_DEBUG "%s: (%s, %u): %pV", file, func, line, &vaf);
va_end(args);
}
-EXPORT_SYMBOL(__jbd2_debug);
#endif
/* Checksumming functions */
@@ -157,7 +156,9 @@ static void commit_timeout(struct timer_list *t)
*
* 1) COMMIT: Every so often we need to commit the current state of the
* filesystem to disk. The journal thread is responsible for writing
- * all of the metadata buffers to disk.
+ * all of the metadata buffers to disk. If a fast commit is ongoing
+ * journal thread waits until it's done and then continues from
+ * there on.
*
* 2) CHECKPOINT: We cannot reuse a used section of the log file until all
* of the data in that part of the log has been rewritten elsewhere on
@@ -199,11 +200,11 @@ loop:
if (journal->j_flags & JBD2_UNMOUNT)
goto end_loop;
- jbd_debug(1, "commit_sequence=%u, commit_request=%u\n",
+ jbd2_debug(1, "commit_sequence=%u, commit_request=%u\n",
journal->j_commit_sequence, journal->j_commit_request);
if (journal->j_commit_sequence != journal->j_commit_request) {
- jbd_debug(1, "OK, requests differ\n");
+ jbd2_debug(1, "OK, requests differ\n");
write_unlock(&journal->j_state_lock);
del_timer_sync(&journal->j_commit_timer);
jbd2_journal_commit_transaction(journal);
@@ -218,7 +219,7 @@ loop:
* good idea, because that depends on threads that may
* be already stopped.
*/
- jbd_debug(1, "Now suspending kjournald2\n");
+ jbd2_debug(1, "Now suspending kjournald2\n");
write_unlock(&journal->j_state_lock);
try_to_freeze();
write_lock(&journal->j_state_lock);
@@ -248,7 +249,7 @@ loop:
finish_wait(&journal->j_wait_commit, &wait);
}
- jbd_debug(1, "kjournald2 wakes\n");
+ jbd2_debug(1, "kjournald2 wakes\n");
/*
* Were we woken up by a commit wakeup event?
@@ -256,7 +257,7 @@ loop:
transaction = journal->j_running_transaction;
if (transaction && time_after_eq(jiffies, transaction->t_expires)) {
journal->j_commit_request = transaction->t_tid;
- jbd_debug(1, "woke because of timeout\n");
+ jbd2_debug(1, "woke because of timeout\n");
}
goto loop;
@@ -264,7 +265,7 @@ end_loop:
del_timer_sync(&journal->j_commit_timer);
journal->j_task = NULL;
wake_up(&journal->j_wait_done_commit);
- jbd_debug(1, "Journal thread exiting.\n");
+ jbd2_debug(1, "Journal thread exiting.\n");
write_unlock(&journal->j_state_lock);
return 0;
}
@@ -477,7 +478,7 @@ repeat:
* Called with j_state_lock locked for writing.
* Returns true if a transaction commit was started.
*/
-int __jbd2_log_start_commit(journal_t *journal, tid_t target)
+static int __jbd2_log_start_commit(journal_t *journal, tid_t target)
{
/* Return if the txn has already requested to be committed */
if (journal->j_commit_request == target)
@@ -496,7 +497,7 @@ int __jbd2_log_start_commit(journal_t *journal, tid_t target)
*/
journal->j_commit_request = target;
- jbd_debug(1, "JBD2: requesting commit %u/%u\n",
+ jbd2_debug(1, "JBD2: requesting commit %u/%u\n",
journal->j_commit_request,
journal->j_commit_sequence);
journal->j_running_transaction->t_requested = jiffies;
@@ -562,12 +563,14 @@ static int __jbd2_journal_force_commit(journal_t *journal)
}
/**
- * Force and wait upon a commit if the calling process is not within
- * transaction. This is used for forcing out undo-protected data which contains
- * bitmaps, when the fs is running out of space.
+ * jbd2_journal_force_commit_nested - Force and wait upon a commit if the
+ * calling process is not within transaction.
*
* @journal: journal to force
* Returns true if progress was made.
+ *
+ * This is used for forcing out undo-protected data which contains
+ * bitmaps, when the fs is running out of space.
*/
int jbd2_journal_force_commit_nested(journal_t *journal)
{
@@ -578,7 +581,7 @@ int jbd2_journal_force_commit_nested(journal_t *journal)
}
/**
- * int journal_force_commit() - force any uncommitted transactions
+ * jbd2_journal_force_commit() - force any uncommitted transactions
* @journal: journal to force
*
* Caller want unconditional commit. We can only force the running transaction
@@ -699,7 +702,7 @@ int jbd2_log_wait_commit(journal_t *journal, tid_t tid)
}
#endif
while (tid_gt(tid, journal->j_commit_sequence)) {
- jbd_debug(1, "JBD2: want %u, j_commit_sequence=%u\n",
+ jbd2_debug(1, "JBD2: want %u, j_commit_sequence=%u\n",
tid, journal->j_commit_sequence);
read_unlock(&journal->j_state_lock);
wake_up(&journal->j_wait_commit);
@@ -714,6 +717,87 @@ int jbd2_log_wait_commit(journal_t *journal, tid_t tid)
return err;
}
+/*
+ * Start a fast commit. If there's an ongoing fast or full commit wait for
+ * it to complete. Returns 0 if a new fast commit was started. Returns -EALREADY
+ * if a fast commit is not needed, either because there's an already a commit
+ * going on or this tid has already been committed. Returns -EINVAL if no jbd2
+ * commit has yet been performed.
+ */
+int jbd2_fc_begin_commit(journal_t *journal, tid_t tid)
+{
+ if (unlikely(is_journal_aborted(journal)))
+ return -EIO;
+ /*
+ * Fast commits only allowed if at least one full commit has
+ * been processed.
+ */
+ if (!journal->j_stats.ts_tid)
+ return -EINVAL;
+
+ write_lock(&journal->j_state_lock);
+ if (tid <= journal->j_commit_sequence) {
+ write_unlock(&journal->j_state_lock);
+ return -EALREADY;
+ }
+
+ if (journal->j_flags & JBD2_FULL_COMMIT_ONGOING ||
+ (journal->j_flags & JBD2_FAST_COMMIT_ONGOING)) {
+ DEFINE_WAIT(wait);
+
+ prepare_to_wait(&journal->j_fc_wait, &wait,
+ TASK_UNINTERRUPTIBLE);
+ write_unlock(&journal->j_state_lock);
+ schedule();
+ finish_wait(&journal->j_fc_wait, &wait);
+ return -EALREADY;
+ }
+ journal->j_flags |= JBD2_FAST_COMMIT_ONGOING;
+ write_unlock(&journal->j_state_lock);
+ jbd2_journal_lock_updates(journal);
+
+ return 0;
+}
+EXPORT_SYMBOL(jbd2_fc_begin_commit);
+
+/*
+ * Stop a fast commit. If fallback is set, this function starts commit of
+ * TID tid before any other fast commit can start.
+ */
+static int __jbd2_fc_end_commit(journal_t *journal, tid_t tid, bool fallback)
+{
+ jbd2_journal_unlock_updates(journal);
+ if (journal->j_fc_cleanup_callback)
+ journal->j_fc_cleanup_callback(journal, 0, tid);
+ write_lock(&journal->j_state_lock);
+ journal->j_flags &= ~JBD2_FAST_COMMIT_ONGOING;
+ if (fallback)
+ journal->j_flags |= JBD2_FULL_COMMIT_ONGOING;
+ write_unlock(&journal->j_state_lock);
+ wake_up(&journal->j_fc_wait);
+ if (fallback)
+ return jbd2_complete_transaction(journal, tid);
+ return 0;
+}
+
+int jbd2_fc_end_commit(journal_t *journal)
+{
+ return __jbd2_fc_end_commit(journal, 0, false);
+}
+EXPORT_SYMBOL(jbd2_fc_end_commit);
+
+int jbd2_fc_end_commit_fallback(journal_t *journal)
+{
+ tid_t tid;
+
+ read_lock(&journal->j_state_lock);
+ tid = journal->j_running_transaction ?
+ journal->j_running_transaction->t_tid : 0;
+ read_unlock(&journal->j_state_lock);
+ return __jbd2_fc_end_commit(journal, tid, true);
+}
+EXPORT_SYMBOL(jbd2_fc_end_commit_fallback);
+
/* Return 1 when transaction with given tid has already committed. */
int jbd2_transaction_committed(journal_t *journal, tid_t tid)
{
@@ -782,6 +866,98 @@ int jbd2_journal_next_log_block(journal_t *journal, unsigned long long *retp)
return jbd2_journal_bmap(journal, blocknr, retp);
}
+/* Map one fast commit buffer for use by the file system */
+int jbd2_fc_get_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;
+
+ *bh_out = NULL;
+
+ if (journal->j_fc_off + journal->j_fc_first < journal->j_fc_last) {
+ fc_off = journal->j_fc_off;
+ blocknr = journal->j_fc_first + fc_off;
+ journal->j_fc_off++;
+ } else {
+ ret = -EINVAL;
+ }
+
+ 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;
+
+
+ journal->j_fc_wbuf[fc_off] = bh;
+
+ *bh_out = bh;
+
+ return 0;
+}
+EXPORT_SYMBOL(jbd2_fc_get_buf);
+
+/*
+ * Wait on fast commit buffers that were allocated by jbd2_fc_get_buf
+ * for completion.
+ */
+int jbd2_fc_wait_bufs(journal_t *journal, int num_blks)
+{
+ struct buffer_head *bh;
+ int i, j_fc_off;
+
+ j_fc_off = journal->j_fc_off;
+
+ /*
+ * 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);
+ /*
+ * Update j_fc_off so jbd2_fc_release_bufs can release remain
+ * buffer head.
+ */
+ if (unlikely(!buffer_uptodate(bh))) {
+ journal->j_fc_off = i + 1;
+ return -EIO;
+ }
+ put_bh(bh);
+ journal->j_fc_wbuf[i] = NULL;
+ }
+
+ return 0;
+}
+EXPORT_SYMBOL(jbd2_fc_wait_bufs);
+
+int jbd2_fc_release_bufs(journal_t *journal)
+{
+ struct buffer_head *bh;
+ int i, j_fc_off;
+
+ j_fc_off = journal->j_fc_off;
+
+ for (i = j_fc_off - 1; i >= 0; i--) {
+ bh = journal->j_fc_wbuf[i];
+ if (!bh)
+ break;
+ put_bh(bh);
+ journal->j_fc_wbuf[i] = NULL;
+ }
+
+ return 0;
+}
+EXPORT_SYMBOL(jbd2_fc_release_bufs);
+
/*
* Conversion of logical to physical block numbers for the journal
*
@@ -944,7 +1120,7 @@ int __jbd2_update_log_tail(journal_t *journal, tid_t tid, unsigned long block)
freed += journal->j_last - journal->j_first;
trace_jbd2_update_log_tail(journal, tid, block, freed);
- jbd_debug(1,
+ jbd2_debug(1,
"Cleaning journal tail from %u to %u (offset %lu), "
"freeing %lu\n",
journal->j_tail_sequence, tid, block, freed);
@@ -1039,7 +1215,7 @@ static const struct seq_operations jbd2_seq_info_ops = {
static int jbd2_seq_info_open(struct inode *inode, struct file *file)
{
- journal_t *journal = PDE_DATA(inode);
+ journal_t *journal = pde_data(inode);
struct jbd2_stats_proc_session *s;
int rc, size;
@@ -1112,6 +1288,52 @@ static int jbd2_min_tag_size(void)
return sizeof(journal_block_tag_t) - 4;
}
+/**
+ * jbd2_journal_shrink_scan()
+ * @shrink: shrinker to work on
+ * @sc: reclaim request to process
+ *
+ * Scan the checkpointed buffer on the checkpoint list and release the
+ * journal_head.
+ */
+static unsigned long jbd2_journal_shrink_scan(struct shrinker *shrink,
+ struct shrink_control *sc)
+{
+ journal_t *journal = container_of(shrink, journal_t, j_shrinker);
+ unsigned long nr_to_scan = sc->nr_to_scan;
+ unsigned long nr_shrunk;
+ unsigned long count;
+
+ count = percpu_counter_read_positive(&journal->j_checkpoint_jh_count);
+ trace_jbd2_shrink_scan_enter(journal, sc->nr_to_scan, count);
+
+ nr_shrunk = jbd2_journal_shrink_checkpoint_list(journal, &nr_to_scan);
+
+ count = percpu_counter_read_positive(&journal->j_checkpoint_jh_count);
+ trace_jbd2_shrink_scan_exit(journal, nr_to_scan, nr_shrunk, count);
+
+ return nr_shrunk;
+}
+
+/**
+ * jbd2_journal_shrink_count()
+ * @shrink: shrinker to work on
+ * @sc: reclaim request to process
+ *
+ * Count the number of checkpoint buffers on the checkpoint list.
+ */
+static unsigned long jbd2_journal_shrink_count(struct shrinker *shrink,
+ struct shrink_control *sc)
+{
+ journal_t *journal = container_of(shrink, journal_t, j_shrinker);
+ unsigned long count;
+
+ count = percpu_counter_read_positive(&journal->j_checkpoint_jh_count);
+ trace_jbd2_shrink_count(journal, sc->nr_to_scan, count);
+
+ return count;
+}
+
/*
* Management for journal control blocks: functions to create and
* destroy journal_t structures, and to initialise and read existing
@@ -1140,6 +1362,8 @@ 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_fc_wait);
+ mutex_init(&journal->j_abort_mutex);
mutex_init(&journal->j_barrier);
mutex_init(&journal->j_checkpoint_mutex);
spin_lock_init(&journal->j_revoke_lock);
@@ -1169,10 +1393,11 @@ static journal_t *journal_init_common(struct block_device *bdev,
journal->j_dev = bdev;
journal->j_fs_dev = fs_dev;
journal->j_blk_offset = start;
- journal->j_maxlen = len;
+ journal->j_total_len = len;
/* We need enough buffers to write out full descriptor block. */
n = journal->j_blocksize / jbd2_min_tag_size();
journal->j_wbufsize = n;
+ journal->j_fc_wbuf = NULL;
journal->j_wbuf = kmalloc_array(n, sizeof(struct buffer_head *),
GFP_KERNEL);
if (!journal->j_wbuf)
@@ -1187,9 +1412,24 @@ static journal_t *journal_init_common(struct block_device *bdev,
journal->j_sb_buffer = bh;
journal->j_superblock = (journal_superblock_t *)bh->b_data;
+ journal->j_shrink_transaction = NULL;
+ journal->j_shrinker.scan_objects = jbd2_journal_shrink_scan;
+ journal->j_shrinker.count_objects = jbd2_journal_shrink_count;
+ journal->j_shrinker.seeks = DEFAULT_SEEKS;
+ journal->j_shrinker.batch = journal->j_max_transaction_buffers;
+
+ if (percpu_counter_init(&journal->j_checkpoint_jh_count, 0, GFP_KERNEL))
+ goto err_cleanup;
+
+ if (register_shrinker(&journal->j_shrinker, "jbd2-journal:(%u:%u)",
+ MAJOR(bdev->bd_dev), MINOR(bdev->bd_dev))) {
+ percpu_counter_destroy(&journal->j_checkpoint_jh_count);
+ goto err_cleanup;
+ }
return journal;
err_cleanup:
+ brelse(journal->j_sb_buffer);
kfree(journal->j_wbuf);
jbd2_journal_destroy_revoke(journal);
kfree(journal);
@@ -1229,7 +1469,8 @@ journal_t *jbd2_journal_init_dev(struct block_device *bdev,
if (!journal)
return NULL;
- bdevname(journal->j_dev, journal->j_devname);
+ snprintf(journal->j_devname, sizeof(journal->j_devname),
+ "%pg", journal->j_dev);
strreplace(journal->j_devname, '/', '!');
jbd2_stats_proc_init(journal);
@@ -1260,7 +1501,7 @@ journal_t *jbd2_journal_init_inode(struct inode *inode)
return NULL;
}
- jbd_debug(1, "JBD2: inode %s/%ld, size %lld, bits %d, blksize %ld\n",
+ jbd2_debug(1, "JBD2: inode %s/%ld, size %lld, bits %d, blksize %ld\n",
inode->i_sb->s_id, inode->i_ino, (long long) inode->i_size,
inode->i_sb->s_blocksize_bits, inode->i_sb->s_blocksize);
@@ -1271,7 +1512,8 @@ journal_t *jbd2_journal_init_inode(struct inode *inode)
return NULL;
journal->j_inode = inode;
- bdevname(journal->j_dev, journal->j_devname);
+ snprintf(journal->j_devname, sizeof(journal->j_devname),
+ "%pg", journal->j_dev);
p = strreplace(journal->j_devname, '/', '!');
sprintf(p, "-%lu", journal->j_inode->i_ino);
jbd2_stats_proc_init(journal);
@@ -1284,7 +1526,7 @@ journal_t *jbd2_journal_init_inode(struct inode *inode)
* superblock as being NULL to prevent the journal destroy from writing
* back a bogus superblock.
*/
-static void journal_fail_superblock (journal_t *journal)
+static void journal_fail_superblock(journal_t *journal)
{
struct buffer_head *bh = journal->j_sb_buffer;
brelse(bh);
@@ -1315,15 +1557,22 @@ static int journal_reset(journal_t *journal)
journal->j_first = first;
journal->j_last = last;
- journal->j_head = first;
- journal->j_tail = first;
- journal->j_free = last - first;
+ journal->j_head = journal->j_first;
+ journal->j_tail = journal->j_first;
+ journal->j_free = journal->j_last - journal->j_first;
journal->j_tail_sequence = journal->j_transaction_sequence;
journal->j_commit_sequence = journal->j_transaction_sequence - 1;
journal->j_commit_request = journal->j_commit_sequence;
- journal->j_max_transaction_buffers = journal->j_maxlen / 4;
+ journal->j_max_transaction_buffers = jbd2_journal_get_max_txn_bufs(journal);
+
+ /*
+ * Now that journal recovery is done, turn fast commits off here. This
+ * way, if fast commit was enabled before the crash but if now FS has
+ * disabled it, we don't enable fast commits.
+ */
+ jbd2_clear_feature_fast_commit(journal);
/*
* As a special case, if the on-disk copy is already marked as needing
@@ -1332,7 +1581,7 @@ static int journal_reset(journal_t *journal)
* attempting a write to a potential-readonly device.
*/
if (sb->s_start == 0) {
- jbd_debug(1, "JBD2: Skipping superblock update on recovered sb "
+ jbd2_debug(1, "JBD2: Skipping superblock update on recovered sb "
"(start %ld, seq %u, errno %d)\n",
journal->j_tail, journal->j_tail_sequence,
journal->j_errno);
@@ -1359,15 +1608,17 @@ static int journal_reset(journal_t *journal)
* This function expects that the caller will have locked the journal
* buffer head, and will return with it unlocked
*/
-static int jbd2_write_superblock(journal_t *journal, int write_flags)
+static int jbd2_write_superblock(journal_t *journal, blk_opf_t write_flags)
{
struct buffer_head *bh = journal->j_sb_buffer;
journal_superblock_t *sb = journal->j_superblock;
- int ret;
+ int ret = 0;
/* Buffer got discarded which means block device got invalidated */
- if (!buffer_mapped(bh))
+ if (!buffer_mapped(bh)) {
+ unlock_buffer(bh);
return -EIO;
+ }
trace_jbd2_write_superblock(journal, write_flags);
if (!(journal->j_flags & JBD2_BARRIER))
@@ -1391,7 +1642,7 @@ static int jbd2_write_superblock(journal_t *journal, int write_flags)
sb->s_checksum = jbd2_superblock_csum(journal, sb);
get_bh(bh);
bh->b_end_io = end_buffer_write_sync;
- ret = submit_bh(REQ_OP_WRITE, write_flags, bh);
+ submit_bh(REQ_OP_WRITE | write_flags, bh);
wait_on_buffer(bh);
if (buffer_write_io_error(bh)) {
clear_buffer_write_io_error(bh);
@@ -1399,10 +1650,10 @@ static int jbd2_write_superblock(journal_t *journal, int write_flags)
ret = -EIO;
}
if (ret) {
- printk(KERN_ERR "JBD2: Error %d detected when updating "
- "journal superblock for %s.\n", ret,
- journal->j_devname);
- jbd2_journal_abort(journal, ret);
+ printk(KERN_ERR "JBD2: I/O error when updating journal superblock for %s.\n",
+ journal->j_devname);
+ if (!is_journal_aborted(journal))
+ jbd2_journal_abort(journal, ret);
}
return ret;
@@ -1413,29 +1664,34 @@ static int jbd2_write_superblock(journal_t *journal, int write_flags)
* @journal: The journal to update.
* @tail_tid: TID of the new transaction at the tail of the log
* @tail_block: The first block of the transaction at the tail of the log
- * @write_op: With which operation should we write the journal sb
+ * @write_flags: Flags for the journal sb write operation
*
* Update a journal's superblock information about log tail and write it to
* disk, waiting for the IO to complete.
*/
int jbd2_journal_update_sb_log_tail(journal_t *journal, tid_t tail_tid,
- unsigned long tail_block, int write_op)
+ unsigned long tail_block,
+ blk_opf_t write_flags)
{
journal_superblock_t *sb = journal->j_superblock;
int ret;
if (is_journal_aborted(journal))
return -EIO;
+ if (test_bit(JBD2_CHECKPOINT_IO_ERROR, &journal->j_atomic_flags)) {
+ jbd2_journal_abort(journal, -EIO);
+ return -EIO;
+ }
BUG_ON(!mutex_is_locked(&journal->j_checkpoint_mutex));
- jbd_debug(1, "JBD2: updating superblock (start %lu, seq %u)\n",
+ jbd2_debug(1, "JBD2: updating superblock (start %lu, seq %u)\n",
tail_block, tail_tid);
lock_buffer(journal->j_sb_buffer);
sb->s_sequence = cpu_to_be32(tail_tid);
sb->s_start = cpu_to_be32(tail_block);
- ret = jbd2_write_superblock(journal, write_op);
+ ret = jbd2_write_superblock(journal, write_flags);
if (ret)
goto out;
@@ -1452,14 +1708,15 @@ out:
/**
* jbd2_mark_journal_empty() - Mark on disk journal as empty.
* @journal: The journal to update.
- * @write_op: With which operation should we write the journal sb
+ * @write_flags: Flags for the journal sb write operation
*
* Update a journal's dynamic superblock fields to show that journal is empty.
* Write updated superblock to disk waiting for IO to complete.
*/
-static void jbd2_mark_journal_empty(journal_t *journal, int write_op)
+static void jbd2_mark_journal_empty(journal_t *journal, blk_opf_t write_flags)
{
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);
@@ -1468,13 +1725,24 @@ static void jbd2_mark_journal_empty(journal_t *journal, int write_op)
return;
}
- jbd_debug(1, "JBD2: Marking journal as empty (seq %u)\n",
+ jbd2_debug(1, "JBD2: Marking journal as empty (seq %u)\n",
journal->j_tail_sequence);
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_flags);
- 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);
@@ -1482,6 +1750,107 @@ static void jbd2_mark_journal_empty(journal_t *journal, int write_op)
write_unlock(&journal->j_state_lock);
}
+/**
+ * __jbd2_journal_erase() - Discard or zeroout journal blocks (excluding superblock)
+ * @journal: The journal to erase.
+ * @flags: A discard/zeroout request is sent for each physically contigous
+ * region of the journal. Either JBD2_JOURNAL_FLUSH_DISCARD or
+ * JBD2_JOURNAL_FLUSH_ZEROOUT must be set to determine which operation
+ * to perform.
+ *
+ * Note: JBD2_JOURNAL_FLUSH_ZEROOUT attempts to use hardware offload. Zeroes
+ * will be explicitly written if no hardware offload is available, see
+ * blkdev_issue_zeroout for more details.
+ */
+static int __jbd2_journal_erase(journal_t *journal, unsigned int flags)
+{
+ int err = 0;
+ unsigned long block, log_offset; /* logical */
+ unsigned long long phys_block, block_start, block_stop; /* physical */
+ loff_t byte_start, byte_stop, byte_count;
+
+ /* flags must be set to either discard or zeroout */
+ if ((flags & ~JBD2_JOURNAL_FLUSH_VALID) || !flags ||
+ ((flags & JBD2_JOURNAL_FLUSH_DISCARD) &&
+ (flags & JBD2_JOURNAL_FLUSH_ZEROOUT)))
+ return -EINVAL;
+
+ if ((flags & JBD2_JOURNAL_FLUSH_DISCARD) &&
+ !bdev_max_discard_sectors(journal->j_dev))
+ return -EOPNOTSUPP;
+
+ /*
+ * lookup block mapping and issue discard/zeroout for each
+ * contiguous region
+ */
+ log_offset = be32_to_cpu(journal->j_superblock->s_first);
+ block_start = ~0ULL;
+ for (block = log_offset; block < journal->j_total_len; block++) {
+ err = jbd2_journal_bmap(journal, block, &phys_block);
+ if (err) {
+ pr_err("JBD2: bad block at offset %lu", block);
+ return err;
+ }
+
+ if (block_start == ~0ULL) {
+ block_start = phys_block;
+ block_stop = block_start - 1;
+ }
+
+ /*
+ * last block not contiguous with current block,
+ * process last contiguous region and return to this block on
+ * next loop
+ */
+ if (phys_block != block_stop + 1) {
+ block--;
+ } else {
+ block_stop++;
+ /*
+ * if this isn't the last block of journal,
+ * no need to process now because next block may also
+ * be part of this contiguous region
+ */
+ if (block != journal->j_total_len - 1)
+ continue;
+ }
+
+ /*
+ * end of contiguous region or this is last block of journal,
+ * take care of the region
+ */
+ byte_start = block_start * journal->j_blocksize;
+ byte_stop = block_stop * journal->j_blocksize;
+ byte_count = (block_stop - block_start + 1) *
+ journal->j_blocksize;
+
+ truncate_inode_pages_range(journal->j_dev->bd_inode->i_mapping,
+ byte_start, byte_stop);
+
+ if (flags & JBD2_JOURNAL_FLUSH_DISCARD) {
+ err = blkdev_issue_discard(journal->j_dev,
+ byte_start >> SECTOR_SHIFT,
+ byte_count >> SECTOR_SHIFT,
+ GFP_NOFS);
+ } else if (flags & JBD2_JOURNAL_FLUSH_ZEROOUT) {
+ err = blkdev_issue_zeroout(journal->j_dev,
+ byte_start >> SECTOR_SHIFT,
+ byte_count >> SECTOR_SHIFT,
+ GFP_NOFS, 0);
+ }
+
+ if (unlikely(err != 0)) {
+ pr_err("JBD2: (error %d) unable to wipe journal at physical blocks %llu - %llu",
+ err, block_start, block_stop);
+ return err;
+ }
+
+ /* reset start and stop after processing a region */
+ block_start = ~0ULL;
+ }
+
+ return blkdev_issue_flush(journal->j_dev);
+}
/**
* jbd2_journal_update_sb_errno() - Update error in the journal.
@@ -1499,7 +1868,7 @@ void jbd2_journal_update_sb_errno(journal_t *journal)
errcode = journal->j_errno;
if (errcode == -ESHUTDOWN)
errcode = 0;
- jbd_debug(1, "JBD2: updating superblock error (errno %d)\n", errcode);
+ jbd2_debug(1, "JBD2: updating superblock error (errno %d)\n", errcode);
sb->s_errno = cpu_to_be32(errcode);
jbd2_write_superblock(journal, REQ_SYNC | REQ_FUA);
@@ -1529,19 +1898,16 @@ static int journal_get_superblock(journal_t *journal)
{
struct buffer_head *bh;
journal_superblock_t *sb;
- int err = -EIO;
+ int err;
bh = journal->j_sb_buffer;
J_ASSERT(bh != NULL);
- if (!buffer_uptodate(bh)) {
- ll_rw_block(REQ_OP_READ, 0, 1, &bh);
- wait_on_buffer(bh);
- if (!buffer_uptodate(bh)) {
- printk(KERN_ERR
- "JBD2: IO error reading journal superblock\n");
- goto out;
- }
+ err = bh_read(bh, 0);
+ if (err < 0) {
+ printk(KERN_ERR
+ "JBD2: IO error reading journal superblock\n");
+ goto out;
}
if (buffer_verified(bh))
@@ -1569,15 +1935,15 @@ static int journal_get_superblock(journal_t *journal)
goto out;
}
- if (be32_to_cpu(sb->s_maxlen) < journal->j_maxlen)
- journal->j_maxlen = be32_to_cpu(sb->s_maxlen);
- else if (be32_to_cpu(sb->s_maxlen) > journal->j_maxlen) {
+ if (be32_to_cpu(sb->s_maxlen) < journal->j_total_len)
+ journal->j_total_len = be32_to_cpu(sb->s_maxlen);
+ else if (be32_to_cpu(sb->s_maxlen) > journal->j_total_len) {
printk(KERN_WARNING "JBD2: journal file too short\n");
goto out;
}
if (be32_to_cpu(sb->s_first) == 0 ||
- be32_to_cpu(sb->s_first) >= journal->j_maxlen) {
+ be32_to_cpu(sb->s_first) >= journal->j_total_len) {
printk(KERN_WARNING
"JBD2: Invalid start block of journal: %u\n",
be32_to_cpu(sb->s_first));
@@ -1649,6 +2015,7 @@ static int load_superblock(journal_t *journal)
{
int err;
journal_superblock_t *sb;
+ int num_fc_blocks;
err = journal_get_superblock(journal);
if (err)
@@ -1659,15 +2026,24 @@ static int load_superblock(journal_t *journal)
journal->j_tail_sequence = be32_to_cpu(sb->s_sequence);
journal->j_tail = be32_to_cpu(sb->s_start);
journal->j_first = be32_to_cpu(sb->s_first);
- journal->j_last = be32_to_cpu(sb->s_maxlen);
journal->j_errno = be32_to_cpu(sb->s_errno);
+ journal->j_last = be32_to_cpu(sb->s_maxlen);
+
+ if (jbd2_has_feature_fast_commit(journal)) {
+ journal->j_fc_last = be32_to_cpu(sb->s_maxlen);
+ num_fc_blocks = jbd2_journal_get_num_fc_blks(sb);
+ if (journal->j_last - num_fc_blocks >= JBD2_MIN_JOURNAL_BLOCKS)
+ journal->j_last = journal->j_fc_last - num_fc_blocks;
+ journal->j_fc_first = journal->j_last + 1;
+ journal->j_fc_off = 0;
+ }
return 0;
}
/**
- * int jbd2_journal_load() - Read journal from disk.
+ * jbd2_journal_load() - Read journal from disk.
* @journal: Journal to act on.
*
* Given a journal_t structure which tells us which disk blocks contain
@@ -1737,7 +2113,7 @@ recovery_error:
}
/**
- * void jbd2_journal_destroy() - Release a journal_t structure.
+ * jbd2_journal_destroy() - Release a journal_t structure.
* @journal: Journal to act on.
*
* Release a journal_t structure once it is no longer in use by the
@@ -1781,6 +2157,16 @@ int jbd2_journal_destroy(journal_t *journal)
J_ASSERT(journal->j_checkpoint_transactions == NULL);
spin_unlock(&journal->j_list_lock);
+ /*
+ * OK, all checkpoint transactions have been checked, now check the
+ * write out io error flag and abort the journal if some buffer failed
+ * to write back to the original location, otherwise the filesystem
+ * may become inconsistent.
+ */
+ if (!is_journal_aborted(journal) &&
+ test_bit(JBD2_CHECKPOINT_IO_ERROR, &journal->j_atomic_flags))
+ jbd2_journal_abort(journal, -EIO);
+
if (journal->j_sb_buffer) {
if (!is_journal_aborted(journal)) {
mutex_lock_io(&journal->j_checkpoint_mutex);
@@ -1798,6 +2184,10 @@ int jbd2_journal_destroy(journal_t *journal)
brelse(journal->j_sb_buffer);
}
+ if (journal->j_shrinker.flags & SHRINKER_REGISTERED) {
+ percpu_counter_destroy(&journal->j_checkpoint_jh_count);
+ unregister_shrinker(&journal->j_shrinker);
+ }
if (journal->j_proc_entry)
jbd2_stats_proc_exit(journal);
iput(journal->j_inode);
@@ -1805,6 +2195,7 @@ int jbd2_journal_destroy(journal_t *journal)
jbd2_journal_destroy_revoke(journal);
if (journal->j_chksum_driver)
crypto_free_shash(journal->j_chksum_driver);
+ kfree(journal->j_fc_wbuf);
kfree(journal->j_wbuf);
kfree(journal);
@@ -1813,7 +2204,7 @@ int jbd2_journal_destroy(journal_t *journal)
/**
- *int jbd2_journal_check_used_features () - Check if features specified are used.
+ * jbd2_journal_check_used_features() - Check if features specified are used.
* @journal: Journal to check.
* @compat: bitmask of compatible features
* @ro: bitmask of features that force read-only mount
@@ -1823,7 +2214,7 @@ int jbd2_journal_destroy(journal_t *journal)
* features. Return true (non-zero) if it does.
**/
-int jbd2_journal_check_used_features (journal_t *journal, unsigned long compat,
+int jbd2_journal_check_used_features(journal_t *journal, unsigned long compat,
unsigned long ro, unsigned long incompat)
{
journal_superblock_t *sb;
@@ -1848,7 +2239,7 @@ int jbd2_journal_check_used_features (journal_t *journal, unsigned long compat,
}
/**
- * int jbd2_journal_check_available_features() - Check feature set in journalling layer
+ * jbd2_journal_check_available_features() - Check feature set in journalling layer
* @journal: Journal to check.
* @compat: bitmask of compatible features
* @ro: bitmask of features that force read-only mount
@@ -1858,7 +2249,7 @@ int jbd2_journal_check_used_features (journal_t *journal, unsigned long compat,
* all of a given set of features on this journal. Return true
* (non-zero) if it can. */
-int jbd2_journal_check_available_features (journal_t *journal, unsigned long compat,
+int jbd2_journal_check_available_features(journal_t *journal, unsigned long compat,
unsigned long ro, unsigned long incompat)
{
if (!compat && !ro && !incompat)
@@ -1879,8 +2270,37 @@ int jbd2_journal_check_available_features (journal_t *journal, unsigned long com
return 0;
}
+static int
+jbd2_journal_initialize_fast_commit(journal_t *journal)
+{
+ journal_superblock_t *sb = journal->j_superblock;
+ unsigned long long num_fc_blks;
+
+ num_fc_blks = jbd2_journal_get_num_fc_blks(sb);
+ if (journal->j_last - num_fc_blks < JBD2_MIN_JOURNAL_BLOCKS)
+ return -ENOSPC;
+
+ /* Are we called twice? */
+ WARN_ON(journal->j_fc_wbuf != NULL);
+ journal->j_fc_wbuf = kmalloc_array(num_fc_blks,
+ sizeof(struct buffer_head *), GFP_KERNEL);
+ if (!journal->j_fc_wbuf)
+ return -ENOMEM;
+
+ journal->j_fc_wbufsize = num_fc_blks;
+ journal->j_fc_last = journal->j_last;
+ journal->j_last = journal->j_fc_last - num_fc_blks;
+ journal->j_fc_first = journal->j_last + 1;
+ journal->j_fc_off = 0;
+ journal->j_free = journal->j_last - journal->j_first;
+ journal->j_max_transaction_buffers =
+ jbd2_journal_get_max_txn_bufs(journal);
+
+ return 0;
+}
+
/**
- * int jbd2_journal_set_features () - Mark a given journal feature in the superblock
+ * jbd2_journal_set_features() - Mark a given journal feature in the superblock
* @journal: Journal to act on.
* @compat: bitmask of compatible features
* @ro: bitmask of features that force read-only mount
@@ -1891,7 +2311,7 @@ int jbd2_journal_check_available_features (journal_t *journal, unsigned long com
*
*/
-int jbd2_journal_set_features (journal_t *journal, unsigned long compat,
+int jbd2_journal_set_features(journal_t *journal, unsigned long compat,
unsigned long ro, unsigned long incompat)
{
#define INCOMPAT_FEATURE_ON(f) \
@@ -1917,11 +2337,18 @@ int jbd2_journal_set_features (journal_t *journal, unsigned long compat,
compat & JBD2_FEATURE_COMPAT_CHECKSUM)
compat &= ~JBD2_FEATURE_COMPAT_CHECKSUM;
- jbd_debug(1, "Setting new features 0x%lx/0x%lx/0x%lx\n",
+ jbd2_debug(1, "Setting new features 0x%lx/0x%lx/0x%lx\n",
compat, ro, incompat);
sb = journal->j_superblock;
+ if (incompat & JBD2_FEATURE_INCOMPAT_FAST_COMMIT) {
+ if (jbd2_journal_initialize_fast_commit(journal)) {
+ pr_err("JBD2: Cannot enable fast commits.\n");
+ return 0;
+ }
+ }
+
/* Load the checksum driver if necessary */
if ((journal->j_chksum_driver == NULL) &&
INCOMPAT_FEATURE_ON(JBD2_FEATURE_INCOMPAT_CSUM_V3)) {
@@ -1964,7 +2391,7 @@ int jbd2_journal_set_features (journal_t *journal, unsigned long compat,
}
/*
- * jbd2_journal_clear_features () - Clear a given journal feature in the
+ * jbd2_journal_clear_features() - Clear a given journal feature in the
* superblock
* @journal: Journal to act on.
* @compat: bitmask of compatible features
@@ -1979,7 +2406,7 @@ void jbd2_journal_clear_features(journal_t *journal, unsigned long compat,
{
journal_superblock_t *sb;
- jbd_debug(1, "Clear features 0x%lx/0x%lx/0x%lx\n",
+ jbd2_debug(1, "Clear features 0x%lx/0x%lx/0x%lx\n",
compat, ro, incompat);
sb = journal->j_superblock;
@@ -1993,15 +2420,20 @@ void jbd2_journal_clear_features(journal_t *journal, unsigned long compat,
EXPORT_SYMBOL(jbd2_journal_clear_features);
/**
- * int jbd2_journal_flush () - Flush journal
+ * jbd2_journal_flush() - Flush journal
* @journal: Journal to act on.
+ * @flags: optional operation on the journal blocks after the flush (see below)
*
* Flush all data for a given journal to disk and empty the journal.
* Filesystems can use this when remounting readonly to ensure that
- * recovery does not need to happen on remount.
+ * recovery does not need to happen on remount. Optionally, a discard or zeroout
+ * can be issued on the journal blocks after flushing.
+ *
+ * flags:
+ * JBD2_JOURNAL_FLUSH_DISCARD: issues discards for the journal blocks
+ * JBD2_JOURNAL_FLUSH_ZEROOUT: issues zeroouts for the journal blocks
*/
-
-int jbd2_journal_flush(journal_t *journal)
+int jbd2_journal_flush(journal_t *journal, unsigned int flags)
{
int err = 0;
transaction_t *transaction = NULL;
@@ -2055,6 +2487,10 @@ int jbd2_journal_flush(journal_t *journal)
* commits of data to the journal will restore the current
* s_start value. */
jbd2_mark_journal_empty(journal, REQ_SYNC | REQ_FUA);
+
+ if (flags)
+ err = __jbd2_journal_erase(journal, flags);
+
mutex_unlock(&journal->j_checkpoint_mutex);
write_lock(&journal->j_state_lock);
J_ASSERT(!journal->j_running_transaction);
@@ -2068,7 +2504,7 @@ out:
}
/**
- * int jbd2_journal_wipe() - Wipe journal contents
+ * jbd2_journal_wipe() - Wipe journal contents
* @journal: Journal to act on.
* @write: flag (see below)
*
@@ -2109,7 +2545,7 @@ int jbd2_journal_wipe(journal_t *journal, int write)
}
/**
- * void jbd2_journal_abort () - Shutdown the journal immediately.
+ * jbd2_journal_abort () - Shutdown the journal immediately.
* @journal: the journal to shutdown.
* @errno: an error number to record in the journal indicating
* the reason for the shutdown.
@@ -2154,6 +2590,13 @@ void jbd2_journal_abort(journal_t *journal, int errno)
transaction_t *transaction;
/*
+ * Lock the aborting procedure until everything is done, this avoid
+ * races between filesystem's error handling flow (e.g. ext4_abort()),
+ * ensure panic after the error info is written into journal's
+ * superblock.
+ */
+ mutex_lock(&journal->j_abort_mutex);
+ /*
* ESHUTDOWN always takes precedence because a file system check
* caused by any other journal abort error is not required after
* a shutdown triggered.
@@ -2167,6 +2610,7 @@ void jbd2_journal_abort(journal_t *journal, int errno)
journal->j_errno = errno;
jbd2_journal_update_sb_errno(journal);
}
+ mutex_unlock(&journal->j_abort_mutex);
return;
}
@@ -2188,14 +2632,11 @@ void jbd2_journal_abort(journal_t *journal, int errno)
* layer could realise that a filesystem check is needed.
*/
jbd2_journal_update_sb_errno(journal);
-
- write_lock(&journal->j_state_lock);
- journal->j_flags |= JBD2_REC_ERR;
- write_unlock(&journal->j_state_lock);
+ mutex_unlock(&journal->j_abort_mutex);
}
/**
- * int jbd2_journal_errno () - returns the journal's error state.
+ * jbd2_journal_errno() - returns the journal's error state.
* @journal: journal to examine.
*
* This is the errno number set with jbd2_journal_abort(), the last
@@ -2219,7 +2660,7 @@ int jbd2_journal_errno(journal_t *journal)
}
/**
- * int jbd2_journal_clear_err () - clears the journal's error state
+ * jbd2_journal_clear_err() - clears the journal's error state
* @journal: journal to act on.
*
* An error must be cleared or acked to take a FS out of readonly
@@ -2239,7 +2680,7 @@ int jbd2_journal_clear_err(journal_t *journal)
}
/**
- * void jbd2_journal_ack_err() - Ack journal err.
+ * jbd2_journal_ack_err() - Ack journal err.
* @journal: journal to act on.
*
* An error must be cleared or acked to take a FS out of readonly
@@ -2422,7 +2863,7 @@ static struct journal_head *journal_alloc_journal_head(void)
#endif
ret = kmem_cache_zalloc(jbd2_journal_head_cache, GFP_NOFS);
if (!ret) {
- jbd_debug(1, "out of memory for journal_head\n");
+ jbd2_debug(1, "out of memory for journal_head\n");
pr_notice_ratelimited("ENOMEM in %s, retrying.\n", __func__);
ret = kmem_cache_zalloc(jbd2_journal_head_cache,
GFP_NOFS | __GFP_NOFAIL);
@@ -2535,6 +2976,7 @@ struct journal_head *jbd2_journal_grab_journal_head(struct buffer_head *bh)
jbd_unlock_bh_journal_head(bh);
return jh;
}
+EXPORT_SYMBOL(jbd2_journal_grab_journal_head);
static void __journal_remove_journal_head(struct buffer_head *bh)
{
@@ -2587,6 +3029,7 @@ void jbd2_journal_put_journal_head(struct journal_head *jh)
jbd_unlock_bh_journal_head(bh);
}
}
+EXPORT_SYMBOL(jbd2_journal_put_journal_head);
/*
* Initialize jbd inode head
diff --git a/fs/jbd2/recovery.c b/fs/jbd2/recovery.c
index a4967b27ffb6..8286a9ec122f 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 *,
@@ -75,8 +74,8 @@ static int do_readahead(journal_t *journal, unsigned int start)
/* Do up to 128K of readahead */
max = start + (128 * 1024 / journal->j_blocksize);
- if (max > journal->j_maxlen)
- max = journal->j_maxlen;
+ if (max > journal->j_total_len)
+ max = journal->j_total_len;
/* Do the readahead itself. We'll submit MAXBUF buffer_heads at
* a time to the block device IO layer. */
@@ -101,7 +100,7 @@ static int do_readahead(journal_t *journal, unsigned int start)
if (!buffer_uptodate(bh) && !buffer_locked(bh)) {
bufs[nbufs++] = bh;
if (nbufs == MAXBUF) {
- ll_rw_block(REQ_OP_READ, 0, nbufs, bufs);
+ bh_readahead_batch(nbufs, bufs, 0);
journal_brelse_array(bufs, nbufs);
nbufs = 0;
}
@@ -110,7 +109,7 @@ static int do_readahead(journal_t *journal, unsigned int start)
}
if (nbufs)
- ll_rw_block(REQ_OP_READ, 0, nbufs, bufs);
+ bh_readahead_batch(nbufs, bufs, 0);
err = 0;
failed:
@@ -135,7 +134,7 @@ static int jread(struct buffer_head **bhp, journal_t *journal,
*bhp = NULL;
- if (offset >= journal->j_maxlen) {
+ if (offset >= journal->j_total_len) {
printk(KERN_ERR "JBD2: corrupted journal superblock\n");
return -EFSCORRUPTED;
}
@@ -153,9 +152,14 @@ static int jread(struct buffer_head **bhp, journal_t *journal,
return -ENOMEM;
if (!buffer_uptodate(bh)) {
- /* If this is a brand new buffer, start readahead.
- Otherwise, we assume we are already reading it. */
- if (!buffer_req(bh))
+ /*
+ * If this is a brand new buffer, start readahead.
+ * Otherwise, we assume we are already reading it.
+ */
+ bool need_readahead = !buffer_req(bh);
+
+ bh_read_nowait(bh, 0);
+ if (need_readahead)
do_readahead(journal, offset);
wait_on_buffer(bh);
}
@@ -180,8 +184,8 @@ static int jbd2_descriptor_block_csum_verify(journal_t *j, void *buf)
if (!jbd2_journal_has_csum_v2or3(j))
return 1;
- tail = (struct jbd2_journal_block_tail *)(buf + j->j_blocksize -
- sizeof(struct jbd2_journal_block_tail));
+ tail = (struct jbd2_journal_block_tail *)((char *)buf +
+ j->j_blocksize - sizeof(struct jbd2_journal_block_tail));
provided = tail->t_checksum;
tail->t_checksum = 0;
calculated = jbd2_chksum(j, j->j_csum_seed, buf, j->j_blocksize);
@@ -197,7 +201,7 @@ static int jbd2_descriptor_block_csum_verify(journal_t *j, void *buf)
static int count_tags(journal_t *journal, struct buffer_head *bh)
{
char * tagp;
- journal_block_tag_t * tag;
+ journal_block_tag_t tag;
int nr = 0, size = journal->j_blocksize;
int tag_bytes = journal_tag_bytes(journal);
@@ -207,14 +211,14 @@ static int count_tags(journal_t *journal, struct buffer_head *bh)
tagp = &bh->b_data[sizeof(journal_header_t)];
while ((tagp - bh->b_data + tag_bytes) <= size) {
- tag = (journal_block_tag_t *) tagp;
+ memcpy(&tag, tagp, sizeof(tag));
nr++;
tagp += tag_bytes;
- if (!(tag->t_flags & cpu_to_be16(JBD2_FLAG_SAME_UUID)))
+ if (!(tag.t_flags & cpu_to_be16(JBD2_FLAG_SAME_UUID)))
tagp += 16;
- if (tag->t_flags & cpu_to_be16(JBD2_FLAG_LAST_TAG))
+ if (tag.t_flags & cpu_to_be16(JBD2_FLAG_LAST_TAG))
break;
}
@@ -225,10 +229,51 @@ 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_fc_last : (journal)->j_last; \
+ \
+ if (var >= _wrap_last) \
+ var -= (_wrap_last - (journal)->j_first); \
} while (0)
+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;
+ int err = 0;
+
+ next_fc_block = journal->j_fc_first;
+ if (!journal->j_fc_replay_callback)
+ return 0;
+
+ while (next_fc_block <= journal->j_fc_last) {
+ jbd2_debug(3, "Fast commit replay: next block %ld\n",
+ next_fc_block);
+ err = jread(&bh, journal, next_fc_block);
+ if (err) {
+ jbd2_debug(3, "Fast commit replay: read error\n");
+ break;
+ }
+
+ err = journal->j_fc_replay_callback(journal, bh, pass,
+ next_fc_block - journal->j_fc_first,
+ expected_commit_id);
+ brelse(bh);
+ next_fc_block++;
+ if (err < 0 || err == JBD2_FC_REPLAY_STOP)
+ break;
+ err = 0;
+ }
+
+ if (err)
+ jbd2_debug(3, "Fast commit replay failed, err = %d\n", err);
+
+ return err;
+}
+
/**
* jbd2_journal_recover - recovers a on-disk journal
* @journal: the journal to recover
@@ -258,7 +303,7 @@ int jbd2_journal_recover(journal_t *journal)
*/
if (!sb->s_start) {
- jbd_debug(1, "No recovery required, last transaction %d\n",
+ jbd2_debug(1, "No recovery required, last transaction %d\n",
be32_to_cpu(sb->s_sequence));
journal->j_transaction_sequence = be32_to_cpu(sb->s_sequence) + 1;
return 0;
@@ -270,10 +315,10 @@ int jbd2_journal_recover(journal_t *journal)
if (!err)
err = do_one_pass(journal, &info, PASS_REPLAY);
- jbd_debug(1, "JBD2: recovery, exit status %d, "
+ jbd2_debug(1, "JBD2: recovery, exit status %d, "
"recovered transactions %u to %u\n",
err, info.start_transaction, info.end_transaction);
- jbd_debug(1, "JBD2: Replayed %d and revoked %d/%d blocks\n",
+ jbd2_debug(1, "JBD2: Replayed %d and revoked %d/%d blocks\n",
info.nr_replays, info.nr_revoke_hits, info.nr_revokes);
/* Restart the log at the next transaction ID, thus invalidating
@@ -286,7 +331,7 @@ int jbd2_journal_recover(journal_t *journal)
err = err2;
/* Make sure all replayed data is on permanent storage */
if (journal->j_flags & JBD2_BARRIER) {
- err2 = blkdev_issue_flush(journal->j_fs_dev, GFP_KERNEL, NULL);
+ err2 = blkdev_issue_flush(journal->j_fs_dev);
if (!err)
err = err2;
}
@@ -323,7 +368,7 @@ int jbd2_journal_skip_recovery(journal_t *journal)
#ifdef CONFIG_JBD2_DEBUG
int dropped = info.end_transaction -
be32_to_cpu(journal->j_superblock->s_sequence);
- jbd_debug(1,
+ jbd2_debug(1,
"JBD2: ignoring %d transaction%s from the journal.\n",
dropped, (dropped == 1) ? "" : "s");
#endif
@@ -394,9 +439,9 @@ static int jbd2_commit_block_csum_verify(journal_t *j, void *buf)
}
static int jbd2_block_tag_csum_verify(journal_t *j, journal_block_tag_t *tag,
+ journal_block_tag3_t *tag3,
void *buf, __u32 sequence)
{
- journal_block_tag3_t *tag3 = (journal_block_tag3_t *)tag;
__u32 csum32;
__be32 seq;
@@ -428,6 +473,8 @@ static int do_one_pass(journal_t *journal,
__u32 crc32_sum = ~0; /* Transactional Checksums */
int descr_csum_size = 0;
int block_error = 0;
+ bool need_check_commit_time = false;
+ __u64 last_trans_commit_time = 0, commit_time;
/*
* First thing is to establish what we expect to find in the log
@@ -443,7 +490,7 @@ static int do_one_pass(journal_t *journal,
if (pass == PASS_SCAN)
info->start_transaction = first_commit_ID;
- jbd_debug(1, "Starting recovery pass %d\n", pass);
+ jbd2_debug(1, "Starting recovery pass %d\n", pass);
/*
* Now we walk through the log, transaction by transaction,
@@ -455,7 +502,7 @@ static int do_one_pass(journal_t *journal,
while (1) {
int flags;
char * tagp;
- journal_block_tag_t * tag;
+ journal_block_tag_t tag;
struct buffer_head * obh;
struct buffer_head * nbh;
@@ -469,14 +516,16 @@ static int do_one_pass(journal_t *journal,
if (tid_geq(next_commit_ID, info->end_transaction))
break;
- jbd_debug(2, "Scanning for sequence ID %u at %lu/%lu\n",
- next_commit_ID, next_log_block, journal->j_last);
+ jbd2_debug(2, "Scanning for sequence ID %u at %lu/%lu\n",
+ next_commit_ID, next_log_block,
+ jbd2_has_feature_fast_commit(journal) ?
+ journal->j_fc_last : journal->j_last);
/* Skip over each chunk of the transaction looking
* either the next descriptor block or the final commit
* record. */
- jbd_debug(3, "JBD2: checking block %ld\n", next_log_block);
+ jbd2_debug(3, "JBD2: checking block %ld\n", next_log_block);
err = jread(&bh, journal, next_log_block);
if (err)
goto failed;
@@ -499,7 +548,7 @@ static int do_one_pass(journal_t *journal,
blocktype = be32_to_cpu(tmp->h_blocktype);
sequence = be32_to_cpu(tmp->h_sequence);
- jbd_debug(3, "Found magic %d, sequence %d\n",
+ jbd2_debug(3, "Found magic %d, sequence %d\n",
blocktype, sequence);
if (sequence != next_commit_ID) {
@@ -520,12 +569,21 @@ static int do_one_pass(journal_t *journal,
if (descr_csum_size > 0 &&
!jbd2_descriptor_block_csum_verify(journal,
bh->b_data)) {
- printk(KERN_ERR "JBD2: Invalid checksum "
- "recovering block %lu in log\n",
- next_log_block);
- err = -EFSBADCRC;
- brelse(bh);
- goto failed;
+ /*
+ * PASS_SCAN can see stale blocks due to lazy
+ * journal init. Don't error out on those yet.
+ */
+ if (pass != PASS_SCAN) {
+ pr_err("JBD2: Invalid checksum recovering block %lu in log\n",
+ next_log_block);
+ err = -EFSBADCRC;
+ brelse(bh);
+ goto failed;
+ }
+ need_check_commit_time = true;
+ jbd2_debug(1,
+ "invalid descriptor block found in %lu\n",
+ next_log_block);
}
/* If it is a valid descriptor block, replay it
@@ -535,6 +593,7 @@ static int do_one_pass(journal_t *journal,
if (pass != PASS_REPLAY) {
if (pass == PASS_SCAN &&
jbd2_has_feature_checksum(journal) &&
+ !need_check_commit_time &&
!info->end_transaction) {
if (calc_chksums(journal, bh,
&next_log_block,
@@ -560,8 +619,8 @@ static int do_one_pass(journal_t *journal,
<= journal->j_blocksize - descr_csum_size) {
unsigned long io_block;
- tag = (journal_block_tag_t *) tagp;
- flags = be16_to_cpu(tag->t_flags);
+ memcpy(&tag, tagp, sizeof(tag));
+ flags = be16_to_cpu(tag.t_flags);
io_block = next_log_block++;
wrap(journal, next_log_block);
@@ -579,7 +638,7 @@ static int do_one_pass(journal_t *journal,
J_ASSERT(obh != NULL);
blocknr = read_tag_block(journal,
- tag);
+ &tag);
/* If the block has been
* revoked, then we're all done
@@ -594,8 +653,8 @@ static int do_one_pass(journal_t *journal,
/* Look for block corruption */
if (!jbd2_block_tag_csum_verify(
- journal, tag, obh->b_data,
- be32_to_cpu(tmp->h_sequence))) {
+ journal, &tag, (journal_block_tag3_t *)tagp,
+ obh->b_data, be32_to_cpu(tmp->h_sequence))) {
brelse(obh);
success = -EFSBADCRC;
printk(KERN_ERR "JBD2: Invalid "
@@ -634,7 +693,6 @@ static int do_one_pass(journal_t *journal,
mark_buffer_dirty(nbh);
BUFFER_TRACE(nbh, "marking uptodate");
++info->nr_replays;
- /* ll_rw_block(WRITE, 1, &nbh); */
unlock_buffer(nbh);
brelse(obh);
brelse(nbh);
@@ -683,21 +741,47 @@ static int do_one_pass(journal_t *journal,
* mentioned conditions. Hence assume
* "Interrupted Commit".)
*/
+ commit_time = be64_to_cpu(
+ ((struct commit_header *)bh->b_data)->h_commit_sec);
+ /*
+ * If need_check_commit_time is set, it means we are in
+ * PASS_SCAN and csum verify failed before. If
+ * commit_time is increasing, it's the same journal,
+ * otherwise it is stale journal block, just end this
+ * recovery.
+ */
+ if (need_check_commit_time) {
+ if (commit_time >= last_trans_commit_time) {
+ pr_err("JBD2: Invalid checksum found in transaction %u\n",
+ next_commit_ID);
+ err = -EFSBADCRC;
+ brelse(bh);
+ goto failed;
+ }
+ ignore_crc_mismatch:
+ /*
+ * It likely does not belong to same journal,
+ * just end this recovery with success.
+ */
+ jbd2_debug(1, "JBD2: Invalid checksum ignored in transaction %u, likely stale data\n",
+ next_commit_ID);
+ brelse(bh);
+ goto done;
+ }
- /* Found an expected commit block: if checksums
- * are present verify them in PASS_SCAN; else not
+ /*
+ * Found an expected commit block: if checksums
+ * are present, verify them in PASS_SCAN; else not
* much to do other than move on to the next sequence
- * number. */
+ * number.
+ */
if (pass == PASS_SCAN &&
jbd2_has_feature_checksum(journal)) {
- int chksum_err, chksum_seen;
struct commit_header *cbh =
(struct commit_header *)bh->b_data;
unsigned found_chksum =
be32_to_cpu(cbh->h_chksum[0]);
- chksum_err = chksum_seen = 0;
-
if (info->end_transaction) {
journal->j_failed_commit =
info->end_transaction;
@@ -705,42 +789,25 @@ static int do_one_pass(journal_t *journal,
break;
}
- if (crc32_sum == found_chksum &&
- cbh->h_chksum_type == JBD2_CRC32_CHKSUM &&
- cbh->h_chksum_size ==
- JBD2_CRC32_CHKSUM_SIZE)
- chksum_seen = 1;
- else if (!(cbh->h_chksum_type == 0 &&
- cbh->h_chksum_size == 0 &&
- found_chksum == 0 &&
- !chksum_seen))
- /*
- * If fs is mounted using an old kernel and then
- * kernel with journal_chksum is used then we
- * get a situation where the journal flag has
- * checksum flag set but checksums are not
- * present i.e chksum = 0, in the individual
- * commit blocks.
- * Hence to avoid checksum failures, in this
- * situation, this extra check is added.
- */
- chksum_err = 1;
-
- if (chksum_err) {
- info->end_transaction = next_commit_ID;
+ /* Neither checksum match nor unused? */
+ if (!((crc32_sum == found_chksum &&
+ cbh->h_chksum_type ==
+ JBD2_CRC32_CHKSUM &&
+ cbh->h_chksum_size ==
+ JBD2_CRC32_CHKSUM_SIZE) ||
+ (cbh->h_chksum_type == 0 &&
+ cbh->h_chksum_size == 0 &&
+ found_chksum == 0)))
+ goto chksum_error;
- if (!jbd2_has_feature_async_commit(journal)) {
- journal->j_failed_commit =
- next_commit_ID;
- brelse(bh);
- break;
- }
- }
crc32_sum = ~0;
}
if (pass == PASS_SCAN &&
!jbd2_commit_block_csum_verify(journal,
bh->b_data)) {
+ chksum_error:
+ if (commit_time < last_trans_commit_time)
+ goto ignore_crc_mismatch;
info->end_transaction = next_commit_ID;
if (!jbd2_has_feature_async_commit(journal)) {
@@ -750,11 +817,24 @@ static int do_one_pass(journal_t *journal,
break;
}
}
+ if (pass == PASS_SCAN)
+ last_trans_commit_time = commit_time;
brelse(bh);
next_commit_ID++;
continue;
case JBD2_REVOKE_BLOCK:
+ /*
+ * Check revoke block crc in pass_scan, if csum verify
+ * failed, check commit block time later.
+ */
+ if (pass == PASS_SCAN &&
+ !jbd2_descriptor_block_csum_verify(journal,
+ bh->b_data)) {
+ jbd2_debug(1, "JBD2: invalid revoke block found in %lu\n",
+ next_log_block);
+ need_check_commit_time = true;
+ }
/* If we aren't in the REVOKE pass, then we can
* just skip over this block. */
if (pass != PASS_REVOKE) {
@@ -770,7 +850,7 @@ static int do_one_pass(journal_t *journal,
continue;
default:
- jbd_debug(3, "Unrecognised magic %d, end of scan.\n",
+ jbd2_debug(3, "Unrecognised magic %d, end of scan.\n",
blocktype);
brelse(bh);
goto done;
@@ -799,6 +879,13 @@ static int do_one_pass(journal_t *journal,
success = -EIO;
}
}
+
+ if (jbd2_has_feature_fast_commit(journal) && pass != PASS_REVOKE) {
+ err = fc_do_one_pass(journal, info, pass);
+ if (err)
+ success = err;
+ }
+
if (block_error && success == 0)
success = -EIO;
return success;
@@ -814,7 +901,7 @@ static int scan_revoke_records(journal_t *journal, struct buffer_head *bh,
{
jbd2_journal_revoke_header_t *header;
int offset, max;
- int csum_size = 0;
+ unsigned csum_size = 0;
__u32 rcount;
int record_len = 4;
@@ -822,9 +909,6 @@ static int scan_revoke_records(journal_t *journal, struct buffer_head *bh,
offset = sizeof(jbd2_journal_revoke_header_t);
rcount = be32_to_cpu(header->r_count);
- if (!jbd2_descriptor_block_csum_verify(journal, header))
- return -EFSBADCRC;
-
if (jbd2_journal_has_csum_v2or3(journal))
csum_size = sizeof(struct jbd2_journal_block_tail);
if (rcount > journal->j_blocksize - csum_size)
diff --git a/fs/jbd2/revoke.c b/fs/jbd2/revoke.c
index fa608788b93d..4556e4689024 100644
--- a/fs/jbd2/revoke.c
+++ b/fs/jbd2/revoke.c
@@ -398,7 +398,7 @@ int jbd2_journal_revoke(handle_t *handle, unsigned long long blocknr,
}
handle->h_revoke_credits--;
- jbd_debug(2, "insert revoke for block %llu, bh_in=%p\n",blocknr, bh_in);
+ jbd2_debug(2, "insert revoke for block %llu, bh_in=%p\n",blocknr, bh_in);
err = insert_revoke_hash(journal, blocknr,
handle->h_transaction->t_tid);
BUFFER_TRACE(bh_in, "exit");
@@ -428,7 +428,7 @@ int jbd2_journal_cancel_revoke(handle_t *handle, struct journal_head *jh)
int did_revoke = 0; /* akpm: debug */
struct buffer_head *bh = jh2bh(jh);
- jbd_debug(4, "journal_head %p, cancelling revoke\n", jh);
+ jbd2_debug(4, "journal_head %p, cancelling revoke\n", jh);
/* Is the existing Revoke bit valid? If so, we trust it, and
* only perform the full cancel if the revoke bit is set. If
@@ -444,7 +444,7 @@ int jbd2_journal_cancel_revoke(handle_t *handle, struct journal_head *jh)
if (need_cancel) {
record = find_revoke_record(journal, bh->b_blocknr);
if (record) {
- jbd_debug(4, "cancelled existing revoke on "
+ jbd2_debug(4, "cancelled existing revoke on "
"blocknr %llu\n", (unsigned long long)bh->b_blocknr);
spin_lock(&journal->j_revoke_lock);
list_del(&record->hash);
@@ -560,7 +560,7 @@ void jbd2_journal_write_revoke_records(transaction_t *transaction,
}
if (descriptor)
flush_descriptor(journal, descriptor, offset);
- jbd_debug(1, "Wrote %d revoke records\n", count);
+ jbd2_debug(1, "Wrote %d revoke records\n", count);
}
/*
diff --git a/fs/jbd2/transaction.c b/fs/jbd2/transaction.c
index 3dccc23cf010..6a404ac1c178 100644
--- a/fs/jbd2/transaction.c
+++ b/fs/jbd2/transaction.c
@@ -107,7 +107,6 @@ static void jbd2_get_transaction(journal_t *journal,
transaction->t_start_time = ktime_get();
transaction->t_tid = journal->j_transaction_sequence++;
transaction->t_expires = jiffies + journal->j_commit_interval;
- spin_lock_init(&transaction->t_handle_lock);
atomic_set(&transaction->t_updates, 0);
atomic_set(&transaction->t_outstanding_credits,
jbd2_descriptor_blocks_per_trans(journal) +
@@ -139,26 +138,22 @@ static void jbd2_get_transaction(journal_t *journal,
/*
* Update transaction's maximum wait time, if debugging is enabled.
*
- * In order for t_max_wait to be reliable, it must be protected by a
- * lock. But doing so will mean that start_this_handle() can not be
- * run in parallel on SMP systems, which limits our scalability. So
- * unless debugging is enabled, we no longer update t_max_wait, which
- * means that maximum wait time reported by the jbd2_run_stats
- * tracepoint will always be zero.
+ * t_max_wait is carefully updated here with use of atomic compare exchange.
+ * Note that there could be multiplre threads trying to do this simultaneously
+ * hence using cmpxchg to avoid any use of locks in this case.
+ * With this t_max_wait can be updated w/o enabling jbd2_journal_enable_debug.
*/
static inline void update_t_max_wait(transaction_t *transaction,
unsigned long ts)
{
-#ifdef CONFIG_JBD2_DEBUG
- if (jbd2_journal_enable_debug &&
- time_after(transaction->t_start, ts)) {
- ts = jbd2_time_diff(ts, transaction->t_start);
- spin_lock(&transaction->t_handle_lock);
- if (ts > transaction->t_max_wait)
- transaction->t_max_wait = ts;
- spin_unlock(&transaction->t_handle_lock);
+ unsigned long oldts, newts;
+
+ if (time_after(transaction->t_start, ts)) {
+ newts = jbd2_time_diff(ts, transaction->t_start);
+ oldts = READ_ONCE(transaction->t_max_wait);
+ while (oldts < newts)
+ oldts = cmpxchg(&transaction->t_max_wait, oldts, newts);
}
-#endif
}
/*
@@ -173,7 +168,7 @@ static void wait_transaction_locked(journal_t *journal)
int need_to_start;
tid_t tid = journal->j_running_transaction->t_tid;
- prepare_to_wait(&journal->j_wait_transaction_locked, &wait,
+ prepare_to_wait_exclusive(&journal->j_wait_transaction_locked, &wait,
TASK_UNINTERRUPTIBLE);
need_to_start = !tid_geq(journal->j_commit_request, tid);
read_unlock(&journal->j_state_lock);
@@ -195,9 +190,11 @@ static void wait_transaction_switching(journal_t *journal)
DEFINE_WAIT(wait);
if (WARN_ON(!journal->j_running_transaction ||
- journal->j_running_transaction->t_state != T_SWITCH))
+ journal->j_running_transaction->t_state != T_SWITCH)) {
+ read_unlock(&journal->j_state_lock);
return;
- prepare_to_wait(&journal->j_wait_transaction_locked, &wait,
+ }
+ prepare_to_wait_exclusive(&journal->j_wait_transaction_locked, &wait,
TASK_UNINTERRUPTIBLE);
read_unlock(&journal->j_state_lock);
/*
@@ -221,9 +218,15 @@ static void sub_reserved_credits(journal_t *journal, int blocks)
* with j_state_lock held for reading. Returns 0 if handle joined the running
* transaction. Returns 1 if we had to wait, j_state_lock is dropped, and
* caller must retry.
+ *
+ * Note: because j_state_lock may be dropped depending on the return
+ * value, we need to fake out sparse so ti doesn't complain about a
+ * locking imbalance. Callers of add_transaction_credits will need to
+ * make a similar accomodation.
*/
static int add_transaction_credits(journal_t *journal, int blocks,
int rsv_blocks)
+__must_hold(&journal->j_state_lock)
{
transaction_t *t = journal->j_running_transaction;
int needed;
@@ -236,6 +239,7 @@ static int add_transaction_credits(journal_t *journal, int blocks,
if (t->t_state != T_RUNNING) {
WARN_ON_ONCE(t->t_state >= T_FLUSH);
wait_transaction_locked(journal);
+ __acquire(&journal->j_state_lock); /* fake out sparse */
return 1;
}
@@ -264,10 +268,12 @@ static int add_transaction_credits(journal_t *journal, int blocks,
wait_event(journal->j_wait_reserved,
atomic_read(&journal->j_reserved_credits) + total <=
journal->j_max_transaction_buffers);
+ __acquire(&journal->j_state_lock); /* fake out sparse */
return 1;
}
wait_transaction_locked(journal);
+ __acquire(&journal->j_state_lock); /* fake out sparse */
return 1;
}
@@ -291,6 +297,7 @@ static int add_transaction_credits(journal_t *journal, int blocks,
journal->j_max_transaction_buffers)
__jbd2_log_wait_for_space(journal);
write_unlock(&journal->j_state_lock);
+ __acquire(&journal->j_state_lock); /* fake out sparse */
return 1;
}
@@ -308,6 +315,7 @@ static int add_transaction_credits(journal_t *journal, int blocks,
wait_event(journal->j_wait_reserved,
atomic_read(&journal->j_reserved_credits) + rsv_blocks
<= journal->j_max_transaction_buffers / 2);
+ __acquire(&journal->j_state_lock); /* fake out sparse */
return 1;
}
return 0;
@@ -347,7 +355,12 @@ static int start_this_handle(journal_t *journal, handle_t *handle,
}
alloc_transaction:
- if (!journal->j_running_transaction) {
+ /*
+ * This check is racy but it is just an optimization of allocating new
+ * transaction early if there are high chances we'll need it. If we
+ * guess wrong, we'll retry or free unused transaction.
+ */
+ if (!data_race(journal->j_running_transaction)) {
/*
* If __GFP_FS is not present, then we may be being called from
* inside the fs writeback layer, so we MUST NOT fail.
@@ -360,7 +373,7 @@ alloc_transaction:
return -ENOMEM;
}
- jbd_debug(3, "New handle %p going live.\n", handle);
+ jbd2_debug(3, "New handle %p going live.\n", handle);
/*
* We need to hold j_state_lock until t_updates has been incremented,
@@ -406,8 +419,14 @@ repeat:
if (!handle->h_reserved) {
/* We may have dropped j_state_lock - restart in that case */
- if (add_transaction_credits(journal, blocks, rsv_blocks))
+ if (add_transaction_credits(journal, blocks, rsv_blocks)) {
+ /*
+ * add_transaction_credits releases
+ * j_state_lock on a non-zero return
+ */
+ __release(&journal->j_state_lock);
goto repeat;
+ }
} else {
/*
* We have handle reserved so we are allowed to join T_LOCKED
@@ -425,7 +444,7 @@ repeat:
}
/* OK, account for the buffers that this operation expects to
- * use and add the handle to the running transaction.
+ * use and add the handle to the running transaction.
*/
update_t_max_wait(transaction, ts);
handle->h_transaction = transaction;
@@ -434,7 +453,7 @@ repeat:
handle->h_start_jiffies = jiffies;
atomic_inc(&transaction->t_updates);
atomic_inc(&transaction->t_handle_count);
- jbd_debug(4, "Handle %p given %d credits (total %d, free %lu)\n",
+ jbd2_debug(4, "Handle %p given %d credits (total %d, free %lu)\n",
handle, blocks,
atomic_read(&transaction->t_outstanding_credits),
jbd2_log_space_left(journal));
@@ -517,7 +536,7 @@ EXPORT_SYMBOL(jbd2__journal_start);
/**
- * handle_t *jbd2_journal_start() - Obtain a new handle.
+ * jbd2_journal_start() - Obtain a new handle.
* @journal: Journal to start transaction on.
* @nblocks: number of block buffer we might modify
*
@@ -541,23 +560,30 @@ handle_t *jbd2_journal_start(journal_t *journal, int nblocks)
}
EXPORT_SYMBOL(jbd2_journal_start);
-static void __jbd2_journal_unreserve_handle(handle_t *handle)
+static void __jbd2_journal_unreserve_handle(handle_t *handle, transaction_t *t)
{
journal_t *journal = handle->h_journal;
WARN_ON(!handle->h_reserved);
sub_reserved_credits(journal, handle->h_total_credits);
+ if (t)
+ atomic_sub(handle->h_total_credits, &t->t_outstanding_credits);
}
void jbd2_journal_free_reserved(handle_t *handle)
{
- __jbd2_journal_unreserve_handle(handle);
+ journal_t *journal = handle->h_journal;
+
+ /* Get j_state_lock to pin running transaction if it exists */
+ read_lock(&journal->j_state_lock);
+ __jbd2_journal_unreserve_handle(handle, journal->j_running_transaction);
+ read_unlock(&journal->j_state_lock);
jbd2_free_handle(handle);
}
EXPORT_SYMBOL(jbd2_journal_free_reserved);
/**
- * int jbd2_journal_start_reserved() - start reserved handle
+ * jbd2_journal_start_reserved() - start reserved handle
* @handle: handle to start
* @type: for handle statistics
* @line_no: for handle statistics
@@ -611,7 +637,7 @@ int jbd2_journal_start_reserved(handle_t *handle, unsigned int type,
EXPORT_SYMBOL(jbd2_journal_start_reserved);
/**
- * int jbd2_journal_extend() - extend buffer credits.
+ * jbd2_journal_extend() - extend buffer credits.
* @handle: handle to 'extend'
* @nblocks: nr blocks to try to extend by.
* @revoke_records: number of revoke records to try to extend by.
@@ -648,7 +674,7 @@ int jbd2_journal_extend(handle_t *handle, int nblocks, int revoke_records)
/* Don't extend a locked-down transaction! */
if (transaction->t_state != T_RUNNING) {
- jbd_debug(3, "denied handle %p %d blocks: "
+ jbd2_debug(3, "denied handle %p %d blocks: "
"transaction not running\n", handle, nblocks);
goto error_out;
}
@@ -659,15 +685,14 @@ int jbd2_journal_extend(handle_t *handle, int nblocks, int revoke_records)
DIV_ROUND_UP(
handle->h_revoke_credits_requested,
journal->j_revoke_records_per_block);
- spin_lock(&transaction->t_handle_lock);
wanted = atomic_add_return(nblocks,
&transaction->t_outstanding_credits);
if (wanted > journal->j_max_transaction_buffers) {
- jbd_debug(3, "denied handle %p %d blocks: "
+ jbd2_debug(3, "denied handle %p %d blocks: "
"transaction too large\n", handle, nblocks);
atomic_sub(nblocks, &transaction->t_outstanding_credits);
- goto unlock;
+ goto error_out;
}
trace_jbd2_handle_extend(journal->j_fs_dev->bd_dev,
@@ -682,9 +707,7 @@ int jbd2_journal_extend(handle_t *handle, int nblocks, int revoke_records)
handle->h_revoke_credits_requested += revoke_records;
result = 0;
- jbd_debug(3, "extended handle %p by %d\n", handle, nblocks);
-unlock:
- spin_unlock(&transaction->t_handle_lock);
+ jbd2_debug(3, "extended handle %p by %d\n", handle, nblocks);
error_out:
read_unlock(&journal->j_state_lock);
return result;
@@ -722,7 +745,8 @@ static void stop_this_handle(handle_t *handle)
atomic_sub(handle->h_total_credits,
&transaction->t_outstanding_credits);
if (handle->h_rsv_handle)
- __jbd2_journal_unreserve_handle(handle->h_rsv_handle);
+ __jbd2_journal_unreserve_handle(handle->h_rsv_handle,
+ transaction);
if (atomic_dec_and_test(&transaction->t_updates))
wake_up(&journal->j_wait_updates);
@@ -735,7 +759,7 @@ static void stop_this_handle(handle_t *handle)
}
/**
- * int jbd2_journal_restart() - restart a handle .
+ * jbd2__journal_restart() - restart a handle .
* @handle: handle to restart
* @nblocks: nr credits requested
* @revoke_records: number of revoke record credits requested
@@ -771,7 +795,7 @@ int jbd2__journal_restart(handle_t *handle, int nblocks, int revoke_records,
* First unlink the handle from its current transaction, and start the
* commit on that.
*/
- jbd_debug(2, "restarting handle %p\n", handle);
+ jbd2_debug(2, "restarting handle %p\n", handle);
stop_this_handle(handle);
handle->h_transaction = NULL;
@@ -804,8 +828,45 @@ int jbd2_journal_restart(handle_t *handle, int nblocks)
}
EXPORT_SYMBOL(jbd2_journal_restart);
+/*
+ * Waits for any outstanding t_updates to finish.
+ * This is called with write j_state_lock held.
+ */
+void jbd2_journal_wait_updates(journal_t *journal)
+{
+ DEFINE_WAIT(wait);
+
+ while (1) {
+ /*
+ * Note that the running transaction can get freed under us if
+ * this transaction is getting committed in
+ * jbd2_journal_commit_transaction() ->
+ * jbd2_journal_free_transaction(). This can only happen when we
+ * release j_state_lock -> schedule() -> acquire j_state_lock.
+ * Hence we should everytime retrieve new j_running_transaction
+ * value (after j_state_lock release acquire cycle), else it may
+ * lead to use-after-free of old freed transaction.
+ */
+ transaction_t *transaction = journal->j_running_transaction;
+
+ if (!transaction)
+ break;
+
+ prepare_to_wait(&journal->j_wait_updates, &wait,
+ TASK_UNINTERRUPTIBLE);
+ if (!atomic_read(&transaction->t_updates)) {
+ finish_wait(&journal->j_wait_updates, &wait);
+ break;
+ }
+ write_unlock(&journal->j_state_lock);
+ schedule();
+ finish_wait(&journal->j_wait_updates, &wait);
+ write_lock(&journal->j_state_lock);
+ }
+}
+
/**
- * void jbd2_journal_lock_updates () - establish a transaction barrier.
+ * jbd2_journal_lock_updates () - establish a transaction barrier.
* @journal: Journal to establish a barrier on.
*
* This locks out any further updates from being started, and blocks
@@ -816,8 +877,6 @@ EXPORT_SYMBOL(jbd2_journal_restart);
*/
void jbd2_journal_lock_updates(journal_t *journal)
{
- DEFINE_WAIT(wait);
-
jbd2_might_wait_for_commit(journal);
write_lock(&journal->j_state_lock);
@@ -831,27 +890,9 @@ void jbd2_journal_lock_updates(journal_t *journal)
write_lock(&journal->j_state_lock);
}
- /* Wait until there are no running updates */
- while (1) {
- transaction_t *transaction = journal->j_running_transaction;
-
- if (!transaction)
- break;
+ /* Wait until there are no running t_updates */
+ jbd2_journal_wait_updates(journal);
- spin_lock(&transaction->t_handle_lock);
- prepare_to_wait(&journal->j_wait_updates, &wait,
- TASK_UNINTERRUPTIBLE);
- if (!atomic_read(&transaction->t_updates)) {
- spin_unlock(&transaction->t_handle_lock);
- finish_wait(&journal->j_wait_updates, &wait);
- break;
- }
- spin_unlock(&transaction->t_handle_lock);
- write_unlock(&journal->j_state_lock);
- schedule();
- finish_wait(&journal->j_wait_updates, &wait);
- write_lock(&journal->j_state_lock);
- }
write_unlock(&journal->j_state_lock);
/*
@@ -864,7 +905,7 @@ void jbd2_journal_lock_updates(journal_t *journal)
}
/**
- * void jbd2_journal_unlock_updates (journal_t* journal) - release barrier
+ * jbd2_journal_unlock_updates () - release barrier
* @journal: Journal to release the barrier on.
*
* Release a transaction barrier obtained with jbd2_journal_lock_updates().
@@ -879,7 +920,7 @@ void jbd2_journal_unlock_updates (journal_t *journal)
write_lock(&journal->j_state_lock);
--journal->j_barrier_count;
write_unlock(&journal->j_state_lock);
- wake_up(&journal->j_wait_transaction_locked);
+ wake_up_all(&journal->j_wait_transaction_locked);
}
static void warn_dirty_buffer(struct buffer_head *bh)
@@ -938,7 +979,7 @@ do_get_write_access(handle_t *handle, struct journal_head *jh,
journal = transaction->t_journal;
- jbd_debug(5, "journal_head %p, force_copy %d\n", jh, force_copy);
+ jbd2_debug(5, "journal_head %p, force_copy %d\n", jh, force_copy);
JBUFFER_TRACE(jh, "entry");
repeat:
@@ -1172,7 +1213,8 @@ out:
}
/**
- * int jbd2_journal_get_write_access() - notify intent to modify a buffer for metadata (not data) update.
+ * jbd2_journal_get_write_access() - notify intent to modify a buffer
+ * for metadata (not data) update.
* @handle: transaction to add buffer modifications to
* @bh: bh to be used for metadata writes
*
@@ -1216,7 +1258,7 @@ int jbd2_journal_get_write_access(handle_t *handle, struct buffer_head *bh)
* unlocked buffer beforehand. */
/**
- * int jbd2_journal_get_create_access () - notify intent to use newly created bh
+ * jbd2_journal_get_create_access () - notify intent to use newly created bh
* @handle: transaction to new buffer to
* @bh: new buffer.
*
@@ -1229,7 +1271,7 @@ int jbd2_journal_get_create_access(handle_t *handle, struct buffer_head *bh)
struct journal_head *jh = jbd2_journal_add_journal_head(bh);
int err;
- jbd_debug(5, "journal_head %p\n", jh);
+ jbd2_debug(5, "journal_head %p\n", jh);
err = -EROFS;
if (is_handle_aborted(handle))
goto out;
@@ -1296,7 +1338,7 @@ out:
}
/**
- * int jbd2_journal_get_undo_access() - Notify intent to modify metadata with
+ * jbd2_journal_get_undo_access() - Notify intent to modify metadata with
* non-rewindable consequences
* @handle: transaction
* @bh: buffer to undo
@@ -1373,7 +1415,7 @@ out:
}
/**
- * void jbd2_journal_set_triggers() - Add triggers for commit writeout
+ * jbd2_journal_set_triggers() - Add triggers for commit writeout
* @bh: buffer to trigger on
* @type: struct jbd2_buffer_trigger_type containing the trigger(s).
*
@@ -1388,7 +1430,7 @@ void jbd2_journal_set_triggers(struct buffer_head *bh,
{
struct journal_head *jh = jbd2_journal_grab_journal_head(bh);
- if (WARN_ON(!jh))
+ if (WARN_ON_ONCE(!jh))
return;
jh->b_triggers = type;
jbd2_journal_put_journal_head(jh);
@@ -1415,7 +1457,7 @@ void jbd2_buffer_abort_trigger(struct journal_head *jh,
}
/**
- * int jbd2_journal_dirty_metadata() - mark a buffer as containing dirty metadata
+ * jbd2_journal_dirty_metadata() - mark a buffer as containing dirty metadata
* @handle: transaction to add buffer to.
* @bh: buffer to mark
*
@@ -1444,8 +1486,6 @@ int jbd2_journal_dirty_metadata(handle_t *handle, struct buffer_head *bh)
struct journal_head *jh;
int ret = 0;
- if (is_handle_aborted(handle))
- return -EROFS;
if (!buffer_jbd(bh))
return -EUCLEAN;
@@ -1454,7 +1494,7 @@ int jbd2_journal_dirty_metadata(handle_t *handle, struct buffer_head *bh)
* of the running transaction.
*/
jh = bh2jh(bh);
- jbd_debug(5, "journal_head %p\n", jh);
+ jbd2_debug(5, "journal_head %p\n", jh);
JBUFFER_TRACE(jh, "entry");
/*
@@ -1463,8 +1503,8 @@ int jbd2_journal_dirty_metadata(handle_t *handle, struct buffer_head *bh)
* crucial to catch bugs so let's do a reliable check until the
* lockless handling is fully proven.
*/
- if (jh->b_transaction != transaction &&
- jh->b_next_transaction != transaction) {
+ if (data_race(jh->b_transaction != transaction &&
+ jh->b_next_transaction != transaction)) {
spin_lock(&jh->b_state_lock);
J_ASSERT_JH(jh, jh->b_transaction == transaction ||
jh->b_next_transaction == transaction);
@@ -1472,8 +1512,8 @@ int jbd2_journal_dirty_metadata(handle_t *handle, struct buffer_head *bh)
}
if (jh->b_modified == 1) {
/* If it's in our transaction it must be in BJ_Metadata list. */
- if (jh->b_transaction == transaction &&
- jh->b_jlist != BJ_Metadata) {
+ if (data_race(jh->b_transaction == transaction &&
+ jh->b_jlist != BJ_Metadata)) {
spin_lock(&jh->b_state_lock);
if (jh->b_transaction == transaction &&
jh->b_jlist != BJ_Metadata)
@@ -1492,6 +1532,18 @@ int jbd2_journal_dirty_metadata(handle_t *handle, struct buffer_head *bh)
journal = transaction->t_journal;
spin_lock(&jh->b_state_lock);
+ if (is_handle_aborted(handle)) {
+ /*
+ * Check journal aborting with @jh->b_state_lock locked,
+ * since 'jh->b_transaction' could be replaced with
+ * 'jh->b_next_transaction' during old transaction
+ * committing if journal aborted, which may fail
+ * assertion on 'jh->b_frozen_data == NULL'.
+ */
+ ret = -EROFS;
+ goto out_unlock_bh;
+ }
+
if (jh->b_modified == 0) {
/*
* This buffer's got modified and becoming part
@@ -1583,7 +1635,7 @@ out:
}
/**
- * void jbd2_journal_forget() - bforget() for potentially-journaled buffers.
+ * jbd2_journal_forget() - bforget() for potentially-journaled buffers.
* @handle: transaction handle
* @bh: bh to 'forget'
*
@@ -1752,7 +1804,7 @@ drop:
}
/**
- * int jbd2_journal_stop() - complete a transaction
+ * jbd2_journal_stop() - complete a transaction
* @handle: transaction to complete.
*
* All done for a particular handle.
@@ -1776,7 +1828,7 @@ int jbd2_journal_stop(handle_t *handle)
pid_t pid;
if (--handle->h_ref > 0) {
- jbd_debug(4, "h_ref %d -> %d\n", handle->h_ref + 1,
+ jbd2_debug(4, "h_ref %d -> %d\n", handle->h_ref + 1,
handle->h_ref);
if (is_handle_aborted(handle))
return -EIO;
@@ -1796,7 +1848,7 @@ int jbd2_journal_stop(handle_t *handle)
if (is_handle_aborted(handle))
err = -EIO;
- jbd_debug(4, "Handle %p going down\n", handle);
+ jbd2_debug(4, "Handle %p going down\n", handle);
trace_jbd2_handle_stats(journal->j_fs_dev->bd_dev,
tid, handle->h_type, handle->h_line_no,
jiffies - handle->h_start_jiffies,
@@ -1874,7 +1926,7 @@ int jbd2_journal_stop(handle_t *handle)
* completes the commit thread, it just doesn't write
* anything to disk. */
- jbd_debug(2, "transaction too old, requesting commit for "
+ jbd2_debug(2, "transaction too old, requesting commit for "
"handle %p\n", handle);
/* This is non-blocking */
jbd2_log_start_commit(journal, tid);
@@ -2018,6 +2070,9 @@ static void __jbd2_journal_temp_unlink_buffer(struct journal_head *jh)
*/
static void __jbd2_journal_unfile_buffer(struct journal_head *jh)
{
+ J_ASSERT_JH(jh, jh->b_transaction != NULL);
+ J_ASSERT_JH(jh, jh->b_next_transaction == NULL);
+
__jbd2_journal_temp_unlink_buffer(jh);
jh->b_transaction = NULL;
}
@@ -2067,13 +2122,9 @@ out:
}
/**
- * int jbd2_journal_try_to_free_buffers() - try to free page buffers.
+ * jbd2_journal_try_to_free_buffers() - try to free page buffers.
* @journal: journal for operation
- * @page: to try and free
- * @gfp_mask: we use the mask to detect how hard should we try to release
- * buffers. If __GFP_DIRECT_RECLAIM and __GFP_FS is set, we wait for commit
- * code to release the buffers.
- *
+ * @folio: Folio to detach data from.
*
* For all the buffers on this page,
* if they are fully written out ordered data, move them onto BUF_CLEAN
@@ -2102,18 +2153,17 @@ out:
* cannot happen because we never reallocate freed data as metadata
* while the data is part of a transaction. Yes?
*
- * Return 0 on failure, 1 on success
+ * Return false on failure, true on success
*/
-int jbd2_journal_try_to_free_buffers(journal_t *journal,
- struct page *page, gfp_t gfp_mask)
+bool jbd2_journal_try_to_free_buffers(journal_t *journal, struct folio *folio)
{
struct buffer_head *head;
struct buffer_head *bh;
- int ret = 0;
+ bool ret = false;
- J_ASSERT(PageLocked(page));
+ J_ASSERT(folio_test_locked(folio));
- head = page_buffers(page);
+ head = folio_buffers(folio);
bh = head;
do {
struct journal_head *jh;
@@ -2135,8 +2185,7 @@ int jbd2_journal_try_to_free_buffers(journal_t *journal,
goto busy;
} while ((bh = bh->b_this_page) != head);
- ret = try_to_free_buffers(page);
-
+ ret = try_to_free_buffers(folio);
busy:
return ret;
}
@@ -2178,14 +2227,14 @@ static int __dispose_buffer(struct journal_head *jh, transaction_t *transaction)
}
/*
- * jbd2_journal_invalidatepage
+ * jbd2_journal_invalidate_folio
*
* This code is tricky. It has a number of cases to deal with.
*
* There are two invariants which this code relies on:
*
- * i_size must be updated on disk before we start calling invalidatepage on the
- * data.
+ * i_size must be updated on disk before we start calling invalidate_folio
+ * on the data.
*
* This is done in ext3 by defining an ext3_setattr method which
* updates i_size before truncate gets going. By maintaining this
@@ -2387,9 +2436,9 @@ zap_buffer_unlocked:
}
/**
- * void jbd2_journal_invalidatepage()
+ * jbd2_journal_invalidate_folio()
* @journal: journal to use for flush...
- * @page: page to flush
+ * @folio: folio to flush
* @offset: start of the range to invalidate
* @length: length of the range to invalidate
*
@@ -2398,30 +2447,29 @@ zap_buffer_unlocked:
* the page is straddling i_size. Caller then has to wait for current commit
* and try again.
*/
-int jbd2_journal_invalidatepage(journal_t *journal,
- struct page *page,
- unsigned int offset,
- unsigned int length)
+int jbd2_journal_invalidate_folio(journal_t *journal, struct folio *folio,
+ size_t offset, size_t length)
{
struct buffer_head *head, *bh, *next;
unsigned int stop = offset + length;
unsigned int curr_off = 0;
- int partial_page = (offset || length < PAGE_SIZE);
+ int partial_page = (offset || length < folio_size(folio));
int may_free = 1;
int ret = 0;
- if (!PageLocked(page))
+ if (!folio_test_locked(folio))
BUG();
- if (!page_has_buffers(page))
+ head = folio_buffers(folio);
+ if (!head)
return 0;
- BUG_ON(stop > PAGE_SIZE || stop < length);
+ BUG_ON(stop > folio_size(folio) || stop < length);
/* We will potentially be playing with lists other than just the
* data lists (especially for journaled data mode), so be
* cautious in our locking. */
- head = bh = page_buffers(page);
+ bh = head;
do {
unsigned int next_off = curr_off + bh->b_size;
next = bh->b_this_page;
@@ -2444,8 +2492,8 @@ int jbd2_journal_invalidatepage(journal_t *journal,
} while (bh != head);
if (!partial_page) {
- if (may_free && try_to_free_buffers(page))
- J_ASSERT(!page_has_buffers(page));
+ if (may_free && try_to_free_buffers(folio))
+ J_ASSERT(!folio_buffers(folio));
}
return 0;
}
@@ -2564,6 +2612,13 @@ bool __jbd2_journal_refile_buffer(struct journal_head *jh)
was_dirty = test_clear_buffer_jbddirty(bh);
__jbd2_journal_temp_unlink_buffer(jh);
+
+ /*
+ * b_transaction must be set, otherwise the new b_transaction won't
+ * be holding jh reference
+ */
+ J_ASSERT_JH(jh, jh->b_transaction != NULL);
+
/*
* We set b_transaction here because b_next_transaction will inherit
* our jh reference and thus __jbd2_journal_file_buffer() must not
@@ -2617,7 +2672,7 @@ static int jbd2_journal_file_inode(handle_t *handle, struct jbd2_inode *jinode,
return -EROFS;
journal = transaction->t_journal;
- jbd_debug(4, "Adding inode %lu, tid:%d\n", jinode->i_vfs_inode->i_ino,
+ jbd2_debug(4, "Adding inode %lu, tid:%d\n", jinode->i_vfs_inode->i_ino,
transaction->t_tid);
spin_lock(&journal->j_list_lock);