Fedora kernel-2.6.17-1.2142_FC4 patched with stable patch-2.6.17.4-vs2.0.2-rc26.diff
[linux-2.6.git] / fs / jbd / transaction.c
index 14d40be..c609f50 100644 (file)
@@ -50,13 +50,11 @@ get_transaction(journal_t *journal, transaction_t *transaction)
        transaction->t_state = T_RUNNING;
        transaction->t_tid = journal->j_transaction_sequence++;
        transaction->t_expires = jiffies + journal->j_commit_interval;
-       INIT_LIST_HEAD(&transaction->t_jcb);
        spin_lock_init(&transaction->t_handle_lock);
-       spin_lock_init(&transaction->t_jcb_lock);
 
        /* Set up the commit timer for the new transaction. */
-       journal->j_commit_timer->expires = transaction->t_expires;
-       add_timer(journal->j_commit_timer);
+       journal->j_commit_timer.expires = transaction->t_expires;
+       add_timer(&journal->j_commit_timer);
 
        J_ASSERT(journal->j_running_transaction == NULL);
        journal->j_running_transaction = transaction;
@@ -229,8 +227,7 @@ repeat_locked:
        spin_unlock(&transaction->t_handle_lock);
        spin_unlock(&journal->j_state_lock);
 out:
-       if (new_transaction)
-               kfree(new_transaction);
+       kfree(new_transaction);
        return ret;
 }
 
@@ -243,7 +240,6 @@ static handle_t *new_handle(int nblocks)
        memset(handle, 0, sizeof(*handle));
        handle->h_buffer_credits = nblocks;
        handle->h_ref = 1;
-       INIT_LIST_HEAD(&handle->h_jcb);
 
        return handle;
 }
@@ -320,7 +316,7 @@ int journal_extend(handle_t *handle, int nblocks)
 
        result = -EIO;
        if (is_handle_aborted(handle))
-               goto error_out;
+               goto out;
 
        result = 1;
 
@@ -357,6 +353,7 @@ unlock:
        spin_unlock(&transaction->t_handle_lock);
 error_out:
        spin_unlock(&journal->j_state_lock);
+out:
        return result;
 }
 
@@ -458,7 +455,7 @@ void journal_lock_updates(journal_t *journal)
         * to make sure that we serialise special journal-locked operations
         * too.
         */
-       down(&journal->j_barrier);
+       mutex_lock(&journal->j_barrier);
 }
 
 /**
@@ -473,7 +470,7 @@ void journal_unlock_updates (journal_t *journal)
 {
        J_ASSERT(journal->j_barrier_count != 0);
 
-       up(&journal->j_barrier);
+       mutex_unlock(&journal->j_barrier);
        spin_lock(&journal->j_state_lock);
        --journal->j_barrier_count;
        spin_unlock(&journal->j_state_lock);
@@ -492,23 +489,21 @@ void journal_unlock_updates (journal_t *journal)
  */
 static void jbd_unexpected_dirty_buffer(struct journal_head *jh)
 {
-       struct buffer_head *bh = jh2bh(jh);
        int jlist;
 
-       if (buffer_dirty(bh)) {
-               /* If this buffer is one which might reasonably be dirty
-                * --- ie. data, or not part of this journal --- then
-                * we're OK to leave it alone, but otherwise we need to
-                * move the dirty bit to the journal's own internal
-                * JBDDirty bit. */
-               jlist = jh->b_jlist;
-
-               if (jlist == BJ_Metadata || jlist == BJ_Reserved || 
-                   jlist == BJ_Shadow || jlist == BJ_Forget) {
-                       if (test_clear_buffer_dirty(jh2bh(jh))) {
-                               set_bit(BH_JBDDirty, &jh2bh(jh)->b_state);
-                       }
-               }
+       /* If this buffer is one which might reasonably be dirty
+        * --- ie. data, or not part of this journal --- then
+        * we're OK to leave it alone, but otherwise we need to
+        * move the dirty bit to the journal's own internal
+        * JBDDirty bit. */
+       jlist = jh->b_jlist;
+
+       if (jlist == BJ_Metadata || jlist == BJ_Reserved ||
+           jlist == BJ_Shadow || jlist == BJ_Forget) {
+               struct buffer_head *bh = jh2bh(jh);
+
+               if (test_clear_buffer_dirty(bh))
+                       set_buffer_jbddirty(bh);
        }
 }
 
@@ -524,7 +519,7 @@ static void jbd_unexpected_dirty_buffer(struct journal_head *jh)
  */
 static int
 do_get_write_access(handle_t *handle, struct journal_head *jh,
-                       int force_copy, int *credits) 
+                       int force_copy)
 {
        struct buffer_head *bh;
        transaction_t *transaction;
@@ -576,9 +571,14 @@ repeat:
                        if (jh->b_next_transaction)
                                J_ASSERT_JH(jh, jh->b_next_transaction ==
                                                        transaction);
-                       JBUFFER_TRACE(jh, "Unexpected dirty buffer");
-                       jbd_unexpected_dirty_buffer(jh);
-               }
+               }
+               /*
+                * In any case we need to clean the dirty flag and we must
+                * do it under the buffer lock to be sure we don't race
+                * with running write-out.
+                */
+               JBUFFER_TRACE(jh, "Unexpected dirty buffer");
+               jbd_unexpected_dirty_buffer(jh);
        }
 
        unlock_buffer(bh);
