]> Pileus Git - ~andy/linux/blobdiff - fs/jbd2/transaction.c
jbd2: transaction reservation support
[~andy/linux] / fs / jbd2 / transaction.c
index 10f524c59ea88d48bf4f85f42e6fb2eca0d7a55b..f33342a2a95ef65651b988b91e7d6f0e5db66e02 100644 (file)
@@ -89,7 +89,8 @@ jbd2_get_transaction(journal_t *journal, transaction_t *transaction)
        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, 0);
+       atomic_set(&transaction->t_outstanding_credits,
+                  atomic_read(&journal->j_reserved_credits));
        atomic_set(&transaction->t_handle_count, 0);
        INIT_LIST_HEAD(&transaction->t_inode_list);
        INIT_LIST_HEAD(&transaction->t_private_list);
@@ -140,6 +141,112 @@ static inline void update_t_max_wait(transaction_t *transaction,
 #endif
 }
 
+/*
+ * Wait until running transaction passes T_LOCKED state. Also starts the commit
+ * if needed. The function expects running transaction to exist and releases
+ * j_state_lock.
+ */
+static void wait_transaction_locked(journal_t *journal)
+       __releases(journal->j_state_lock)
+{
+       DEFINE_WAIT(wait);
+       int need_to_start;
+       tid_t tid = journal->j_running_transaction->t_tid;
+
+       prepare_to_wait(&journal->j_wait_transaction_locked, &wait,
+                       TASK_UNINTERRUPTIBLE);
+       need_to_start = !tid_geq(journal->j_commit_request, tid);
+       read_unlock(&journal->j_state_lock);
+       if (need_to_start)
+               jbd2_log_start_commit(journal, tid);
+       schedule();
+       finish_wait(&journal->j_wait_transaction_locked, &wait);
+}
+
+static void sub_reserved_credits(journal_t *journal, int blocks)
+{
+       atomic_sub(blocks, &journal->j_reserved_credits);
+       wake_up(&journal->j_wait_reserved);
+}
+
+/*
+ * Wait until we can add credits for handle to the running transaction.  Called
+ * 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.
+ */
+static int add_transaction_credits(journal_t *journal, int blocks,
+                                  int rsv_blocks)
+{
+       transaction_t *t = journal->j_running_transaction;
+       int needed;
+       int total = blocks + rsv_blocks;
+
+       /*
+        * If the current transaction is locked down for commit, wait
+        * for the lock to be released.
+        */
+       if (t->t_state == T_LOCKED) {
+               wait_transaction_locked(journal);
+               return 1;
+       }
+
+       /*
+        * If there is not enough space left in the log to write all
+        * potential buffers requested by this operation, we need to
+        * stall pending a log checkpoint to free some more log space.
+        */
+       needed = atomic_add_return(total, &t->t_outstanding_credits);
+       if (needed > journal->j_max_transaction_buffers) {
+               /*
+                * If the current transaction is already too large,
+                * then start to commit it: we can then go back and
+                * attach this handle to a new transaction.
+                */
+               atomic_sub(total, &t->t_outstanding_credits);
+               wait_transaction_locked(journal);
+               return 1;
+       }
+
+       /*
+        * The commit code assumes that it can get enough log space
+        * without forcing a checkpoint.  This is *critical* for
+        * correctness: a checkpoint of a buffer which is also
+        * associated with a committing transaction creates a deadlock,
+        * so commit simply cannot force through checkpoints.
+        *
+        * We must therefore ensure the necessary space in the journal
+        * *before* starting to dirty potentially checkpointed buffers
+        * in the new transaction.
+        */
+       if (jbd2_log_space_left(journal) < jbd2_space_needed(journal)) {
+               atomic_sub(total, &t->t_outstanding_credits);
+               read_unlock(&journal->j_state_lock);
+               write_lock(&journal->j_state_lock);
+               if (jbd2_log_space_left(journal) < jbd2_space_needed(journal))
+                       __jbd2_log_wait_for_space(journal);
+               write_unlock(&journal->j_state_lock);
+               return 1;
+       }
+
+       /* No reservation? We are done... */
+       if (!rsv_blocks)
+               return 0;
+
+       needed = atomic_add_return(rsv_blocks, &journal->j_reserved_credits);
+       /* We allow at most half of a transaction to be reserved */
+       if (needed > journal->j_max_transaction_buffers / 2) {
+               sub_reserved_credits(journal, rsv_blocks);
+               atomic_sub(total, &t->t_outstanding_credits);
+               read_unlock(&journal->j_state_lock);
+               wait_event(journal->j_wait_reserved,
+                        atomic_read(&journal->j_reserved_credits) + rsv_blocks
+                        <= journal->j_max_transaction_buffers / 2);
+               return 1;
+       }
+       return 0;
+}
+
 /*
  * start_this_handle: Given a handle, deal with any locking or stalling
  * needed to make sure that there is enough journal space for the handle
@@ -151,18 +258,24 @@ static int start_this_handle(journal_t *journal, handle_t *handle,
                             gfp_t gfp_mask)
 {
        transaction_t   *transaction, *new_transaction = NULL;
-       tid_t           tid;
-       int             needed, need_to_start;
-       int             nblocks = handle->h_buffer_credits;
+       int             blocks = handle->h_buffer_credits;
+       int             rsv_blocks = 0;
        unsigned long ts = jiffies;
 
-       if (nblocks > journal->j_max_transaction_buffers) {
+       /*
+        * 1/2 of transaction can be reserved so we can practically handle
+        * only 1/2 of maximum transaction size per operation
+        */
+       if (WARN_ON(blocks > journal->j_max_transaction_buffers / 2)) {
                printk(KERN_ERR "JBD2: %s wants too many credits (%d > %d)\n",
-                      current->comm, nblocks,
-                      journal->j_max_transaction_buffers);
+                      current->comm, blocks,
+                      journal->j_max_transaction_buffers / 2);
                return -ENOSPC;
        }
 
