

This was a system-wide spinlock.

Simpe transformation: make it a filesystem-wide spinlock, in the JBD journal.

That's a bit lame, and later it might be nice to make it per-trasnaction_t. 
But there are interesting ranking and ordering problems with that, especially
around __journal_refile_buffer().


 fs/jbd/checkpoint.c  |   66 +++++++++++-------------------
 fs/jbd/commit.c      |   48 ++++++++++------------
 fs/jbd/journal.c     |   54 ------------------------
 fs/jbd/transaction.c |  111 ++++++++++++++++++++++-----------------------------
 include/linux/jbd.h  |   13 +----
 5 files changed, 101 insertions(+), 191 deletions(-)

diff -puN fs/jbd/checkpoint.c~jbd-100-remove-journal_datalist_lock fs/jbd/checkpoint.c
--- 25/fs/jbd/checkpoint.c~jbd-100-remove-journal_datalist_lock	2003-05-25 00:21:11.000000000 -0700
+++ 25-akpm/fs/jbd/checkpoint.c	2003-05-25 00:21:11.000000000 -0700
@@ -23,12 +23,10 @@
 #include <linux/errno.h>
 #include <linux/slab.h>
 
-extern spinlock_t journal_datalist_lock;
-
 /*
  * Unlink a buffer from a transaction. 
  *
- * Called with journal_datalist_lock held.
+ * Called with j_list_lock held.
  */
 
 static inline void __buffer_unlink(struct journal_head *jh)