@@ -606,11 +606,6 @@ repeat:
                JBUFFER_TRACE(jh, "has frozen data");
                J_ASSERT_JH(jh, jh->b_next_transaction == NULL);
                jh->b_next_transaction = transaction;
-
-               J_ASSERT_JH(jh, handle->h_buffer_credits > 0);
-               handle->h_buffer_credits--;
-               if (credits)
-                       (*credits)++;
                goto done;
        }
 
@@ -632,13 +627,22 @@ repeat:
                 * 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);
+
                        JBUFFER_TRACE(jh, "on shadow: sleep");
                        jbd_unlock_bh_state(bh);
                        /* commit wakes up all shadow buffers after IO */
-                       wqh = bh_waitq_head(jh2bh(jh));
-                       wait_event(*wqh, (jh->b_jlist != BJ_Shadow));
+                       for ( ; ; ) {
+                               prepare_to_wait(wqh, &wait.wait,
+                                               TASK_UNINTERRUPTIBLE);
+                               if (jh->b_jlist != BJ_Shadow)
+                                       break;
+                               schedule();
+                       }
+                       finish_wait(wqh, &wait.wait);
                        goto repeat;
                }
 
@@ -681,10 +685,6 @@ repeat:
                jh->b_next_transaction = transaction;
        }
 
-       J_ASSERT(handle->h_buffer_credits > 0);
-       handle->h_buffer_credits--;
-       if (credits)
-               (*credits)++;
 
        /*
         * Finally, if the buffer is not journaled right now, we need to make
@@ -724,8 +724,7 @@ done:
        journal_cancel_revoke(handle, jh);
 
 out:
-       if (frozen_buffer)
-               kfree(frozen_buffer);
+       kfree(frozen_buffer);
 
        JBUFFER_TRACE(jh, "exit");
        return error;
@@ -735,6 +734,7 @@ out:
  * int 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
+ * @credits: variable that will receive credits for the buffer
  *
  * Returns an error code or 0 on success.
  *
@@ -742,8 +742,7 @@ out:
  * because we're write()ing a buffer which is also part of a shared mapping.
  */
 