+       if (handle->h_rsv_handle)
+               rsv_blocks = handle->h_rsv_handle->h_buffer_credits;
+
 alloc_transaction:
        if (!journal->j_running_transaction) {
                new_transaction = kmem_cache_zalloc(transaction_cache,
@@ -199,8 +312,12 @@ repeat:
                return -EROFS;
        }
 
-       /* Wait on the journal's transaction barrier if necessary */
-       if (journal->j_barrier_count) {
+       /*
+        * Wait on the journal's transaction barrier if necessary. Specifically
+        * we allow reserved handles to proceed because otherwise commit could
+        * deadlock on page writeback not being able to complete.
+        */
+       if (!handle->h_reserved && journal->j_barrier_count) {
                read_unlock(&journal->j_state_lock);
                wait_event(journal->j_wait_transaction_locked,
                                journal->j_barrier_count == 0);
@@ -213,7 +330,7 @@ repeat:
                        goto alloc_transaction;
                write_lock(&journal->j_state_lock);
                if (!journal->j_running_transaction &&
-                   !journal->j_barrier_count) {
+                   (handle->h_reserved || !journal->j_barrier_count)) {
                        jbd2_get_transaction(journal, new_transaction);
                        new_transaction = NULL;
                }
@@ -223,85 +340,18 @@ repeat:
 
        transaction = journal->j_running_transaction;
 
-       /*
-        * If the current transaction is locked down for commit, wait for the
-        * lock to be released.
-        */
-       if (transaction->t_state == T_LOCKED) {
-               DEFINE_WAIT(wait);
-
-               prepare_to_wait(&journal->j_wait_transaction_locked,
-                                       &wait, TASK_UNINTERRUPTIBLE);
-               read_unlock(&journal->j_state_lock);
-               schedule();
-               finish_wait(&journal->j_wait_transaction_locked, &wait);
-               goto repeat;
-       }
-
-       /*
-        * If there is not enough space left in the log to write all potential
-        * buffers requested by this operation, we need to stall pending a log
-        * checkpoint to free some more log space.
-        */
-       needed = atomic_add_return(nblocks,
-                                  &transaction->t_outstanding_credits);
-
-       if (needed > journal->j_max_transaction_buffers) {
+       if (!handle->h_reserved) {
+               /* We may have dropped j_state_lock - restart in that case */
+               if (add_transaction_credits(journal, blocks, rsv_blocks))
+                       goto repeat;
+       } else {
                /*
-                * If the current transaction is already too large, then start
-                * to commit it: we can then go back and attach this handle to
-                * a new transaction.
+                * We have handle reserved so we are allowed to join T_LOCKED
+                * transaction and we don't have to check for transaction size
+                * and journal space.
                 */
-               DEFINE_WAIT(wait);
-
-               jbd_debug(2, "Handle %p starting new commit...\n", handle);
-               atomic_sub(nblocks, &transaction->t_outstanding_credits);
-               prepare_to_wait(&journal->j_wait_transaction_locked, &wait,
-                               TASK_UNINTERRUPTIBLE);
-               tid = transaction->t_tid;
-               need_to_start = !tid_geq(journal->j_commit_request, tid);
-               read_unlock(&journal->j_state_lock);
-               if (need_to_start)
-                       jbd2_log_start_commit(journal, tid);
-               schedule();
-               finish_wait(&journal->j_wait_transaction_locked, &wait);
-               goto repeat;
-       }
-
-       /*
-        * The commit code assumes that it can get enough log space
-        * without forcing a checkpoint.  This is *critical* for
-        * correctness: a checkpoint of a buffer which is also
-        * associated with a committing transaction creates a deadlock,
-        * so commit simply cannot force through checkpoints.
-        *
-        * We must therefore ensure the necessary space in the journal
-        * *before* starting to dirty potentially checkpointed buffers
-        * in the new transaction.
-        *
-        * The worst part is, any transaction currently committing can
-        * reduce the free space arbitrarily.  Be careful to account for
-        * those buffers when checkpointing.
-        */
-
-       /*
-        * @@@ AKPM: This seems rather over-defensive.  We're giving commit
-        * a _lot_ of headroom: 1/4 of the journal plus the size of
-        * the committing transaction.  Really, we only need to give it
-        * committing_transaction->t_outstanding_credits plus "enough" for
-        * the log control blocks.
-        * Also, this test is inconsistent with the matching one in
-        * jbd2_journal_extend().
-        */
-       if (__jbd2_log_space_left(journal) < jbd_space_needed(journal)) {
-               jbd_debug(2, "Handle %p waiting for checkpoint...\n", handle);
-               atomic_sub(nblocks, &transaction->t_outstanding_credits);
-               read_unlock(&journal->j_state_lock);
-               write_lock(&journal->j_state_lock);
-               if (__jbd2_log_space_left(journal) < jbd_space_needed(journal))
-                       __jbd2_log_wait_for_space(journal);
-               write_unlock(&journal->j_state_lock);
-               goto repeat;
+               sub_reserved_credits(journal, blocks);
+               handle->h_reserved = 0;
        }
 
        /* OK, account for the buffers that this operation expects to
@@ -309,14 +359,14 @@ repeat:
         */
        update_t_max_wait(transaction, ts);
        handle->h_transaction = transaction;
-       handle->h_requested_credits = nblocks;
+       handle->h_requested_credits = blocks;
        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 %d)\n",
-                 handle, nblocks,
+       jbd_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));
+                 jbd2_log_space_left(journal));
        read_unlock(&journal->j_state_lock);
 
        lock_map_acquire(&handle->h_lockdep_map);