@@ -49,7 +47,7 @@ static inline void __buffer_unlink(struc
 /*
  * Try to release a checkpointed buffer from its transaction.
  * Returns 1 if we released it.
- * Requires journal_datalist_lock
+ * Requires j_list_lock
  * Called under jbd_lock_bh_state(jh2bh(jh)), and drops it
  */
 static int __try_to_free_cp_buf(struct journal_head *jh)
@@ -107,7 +105,7 @@ void log_wait_for_space(journal_t *journ
  * the last checkpoint buffer is cleansed)
  *
  * Called with the journal locked.
- * Called with journal_datalist_lock held.
+ * Called with j_list_lock held.
  */
 static int __cleanup_transaction(journal_t *journal, transaction_t *transaction)
 {
@@ -115,7 +113,7 @@ static int __cleanup_transaction(journal
 	struct buffer_head *bh;
 	int ret = 0;
 
-	assert_spin_locked(&journal_datalist_lock);
+	assert_spin_locked(&journal->j_list_lock);
 	jh = transaction->t_checkpoint_list;
 	if (!jh)
 		return 0;
@@ -127,7 +125,7 @@ static int __cleanup_transaction(journal
 		bh = jh2bh(jh);
 		if (buffer_locked(bh)) {
 			atomic_inc(&bh->b_count);
-			spin_unlock(&journal_datalist_lock);
+			spin_unlock(&journal->j_list_lock);
 			unlock_journal(journal);
 			wait_on_buffer(bh);
 			/* the journal_head may have gone by now */
@@ -142,7 +140,7 @@ static int __cleanup_transaction(journal
 			tid_t tid = transaction->t_tid;
 
 			jbd_unlock_bh_state(bh);
-			spin_unlock(&journal_datalist_lock);
+			spin_unlock(&journal->j_list_lock);
 			log_start_commit(journal, transaction);
 			unlock_journal(journal);
 			log_wait_commit(journal, tid);
@@ -175,20 +173,20 @@ static int __cleanup_transaction(journal
 	return ret;
 out_return_1:
 	lock_journal(journal);
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	return 1;
 }
 
 #define NR_BATCH	64
 
-static void __flush_batch(struct buffer_head **bhs, int *batch_count)
+static void
+__flush_batch(journal_t *journal, struct buffer_head **bhs, int *batch_count)
 {
 	int i;
 
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	ll_rw_block(WRITE, *batch_count, bhs);
-	blk_run_queues();
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	for (i = 0; i < *batch_count; i++) {
 		struct buffer_head *bh = bhs[i];
 		clear_bit(BH_JWrite, &bh->b_state);
@@ -204,7 +202,7 @@ static void __flush_batch(struct buffer_
  * Return 1 if something happened which requires us to abort the current
  * scan of the checkpoint list.  
  *
- * Called with journal_datalist_lock held.
+ * Called with j_list_lock held.
  * Called under jbd_lock_bh_state(jh2bh(jh)), and drops it
  */
 static int __flush_buffer(journal_t *journal, struct journal_head *jh,
@@ -232,7 +230,7 @@ static int __flush_buffer(journal_t *jou
 		jbd_unlock_bh_state(bh);
 		(*batch_count)++;
 		if (*batch_count == NR_BATCH) {
-			__flush_batch(bhs, batch_count);
+			__flush_batch(journal, bhs, batch_count);
 			ret = 1;
 		}
 	} else {
@@ -266,7 +264,7 @@ static int __flush_buffer(journal_t *jou
  */
 
 /* @@@ `nblocks' is unused.  Should it be used? */
-int log_do_checkpoint (journal_t *journal, int nblocks)
+int log_do_checkpoint(journal_t *journal, int nblocks)
 {
 	transaction_t *transaction, *last_transaction, *next_transaction;
 	int result;
@@ -293,7 +291,7 @@ int log_do_checkpoint (journal_t *journa
 	 * AKPM: check this code.  I had a feeling a while back that it
 	 * degenerates into a busy loop at unmount time.
 	 */
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 repeat:
 	transaction = journal->j_checkpoint_transactions;
 	if (transaction == NULL)
@@ -319,7 +317,7 @@ repeat:
 						&drop_count);
 		} while (jh != last_jh && !retry);
 		if (batch_count) {
-			__flush_batch(bhs, &batch_count);
+			__flush_batch(journal, bhs, &batch_count);
 			goto repeat;
 		}
 		if (retry)
@@ -335,7 +333,7 @@ repeat:
 	} while (transaction != last_transaction);
 
 done:
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	result = cleanup_journal_tail(journal);
 	if (result < 0)
 		return result;
@@ -374,7 +372,7 @@ int cleanup_journal_tail(journal_t *jour
 	 * start. */
 
 	/* j_checkpoint_transactions needs locking */
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	transaction = journal->j_checkpoint_transactions;
 	if (transaction) {
 		first_tid = transaction->t_tid;
@@ -389,7 +387,7 @@ int cleanup_journal_tail(journal_t *jour
 		first_tid = journal->j_transaction_sequence;
 		blocknr = journal->j_head;
 	}
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	J_ASSERT (blocknr != 0);
 
 	/* If the oldest pinned transaction is at the tail of the log
@@ -426,7 +424,7 @@ int cleanup_journal_tail(journal_t *jour
  * Find all the written-back checkpoint buffers in the journal and release them.
  *
  * Called with the journal locked.
- * Called with journal_datalist_lock held.
+ * Called with j_list_lock held.
  * Returns number of bufers reaped (for debug)
  */
 
@@ -477,7 +475,7 @@ out:
  * checkpoint list.  
  *
  * This function is called with the journal locked.
- * This function is called with journal_datalist_lock held.
+ * This function is called with j_list_lock held.
  */
 
 void __journal_remove_checkpoint(struct journal_head *jh)
@@ -522,20 +520,13 @@ out:
 	JBUFFER_TRACE(jh, "exit");
 }
 
-void journal_remove_checkpoint(struct journal_head *jh)
-{
-	spin_lock(&journal_datalist_lock);
-	__journal_remove_checkpoint(jh);
-	spin_unlock(&journal_datalist_lock);
-}
-
 /*
  * journal_insert_checkpoint: put a committed buffer onto a checkpoint
  * list so that we know when it is safe to clean the transaction out of
  * the log.
  *
  * Called with the journal locked.
- * Called with journal_datalist_lock held.
+ * Called with j_list_lock held.
  */
 void __journal_insert_checkpoint(struct journal_head *jh, 
 			       transaction_t *transaction)
@@ -544,7 +535,6 @@ void __journal_insert_checkpoint(struct 
 	J_ASSERT_JH(jh, buffer_dirty(jh2bh(jh)) || buffer_jbddirty(jh2bh(jh)));
 	J_ASSERT_JH(jh, jh->b_cp_transaction == NULL);
 
-	assert_spin_locked(&journal_datalist_lock);
 	jh->b_cp_transaction = transaction;
 
 	if (!transaction->t_checkpoint_list) {
@@ -558,14 +548,6 @@ void __journal_insert_checkpoint(struct 
 	transaction->t_checkpoint_list = jh;
 }
 
-void journal_insert_checkpoint(struct journal_head *jh, 
-			       transaction_t *transaction)
-{
-	spin_lock(&journal_datalist_lock);
-	__journal_insert_checkpoint(jh, transaction);
-	spin_unlock(&journal_datalist_lock);
-}
-
 /*
  * We've finished with this transaction structure: adios...
  * 
@@ -573,12 +555,12 @@ void journal_insert_checkpoint(struct jo
  * point.
  *
  * Called with the journal locked.
- * Called with journal_datalist_lock held.
+ * Called with j_list_lock held.
  */
 
 void __journal_drop_transaction(journal_t *journal, transaction_t *transaction)
 {
-	assert_spin_locked(&journal_datalist_lock);
+	assert_spin_locked(&journal->j_list_lock);
 	if (transaction->t_cpnext) {
 		transaction->t_cpnext->t_cpprev = transaction->t_cpprev;
 		transaction->t_cpprev->t_cpnext = transaction->t_cpnext;
diff -puN fs/jbd/commit.c~jbd-100-remove-journal_datalist_lock fs/jbd/commit.c
--- 25/fs/jbd/commit.c~jbd-100-remove-journal_datalist_lock	2003-05-25 00:21:11.000000000 -0700
+++ 25-akpm/fs/jbd/commit.c	2003-05-25 00:21:11.000000000 -0700
@@ -20,8 +20,6 @@
 #include <linux/slab.h>
 #include <linux/smp_lock.h>
 
-extern spinlock_t journal_datalist_lock;
-
 /*
  * Default IO end handler for temporary BJ_IO buffer_heads.
  */
@@ -67,9 +65,9 @@ void journal_commit_transaction(journal_
 	lock_journal(journal); /* Protect journal->j_running_transaction */
 
 #ifdef COMMIT_STATS
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	summarise_journal_usage(journal);
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 #endif
 
 	lock_kernel();
@@ -121,7 +119,7 @@ void journal_commit_transaction(journal_
 	while (commit_transaction->t_reserved_list) {
 		jh = commit_transaction->t_reserved_list;
 		JBUFFER_TRACE(jh, "reserved, unused: refile");
-		journal_refile_buffer(jh);
+		journal_refile_buffer(journal, jh);
 	}
 
 	/*
@@ -129,9 +127,9 @@ void journal_commit_transaction(journal_
 	 * checkpoint lists.  We do this *before* commit because it potentially
 	 * frees some memory
 	 */
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	__journal_clean_checkpoint_list(journal);
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 
 	/* First part of the commit: force the revoke list out to disk.
 	 * The revoke code generates its own metadata blocks on disk for this.
@@ -185,10 +183,10 @@ write_out_data:
 	 * Cleanup any flushed data buffers from the data list.  Even in
 	 * abort mode, we want to flush this out as soon as possible.
 	 *
-	 * We take journal_datalist_lock to protect the lists from
+	 * We take j_list_lock to protect the lists from
 	 * journal_try_to_free_buffers().
 	 */
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 
 write_out_data_locked:
 	bufs = 0;
@@ -214,9 +212,9 @@ write_out_data_locked:
 				 * We have a lock ranking problem..
 				 */
 				if (!jbd_trylock_bh_state(bh)) {
-					spin_unlock(&journal_datalist_lock);
+					spin_unlock(&journal->j_list_lock);
 					schedule();
-					spin_lock(&journal_datalist_lock);
+					spin_lock(&journal->j_list_lock);
 					break;
 				}
 				__journal_unfile_buffer(jh);
@@ -238,14 +236,14 @@ write_out_data_locked:
 
 	if (bufs || need_resched()) {
 		jbd_debug(2, "submit %d writes\n", bufs);
-		spin_unlock(&journal_datalist_lock);
+		spin_unlock(&journal->j_list_lock);
 		unlock_journal(journal);
 		if (bufs)
 			ll_rw_block(WRITE, bufs, wbuf);
 		cond_resched();
 		journal_brelse_array(wbuf, bufs);
 		lock_journal(journal);
-		spin_lock(&journal_datalist_lock);
+		spin_lock(&journal->j_list_lock);
 		if (bufs)
 			goto write_out_data_locked;
 	}
@@ -263,7 +261,7 @@ write_out_data_locked:
 		bh = jh2bh(jh);
 		if (buffer_locked(bh)) {
 			get_bh(bh);
-			spin_unlock(&journal_datalist_lock);
+			spin_unlock(&journal->j_list_lock);
 			unlock_journal(journal);
 			wait_on_buffer(bh);
 			if (unlikely(!buffer_uptodate(bh)))
@@ -279,7 +277,7 @@ write_out_data_locked:
 	goto write_out_data_locked;
 
 sync_datalist_empty:
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 
 	/*
 	 * If we found any dirty or locked buffers, then we should have
@@ -311,7 +309,7 @@ sync_datalist_empty:
 
 		if (is_journal_aborted(journal)) {
 			JBUFFER_TRACE(jh, "journal is aborting: refile");
-			journal_refile_buffer(jh);
+			journal_refile_buffer(journal, jh);
 			/* If that was the last one, we need to clean up
 			 * any descriptor buffers which may have been
 			 * already allocated, even if we are now
@@ -355,7 +353,7 @@ sync_datalist_empty:
                            completion later */
 			BUFFER_TRACE(bh, "ph3: file as descriptor");
 			journal_file_buffer(descriptor, commit_transaction,
-						BJ_LogCtl);
+					BJ_LogCtl);
 		}
 
 		/* Where is the buffer to be written? */
@@ -462,7 +460,7 @@ start_journal_io:
 	jbd_debug(3, "JBD: commit phase 4\n");
 
 	/*
-	 * akpm: these are BJ_IO, and journal_datalist_lock is not needed.
+	 * akpm: these are BJ_IO, and j_list_lock is not needed.
 	 * See __journal_try_to_free_buffer.
 	 */
 wait_for_iobuf:
@@ -483,7 +481,7 @@ wait_for_iobuf:
 		clear_buffer_jwrite(bh);
 
 		JBUFFER_TRACE(jh, "ph4: unfile after journal write");
-		journal_unfile_buffer(jh);
+		journal_unfile_buffer(journal, jh);
 
 		/*
 		 * akpm: don't put back a buffer_head with stale pointers
@@ -543,8 +541,8 @@ wait_for_iobuf:
 		}
 
 		BUFFER_TRACE(bh, "ph5: control buffer writeout done: unfile");
-		clear_bit(BH_JWrite, &bh->b_state);
-		journal_unfile_buffer(jh);
+		clear_buffer_jwrite(bh);
+		journal_unfile_buffer(journal, jh);
 		jh->b_transaction = NULL;
 		journal_put_journal_head(jh);
 		__brelse(bh);		/* One for getblk */
@@ -664,7 +662,7 @@ skip_commit: /* The journal should be un
 			jh->b_frozen_data = NULL;
 		}
 
-		spin_lock(&journal_datalist_lock);
+		spin_lock(&journal->j_list_lock);
 		cp_transaction = jh->b_cp_transaction;
 		if (cp_transaction) {
 			JBUFFER_TRACE(jh, "remove from old cp transaction");
@@ -706,7 +704,7 @@ skip_commit: /* The journal should be un
 			journal_remove_journal_head(bh);
 			__brelse(bh);
 		}
-		spin_unlock(&journal_datalist_lock);
+		spin_unlock(&journal->j_list_lock);
 	}
 
 	/* Done with this transaction! */
@@ -720,7 +718,7 @@ skip_commit: /* The journal should be un
 	journal->j_commit_sequence = commit_transaction->t_tid;
 	journal->j_committing_transaction = NULL;
 
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	if (commit_transaction->t_checkpoint_list == NULL) {
 		__journal_drop_transaction(journal, commit_transaction);
 	} else {
@@ -739,7 +737,7 @@ skip_commit: /* The journal should be un
 				commit_transaction;
 		}
 	}
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 
 	jbd_debug(1, "JBD: commit %d complete, head %d\n",
 		  journal->j_commit_sequence, journal->j_tail_sequence);
diff -puN fs/jbd/journal.c~jbd-100-remove-journal_datalist_lock fs/jbd/journal.c
--- 25/fs/jbd/journal.c~jbd-100-remove-journal_datalist_lock	2003-05-25 00:21:11.000000000 -0700
+++ 25-akpm/fs/jbd/journal.c	2003-05-25 00:21:11.000000000 -0700
@@ -84,39 +84,6 @@ EXPORT_SYMBOL(journal_force_commit);
 static int journal_convert_superblock_v1(journal_t *, journal_superblock_t *);
 
 /*
- * journal_datalist_lock is used to protect data buffers:
- *
- *	bh->b_transaction
- *	bh->b_tprev
- *	bh->b_tnext
- *
- * journal_free_buffer() is called from journal_try_to_free_buffer(), and is
- * async wrt everything else.
- *
- * It is also used for checkpoint data, also to protect against
- * journal_try_to_free_buffer():
- *
- *	bh->b_cp_transaction
- *	bh->b_cpnext
- *	bh->b_cpprev
- *	transaction->t_checkpoint_list
- *	transaction->t_cpnext
- *	transaction->t_cpprev
- *	journal->j_checkpoint_transactions
- *
- * It is global at this time rather than per-journal because it's
- * impossible for __journal_free_buffer to go from a buffer_head
- * back to a journal_t unracily (well, not true.  Fix later)
- *
- *
- * The `datalist' and `checkpoint list' functions are quite
- * separate and we could use two spinlocks here.
- *
- * lru_list_lock nests inside journal_datalist_lock.
- */
-spinlock_t journal_datalist_lock = SPIN_LOCK_UNLOCKED;
-
-/*
  * List of all journals in the system.  Protected by the BKL.
  */
 static LIST_HEAD(all_journals);
@@ -1516,26 +1483,6 @@ int journal_blocks_per_page(struct inode
 }
 
 /*
- * shrink_journal_memory().
- * Called when we're under memory pressure.  Free up all the written-back
- * checkpointed metadata buffers.
- */
-void shrink_journal_memory(void)
-{
-	struct list_head *list;
-
-	lock_kernel();
-	list_for_each(list, &all_journals) {
-		journal_t *journal =
-			list_entry(list, journal_t, j_all_journals);
-		spin_lock(&journal_datalist_lock);
-		__journal_clean_checkpoint_list(journal);
-		spin_unlock(&journal_datalist_lock);
-	}
-	unlock_kernel();
-}
-
-/*
  * Simple support for retying memory allocations.  Introduced to help to
  * debug different VM deadlock avoidance strategies. 
  */
@@ -1660,7 +1607,6 @@ static void journal_free_journal_head(st
  *
  * Doesn't need the journal lock.
  * May sleep.
- * Cannot be called with journal_datalist_lock held.
  */
 struct journal_head *journal_add_journal_head(struct buffer_head *bh)
 {
diff -puN fs/jbd/transaction.c~jbd-100-remove-journal_datalist_lock fs/jbd/transaction.c
--- 25/fs/jbd/transaction.c~jbd-100-remove-journal_datalist_lock	2003-05-25 00:21:11.000000000 -0700
+++ 25-akpm/fs/jbd/transaction.c	2003-05-25 00:21:11.000000000 -0700
@@ -27,8 +27,6 @@
 #include <linux/mm.h>
 #include <linux/highmem.h>
 
-extern spinlock_t journal_datalist_lock;
-
 /*
  * get_transaction: obtain a new transaction_t object.
  *
@@ -457,7 +455,7 @@ void journal_unlock_updates (journal_t *
  * continuing as gracefully as possible.  #
  *
  * The caller should already hold the journal lock and
- * journal_datalist_lock spinlock: most callers will need those anyway
+ * j_list_lock spinlock: most callers will need those anyway
  * in order to probe the buffer's journaling state safely.
  */
 static void jbd_unexpected_dirty_buffer(struct journal_head *jh)
@@ -513,7 +511,7 @@ repeat:
 
 	lock_buffer(bh);
 	jbd_lock_bh_state(bh);
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 
 	/* We now hold the buffer lock so it is safe to query the buffer
 	 * state.  Is the buffer dirty? 
@@ -549,7 +547,7 @@ repeat:
 
 	error = -EROFS;
 	if (is_handle_aborted(handle)) {
-		spin_unlock(&journal_datalist_lock);
+		spin_unlock(&journal->j_list_lock);
 		jbd_unlock_bh_state(bh);
 		goto out_unlocked;
 	}
@@ -596,7 +594,7 @@ repeat:
 			wait_queue_head_t *wqh;
 
 			JBUFFER_TRACE(jh, "on shadow: sleep");
-			spin_unlock(&journal_datalist_lock);
+			spin_unlock(&journal->j_list_lock);
 			jbd_unlock_bh_state(bh);
 			unlock_journal(journal);
 			/* commit wakes up all shadow buffers after IO */
@@ -624,7 +622,7 @@ repeat:
 			JBUFFER_TRACE(jh, "generate frozen data");
 			if (!frozen_buffer) {
 				JBUFFER_TRACE(jh, "allocate memory for buffer");
-				spin_unlock(&journal_datalist_lock);
+				spin_unlock(&journal->j_list_lock);
 				jbd_unlock_bh_state(bh);
 				frozen_buffer = jbd_kmalloc(jh2bh(jh)->b_size,
 							    GFP_NOFS);
@@ -635,7 +633,7 @@ repeat:
 					JBUFFER_TRACE(jh, "oom!");
 					error = -ENOMEM;
 					jbd_lock_bh_state(bh);
-					spin_lock(&journal_datalist_lock);
+					spin_lock(&journal->j_list_lock);
 					goto done_locked;
 				}
 				goto repeat;
@@ -677,7 +675,7 @@ done_locked:
 		memcpy(jh->b_frozen_data, source+offset, jh2bh(jh)->b_size);
 		kunmap_atomic(source, KM_USER0);
 	}
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	jbd_unlock_bh_state(bh);
 
 	/* If we are about to journal a buffer, then any revoke pending
@@ -763,7 +761,7 @@ int journal_get_create_access(handle_t *
 	 * reused here.
 	 */
 	jbd_lock_bh_state(bh);
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	J_ASSERT_JH(jh, (jh->b_transaction == transaction ||
 		jh->b_transaction == NULL ||
 		(jh->b_transaction == journal->j_committing_transaction &&
@@ -783,7 +781,7 @@ int journal_get_create_access(handle_t *
 		JBUFFER_TRACE(jh, "set next transaction");
 		jh->b_next_transaction = transaction;
 	}
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	jbd_unlock_bh_state(bh);
 
 	/*
@@ -936,7 +934,7 @@ int journal_dirty_data (handle_t *handle
 	 * about it in this layer.
 	 */
 	jbd_lock_bh_state(bh);
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	if (jh->b_transaction) {
 		JBUFFER_TRACE(jh, "has transaction");
 		if (jh->b_transaction != handle->h_transaction) {
@@ -991,12 +989,12 @@ int journal_dirty_data (handle_t *handle
 			 */
 			if (buffer_dirty(bh)) {
 				get_bh(bh);
-				spin_unlock(&journal_datalist_lock);
+				spin_unlock(&journal->j_list_lock);
 				jbd_unlock_bh_state(bh);
 				need_brelse = 1;
 				sync_dirty_buffer(bh);
 				jbd_lock_bh_state(bh);
-				spin_lock(&journal_datalist_lock);
+				spin_lock(&journal->j_list_lock);
 				/* The buffer may become locked again at any
 				   time if it is redirtied */
 			}
@@ -1030,7 +1028,7 @@ int journal_dirty_data (handle_t *handle
 		__journal_file_buffer(jh, handle->h_transaction, BJ_SyncData);
 	}
 no_journal:
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	jbd_unlock_bh_state(bh);
 	if (need_brelse) {
 		BUFFER_TRACE(bh, "brelse");
@@ -1088,7 +1086,7 @@ int journal_dirty_metadata(handle_t *han
 		goto out_unlock;
 	
 	jbd_lock_bh_state(bh);
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	set_buffer_jbddirty(bh);
 
 	J_ASSERT_JH(jh, jh->b_transaction != NULL);
@@ -1118,7 +1116,7 @@ int journal_dirty_metadata(handle_t *han
 	__journal_file_buffer(jh, handle->h_transaction, BJ_Metadata);
 
 done_locked:
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	jbd_unlock_bh_state(bh);
 	JBUFFER_TRACE(jh, "exit");
 out_unlock:
@@ -1149,14 +1147,14 @@ void journal_release_buffer(handle_t *ha
 	 * cases, just leave the buffer as it is. */
 
 	jbd_lock_bh_state(bh);
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	if (jh->b_jlist == BJ_Reserved && jh->b_transaction == transaction &&
 	    !buffer_jbddirty(jh2bh(jh))) {
 		JBUFFER_TRACE(jh, "unused: refiling it");
 		handle->h_buffer_credits++;
 		__journal_refile_buffer(jh);
 	}
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	jbd_unlock_bh_state(bh);
 
 	JBUFFER_TRACE(jh, "exit");
@@ -1190,7 +1188,7 @@ void journal_forget (handle_t *handle, s
 
 	lock_journal(journal);
 	jbd_lock_bh_state(bh);
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 
 	if (!buffer_jbd(bh))
 		goto not_jbd;
@@ -1229,7 +1227,7 @@ void journal_forget (handle_t *handle, s
 			journal_remove_journal_head(bh);
 			__brelse(bh);
 			if (!buffer_jbd(bh)) {
-				spin_unlock(&journal_datalist_lock);
+				spin_unlock(&journal->j_list_lock);
 				jbd_unlock_bh_state(bh);
 				unlock_journal(journal);
 				__bforget(bh);
@@ -1253,7 +1251,7 @@ void journal_forget (handle_t *handle, s
 	}
 
 not_jbd:
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	jbd_unlock_bh_state(bh);
 	unlock_journal(journal);
 	__brelse(bh);
@@ -1430,7 +1428,7 @@ out:
  * Append a buffer to a transaction list, given the transaction's list head
  * pointer.
  *
- * journal_datalist_lock is held.
+ * j_list_lock is held.
  *
  * jbd_lock_bh_state(jh2bh(jh)) is held.
  */
@@ -1454,8 +1452,7 @@ __blist_add_buffer(struct journal_head *
  * Remove a buffer from a transaction list, given the transaction's list
  * head pointer.
  *
- * Called with journal_datalist_lock held, and the journal may not
- * be locked.
+ * Called with j_list_lock held, and the journal may not be locked.
  *
  * jbd_lock_bh_state(jh2bh(jh)) is held.
  */
@@ -1483,7 +1480,7 @@ __blist_del_buffer(struct journal_head *
  *
  * If bh->b_jlist is BJ_SyncData then we may have been called
  * via journal_try_to_free_buffer() or journal_clean_data_list().  In that
- * case, journal_datalist_lock will be held, and the journal may not be locked.
+ * case, j_list_lock will be held, and the journal may not be locked.
  */
 void __journal_unfile_buffer(struct journal_head *jh)
 {
@@ -1491,7 +1488,6 @@ void __journal_unfile_buffer(struct jour
 	transaction_t *transaction;
 	struct buffer_head *bh = jh2bh(jh);
 
-	assert_spin_locked(&journal_datalist_lock);
 	WARN_ON(!buffer_state(bh));
 	transaction = jh->b_transaction;
 
@@ -1533,31 +1529,24 @@ void __journal_unfile_buffer(struct jour
 		mark_buffer_dirty(bh);	/* Expose it to the VM */
 }
 
-void journal_unfile_buffer(struct journal_head *jh)
+void journal_unfile_buffer(journal_t *journal, struct journal_head *jh)
 {
 	jbd_lock_bh_state(jh2bh(jh));
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	__journal_unfile_buffer(jh);
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	jbd_unlock_bh_state(jh2bh(jh));
 }
 
 /*
- * Called from journal_try_to_free_buffers().  The journal is not
- * locked. lru_list_lock is not held.
- *
- * Here we see why journal_datalist_lock is global and not per-journal.
- * We cannot get back to this buffer's journal pointer without locking
- * out journal_clean_data_list() in some manner.
- *
- * One could use journal_datalist_lock to get unracy access to a
- * per-journal lock.
+ * Called from journal_try_to_free_buffers().
  *
  * Called under jbd_lock_bh_state(bh)
  *
  * Returns non-zero iff we were able to free the journal_head.
  */
-static inline int __journal_try_to_free_buffer(struct buffer_head *bh)
+static inline int
+__journal_try_to_free_buffer(journal_t *journal, struct buffer_head *bh)
 {
 	struct journal_head *jh;
 
@@ -1569,7 +1558,7 @@ static inline int __journal_try_to_free_
 	if (jh->b_next_transaction != 0)
 		goto out;
 
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	if (jh->b_transaction != 0 && jh->b_cp_transaction == 0) {
 		if (jh->b_jlist == BJ_SyncData) {
 			/* A written-back ordered data buffer */
@@ -1588,7 +1577,7 @@ static inline int __journal_try_to_free_
 			__brelse(bh);
 		}
 	}
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	return !buffer_jbd(bh);
 
 out:
@@ -1643,7 +1632,8 @@ int journal_try_to_free_buffers(journal_
 	bh = head;
 	do {
 		jbd_lock_bh_state(bh);
-		if (buffer_jbd(bh) && !__journal_try_to_free_buffer(bh)) {
+		if (buffer_jbd(bh) &&
+				!__journal_try_to_free_buffer(journal, bh)) {
 			jbd_unlock_bh_state(bh);
 			goto busy;
 		}
@@ -1662,7 +1652,7 @@ busy:
  * release it.
  * Returns non-zero if JBD no longer has an interest in the buffer.
  *
- * Called under journal_datalist_lock.
+ * Called under j_list_lock.
  *
  * Called under jbd_lock_bh_state(bh).
  */
@@ -1743,16 +1733,17 @@ static int journal_unmap_buffer(journal_
 
 	BUFFER_TRACE(bh, "entry");
 
-	/* It is safe to proceed here without the
-	 * journal_datalist_spinlock because the buffers cannot be
-	 * stolen by try_to_free_buffers as long as we are holding the
-	 * page lock. --sct */
+	/*
+	 * It is safe to proceed here without the j_list_lock because the
+	 * buffers cannot be stolen by try_to_free_buffers as long as we are
+	 * holding the page lock. --sct
+	 */
 
 	if (!buffer_jbd(bh))
 		goto zap_buffer_unlocked;
 
 	jbd_lock_bh_state(bh);
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	jh = bh2jh(bh);
 	transaction = jh->b_transaction;
 	if (transaction == NULL) {
@@ -1781,7 +1772,7 @@ static int journal_unmap_buffer(journal_
 			JBUFFER_TRACE(jh, "checkpointed: add to BJ_Forget");
 			ret = __dispose_buffer(jh,
 					journal->j_running_transaction);
-			spin_unlock(&journal_datalist_lock);
+			spin_unlock(&journal->j_list_lock);
 			jbd_unlock_bh_state(bh);
 			return ret;
 		} else {
@@ -1793,7 +1784,7 @@ static int journal_unmap_buffer(journal_
 				JBUFFER_TRACE(jh, "give to committing trans");
 				ret = __dispose_buffer(jh,
 					journal->j_committing_transaction);
-				spin_unlock(&journal_datalist_lock);
+				spin_unlock(&journal->j_list_lock);
 				jbd_unlock_bh_state(bh);
 				return ret;
 			} else {
@@ -1815,7 +1806,7 @@ static int journal_unmap_buffer(journal_
 					journal->j_running_transaction);
 			jh->b_next_transaction = NULL;
 		}
-		spin_unlock(&journal_datalist_lock);
+		spin_unlock(&journal->j_list_lock);
 		jbd_unlock_bh_state(bh);
 		return 0;
 	} else {
@@ -1830,7 +1821,7 @@ static int journal_unmap_buffer(journal_
 	}
 
 zap_buffer:
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	jbd_unlock_bh_state(bh);
 zap_buffer_unlocked:
 	clear_buffer_dirty(bh);
@@ -1907,7 +1898,6 @@ void __journal_file_buffer(struct journa
 	int was_dirty = 0;
 	struct buffer_head *bh = jh2bh(jh);
 
-	assert_spin_locked(&journal_datalist_lock);
 	WARN_ON(!buffer_state(bh));
 
 #ifdef __SMP__
@@ -1976,9 +1966,9 @@ void journal_file_buffer(struct journal_
 				transaction_t *transaction, int jlist)
 {
 	jbd_lock_bh_state(jh2bh(jh));
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&transaction->t_journal->j_list_lock);
 	__journal_file_buffer(jh, transaction, jlist);
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&transaction->t_journal->j_list_lock);
 	jbd_unlock_bh_state(jh2bh(jh));
 }
 
@@ -1988,7 +1978,7 @@ void journal_file_buffer(struct journal_
  * already started to be used by a subsequent transaction, refile the
  * buffer on that transaction's metadata list.
  *
- * Called under journal_datalist_lock
+ * Called under journal->j_list_lock
  *
  * Called under jbd_lock_bh_state(jh2bh(jh))
  */
@@ -1997,7 +1987,6 @@ void __journal_refile_buffer(struct jour
 	int was_dirty;
 	struct buffer_head *bh = jh2bh(jh);
 
-	assert_spin_locked(&journal_datalist_lock);
 	WARN_ON(!buffer_state(bh));
 
 	/* If the buffer is now unused, just drop it. */
@@ -2037,16 +2026,16 @@ void __journal_refile_buffer(struct jour
  *
  * *** The journal_head may be freed by this call! ***
  */
-void journal_refile_buffer(struct journal_head *jh)
+void journal_refile_buffer(journal_t *journal, struct journal_head *jh)
 {
 	struct buffer_head *bh;
 
-	spin_lock(&journal_datalist_lock);
+	spin_lock(&journal->j_list_lock);
 	bh = jh2bh(jh);
 
 	__journal_refile_buffer(jh);
 	journal_remove_journal_head(bh);
 
-	spin_unlock(&journal_datalist_lock);
+	spin_unlock(&journal->j_list_lock);
 	__brelse(bh);
 }
diff -puN include/linux/jbd.h~jbd-100-remove-journal_datalist_lock include/linux/jbd.h
--- 25/include/linux/jbd.h~jbd-100-remove-journal_datalist_lock	2003-05-25 00:21:11.000000000 -0700
+++ 25-akpm/include/linux/jbd.h	2003-05-25 00:21:11.000000000 -0700
@@ -838,10 +838,10 @@ struct journal_s
  */
 
 /* Filing buffers */
+extern void journal_unfile_buffer(journal_t *, struct journal_head *);
 extern void __journal_unfile_buffer(struct journal_head *);
-extern void journal_unfile_buffer(struct journal_head *);
 extern void __journal_refile_buffer(struct journal_head *);
-extern void journal_refile_buffer(struct journal_head *);
+extern void journal_refile_buffer(journal_t *, struct journal_head *);
 extern void __journal_file_buffer(struct journal_head *, transaction_t *, int);
 extern void __journal_free_buffer(struct journal_head *bh);
 extern void journal_file_buffer(struct journal_head *, transaction_t *, int);
@@ -856,10 +856,8 @@ extern void journal_commit_transaction(j
 
 /* Checkpoint list management */
 int __journal_clean_checkpoint_list(journal_t *journal);
-extern void journal_remove_checkpoint(struct journal_head *);
-extern void __journal_remove_checkpoint(struct journal_head *);
-extern void journal_insert_checkpoint(struct journal_head *, transaction_t *);
-extern void __journal_insert_checkpoint(struct journal_head *,transaction_t *);
+void __journal_remove_checkpoint(struct journal_head *);
+void __journal_insert_checkpoint(struct journal_head *, transaction_t *);
 
 /* Buffer IO */
 extern int 
@@ -1019,9 +1017,6 @@ extern void	log_wait_for_space(journal_t
 extern void	__journal_drop_transaction(journal_t *, transaction_t *);
 extern int	cleanup_journal_tail(journal_t *);
 
-/* Reduce journal memory usage by flushing */
-extern void shrink_journal_memory(void);
-
 /* Debugging code only: */
 
 #define jbd_ENOSYS() \

_