-int journal_get_write_access(handle_t *handle,
-                       struct buffer_head *bh, int *credits)
+int journal_get_write_access(handle_t *handle, struct buffer_head *bh)
 {
        struct journal_head *jh = journal_add_journal_head(bh);
        int rc;
@@ -751,7 +750,7 @@ int journal_get_write_access(handle_t *handle,
        /* We do not want to get caught playing with fields which the
         * log thread also manipulates.  Make sure that the buffer
         * completes any outstanding IO before proceeding. */
-       rc = do_get_write_access(handle, jh, 0, credits);
+       rc = do_get_write_access(handle, jh, 0);
        journal_put_journal_head(jh);
        return rc;
 }
@@ -807,9 +806,6 @@ int journal_get_create_access(handle_t *handle, struct buffer_head *bh)
        J_ASSERT_JH(jh, jh->b_next_transaction == NULL);
        J_ASSERT_JH(jh, buffer_locked(jh2bh(jh)));
 
-       J_ASSERT_JH(jh, handle->h_buffer_credits > 0);
-       handle->h_buffer_credits--;
-
        if (jh->b_transaction == NULL) {
                jh->b_transaction = transaction;
                JBUFFER_TRACE(jh, "file as BJ_Reserved");
@@ -862,8 +858,7 @@ out:
  *
  * Returns error number or 0 on success.
  */
-int journal_get_undo_access(handle_t *handle, struct buffer_head *bh,
-                               int *credits)
+int journal_get_undo_access(handle_t *handle, struct buffer_head *bh)
 {
        int err;
        struct journal_head *jh = journal_add_journal_head(bh);
@@ -876,7 +871,7 @@ int journal_get_undo_access(handle_t *handle, struct buffer_head *bh,
         * make sure that obtaining the committed_data is done
         * atomically wrt. completion of any outstanding commits.
         */
-       err = do_get_write_access(handle, jh, 1, credits);
+       err = do_get_write_access(handle, jh, 1);
        if (err)
                goto out;
 
@@ -908,8 +903,7 @@ repeat:
        jbd_unlock_bh_state(bh);
 out:
        journal_put_journal_head(jh);
-       if (committed_data)
-               kfree(committed_data);
+       kfree(committed_data);
        return err;
 }
 
@@ -1037,7 +1031,12 @@ int journal_dirty_data(handle_t *handle, struct buffer_head *bh)
                        /* journal_clean_data_list() may have got there first */
                        if (jh->b_transaction != NULL) {
                                JBUFFER_TRACE(jh, "unfile from commit");
-                               __journal_unfile_buffer(jh);
+                               __journal_temp_unlink_buffer(jh);
+                               /* It still points to the committing
+                                * transaction; move it to this one so
+                                * that the refile assert checks are
+                                * happy. */
+                               jh->b_transaction = handle->h_transaction;
                        }
                        /* The buffer will be refiled below */
 
@@ -1051,7 +1050,8 @@ int journal_dirty_data(handle_t *handle, struct buffer_head *bh)
                if (jh->b_jlist != BJ_SyncData && jh->b_jlist != BJ_Locked) {
                        JBUFFER_TRACE(jh, "not on correct data list: unfile");
                        J_ASSERT_JH(jh, jh->b_jlist != BJ_Shadow);
-                       __journal_unfile_buffer(jh);
+                       __journal_temp_unlink_buffer(jh);
+                       jh->b_transaction = handle->h_transaction;
                        JBUFFER_TRACE(jh, "file as data");
                        __journal_file_buffer(jh, handle->h_transaction,
                                                BJ_SyncData);
@@ -1104,6 +1104,17 @@ int journal_dirty_metadata(handle_t *handle, struct buffer_head *bh)
 
        jbd_lock_bh_state(bh);
 
+       if (jh->b_modified == 0) {
+               /*
+                * This buffer's got modified and becoming part
+                * of the transaction. This needs to be done
+                * once a transaction -bzzz
+                */
+               jh->b_modified = 1;
+               J_ASSERT_JH(jh, handle->h_buffer_credits > 0);
+               handle->h_buffer_credits--;
+       }
+
        /*
         * fastpath, to avoid expensive locking.  If this buffer is already
         * on the running transaction's metadata list there is nothing to do.
@@ -1154,24 +1165,11 @@ out:
  * journal_release_buffer: undo a get_write_access without any buffer
  * updates, if the update decided in the end that it didn't need access.
  *
- * The caller passes in the number of credits which should be put back for
- * this buffer (zero or one).
- *
- * We leave the buffer attached to t_reserved_list because even though this
- * handle doesn't want it, some other concurrent handle may want to journal
- * this buffer.  If that handle is curently in between get_write_access() and
- * journal_dirty_metadata() then it expects the buffer to be reserved.  If
- * we were to rip it off t_reserved_list here, the other handle will explode
- * when journal_dirty_metadata is presented with a non-reserved buffer.
- *
- * If nobody really wants to journal this buffer then it will be thrown
- * away at the start of commit.
  */
 void
-journal_release_buffer(handle_t *handle, struct buffer_head *bh, int credits)
+journal_release_buffer(handle_t *handle, struct buffer_head *bh)
 {
        BUFFER_TRACE(bh, "entry");
-       handle->h_buffer_credits += credits;
 }
 
 /** 
@@ -1191,11 +1189,13 @@ journal_release_buffer(handle_t *handle, struct buffer_head *bh, int credits)
  * Allow this call even if the handle has aborted --- it may be part of
  * the caller's cleanup after an abort.
  */
-void journal_forget(handle_t *handle, struct buffer_head *bh)
+int journal_forget (handle_t *handle, struct buffer_head *bh)
 {
        transaction_t *transaction = handle->h_transaction;
        journal_t *journal = transaction->t_journal;
        struct journal_head *jh;
+       int drop_reserve = 0;
+       int err = 0;
 
        BUFFER_TRACE(bh, "entry");
 
@@ -1206,6 +1206,20 @@ void journal_forget(handle_t *handle, struct buffer_head *bh)
                goto not_jbd;
        jh = bh2jh(bh);
 
+       /* Critical error: attempting to delete a bitmap buffer, maybe?
+        * Don't do any jbd operations, and return an error. */
+       if (!J_EXPECT_JH(jh, !jh->b_committed_data,
+                        "inconsistent data on disk")) {
+               err = -EIO;
+               goto not_jbd;
+       }
+
+       /*
+        * The buffer's going from the transaction, we must drop
+        * all references -bzzz
+        */
+       jh->b_modified = 0;
+
        if (jh->b_transaction == handle->h_transaction) {
                J_ASSERT_JH(jh, !jh->b_frozen_data);
 
@@ -1216,9 +1230,8 @@ void journal_forget(handle_t *handle, struct buffer_head *bh)
                clear_buffer_jbddirty(bh);
 
                JBUFFER_TRACE(jh, "belongs to current transaction: unfile");
-               J_ASSERT_JH(jh, !jh->b_committed_data);
 
-               __journal_unfile_buffer(jh);
+               drop_reserve = 1;
 
                /* 
                 * We are no longer going to journal this buffer.
@@ -1233,15 +1246,17 @@ void journal_forget(handle_t *handle, struct buffer_head *bh)
                 */
 
                if (jh->b_cp_transaction) {
+                       __journal_temp_unlink_buffer(jh);
                        __journal_file_buffer(jh, transaction, BJ_Forget);
                } else {
+                       __journal_unfile_buffer(jh);
                        journal_remove_journal_head(bh);
                        __brelse(bh);
                        if (!buffer_jbd(bh)) {
                                spin_unlock(&journal->j_list_lock);
                                jbd_unlock_bh_state(bh);
                                __bforget(bh);
-                               return;
+                               goto drop;
                        }
                }
        } else if (jh->b_transaction) {
@@ -1256,6 +1271,7 @@ void journal_forget(handle_t *handle, struct buffer_head *bh)
                if (jh->b_next_transaction) {
                        J_ASSERT(jh->b_next_transaction == transaction);
                        jh->b_next_transaction = NULL;
+                       drop_reserve = 1;
                }
        }
 
@@ -1263,37 +1279,12 @@ not_jbd:
        spin_unlock(&journal->j_list_lock);
        jbd_unlock_bh_state(bh);
        __brelse(bh);
-       return;
-}
-
-/**
- * void journal_callback_set() -  Register a callback function for this handle.
- * @handle: handle to attach the callback to.
- * @func: function to callback.
- * @jcb:  structure with additional information required by func() , and
- *        some space for jbd internal information.
- * 
- * The function will be
- * called when the transaction that this handle is part of has been
- * committed to disk with the original callback data struct and the
- * error status of the journal as parameters.  There is no guarantee of
- * ordering between handles within a single transaction, nor between
- * callbacks registered on the same handle.
- *
- * The caller is responsible for allocating the journal_callback struct.
- * This is to allow the caller to add as much extra data to the callback
- * as needed, but reduce the overhead of multiple allocations.  The caller
- * allocated struct must start with a struct journal_callback at offset 0,
- * and has the caller-specific data afterwards.
- */
-void journal_callback_set(handle_t *handle,
-                       void (*func)(struct journal_callback *jcb, int error),
-                       struct journal_callback *jcb)
-{
-       spin_lock(&handle->h_transaction->t_jcb_lock);
-       list_add_tail(&jcb->jcb_list, &handle->h_jcb);
-       spin_unlock(&handle->h_transaction->t_jcb_lock);
-       jcb->jcb_func = func;
+drop:
+       if (drop_reserve) {
+               /* no need to reserve log space for this block -bzzz */
+               handle->h_buffer_credits++;
+       }
+       return err;
 }
 
 /**
@@ -1317,6 +1308,7 @@ int journal_stop(handle_t *handle)
        transaction_t *transaction = handle->h_transaction;
        journal_t *journal = transaction->t_journal;
        int old_handle_count, err;
+       pid_t pid;
 
        J_ASSERT(transaction->t_updates > 0);
        J_ASSERT(journal_current_handle() == handle);
@@ -1342,12 +1334,18 @@ int journal_stop(handle_t *handle)
         * It doesn't cost much - we're about to run a commit and sleep
         * on IO anyway.  Speeds up many-threaded, many-dir operations
         * by 30x or more...
+        *
+        * But don't do this if this process was the most recent one to
+        * perform a synchronous write.  We do this to detect the case where a
+        * single process is doing a stream of sync writes.  No point in waiting
+        * for joiners in that case.
         */
-       if (handle->h_sync) {
+       pid = current->pid;
+       if (handle->h_sync && journal->j_last_sync_writer != pid) {
+               journal->j_last_sync_writer = pid;
                do {
                        old_handle_count = transaction->t_handle_count;
-                       set_current_state(TASK_UNINTERRUPTIBLE);
-                       schedule_timeout(1);
+                       schedule_timeout_uninterruptible(1);
                } while (old_handle_count != transaction->t_handle_count);
        }
 
@@ -1362,11 +1360,6 @@ int journal_stop(handle_t *handle)
                        wake_up(&journal->j_wait_transaction_locked);
        }
 
-       /* Move callbacks from the handle to the transaction. */
-       spin_lock(&transaction->t_jcb_lock);
-       list_splice(&handle->h_jcb, &transaction->t_jcb);
-       spin_unlock(&transaction->t_jcb_lock);
-
        /*
         * If the handle is marked SYNC, we need to set another commit
         * going!  We also want to force a commit if the current
@@ -1472,7 +1465,7 @@ __blist_del_buffer(struct journal_head **list, struct journal_head *jh)
        if (*list == jh) {
                *list = jh->b_tnext;
                if (*list == jh)
-                       *list = 0;
+                       *list = NULL;
        }
        jh->b_tprev->b_tnext = jh->b_tnext;
        jh->b_tnext->b_tprev = jh->b_tprev;
@@ -1489,9 +1482,9 @@ __blist_del_buffer(struct journal_head **list, struct journal_head *jh)
  *
  * Called under j_list_lock.  The journal may not be locked.
  */
-void __journal_unfile_buffer(struct journal_head *jh)
+void __journal_temp_unlink_buffer(struct journal_head *jh)
 {
-       struct journal_head **list = 0;
+       struct journal_head **list = NULL;
        transaction_t *transaction;
        struct buffer_head *bh = jh2bh(jh);
 
@@ -1506,7 +1499,7 @@ void __journal_unfile_buffer(struct journal_head *jh)
 
        switch (jh->b_jlist) {
        case BJ_None:
-               goto out;
+               return;
        case BJ_SyncData:
                list = &transaction->t_sync_datalist;
                break;
@@ -1539,7 +1532,11 @@ void __journal_unfile_buffer(struct journal_head *jh)
        jh->b_jlist = BJ_None;
        if (test_clear_buffer_jbddirty(bh))
                mark_buffer_dirty(bh);  /* Expose it to the VM */
-out:
+}
+
+void __journal_unfile_buffer(struct journal_head *jh)
+{
+       __journal_temp_unlink_buffer(jh);
        jh->b_transaction = NULL;
 }
 
@@ -1598,7 +1595,7 @@ out:
  * int journal_try_to_free_buffers() - try to free page buffers.
  * @journal: journal for operation
  * @page: to try and free
- * @gfp_mask: 'IO' mode for try_to_free_buffers()
+ * @unused_gfp_mask: unused
  *
  * 
  * For all the buffers on this page,
@@ -1629,7 +1626,7 @@ out:
  * while the data is part of a transaction.  Yes?
  */
 int journal_try_to_free_buffers(journal_t *journal, 
-                               struct page *page, int unused_gfp_mask)
+                               struct page *page, gfp_t unused_gfp_mask)
 {
        struct buffer_head *head;
        struct buffer_head *bh;
@@ -1764,14 +1761,10 @@ static int journal_unmap_buffer(journal_t *journal, struct buffer_head *bh)
        jbd_lock_bh_state(bh);
        spin_lock(&journal->j_list_lock);
 
-       /*
-        * Now we have the locks, check again to see whether kjournald has
-        * taken the buffer off the transaction.
-        */
-       if (!buffer_jbd(bh))
-               goto zap_buffer;
+       jh = journal_grab_journal_head(bh);
+       if (!jh)
+               goto zap_buffer_no_jh;
 
-       jh = bh2jh(bh);
        transaction = jh->b_transaction;
        if (transaction == NULL) {
                /* First case: not on any transaction.  If it
@@ -1799,6 +1792,7 @@ static int journal_unmap_buffer(journal_t *journal, struct buffer_head *bh)
                        JBUFFER_TRACE(jh, "checkpointed: add to BJ_Forget");
                        ret = __dispose_buffer(jh,
                                        journal->j_running_transaction);
+                       journal_put_journal_head(jh);
                        spin_unlock(&journal->j_list_lock);
                        jbd_unlock_bh_state(bh);
                        spin_unlock(&journal->j_state_lock);
@@ -1812,6 +1806,7 @@ static int journal_unmap_buffer(journal_t *journal, struct buffer_head *bh)
                                JBUFFER_TRACE(jh, "give to committing trans");
                                ret = __dispose_buffer(jh,
                                        journal->j_committing_transaction);
+                               journal_put_journal_head(jh);
                                spin_unlock(&journal->j_list_lock);
                                jbd_unlock_bh_state(bh);
                                spin_unlock(&journal->j_state_lock);
@@ -1824,7 +1819,17 @@ static int journal_unmap_buffer(journal_t *journal, struct buffer_head *bh)
                        }
                }
        } else if (transaction == journal->j_committing_transaction) {
-               /* If it is committing, we simply cannot touch it.  We
+               if (jh->b_jlist == BJ_Locked) {
+                       /*
+                        * The buffer is on the committing transaction's locked
+                        * list.  We have the buffer locked, so I/O has
+                        * completed.  So we can nail the buffer now.
+                        */
+                       may_free = __dispose_buffer(jh, transaction);
+                       goto zap_buffer;
+               }
+               /*
+                * If it is committing, we simply cannot touch it.  We
                 * can remove it's next_transaction pointer from the
                 * running transaction if that is set, but nothing
                 * else. */
@@ -1835,6 +1840,7 @@ static int journal_unmap_buffer(journal_t *journal, struct buffer_head *bh)
                                        journal->j_running_transaction);
                        jh->b_next_transaction = NULL;
                }
+               journal_put_journal_head(jh);
                spin_unlock(&journal->j_list_lock);
                jbd_unlock_bh_state(bh);
                spin_unlock(&journal->j_state_lock);
@@ -1851,6 +1857,8 @@ static int journal_unmap_buffer(journal_t *journal, struct buffer_head *bh)
        }
 
 zap_buffer:
+       journal_put_journal_head(jh);
+zap_buffer_no_jh:
        spin_unlock(&journal->j_list_lock);
        jbd_unlock_bh_state(bh);
        spin_unlock(&journal->j_state_lock);
@@ -1865,16 +1873,15 @@ zap_buffer_unlocked:
 }
 
 /** 
- * int journal_invalidatepage() 
+ * void journal_invalidatepage()
  * @journal: journal to use for flush... 
  * @page:    page to flush
  * @offset:  length of page to invalidate.
  *
  * Reap page buffers containing data after offset in page.
  *
- * Return non-zero if the page's buffers were successfully reaped.
  */
-int journal_invalidatepage(journal_t *journal, 
+void journal_invalidatepage(journal_t *journal,
                      struct page *page, 
                      unsigned long offset)
 {
@@ -1885,7 +1892,7 @@ int journal_invalidatepage(journal_t *journal,
        if (!PageLocked(page))
                BUG();
        if (!page_has_buffers(page))
-               return 1;
+               return;
 
        /* We will potentially be playing with lists other than just the
         * data lists (especially for journaled data mode), so be
@@ -1896,7 +1903,6 @@ int journal_invalidatepage(journal_t *journal,
                unsigned int next_off = curr_off + bh->b_size;
                next = bh->b_this_page;
 
-               /* AKPM: doing lock_buffer here may be overly paranoid */
                if (offset <= curr_off) {
                        /* This block is wholly outside the truncation point */
                        lock_buffer(bh);
@@ -1909,11 +1915,9 @@ int journal_invalidatepage(journal_t *journal,
        } while (bh != head);
 
        if (!offset) {
-               if (!may_free || !try_to_free_buffers(page))
-                       return 0;
-               J_ASSERT(!page_has_buffers(page));
+               if (may_free && try_to_free_buffers(page))
+                       J_ASSERT(!page_has_buffers(page));
        }
-       return 1;
 }
 
 /* 
@@ -1922,7 +1926,7 @@ int journal_invalidatepage(journal_t *journal,
 void __journal_file_buffer(struct journal_head *jh,
                        transaction_t *transaction, int jlist)
 {
-       struct journal_head **list = 0;
+       struct journal_head **list = NULL;
        int was_dirty = 0;
        struct buffer_head *bh = jh2bh(jh);
 
@@ -1948,7 +1952,7 @@ void __journal_file_buffer(struct journal_head *jh,
        }
 
        if (jh->b_transaction)
-               __journal_unfile_buffer(jh);
+               __journal_temp_unlink_buffer(jh);
        jh->b_transaction = transaction;
 
        switch (jlist) {
@@ -2031,7 +2035,7 @@ void __journal_refile_buffer(struct journal_head *jh)
         */
 
        was_dirty = test_clear_buffer_jbddirty(bh);
-       __journal_unfile_buffer(jh);
+       __journal_temp_unlink_buffer(jh);
        jh->b_transaction = jh->b_next_transaction;
        jh->b_next_transaction = NULL;
        __journal_file_buffer(jh, jh->b_transaction, BJ_Metadata);