@@ -348,16 +398,21 @@ static handle_t *new_handle(int nblocks)
  *
  * We make sure that the transaction can guarantee at least nblocks of
  * modified buffers in the log.  We block until the log can guarantee
- * that much space.
- *
- * This function is visible to journal users (like ext3fs), so is not
- * called with the journal already locked.
+ * that much space. Additionally, if rsv_blocks > 0, we also create another
+ * handle with rsv_blocks reserved blocks in the journal. This handle is
+ * is stored in h_rsv_handle. It is not attached to any particular transaction
+ * and thus doesn't block transaction commit. If the caller uses this reserved
+ * handle, it has to set h_rsv_handle to NULL as otherwise jbd2_journal_stop()
+ * on the parent handle will dispose the reserved one. Reserved handle has to
+ * be converted to a normal handle using jbd2_journal_start_reserved() before
+ * it can be used.
  *
  * Return a pointer to a newly allocated handle, or an ERR_PTR() value
  * on failure.
  */
-handle_t *jbd2__journal_start(journal_t *journal, int nblocks, gfp_t gfp_mask,
-                             unsigned int type, unsigned int line_no)
+handle_t *jbd2__journal_start(journal_t *journal, int nblocks, int rsv_blocks,
+                             gfp_t gfp_mask, unsigned int type,
+                             unsigned int line_no)
 {
        handle_t *handle = journal_current_handle();
        int err;
@@ -374,11 +429,25 @@ handle_t *jbd2__journal_start(journal_t *journal, int nblocks, gfp_t gfp_mask,
        handle = new_handle(nblocks);
        if (!handle)
                return ERR_PTR(-ENOMEM);
+       if (rsv_blocks) {
+               handle_t *rsv_handle;
+
+               rsv_handle = new_handle(rsv_blocks);
+               if (!rsv_handle) {
+                       jbd2_free_handle(handle);
+                       return ERR_PTR(-ENOMEM);
+               }
+               rsv_handle->h_reserved = 1;
+               rsv_handle->h_journal = journal;
+               handle->h_rsv_handle = rsv_handle;
+       }
 
        current->journal_info = handle;
 
        err = start_this_handle(journal, handle, gfp_mask);
        if (err < 0) {
+               if (handle->h_rsv_handle)
+                       jbd2_free_handle(handle->h_rsv_handle);
                jbd2_free_handle(handle);
                current->journal_info = NULL;
                return ERR_PTR(err);
@@ -395,10 +464,68 @@ EXPORT_SYMBOL(jbd2__journal_start);
 
 handle_t *jbd2_journal_start(journal_t *journal, int nblocks)
 {
-       return jbd2__journal_start(journal, nblocks, GFP_NOFS, 0, 0);
+       return jbd2__journal_start(journal, nblocks, 0, GFP_NOFS, 0, 0);
 }
 EXPORT_SYMBOL(jbd2_journal_start);
 
+void jbd2_journal_free_reserved(handle_t *handle)
+{
+       journal_t *journal = handle->h_journal;
+
+       WARN_ON(!handle->h_reserved);
+       sub_reserved_credits(journal, handle->h_buffer_credits);
+       jbd2_free_handle(handle);
+}
+EXPORT_SYMBOL(jbd2_journal_free_reserved);
+
+/**
+ * int jbd2_journal_start_reserved(handle_t *handle) - start reserved handle
+ * @handle: handle to start
+ *
+ * Start handle that has been previously reserved with jbd2_journal_reserve().
+ * This attaches @handle to the running transaction (or creates one if there's
+ * not transaction running). Unlike jbd2_journal_start() this function cannot
+ * block on journal commit, checkpointing, or similar stuff. It can block on
+ * memory allocation or frozen journal though.
+ *
+ * Return 0 on success, non-zero on error - handle is freed in that case.
+ */
+int jbd2_journal_start_reserved(handle_t *handle, unsigned int type,
+                               unsigned int line_no)
+{
+       journal_t *journal = handle->h_journal;
+       int ret = -EIO;
+
+       if (WARN_ON(!handle->h_reserved)) {
+               /* Someone passed in normal handle? Just stop it. */
+               jbd2_journal_stop(handle);
+               return ret;
+       }
+       /*
+        * Usefulness of mixing of reserved and unreserved handles is
+        * questionable. So far nobody seems to need it so just error out.
+        */
+       if (WARN_ON(current->journal_info)) {
+               jbd2_journal_free_reserved(handle);
+               return ret;
+       }
+
+       handle->h_journal = NULL;
+       current->journal_info = handle;
+       /*
+        * GFP_NOFS is here because callers are likely from writeback or
+        * similarly constrained call sites
+        */
+       ret = start_this_handle(journal, handle, GFP_NOFS);
+       if (ret < 0) {
+               current->journal_info = NULL;
+               jbd2_journal_free_reserved(handle);
+       }
+       handle->h_type = type;
+       handle->h_line_no = line_no;
+       return ret;
+}
+EXPORT_SYMBOL(jbd2_journal_start_reserved);
 
 /**
  * int jbd2_journal_extend() - extend buffer credits.
@@ -443,17 +570,21 @@ int jbd2_journal_extend(handle_t *handle, int nblocks)
        }
 
        spin_lock(&transaction->t_handle_lock);
-       wanted = atomic_read(&transaction->t_outstanding_credits) + nblocks;
+       wanted = atomic_add_return(nblocks,
+                                  &transaction->t_outstanding_credits);
 
        if (wanted > journal->j_max_transaction_buffers) {
                jbd_debug(3, "denied handle %p %d blocks: "
                          "transaction too large\n", handle, nblocks);
+               atomic_sub(nblocks, &transaction->t_outstanding_credits);
                goto unlock;
        }
 
-       if (wanted > __jbd2_log_space_left(journal)) {
+       if (wanted + (wanted >> JBD2_CONTROL_BLOCKS_SHIFT) >
+           jbd2_log_space_left(journal)) {
                jbd_debug(3, "denied handle %p %d blocks: "
                          "insufficient log space\n", handle, nblocks);
+               atomic_sub(nblocks, &transaction->t_outstanding_credits);
                goto unlock;
        }
 
@@ -465,7 +596,6 @@ int jbd2_journal_extend(handle_t *handle, int nblocks)
 
        handle->h_buffer_credits += nblocks;
        handle->h_requested_credits += nblocks;
-       atomic_add(nblocks, &transaction->t_outstanding_credits);
        result = 0;
 
        jbd_debug(3, "extended handle %p by %d\n", handle, nblocks);
@@ -490,7 +620,8 @@ out:
  * to a running handle, a call to jbd2_journal_restart will commit the
  * handle's transaction so far and reattach the handle to a new
  * transaction capabable of guaranteeing the requested number of
- * credits.
+ * credits. We preserve reserved handle if there's any attached to the
+ * passed in handle.
  */
 int jbd2__journal_restart(handle_t *handle, int nblocks, gfp_t gfp_mask)
 {
@@ -515,6 +646,10 @@ int jbd2__journal_restart(handle_t *handle, int nblocks, gfp_t gfp_mask)
        spin_lock(&transaction->t_handle_lock);
        atomic_sub(handle->h_buffer_credits,
                   &transaction->t_outstanding_credits);
+       if (handle->h_rsv_handle) {
+               sub_reserved_credits(journal,
+                                    handle->h_rsv_handle->h_buffer_credits);
+       }
        if (atomic_dec_and_test(&transaction->t_updates))
                wake_up(&journal->j_wait_updates);
        spin_unlock(&transaction->t_handle_lock);
@@ -557,6 +692,14 @@ void jbd2_journal_lock_updates(journal_t *journal)
        write_lock(&journal->j_state_lock);
        ++journal->j_barrier_count;
 
+       /* Wait until there are no reserved handles */
+       if (atomic_read(&journal->j_reserved_credits)) {
+               write_unlock(&journal->j_state_lock);
+               wait_event(journal->j_wait_reserved,
+                          atomic_read(&journal->j_reserved_credits) == 0);
+               write_lock(&journal->j_state_lock);
+       }
+
        /* Wait until there are no running updates */
        while (1) {
                transaction_t *transaction = journal->j_running_transaction;
@@ -619,6 +762,12 @@ static void warn_dirty_buffer(struct buffer_head *bh)
               bdevname(bh->b_bdev, b), (unsigned long long)bh->b_blocknr);
 }
 
+static int sleep_on_shadow_bh(void *word)
+{
+       io_schedule();
+       return 0;
+}
+
 /*
  * If the buffer is already part of the current transaction, then there
  * is nothing we need to do.  If it is already part of a prior
@@ -754,41 +903,29 @@ repeat:
                 * journaled.  If the primary copy is already going to
                 * disk then we cannot do copy-out here. */
 
-               if (jh->b_jlist == BJ_Shadow) {
-                       DEFINE_WAIT_BIT(wait, &bh->b_state, BH_Unshadow);
-                       wait_queue_head_t *wqh;
-
-                       wqh = bit_waitqueue(&bh->b_state, BH_Unshadow);
-
+               if (buffer_shadow(bh)) {
                        JBUFFER_TRACE(jh, "on shadow: sleep");
                        jbd_unlock_bh_state(bh);
-                       /* commit wakes up all shadow buffers after IO */
-                       for ( ; ; ) {
-                               prepare_to_wait(wqh, &wait.wait,
-                                               TASK_UNINTERRUPTIBLE);
-                               if (jh->b_jlist != BJ_Shadow)
-                                       break;
-                               schedule();
-                       }
-                       finish_wait(wqh, &wait.wait);
+                       wait_on_bit(&bh->b_state, BH_Shadow,
+                                   sleep_on_shadow_bh, TASK_UNINTERRUPTIBLE);
                        goto repeat;
                }
 
-               /* Only do the copy if the currently-owning transaction
-                * still needs it.  If it is on the Forget list, the
-                * committing transaction is past that stage.  The
-                * buffer had better remain locked during the kmalloc,
-                * but that should be true --- we hold the journal lock
-                * still and the buffer is already on the BUF_JOURNAL
-                * list so won't be flushed.
+               /*
+                * Only do the copy if the currently-owning transaction still
+                * needs it. If buffer isn't on BJ_Metadata list, the
+                * committing transaction is past that stage (here we use the
+                * fact that BH_Shadow is set under bh_state lock together with
+                * refiling to BJ_Shadow list and at this point we know the
+                * buffer doesn't have BH_Shadow set).
                 *
                 * Subtle point, though: if this is a get_undo_access,
                 * then we will be relying on the frozen_data to contain
                 * the new value of the committed_data record after the
                 * transaction, so we HAVE to force the frozen_data copy
-                * in that case. */
-
-               if (jh->b_jlist != BJ_Forget || force_copy) {
+                * in that case.
+                */
+               if (jh->b_jlist == BJ_Metadata || force_copy) {
                        JBUFFER_TRACE(jh, "generate frozen data");
                        if (!frozen_buffer) {
                                JBUFFER_TRACE(jh, "allocate memory for buffer");
@@ -1518,6 +1655,8 @@ int jbd2_journal_stop(handle_t *handle)
 
        lock_map_release(&handle->h_lockdep_map);
 
+       if (handle->h_rsv_handle)
+               jbd2_journal_free_reserved(handle->h_rsv_handle);
        jbd2_free_handle(handle);
        return err;
 }
@@ -1601,10 +1740,10 @@ __blist_del_buffer(struct journal_head **list, struct journal_head *jh)
  * Remove a buffer from the appropriate transaction list.
  *
  * Note that this function can *change* the value of
- * bh->b_transaction->t_buffers, t_forget, t_iobuf_list, t_shadow_list,
- * t_log_list or t_reserved_list.  If the caller is holding onto a copy of one
- * of these pointers, it could go bad.  Generally the caller needs to re-read
- * the pointer from the transaction_t.
+ * bh->b_transaction->t_buffers, t_forget, t_shadow_list, t_log_list or
+ * t_reserved_list.  If the caller is holding onto a copy of one of these
+ * pointers, it could go bad.  Generally the caller needs to re-read the
+ * pointer from the transaction_t.
  *
  * Called under j_list_lock.
  */
@@ -1634,15 +1773,9 @@ static void __jbd2_journal_temp_unlink_buffer(struct journal_head *jh)
        case BJ_Forget:
                list = &transaction->t_forget;
                break;
-       case BJ_IO:
-               list = &transaction->t_iobuf_list;
-               break;
        case BJ_Shadow:
                list = &transaction->t_shadow_list;
                break;
-       case BJ_LogCtl:
-               list = &transaction->t_log_list;
-               break;
        case BJ_Reserved:
                list = &transaction->t_reserved_list;
                break;
@@ -2034,18 +2167,23 @@ zap_buffer_unlocked:
  * void jbd2_journal_invalidatepage()
  * @journal: journal to use for flush...
  * @page:    page to flush
- * @offset:  length of page to invalidate.
+ * @offset:  start of the range to invalidate
+ * @length:  length of the range to invalidate
  *
- * Reap page buffers containing data after offset in page. Can return -EBUSY
- * if buffers are part of the committing transaction and the page is straddling
- * i_size. Caller then has to wait for current commit and try again.
+ * Reap page buffers containing data after in the specified range in page.
+ * Can return -EBUSY if buffers are part of the committing transaction and
+ * 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 long offset)
+                               unsigned int offset,
+                               unsigned int length)
 {
        struct buffer_head *head, *bh, *next;
+       unsigned int stop = offset + length;
        unsigned int curr_off = 0;
+       int partial_page = (offset || length < PAGE_CACHE_SIZE);
        int may_free = 1;
        int ret = 0;
 
@@ -2054,6 +2192,8 @@ int jbd2_journal_invalidatepage(journal_t *journal,
        if (!page_has_buffers(page))
                return 0;
 
+       BUG_ON(stop > PAGE_CACHE_SIZE || 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. */
@@ -2063,10 +2203,13 @@ int jbd2_journal_invalidatepage(journal_t *journal,
                unsigned int next_off = curr_off + bh->b_size;
                next = bh->b_this_page;
 
+               if (next_off > stop)
+                       return 0;
+
                if (offset <= curr_off) {
                        /* This block is wholly outside the truncation point */
                        lock_buffer(bh);
-                       ret = journal_unmap_buffer(journal, bh, offset > 0);
+                       ret = journal_unmap_buffer(journal, bh, partial_page);
                        unlock_buffer(bh);
                        if (ret < 0)
                                return ret;
@@ -2077,7 +2220,7 @@ int jbd2_journal_invalidatepage(journal_t *journal,
 
        } while (bh != head);
 
-       if (!offset) {
+       if (!partial_page) {
                if (may_free && try_to_free_buffers(page))
                        J_ASSERT(!page_has_buffers(page));
        }
@@ -2138,15 +2281,9 @@ void __jbd2_journal_file_buffer(struct journal_head *jh,
        case BJ_Forget:
                list = &transaction->t_forget;
                break;
-       case BJ_IO:
-               list = &transaction->t_iobuf_list;
-               break;
        case BJ_Shadow:
                list = &transaction->t_shadow_list;
                break;
-       case BJ_LogCtl:
-               list = &transaction->t_log_list;
-               break;
        case BJ_Reserved:
                list = &transaction->t_reserved_list;
                break;