2009-04-07 02:20:12

by Frederic Weisbecker

[permalink] [raw]
Subject: [PATCH] reiserfs: kill-the-BKL


For the tip:kill-the-BKL tree.

First of all, I'm sorry for such a monolithic patch. I know it doesn't make it
easy to review. But I can't really cut it in several subjects because all
bits are needed here for the "Bkl to a mutex" conversion.

Although I first thought that half of it was only there for performance
optimizations and could also be decoupled, further tests showed me that
they all contribute to correctness and not only performance.

This patch is an attempt to remove the Bkl based locking scheme from
reiserfs and is intended.
It is a bit inspired from an old attempt by Peter Zijlstra:
http://lkml.indiana.edu/hypermail/linux/kernel/0704.2/2174.html


The bkl is heavily used in this filesystem to prevent from
concurrent write accesses on the filesystem.

Reiserfs makes a deep use of the specific properties of the Bkl:

- It can be acqquired recursively by a same task
- It is released on the schedule() calls and reacquired when schedule() returns

The two properties above are a roadmap for the reiserfs write locking so it's
very hard to simply replace it with a common mutex.

- We need a recursive-able locking unless we want to restructure several blocks
of the code.
- We need to identify the sites where the bkl was implictly relaxed
(schedule, wait, sync, etc...) so that we can in turn release and
reacquire our new lock explicitly.
Such implicit releases of the lock are often required to let other
resources producer/consumer do their job or we can suffer unexpected
starvations or deadlocks.

So the new lock that replaces the bkl here is a per superblock mutex with a
specific property: it can be acquired recursively by a same task, like the
bkl.
For such purpose, we integrate a lock owner and a lock depth field on the
superblock information structure.

The first axis on this patch is to turn reiserfs_write_(un)lock() function
into a wrapper to manage this mutex. Also some explicit calls to
lock_kernel() have been converted to reiserfs_write_lock() helpers.

The second axis is to find the important blocking sites (schedule...(),
wait_on_buffer(), sync_dirty_buffer(), etc...) and then apply an explicit
release of the write lock on these locations before blocking. Then we can
safely wait for those who can give us resources or those who need some.
Typically this is a fight between the current writer, the reiserfs workqueue
(aka the async commiter) and the pdflush threads.

The third axis is a consequence of the second. The write lock is usually
on top of a lock dependency chain which can include the journal lock, the
flush lock or the commit lock. So it's dangerous to release and trying to
reacquire the write lock while we still hold other locks.

This is fine with the bkl:

T1 T2

lock_kernel()
mutex_lock(A)
unlock_kernel()
// do something
lock_kernel()
mutex_lock(A) -> already locked by T1
schedule() (and then unlock_kernel())
lock_kernel()
mutex_unlock(A)
....

This is not fine with a mutex:

T1 T2

mutex_lock(write)
mutex_lock(A)
mutex_unlock(write)
// do something
mutex_lock(write)
mutex_lock(A) -> already locked by T1
schedule()

mutex_lock(write) -> already locked by T2
deadlock

The solution in this patch is to provide a helper which releases the write
lock and sleep a bit if we can't lock a mutex that depend on it. It's another
simulation of the bkl behaviour.

The last axis is to locate the fs callbacks that are called with the bkl held,
according to Documentation/filesystem/Locking.

Those are:

- reiserfs_remount
- reiserfs_fill_super
- reiserfs_put_super

Reiserfs didn't need to explicitly lock because of the context of these callbacks.
But now we must take care of that with the new locking.

After this patch, reiserfs suffers from a slight performance regression (for now).
On UP, a high volume write with dd reports an average of 27 MB/s instead
of 30 MB/s without the patch applied.

Signed-off-by: Frederic Weisbecker <[email protected]>
Reviewed-by: Ingo Molnar <[email protected]>
---
fs/reiserfs/Makefile | 2 +-
fs/reiserfs/bitmap.c | 2 +
fs/reiserfs/dir.c | 8 +++
fs/reiserfs/fix_node.c | 10 +++
fs/reiserfs/inode.c | 23 +++++--
fs/reiserfs/ioctl.c | 6 +-
fs/reiserfs/journal.c | 134 +++++++++++++++++++++++++++++++---------
fs/reiserfs/lock.c | 63 +++++++++++++++++++
fs/reiserfs/resize.c | 2 +
fs/reiserfs/stree.c | 2 +
fs/reiserfs/super.c | 41 ++++++++++++-
include/linux/reiserfs_fs.h | 12 ++--
include/linux/reiserfs_fs_sb.h | 9 +++
13 files changed, 271 insertions(+), 43 deletions(-)
create mode 100644 fs/reiserfs/lock.c

diff --git a/fs/reiserfs/Makefile b/fs/reiserfs/Makefile
index 7c5ab63..6a9e30c 100644
--- a/fs/reiserfs/Makefile
+++ b/fs/reiserfs/Makefile
@@ -7,7 +7,7 @@ obj-$(CONFIG_REISERFS_FS) += reiserfs.o
reiserfs-objs := bitmap.o do_balan.o namei.o inode.o file.o dir.o fix_node.o \
super.o prints.o objectid.o lbalance.o ibalance.o stree.o \
hashes.o tail_conversion.o journal.o resize.o \
- item_ops.o ioctl.o procfs.o xattr.o
+ item_ops.o ioctl.o procfs.o xattr.o lock.o

ifeq ($(CONFIG_REISERFS_FS_XATTR),y)
reiserfs-objs += xattr_user.o xattr_trusted.o
diff --git a/fs/reiserfs/bitmap.c b/fs/reiserfs/bitmap.c
index e716161..1470334 100644
--- a/fs/reiserfs/bitmap.c
+++ b/fs/reiserfs/bitmap.c
@@ -1256,7 +1256,9 @@ struct buffer_head *reiserfs_read_bitmap_block(struct super_block *sb,
else {
if (buffer_locked(bh)) {
PROC_INFO_INC(sb, scan_bitmap.wait);
+ reiserfs_write_unlock(sb);
__wait_on_buffer(bh);
+ reiserfs_write_lock(sb);
}
BUG_ON(!buffer_uptodate(bh));
BUG_ON(atomic_read(&bh->b_count) == 0);
diff --git a/fs/reiserfs/dir.c b/fs/reiserfs/dir.c
index 67a80d7..6d71aa0 100644
--- a/fs/reiserfs/dir.c
+++ b/fs/reiserfs/dir.c
@@ -174,14 +174,22 @@ int reiserfs_readdir_dentry(struct dentry *dentry, void *dirent,
// user space buffer is swapped out. At that time
// entry can move to somewhere else
memcpy(local_buf, d_name, d_reclen);
+
+ /*
+ * Since filldir might sleep, we can release
+ * the write lock here for other waiters
+ */
+ reiserfs_write_unlock(inode->i_sb);
if (filldir
(dirent, local_buf, d_reclen, d_off, d_ino,
DT_UNKNOWN) < 0) {
+ reiserfs_write_lock(inode->i_sb);
if (local_buf != small_buf) {
kfree(local_buf);
}
goto end;
}
+ reiserfs_write_lock(inode->i_sb);
if (local_buf != small_buf) {
kfree(local_buf);
}
diff --git a/fs/reiserfs/fix_node.c b/fs/reiserfs/fix_node.c
index 5e5a4e6..bf5f2cb 100644
--- a/fs/reiserfs/fix_node.c
+++ b/fs/reiserfs/fix_node.c
@@ -1022,7 +1022,11 @@ static int get_far_parent(struct tree_balance *tb,
/* Check whether the common parent is locked. */

if (buffer_locked(*pcom_father)) {
+
+ /* Release the write lock while the buffer is busy */
+ reiserfs_write_unlock(tb->tb_sb);
__wait_on_buffer(*pcom_father);
+ reiserfs_write_lock(tb->tb_sb);
if (FILESYSTEM_CHANGED_TB(tb)) {
brelse(*pcom_father);
return REPEAT_SEARCH;
@@ -1927,7 +1931,9 @@ static int get_direct_parent(struct tree_balance *tb, int h)
return REPEAT_SEARCH;

if (buffer_locked(bh)) {
+ reiserfs_write_unlock(tb->tb_sb);
__wait_on_buffer(bh);
+ reiserfs_write_lock(tb->tb_sb);
if (FILESYSTEM_CHANGED_TB(tb))
return REPEAT_SEARCH;
}
@@ -2278,7 +2284,9 @@ static int wait_tb_buffers_until_unlocked(struct tree_balance *tb)
REPEAT_SEARCH : CARRY_ON;
}
#endif
+ reiserfs_write_unlock(tb->tb_sb);
__wait_on_buffer(locked);
+ reiserfs_write_lock(tb->tb_sb);
if (FILESYSTEM_CHANGED_TB(tb))
return REPEAT_SEARCH;
}
@@ -2349,7 +2357,9 @@ int fix_nodes(int op_mode, struct tree_balance *tb,

/* if it possible in indirect_to_direct conversion */
if (buffer_locked(tbS0)) {
+ reiserfs_write_unlock(tb->tb_sb);
__wait_on_buffer(tbS0);
+ reiserfs_write_lock(tb->tb_sb);
if (FILESYSTEM_CHANGED_TB(tb))
return REPEAT_SEARCH;
}
diff --git a/fs/reiserfs/inode.c b/fs/reiserfs/inode.c
index 6fd0f47..88ef0b7 100644
--- a/fs/reiserfs/inode.c
+++ b/fs/reiserfs/inode.c
@@ -489,10 +489,14 @@ static int reiserfs_get_blocks_direct_io(struct inode *inode,
disappeared */
if (REISERFS_I(inode)->i_flags & i_pack_on_close_mask) {
int err;
- lock_kernel();
+
+ reiserfs_write_lock(inode->i_sb);
+
err = reiserfs_commit_for_inode(inode);
REISERFS_I(inode)->i_flags &= ~i_pack_on_close_mask;
- unlock_kernel();
+
+ reiserfs_write_unlock(inode->i_sb);
+
if (err < 0)
ret = err;
}
@@ -616,7 +620,6 @@ int reiserfs_get_block(struct inode *inode, sector_t block,
loff_t new_offset =
(((loff_t) block) << inode->i_sb->s_blocksize_bits) + 1;

- /* bad.... */
reiserfs_write_lock(inode->i_sb);
version = get_inode_item_key_version(inode);

@@ -997,10 +1000,14 @@ int reiserfs_get_block(struct inode *inode, sector_t block,
if (retval)
goto failure;
}
- /* inserting indirect pointers for a hole can take a
- ** long time. reschedule if needed
+ /*
+ * inserting indirect pointers for a hole can take a
+ * long time. reschedule if needed and also release the write
+ * lock for others.
*/
+ reiserfs_write_unlock(inode->i_sb);
cond_resched();
+ reiserfs_write_lock(inode->i_sb);

retval = search_for_position_by_key(inode->i_sb, &key, &path);
if (retval == IO_ERROR) {
@@ -2612,7 +2619,10 @@ int reiserfs_prepare_write(struct file *f, struct page *page,
int ret;
int old_ref = 0;

+ reiserfs_write_unlock(inode->i_sb);
reiserfs_wait_on_write_block(inode->i_sb);
+ reiserfs_write_lock(inode->i_sb);
+
fix_tail_page_for_writing(page);
if (reiserfs_transaction_running(inode->i_sb)) {
struct reiserfs_transaction_handle *th;
@@ -2762,7 +2772,10 @@ int reiserfs_commit_write(struct file *f, struct page *page,
int update_sd = 0;
struct reiserfs_transaction_handle *th = NULL;

+ reiserfs_write_unlock(inode->i_sb);
reiserfs_wait_on_write_block(inode->i_sb);
+ reiserfs_write_lock(inode->i_sb);
+
if (reiserfs_transaction_running(inode->i_sb)) {
th = current->journal_info;
}
diff --git a/fs/reiserfs/ioctl.c b/fs/reiserfs/ioctl.c
index 0ccc3fd..5e40b0c 100644
--- a/fs/reiserfs/ioctl.c
+++ b/fs/reiserfs/ioctl.c
@@ -141,9 +141,11 @@ long reiserfs_compat_ioctl(struct file *file, unsigned int cmd,
default:
return -ENOIOCTLCMD;
}
- lock_kernel();
+
+ reiserfs_write_lock(inode->i_sb);
ret = reiserfs_ioctl(inode, file, cmd, (unsigned long) compat_ptr(arg));
- unlock_kernel();
+ reiserfs_write_unlock(inode->i_sb);
+
return ret;
}
#endif
diff --git a/fs/reiserfs/journal.c b/fs/reiserfs/journal.c
index 77f5bb7..64dcabd 100644
--- a/fs/reiserfs/journal.c
+++ b/fs/reiserfs/journal.c
@@ -429,21 +429,6 @@ static void clear_prepared_bits(struct buffer_head *bh)
clear_buffer_journal_restore_dirty(bh);
}

-/* utility function to force a BUG if it is called without the big
-** kernel lock held. caller is the string printed just before calling BUG()
-*/
-void reiserfs_check_lock_depth(struct super_block *sb, char *caller)
-{
-#ifdef CONFIG_SMP
- if (current->lock_depth < 0) {
- reiserfs_panic(sb, "journal-1", "%s called without kernel "
- "lock held", caller);
- }
-#else
- ;
-#endif
-}
-
/* return a cnode with same dev, block number and size in table, or null if not found */
static inline struct reiserfs_journal_cnode *get_journal_hash_dev(struct
super_block
@@ -552,11 +537,48 @@ static inline void insert_journal_hash(struct reiserfs_journal_cnode **table,
journal_hash(table, cn->sb, cn->blocknr) = cn;
}

+/*
+ * Several mutexes depend on the write lock.
+ * However sometimes we want to relax the write lock while we hold
+ * these mutexes, according to the release/reacquire on schedule()
+ * properties of the Bkl that were used.
+ * Reiserfs performances and locking were based on this scheme.
+ * Now that the write lock is a mutex and not the bkl anymore, doing so
+ * may result in a deadlock:
+ *
+ * A acquire write_lock
+ * A acquire j_commit_mutex
+ * A release write_lock and wait for something
+ * B acquire write_lock
+ * B can't acquire j_commit_mutex and sleep
+ * A can't acquire write lock anymore
+ * deadlock
+ *
+ * What we do here is avoiding such deadlock by playing the same game
+ * than the Bkl: if we can't acquire a mutex that depends on the write lock,
+ * we release the write lock, wait a bit and then retry.
+ *
+ * The mutexes concerned by this hack are:
+ * - The commit mutex of a journal list
+ * - The flush mutex
+ * - The journal lock
+ */
+static inline void reiserfs_mutex_lock_safe(struct mutex *m,
+ struct super_block *s)
+{
+ while (!mutex_trylock(m)) {
+ reiserfs_write_unlock(s);
+ schedule();
+ reiserfs_write_lock(s);
+ }
+}
+
/* lock the current transaction */
static inline void lock_journal(struct super_block *sb)
{
PROC_INFO_INC(sb, journal.lock_journal);
- mutex_lock(&SB_JOURNAL(sb)->j_mutex);
+
+ reiserfs_mutex_lock_safe(&SB_JOURNAL(sb)->j_mutex, sb);
}

/* unlock the current transaction */
@@ -708,7 +730,9 @@ static void check_barrier_completion(struct super_block *s,
disable_barrier(s);
set_buffer_uptodate(bh);
set_buffer_dirty(bh);
+ reiserfs_write_unlock(s);
sync_dirty_buffer(bh);
+ reiserfs_write_lock(s);
}
}

@@ -996,8 +1020,13 @@ static int reiserfs_async_progress_wait(struct super_block *s)
{
DEFINE_WAIT(wait);
struct reiserfs_journal *j = SB_JOURNAL(s);
- if (atomic_read(&j->j_async_throttle))
+
+ if (atomic_read(&j->j_async_throttle)) {
+ reiserfs_write_unlock(s);
congestion_wait(WRITE, HZ / 10);
+ reiserfs_write_lock(s);
+ }
+
return 0;
}

@@ -1043,7 +1072,8 @@ static int flush_commit_list(struct super_block *s,
}

/* make sure nobody is trying to flush this one at the same time */
- mutex_lock(&jl->j_commit_mutex);
+ reiserfs_mutex_lock_safe(&jl->j_commit_mutex, s);
+
if (!journal_list_still_alive(s, trans_id)) {
mutex_unlock(&jl->j_commit_mutex);
goto put_jl;
@@ -1061,12 +1091,17 @@ static int flush_commit_list(struct super_block *s,

if (!list_empty(&jl->j_bh_list)) {
int ret;
- unlock_kernel();
+
+ /*
+ * We might sleep in numerous places inside
+ * write_ordered_buffers. Relax the write lock.
+ */
+ reiserfs_write_unlock(s);
ret = write_ordered_buffers(&journal->j_dirty_buffers_lock,
journal, jl, &jl->j_bh_list);
if (ret < 0 && retval == 0)
retval = ret;
- lock_kernel();
+ reiserfs_write_lock(s);
}
BUG_ON(!list_empty(&jl->j_bh_list));
/*
@@ -1114,12 +1149,19 @@ static int flush_commit_list(struct super_block *s,
bn = SB_ONDISK_JOURNAL_1st_BLOCK(s) +
(jl->j_start + i) % SB_ONDISK_JOURNAL_SIZE(s);
tbh = journal_find_get_block(s, bn);
+
+ reiserfs_write_unlock(s);
wait_on_buffer(tbh);
+ reiserfs_write_lock(s);
// since we're using ll_rw_blk above, it might have skipped over
// a locked buffer. Double check here
//
- if (buffer_dirty(tbh)) /* redundant, sync_dirty_buffer() checks */
+ /* redundant, sync_dirty_buffer() checks */
+ if (buffer_dirty(tbh)) {
+ reiserfs_write_unlock(s);
sync_dirty_buffer(tbh);
+ reiserfs_write_lock(s);
+ }
if (unlikely(!buffer_uptodate(tbh))) {
#ifdef CONFIG_REISERFS_CHECK
reiserfs_warning(s, "journal-601",
@@ -1143,10 +1185,15 @@ static int flush_commit_list(struct super_block *s,
if (buffer_dirty(jl->j_commit_bh))
BUG();
mark_buffer_dirty(jl->j_commit_bh) ;
+ reiserfs_write_unlock(s);
sync_dirty_buffer(jl->j_commit_bh) ;
+ reiserfs_write_lock(s);
}
- } else
+ } else {
+ reiserfs_write_unlock(s);
wait_on_buffer(jl->j_commit_bh);
+ reiserfs_write_lock(s);
+ }

check_barrier_completion(s, jl->j_commit_bh);

@@ -1286,7 +1333,9 @@ static int _update_journal_header_block(struct super_block *sb,

if (trans_id >= journal->j_last_flush_trans_id) {
if (buffer_locked((journal->j_header_bh))) {
+ reiserfs_write_unlock(sb);
wait_on_buffer((journal->j_header_bh));
+ reiserfs_write_lock(sb);
if (unlikely(!buffer_uptodate(journal->j_header_bh))) {
#ifdef CONFIG_REISERFS_CHECK
reiserfs_warning(sb, "journal-699",
@@ -1312,12 +1361,16 @@ static int _update_journal_header_block(struct super_block *sb,
disable_barrier(sb);
goto sync;
}
+ reiserfs_write_unlock(sb);
wait_on_buffer(journal->j_header_bh);
+ reiserfs_write_lock(sb);
check_barrier_completion(sb, journal->j_header_bh);
} else {
sync:
set_buffer_dirty(journal->j_header_bh);
+ reiserfs_write_unlock(sb);
sync_dirty_buffer(journal->j_header_bh);
+ reiserfs_write_lock(sb);
}
if (!buffer_uptodate(journal->j_header_bh)) {
reiserfs_warning(sb, "journal-837",
@@ -1409,7 +1462,7 @@ static int flush_journal_list(struct super_block *s,

/* if flushall == 0, the lock is already held */
if (flushall) {
- mutex_lock(&journal->j_flush_mutex);
+ reiserfs_mutex_lock_safe(&journal->j_flush_mutex, s);
} else if (mutex_trylock(&journal->j_flush_mutex)) {
BUG();
}
@@ -1553,7 +1606,11 @@ static int flush_journal_list(struct super_block *s,
reiserfs_panic(s, "journal-1011",
"cn->bh is NULL");
}
+
+ reiserfs_write_unlock(s);
wait_on_buffer(cn->bh);
+ reiserfs_write_lock(s);
+
if (!cn->bh) {
reiserfs_panic(s, "journal-1012",
"cn->bh is NULL");
@@ -1973,11 +2030,19 @@ static int do_journal_release(struct reiserfs_transaction_handle *th,
reiserfs_mounted_fs_count--;
/* wait for all commits to finish */
cancel_delayed_work(&SB_JOURNAL(sb)->j_work);
+
+ /*
+ * We must release the write lock here because
+ * the workqueue job (flush_async_commit) needs this lock
+ */
+ reiserfs_write_unlock(sb);
flush_workqueue(commit_wq);
+
if (!reiserfs_mounted_fs_count) {
destroy_workqueue(commit_wq);
commit_wq = NULL;
}
+ reiserfs_write_lock(sb);

free_journal_ram(sb);

@@ -2243,7 +2308,11 @@ static int journal_read_transaction(struct super_block *sb,
/* read in the log blocks, memcpy to the corresponding real block */
ll_rw_block(READ, get_desc_trans_len(desc), log_blocks);
for (i = 0; i < get_desc_trans_len(desc); i++) {
+
+ reiserfs_write_unlock(sb);
wait_on_buffer(log_blocks[i]);
+ reiserfs_write_lock(sb);
+
if (!buffer_uptodate(log_blocks[i])) {
reiserfs_warning(sb, "journal-1212",
"REPLAY FAILURE fsck required! "
@@ -2964,8 +3033,11 @@ static void queue_log_writer(struct super_block *s)
init_waitqueue_entry(&wait, current);
add_wait_queue(&journal->j_join_wait, &wait);
set_current_state(TASK_UNINTERRUPTIBLE);
- if (test_bit(J_WRITERS_QUEUED, &journal->j_state))
+ if (test_bit(J_WRITERS_QUEUED, &journal->j_state)) {
+ reiserfs_write_unlock(s);
schedule();
+ reiserfs_write_lock(s);
+ }
__set_current_state(TASK_RUNNING);
remove_wait_queue(&journal->j_join_wait, &wait);
}
@@ -2982,7 +3054,9 @@ static void let_transaction_grow(struct super_block *sb, unsigned int trans_id)
struct reiserfs_journal *journal = SB_JOURNAL(sb);
unsigned long bcount = journal->j_bcount;
while (1) {
+ reiserfs_write_unlock(sb);
schedule_timeout_uninterruptible(1);
+ reiserfs_write_lock(sb);
journal->j_current_jl->j_state |= LIST_COMMIT_PENDING;
while ((atomic_read(&journal->j_wcount) > 0 ||
atomic_read(&journal->j_jlock)) &&
@@ -3033,7 +3107,9 @@ static int do_journal_begin_r(struct reiserfs_transaction_handle *th,

if (test_bit(J_WRITERS_BLOCKED, &journal->j_state)) {
unlock_journal(sb);
+ reiserfs_write_unlock(sb);
reiserfs_wait_on_write_block(sb);
+ reiserfs_write_lock(sb);
PROC_INFO_INC(sb, journal.journal_relock_writers);
goto relock;
}
@@ -3506,14 +3582,14 @@ static void flush_async_commits(struct work_struct *work)
struct reiserfs_journal_list *jl;
struct list_head *entry;

- lock_kernel();
+ reiserfs_write_lock(sb);
if (!list_empty(&journal->j_journal_list)) {
/* last entry is the youngest, commit it and you get everything */
entry = journal->j_journal_list.prev;
jl = JOURNAL_LIST_ENTRY(entry);
flush_commit_list(sb, jl, 1);
}
- unlock_kernel();
+ reiserfs_write_unlock(sb);
}

/*
@@ -4041,7 +4117,7 @@ static int do_journal_end(struct reiserfs_transaction_handle *th,
* the new transaction is fully setup, and we've already flushed the
* ordered bh list
*/
- mutex_lock(&jl->j_commit_mutex);
+ reiserfs_mutex_lock_safe(&jl->j_commit_mutex, sb);

/* save the transaction id in case we need to commit it later */
commit_trans_id = jl->j_trans_id;
@@ -4203,10 +4279,10 @@ static int do_journal_end(struct reiserfs_transaction_handle *th,
* is lost.
*/
if (!list_empty(&jl->j_tail_bh_list)) {
- unlock_kernel();
+ reiserfs_write_unlock(sb);
write_ordered_buffers(&journal->j_dirty_buffers_lock,
journal, jl, &jl->j_tail_bh_list);
- lock_kernel();
+ reiserfs_write_lock(sb);
}
BUG_ON(!list_empty(&jl->j_tail_bh_list));
mutex_unlock(&jl->j_commit_mutex);
diff --git a/fs/reiserfs/lock.c b/fs/reiserfs/lock.c
new file mode 100644
index 0000000..cdd8d9e
--- /dev/null
+++ b/fs/reiserfs/lock.c
@@ -0,0 +1,63 @@
+#include <linux/reiserfs_fs.h>
+#include <linux/mutex.h>
+
+/*
+ * The previous reiserfs locking scheme was heavily based on
+ * the tricky properties of the Bkl:
+ *
+ * - it was acquired recursively by a same task
+ * - the performances relied on the release-while-schedule() property
+ *
+ * Now that we replace it by a mutex, we still want to keep the same
+ * recursive property to avoid big changes in the code structure.
+ * We use our own lock_owner here because the owner field on a mutex
+ * is only available in SMP or mutex debugging, also we only need this field
+ * for this mutex, no need for a system wide mutex facility.
+ *
+ * Also this lock is often released before a call that could block because
+ * reiserfs performances were partialy based on the release while schedule()
+ * property of the Bkl.
+ */
+void reiserfs_write_lock(struct super_block *s)
+{
+ struct reiserfs_sb_info *sb_i = REISERFS_SB(s);
+
+ if (sb_i->lock_owner != current) {
+ mutex_lock(&sb_i->lock);
+ sb_i->lock_owner = current;
+ }
+
+ /* No need to protect it, only the current task touches it */
+ sb_i->lock_depth++;
+}
+
+void reiserfs_write_unlock(struct super_block *s)
+{
+ struct reiserfs_sb_info *sb_i = REISERFS_SB(s);
+
+ /*
+ * Are we unlocking without even holding the lock?
+ * Such a situation could even raise a BUG() if we don't
+ * want the data become corrupted
+ */
+ WARN_ONCE(sb_i->lock_owner != current,
+ "Superblock write lock imbalance");
+
+ if (--sb_i->lock_depth == -1) {
+ sb_i->lock_owner = NULL;
+ mutex_unlock(&sb_i->lock);
+ }
+}
+
+/*
+ * Utility function to force a BUG if it is called without the superblock
+ * write lock held. caller is the string printed just before calling BUG()
+ */
+void reiserfs_check_lock_depth(struct super_block *sb, char *caller)
+{
+ struct reiserfs_sb_info *sb_i = REISERFS_SB(sb);
+
+ if (sb_i->lock_depth < 0)
+ reiserfs_panic(sb, "%s called without kernel lock held %d",
+ caller);
+}
diff --git a/fs/reiserfs/resize.c b/fs/reiserfs/resize.c
index 238e9d9..6a7bfb3 100644
--- a/fs/reiserfs/resize.c
+++ b/fs/reiserfs/resize.c
@@ -142,7 +142,9 @@ int reiserfs_resize(struct super_block *s, unsigned long block_count_new)

set_buffer_uptodate(bh);
mark_buffer_dirty(bh);
+ reiserfs_write_unlock(s);
sync_dirty_buffer(bh);
+ reiserfs_write_lock(s);
// update bitmap_info stuff
bitmap[i].free_count = sb_blocksize(sb) * 8 - 1;
brelse(bh);
diff --git a/fs/reiserfs/stree.c b/fs/reiserfs/stree.c
index d036ee5..6bd99a9 100644
--- a/fs/reiserfs/stree.c
+++ b/fs/reiserfs/stree.c
@@ -629,7 +629,9 @@ int search_by_key(struct super_block *sb, const struct cpu_key *key, /* Key to s
search_by_key_reada(sb, reada_bh,
reada_blocks, reada_count);
ll_rw_block(READ, 1, &bh);
+ reiserfs_write_unlock(sb);
wait_on_buffer(bh);
+ reiserfs_write_lock(sb);
if (!buffer_uptodate(bh))
goto io_error;
} else {
diff --git a/fs/reiserfs/super.c b/fs/reiserfs/super.c
index 0ae6486..1ac4fcb 100644
--- a/fs/reiserfs/super.c
+++ b/fs/reiserfs/super.c
@@ -470,6 +470,13 @@ static void reiserfs_put_super(struct super_block *s)
struct reiserfs_transaction_handle th;
th.t_trans_id = 0;

+ /*
+ * We didn't need to explicitly lock here before, because put_super
+ * is called with the bkl held.
+ * Now that we have our own lock, we must explicitly lock.
+ */
+ reiserfs_write_lock(s);
+
/* change file system state to current state if it was mounted with read-write permissions */
if (!(s->s_flags & MS_RDONLY)) {
if (!journal_begin(&th, s, 10)) {
@@ -499,6 +506,8 @@ static void reiserfs_put_super(struct super_block *s)

reiserfs_proc_info_done(s);

+ reiserfs_write_unlock(s);
+ mutex_destroy(&REISERFS_SB(s)->lock);
kfree(s->s_fs_info);
s->s_fs_info = NULL;

@@ -1191,7 +1200,15 @@ static int reiserfs_remount(struct super_block *s, int *mount_flags, char *arg)
unsigned int qfmt = 0;
#ifdef CONFIG_QUOTA
int i;
+#endif

+ /*
+ * We used to protect using the implicitly acquired bkl here.
+ * Now we must explictly acquire our own lock
+ */
+ reiserfs_write_lock(s);
+
+#ifdef CONFIG_QUOTA
memcpy(qf_names, REISERFS_SB(s)->s_qf_names, sizeof(qf_names));
#endif

@@ -1316,11 +1333,13 @@ static int reiserfs_remount(struct super_block *s, int *mount_flags, char *arg)
}

out_ok:
+ reiserfs_write_unlock(s);
kfree(s->s_options);
s->s_options = new_opts;
return 0;

out_err:
+ reiserfs_write_unlock(s);
kfree(new_opts);
return err;
}
@@ -1425,7 +1444,9 @@ static int read_super_block(struct super_block *s, int offset)
static int reread_meta_blocks(struct super_block *s)
{
ll_rw_block(READ, 1, &(SB_BUFFER_WITH_SB(s)));
+ reiserfs_write_unlock(s);
wait_on_buffer(SB_BUFFER_WITH_SB(s));
+ reiserfs_write_lock(s);
if (!buffer_uptodate(SB_BUFFER_WITH_SB(s))) {
reiserfs_warning(s, "reiserfs-2504", "error reading the super");
return 1;
@@ -1634,7 +1655,7 @@ static int reiserfs_fill_super(struct super_block *s, void *data, int silent)
sbi = kzalloc(sizeof(struct reiserfs_sb_info), GFP_KERNEL);
if (!sbi) {
errval = -ENOMEM;
- goto error;
+ goto error_alloc;
}
s->s_fs_info = sbi;
/* Set default values for options: non-aggressive tails, RO on errors */
@@ -1648,6 +1669,20 @@ static int reiserfs_fill_super(struct super_block *s, void *data, int silent)
/* setup default block allocator options */
reiserfs_init_alloc_options(s);

+ mutex_init(&REISERFS_SB(s)->lock);
+ REISERFS_SB(s)->lock_depth = -1;
+
+ /*
+ * This function is called with the bkl, which also was the old
+ * locking used here.
+ * do_journal_begin() will soon check if we hold the lock (ie: was the
+ * bkl). This is likely because do_journal_begin() has several another
+ * callers because at this time, it doesn't seem to be necessary to
+ * protect against anything.
+ * Anyway, let's be conservative and lock for now.
+ */
+ reiserfs_write_lock(s);
+
jdev_name = NULL;
if (reiserfs_parse_options
(s, (char *)data, &(sbi->s_mount_opt), &blocks, &jdev_name,
@@ -1871,9 +1906,13 @@ static int reiserfs_fill_super(struct super_block *s, void *data, int silent)
init_waitqueue_head(&(sbi->s_wait));
spin_lock_init(&sbi->bitmap_lock);

+ reiserfs_write_unlock(s);
+
return (0);

error:
+ reiserfs_write_unlock(s);
+error_alloc:
if (jinit_done) { /* kill the commit thread, free journal ram */
journal_release_error(NULL, s);
}
diff --git a/include/linux/reiserfs_fs.h b/include/linux/reiserfs_fs.h
index 2245c78..f6b7b7b 100644
--- a/include/linux/reiserfs_fs.h
+++ b/include/linux/reiserfs_fs.h
@@ -52,11 +52,13 @@
#define REISERFS_IOC32_GETVERSION FS_IOC32_GETVERSION
#define REISERFS_IOC32_SETVERSION FS_IOC32_SETVERSION

-/* Locking primitives */
-/* Right now we are still falling back to (un)lock_kernel, but eventually that
- would evolve into real per-fs locks */
-#define reiserfs_write_lock( sb ) lock_kernel()
-#define reiserfs_write_unlock( sb ) unlock_kernel()
+/*
+ * Locking primitives. The write lock is a per superblock
+ * special mutex that has properties close to the Big Kernel Lock
+ * which was used in the previous locking scheme.
+ */
+void reiserfs_write_lock(struct super_block *s);
+void reiserfs_write_unlock(struct super_block *s);

struct fid;

diff --git a/include/linux/reiserfs_fs_sb.h b/include/linux/reiserfs_fs_sb.h
index 5621d87..cec8319 100644
--- a/include/linux/reiserfs_fs_sb.h
+++ b/include/linux/reiserfs_fs_sb.h
@@ -7,6 +7,8 @@
#ifdef __KERNEL__
#include <linux/workqueue.h>
#include <linux/rwsem.h>
+#include <linux/mutex.h>
+#include <linux/sched.h>
#endif

typedef enum {
@@ -355,6 +357,13 @@ struct reiserfs_sb_info {
struct reiserfs_journal *s_journal; /* pointer to journal information */
unsigned short s_mount_state; /* reiserfs state (valid, invalid) */

+ /* Serialize writers access, replace the old bkl */
+ struct mutex lock;
+ /* Owner of the lock (can be recursive) */
+ struct task_struct *lock_owner;
+ /* Depth of the lock, start from -1 like the bkl */
+ int lock_depth;
+
/* Comment? -Hans */
void (*end_io_handler) (struct buffer_head *, int);
hashf_t s_hash_function; /* pointer to function which is used
--
1.6.1


2009-04-07 11:25:13

by Frederic Weisbecker

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL

Note: I forgot to put the RFC prefix for this patch.

Frederic Weisbecker.

2009-04-07 13:40:47

by Ingo Molnar

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL


* Frederic Weisbecker <[email protected]> wrote:

> +/*
> + * Utility function to force a BUG if it is called without the superblock
> + * write lock held. caller is the string printed just before calling BUG()
> + */
> +void reiserfs_check_lock_depth(struct super_block *sb, char *caller)
> +{
> + struct reiserfs_sb_info *sb_i = REISERFS_SB(sb);
> +
> + if (sb_i->lock_depth < 0)
> + reiserfs_panic(sb, "%s called without kernel lock held %d",
> + caller);

s/kernel lock/sb write lock/

> + reiserfs_write_unlock(s);
> + mutex_destroy(&REISERFS_SB(s)->lock);
> kfree(s->s_fs_info);
> s->s_fs_info = NULL;

ah, mutex_destroy() - it's a small detail but still nice ;-)

Anyway, it still looks good to me, after the earlier patch i
reviewed. Would be nice to have the testing feedback of reiserfs
users, and the locking review from anyone who knows this code.

Ingo

2009-04-07 21:34:34

by Alexander Beregalov

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL

Hi Frederic

I've applied your patch.
This is UP system.
This warning seems related to your patch.


[ INFO: possible circular locking dependency detected ]
2.6.29-10247-g8e2c4f2-dirty #2
-------------------------------------------------------
udevadm/1239 is trying to acquire lock:
(&mm->mmap_sem){++++++}, at: [<c0173f52>] might_fault+0x52/0xa0

but task is already holding lock:
(sysfs_mutex){+.+.+.}, at: [<c01ccd36>] sysfs_readdir+0x56/0x200

which lock already depends on the new lock.


the existing dependency chain (in reverse order) is:

-> #3 (sysfs_mutex){+.+.+.}:
[<c014fb2a>] __lock_acquire+0xd1a/0x1160
[<c014ffe4>] lock_acquire+0x74/0x90
[<c0448d70>] __mutex_lock_common+0x50/0x430
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c01cd17c>] sysfs_addrm_start+0x2c/0xc0
[<c01cdd30>] create_dir+0x40/0x80
[<c01cdd9b>] sysfs_create_dir+0x2b/0x50
[<c02c3121>] kobject_add_internal+0xc1/0x220
[<c02c3351>] kobject_add_varg+0x31/0x50
[<c02c33cc>] kobject_add+0x2c/0x60
[<c0337a52>] device_add+0xe2/0x540
[<c01cac02>] add_partition+0x112/0x210
[<c01cb2ef>] rescan_partitions+0x23f/0x320
[<c01b0f13>] __blkdev_get+0x153/0x330
[<c01b10fa>] blkdev_get+0xa/0x10
[<c01caaca>] register_disk+0x10a/0x130
[<c02bd5c9>] add_disk+0x109/0x170
[<c034e85a>] sd_probe_async+0x19a/0x280
[<c0143900>] async_thread+0xd0/0x230
[<c013d86a>] kthread+0x3a/0x70
[<c0103677>] kernel_thread_helper+0x7/0x10
[<ffffffff>] 0xffffffff

-> #2 (&bdev->bd_mutex){+.+.+.}:
[<c014fb2a>] __lock_acquire+0xd1a/0x1160
[<c014ffe4>] lock_acquire+0x74/0x90
[<c0448d70>] __mutex_lock_common+0x50/0x430
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c01b0df3>] __blkdev_get+0x33/0x330
[<c01b10fa>] blkdev_get+0xa/0x10
[<c01b1261>] open_by_devnum+0x21/0x50
[<c01fed2e>] journal_init+0x22e/0x19a0
[<c01ea881>] reiserfs_fill_super+0x3a1/0x1050
[<c018d053>] get_sb_bdev+0x133/0x160
[<c01e8b21>] get_super_block+0x21/0x30
[<c018bd7a>] vfs_kern_mount+0x3a/0xa0
[<c018be39>] do_kern_mount+0x39/0xd0
[<c01a2063>] do_mount+0x3a3/0x790
[<c01a24d9>] sys_mount+0x89/0xc0
[<c06fac3d>] mount_block_root+0xd0/0x261
[<c06fae27>] mount_root+0x59/0x5f
[<c06faf1c>] prepare_namespace+0xef/0x171
[<c06fa4e3>] kernel_init+0xd5/0xee
[<c0103677>] kernel_thread_helper+0x7/0x10
[<ffffffff>] 0xffffffff

-> #1 (&REISERFS_SB(s)->lock){+.+.+.}:
[<c014fb2a>] __lock_acquire+0xd1a/0x1160
[<c014ffe4>] lock_acquire+0x74/0x90
[<c0448d70>] __mutex_lock_common+0x50/0x430
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01df2e6>] reiserfs_get_block+0x66/0x14f0
[<c01b32f0>] do_mpage_readpage+0x340/0x490
[<c01b352e>] mpage_readpages+0x9e/0xe0
[<c01dc729>] reiserfs_readpages+0x19/0x20
[<c01696ff>] __do_page_cache_readahead+0x13f/0x200
[<c016980e>] do_page_cache_readahead+0x4e/0x70
[<c0162caf>] filemap_fault+0x30f/0x430
[<c0175f6d>] __do_fault+0x3d/0x360
[<c0176aa0>] handle_mm_fault+0x100/0x590
[<c011aff1>] do_page_fault+0x121/0x280
[<c044b40a>] error_code+0x6a/0x70
[<c01bf810>] load_elf_binary+0x9d0/0x19f0
[<c018fde3>] search_binary_handler+0x163/0x2f0
[<c019018d>] do_execve+0x21d/0x2b0
[<c01016de>] sys_execve+0x3e/0x70
[<c0102ec9>] syscall_call+0x7/0xb
[<ffffffff>] 0xffffffff

-> #0 (&mm->mmap_sem){++++++}:
[<c014fbde>] __lock_acquire+0xdce/0x1160
[<c014ffe4>] lock_acquire+0x74/0x90
[<c0173f83>] might_fault+0x83/0xa0
[<c02ca436>] copy_to_user+0x36/0x130
[<c0197b24>] filldir64+0xa4/0xf0
[<c01ccdf6>] sysfs_readdir+0x116/0x200
[<c0197d9e>] vfs_readdir+0x7e/0xa0
[<c0197e29>] sys_getdents64+0x69/0xb0
[<c0102e48>] sysenter_do_call+0x12/0x36
[<ffffffff>] 0xffffffff

other info that might help us debug this:

2 locks held by udevadm/1239:
#0: (&type->i_mutex_dir_key){+.+.+.}, at: [<c0197d6f>] vfs_readdir+0x4f/0xa0
#1: (sysfs_mutex){+.+.+.}, at: [<c01ccd36>] sysfs_readdir+0x56/0x200

stack backtrace:
Pid: 1239, comm: udevadm Not tainted 2.6.29-10247-g8e2c4f2-dirty #2
Call Trace:
[<c0447c92>] ? printk+0x18/0x1e
[<c014db5c>] print_circular_bug_tail+0x8c/0xe0
[<c014fbde>] __lock_acquire+0xdce/0x1160
[<c014bb1d>] ? put_lock_stats+0xd/0x30
[<c014ffe4>] lock_acquire+0x74/0x90
[<c0173f52>] ? might_fault+0x52/0xa0
[<c0173f83>] might_fault+0x83/0xa0
[<c0173f52>] ? might_fault+0x52/0xa0
[<c02ca436>] copy_to_user+0x36/0x130
[<c0197b24>] filldir64+0xa4/0xf0
[<c01ccdf6>] sysfs_readdir+0x116/0x200
[<c0197a80>] ? filldir64+0x0/0xf0
[<c0197d9e>] vfs_readdir+0x7e/0xa0
[<c0197a80>] ? filldir64+0x0/0xf0
[<c0197e29>] sys_getdents64+0x69/0xb0
[<c0102e48>] sysenter_do_call+0x12/0x36

2009-04-07 21:57:46

by Frederic Weisbecker

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL

On Wed, Apr 08, 2009 at 01:34:07AM +0400, Alexander Beregalov wrote:
> Hi Frederic
>
> I've applied your patch.
> This is UP system.
> This warning seems related to your patch.
>
>
> [ INFO: possible circular locking dependency detected ]
> 2.6.29-10247-g8e2c4f2-dirty #2
> -------------------------------------------------------
> udevadm/1239 is trying to acquire lock:
> (&mm->mmap_sem){++++++}, at: [<c0173f52>] might_fault+0x52/0xa0
>
> but task is already holding lock:
> (sysfs_mutex){+.+.+.}, at: [<c01ccd36>] sysfs_readdir+0x56/0x200


Ah, I've never hit it. I'm not sure this is related to my patch
but it could be.
Could you send me your config?

Thanks,
Frederic.


>
> which lock already depends on the new lock.
>
>
> the existing dependency chain (in reverse order) is:
>
> -> #3 (sysfs_mutex){+.+.+.}:
> [<c014fb2a>] __lock_acquire+0xd1a/0x1160
> [<c014ffe4>] lock_acquire+0x74/0x90
> [<c0448d70>] __mutex_lock_common+0x50/0x430
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c01cd17c>] sysfs_addrm_start+0x2c/0xc0
> [<c01cdd30>] create_dir+0x40/0x80
> [<c01cdd9b>] sysfs_create_dir+0x2b/0x50
> [<c02c3121>] kobject_add_internal+0xc1/0x220
> [<c02c3351>] kobject_add_varg+0x31/0x50
> [<c02c33cc>] kobject_add+0x2c/0x60
> [<c0337a52>] device_add+0xe2/0x540
> [<c01cac02>] add_partition+0x112/0x210
> [<c01cb2ef>] rescan_partitions+0x23f/0x320
> [<c01b0f13>] __blkdev_get+0x153/0x330
> [<c01b10fa>] blkdev_get+0xa/0x10
> [<c01caaca>] register_disk+0x10a/0x130
> [<c02bd5c9>] add_disk+0x109/0x170
> [<c034e85a>] sd_probe_async+0x19a/0x280
> [<c0143900>] async_thread+0xd0/0x230
> [<c013d86a>] kthread+0x3a/0x70
> [<c0103677>] kernel_thread_helper+0x7/0x10
> [<ffffffff>] 0xffffffff
>
> -> #2 (&bdev->bd_mutex){+.+.+.}:
> [<c014fb2a>] __lock_acquire+0xd1a/0x1160
> [<c014ffe4>] lock_acquire+0x74/0x90
> [<c0448d70>] __mutex_lock_common+0x50/0x430
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c01b0df3>] __blkdev_get+0x33/0x330
> [<c01b10fa>] blkdev_get+0xa/0x10
> [<c01b1261>] open_by_devnum+0x21/0x50
> [<c01fed2e>] journal_init+0x22e/0x19a0
> [<c01ea881>] reiserfs_fill_super+0x3a1/0x1050
> [<c018d053>] get_sb_bdev+0x133/0x160
> [<c01e8b21>] get_super_block+0x21/0x30
> [<c018bd7a>] vfs_kern_mount+0x3a/0xa0
> [<c018be39>] do_kern_mount+0x39/0xd0
> [<c01a2063>] do_mount+0x3a3/0x790
> [<c01a24d9>] sys_mount+0x89/0xc0
> [<c06fac3d>] mount_block_root+0xd0/0x261
> [<c06fae27>] mount_root+0x59/0x5f
> [<c06faf1c>] prepare_namespace+0xef/0x171
> [<c06fa4e3>] kernel_init+0xd5/0xee
> [<c0103677>] kernel_thread_helper+0x7/0x10
> [<ffffffff>] 0xffffffff
>
> -> #1 (&REISERFS_SB(s)->lock){+.+.+.}:
> [<c014fb2a>] __lock_acquire+0xd1a/0x1160
> [<c014ffe4>] lock_acquire+0x74/0x90
> [<c0448d70>] __mutex_lock_common+0x50/0x430
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01df2e6>] reiserfs_get_block+0x66/0x14f0
> [<c01b32f0>] do_mpage_readpage+0x340/0x490
> [<c01b352e>] mpage_readpages+0x9e/0xe0
> [<c01dc729>] reiserfs_readpages+0x19/0x20
> [<c01696ff>] __do_page_cache_readahead+0x13f/0x200
> [<c016980e>] do_page_cache_readahead+0x4e/0x70
> [<c0162caf>] filemap_fault+0x30f/0x430
> [<c0175f6d>] __do_fault+0x3d/0x360
> [<c0176aa0>] handle_mm_fault+0x100/0x590
> [<c011aff1>] do_page_fault+0x121/0x280
> [<c044b40a>] error_code+0x6a/0x70
> [<c01bf810>] load_elf_binary+0x9d0/0x19f0
> [<c018fde3>] search_binary_handler+0x163/0x2f0
> [<c019018d>] do_execve+0x21d/0x2b0
> [<c01016de>] sys_execve+0x3e/0x70
> [<c0102ec9>] syscall_call+0x7/0xb
> [<ffffffff>] 0xffffffff
>
> -> #0 (&mm->mmap_sem){++++++}:
> [<c014fbde>] __lock_acquire+0xdce/0x1160
> [<c014ffe4>] lock_acquire+0x74/0x90
> [<c0173f83>] might_fault+0x83/0xa0
> [<c02ca436>] copy_to_user+0x36/0x130
> [<c0197b24>] filldir64+0xa4/0xf0
> [<c01ccdf6>] sysfs_readdir+0x116/0x200
> [<c0197d9e>] vfs_readdir+0x7e/0xa0
> [<c0197e29>] sys_getdents64+0x69/0xb0
> [<c0102e48>] sysenter_do_call+0x12/0x36
> [<ffffffff>] 0xffffffff
>
> other info that might help us debug this:
>
> 2 locks held by udevadm/1239:
> #0: (&type->i_mutex_dir_key){+.+.+.}, at: [<c0197d6f>] vfs_readdir+0x4f/0xa0
> #1: (sysfs_mutex){+.+.+.}, at: [<c01ccd36>] sysfs_readdir+0x56/0x200
>
> stack backtrace:
> Pid: 1239, comm: udevadm Not tainted 2.6.29-10247-g8e2c4f2-dirty #2
> Call Trace:
> [<c0447c92>] ? printk+0x18/0x1e
> [<c014db5c>] print_circular_bug_tail+0x8c/0xe0
> [<c014fbde>] __lock_acquire+0xdce/0x1160
> [<c014bb1d>] ? put_lock_stats+0xd/0x30
> [<c014ffe4>] lock_acquire+0x74/0x90
> [<c0173f52>] ? might_fault+0x52/0xa0
> [<c0173f83>] might_fault+0x83/0xa0
> [<c0173f52>] ? might_fault+0x52/0xa0
> [<c02ca436>] copy_to_user+0x36/0x130
> [<c0197b24>] filldir64+0xa4/0xf0
> [<c01ccdf6>] sysfs_readdir+0x116/0x200
> [<c0197a80>] ? filldir64+0x0/0xf0
> [<c0197d9e>] vfs_readdir+0x7e/0xa0
> [<c0197a80>] ? filldir64+0x0/0xf0
> [<c0197e29>] sys_getdents64+0x69/0xb0
> [<c0102e48>] sysenter_do_call+0x12/0x36

2009-04-07 22:20:28

by Alexander Beregalov

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL

On Tue, Apr 07, 2009 at 11:57:22PM +0200, Frederic Weisbecker wrote:
> On Wed, Apr 08, 2009 at 01:34:07AM +0400, Alexander Beregalov wrote:
> > Hi Frederic
> >
> > I've applied your patch.
> > This is UP system.
> > This warning seems related to your patch.
> >
> >
> > [ INFO: possible circular locking dependency detected ]
> > 2.6.29-10247-g8e2c4f2-dirty #2
> > -------------------------------------------------------
> > udevadm/1239 is trying to acquire lock:
> > (&mm->mmap_sem){++++++}, at: [<c0173f52>] might_fault+0x52/0xa0
> >
> > but task is already holding lock:
> > (sysfs_mutex){+.+.+.}, at: [<c01ccd36>] sysfs_readdir+0x56/0x200
>
>
> Ah, I've never hit it. I'm not sure this is related to my patch
> but it could be.
> Could you send me your config?
[Sent in private mail]

This should be related.
System is hung after dbench testing for some time.


SysRq : Emergency Sync
SysRq : Show Blocked State
task PC stack pid father
events/0 D 00000000 5672 6 2
f7067ee8 00000046 00000046 00000000 00000000 f7056720 f70569ac b6acece3
00000b44 00000000 f70569ac c018c1df 00000202 00000202 f6982a60 f7056720
f7067f30 c0448e3a 00000000 00000002 00000000 c018c1df 00000000 0018c3b2
Call Trace:
[<c018c1df>] ? lock_super+0x1f/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c018c1df>] ? lock_super+0x1f/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c018c1df>] ? lock_super+0x1f/0x30
[<c018c1df>] lock_super+0x1f/0x30
[<c018c3b9>] sync_supers+0x69/0xc0
[<c01a922c>] do_sync+0x1c/0x60
[<c013a0e6>] ? worker_thread+0xe6/0x250
[<c01a927d>] do_sync_work+0xd/0x20
[<c013a148>] worker_thread+0x148/0x250
[<c013a0e6>] ? worker_thread+0xe6/0x250
[<c01a9270>] ? do_sync_work+0x0/0x20
[<c013dab0>] ? autoremove_wake_function+0x0/0x40
[<c013a000>] ? worker_thread+0x0/0x250
[<c013d86a>] kthread+0x3a/0x70
[<c013d830>] ? kthread+0x0/0x70
[<c0103677>] kernel_thread_helper+0x7/0x10
metalog D 00000000 4936 2183 1
f7161cec 00000046 00000046 00000000 00000000 f6b8d280 f6b8d50c 1eba9be4
00000b44 00000000 f6b8d50c c020375e 00000202 00000202 f670a014 f6b8d280
f7161d34 c0448e3a 00000000 00000002 00000000 c020375e 00000000 0018440c
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01e9674>] reiserfs_dirty_inode+0x24/0xb0
[<c01a0f7a>] ? mnt_want_write+0x1a/0xc0
[<c01a613c>] __mark_inode_dirty+0x2c/0x180
[<c019c6f5>] file_update_time+0xc5/0xe0
[<c01634eb>] __generic_file_aio_write_nolock+0x18b/0x520
[<c0163980>] ? generic_file_aio_write+0x50/0xd0
[<c0163993>] generic_file_aio_write+0x63/0xd0
[<c018a331>] do_sync_write+0xd1/0x110
[<c013dab0>] ? autoremove_wake_function+0x0/0x40
[<c01e23ba>] reiserfs_file_write+0x8a/0xa0
[<c018a94c>] vfs_write+0x9c/0x140
[<c01e2330>] ? reiserfs_file_write+0x0/0xa0
[<c018aaad>] sys_write+0x3d/0x70
[<c0102e48>] sysenter_do_call+0x12/0x36
svscan D 00000000 5384 2223 1
f731dd8c 00000046 00000046 00000000 00000000 f6740000 f674028c eaa8d98d
00000af9 00000000 f674028c c020375e 00000202 00000202 f670a014 f6740000
f731ddd4 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00740000
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01e266a>] reiserfs_readdir_dentry+0x5a/0x740
[<c018b46f>] ? file_move+0x1f/0x50
[<c0197b70>] ? filldir+0x0/0xd0
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c044ae07>] ? _spin_unlock+0x27/0x50
[<c018b48e>] ? file_move+0x3e/0x50
[<c0173f9d>] ? might_fault+0x9d/0xa0
[<c0197d6f>] ? vfs_readdir+0x4f/0xa0
[<c0449013>] ? __mutex_lock_common+0x2f3/0x430
[<c0197d6f>] ? vfs_readdir+0x4f/0xa0
[<c01e2d67>] reiserfs_readdir+0x17/0x20
[<c0197d9e>] vfs_readdir+0x7e/0xa0
[<c0197b70>] ? filldir+0x0/0xd0
[<c0197ed8>] sys_getdents+0x68/0xb0
[<c0102e48>] sysenter_do_call+0x12/0x36
multilog D 00000000 4800 2236 2228
f731bcec 00000046 00000046 00000000 00000000 f6ba8000 f6ba828c 50105c29
00000b26 00000000 f6ba828c c020375e 00000202 00000202 f670a014 f6ba8000
f731bd34 c0448e3a 00000000 00000002 00000000 c020375e 00000000 0014ce9d
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01e9674>] reiserfs_dirty_inode+0x24/0xb0
[<c01a0f7a>] ? mnt_want_write+0x1a/0xc0
[<c01a613c>] __mark_inode_dirty+0x2c/0x180
[<c019c6f5>] file_update_time+0xc5/0xe0
[<c01634eb>] __generic_file_aio_write_nolock+0x18b/0x520
[<c0163980>] ? generic_file_aio_write+0x50/0xd0
[<c0163993>] generic_file_aio_write+0x63/0xd0
[<c018a331>] do_sync_write+0xd1/0x110
[<c013dab0>] ? autoremove_wake_function+0x0/0x40
[<c01b42ba>] ? dnotify_parent+0x2a/0x80
[<c01e23ba>] reiserfs_file_write+0x8a/0xa0
[<c018a94c>] vfs_write+0x9c/0x140
[<c0173f52>] ? might_fault+0x52/0xa0
[<c01e2330>] ? reiserfs_file_write+0x0/0xa0
[<c018aaad>] sys_write+0x3d/0x70
[<c0102e48>] sysenter_do_call+0x12/0x36
agetty D 00000000 5432 2441 1
f73cbba8 00000046 00000046 00000000 00000000 f73dd280 f73dd50c 6a069cd5
00000b33 00000000 f73dd50c c020375e 00000202 00000202 f670a014 f73dd280
f73cbbf0 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000246
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01df2e6>] reiserfs_get_block+0x66/0x14f0
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014e52b>] ? trace_hardirqs_on+0xb/0x10
[<c044b07d>] ? _spin_unlock_irq+0x2d/0x50
[<c0124b02>] ? finish_task_switch+0x62/0xb0
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c0124aa0>] ? finish_task_switch+0x0/0xb0
[<c044b2df>] ? __reacquire_kernel_lock+0x2f/0x40
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c0332d78>] ? serial8250_tx_empty+0x18/0x50
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c016108e>] ? add_to_page_cache_locked+0x5e/0xe0
[<c01b32f0>] do_mpage_readpage+0x340/0x490
[<c01610e9>] ? add_to_page_cache_locked+0xb9/0xe0
[<c016a415>] ? __lru_cache_add+0x65/0xa0
[<c0161184>] ? add_to_page_cache_lru+0x74/0x80
[<c01b352e>] mpage_readpages+0x9e/0xe0
[<c01df280>] ? reiserfs_get_block+0x0/0x14f0
[<c04480b8>] ? __schedule+0x2f8/0x4f0
[<c01dc710>] ? reiserfs_readpages+0x0/0x20
[<c01dc729>] reiserfs_readpages+0x19/0x20
[<c01df280>] ? reiserfs_get_block+0x0/0x14f0
[<c01696ff>] __do_page_cache_readahead+0x13f/0x200
[<c016980e>] do_page_cache_readahead+0x4e/0x70
[<c0162caf>] filemap_fault+0x30f/0x430
[<c0175f6d>] __do_fault+0x3d/0x360
[<c01b42ba>] ? dnotify_parent+0x2a/0x80
[<c0176aa0>] handle_mm_fault+0x100/0x590
[<c014159d>] ? down_read_trylock+0x5d/0x70
[<c011aff1>] do_page_fault+0x121/0x280
[<c011aed0>] ? do_page_fault+0x0/0x280
[<c044b40a>] error_code+0x6a/0x70
[<c011aed0>] ? do_page_fault+0x0/0x280
screen D 00000000 5244 2529 2528
f612fd90 00000046 00000046 00000000 00000000 f6babde0 f6bac06c 33c73fab
00000b1c 00000000 f6bac06c c0193bf3 00000202 00000202 f6f9615c f6babde0
f612fdd8 c0448e3a 00000000 00000002 00000000 c0193bf3 00000000 0012fed0
Call Trace:
[<c0193bf3>] ? do_lookup+0xd3/0x1c0
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c0193bf3>] ? do_lookup+0xd3/0x1c0
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c0193bf3>] ? do_lookup+0xd3/0x1c0
[<c0193bf3>] do_lookup+0xd3/0x1c0
[<c0193e49>] __link_path_walk+0x169/0xcc0
[<c0194b53>] path_walk+0x53/0xb0
[<c0194c8b>] do_path_lookup+0x8b/0x170
[<c01958ea>] user_path_at+0x4a/0x80
[<c01b42ba>] ? dnotify_parent+0x2a/0x80
[<c018e471>] vfs_stat_fd+0x21/0x50
[<c018e571>] vfs_stat+0x11/0x20
[<c018e594>] sys_stat64+0x14/0x30
[<c018abdf>] ? vfs_read+0xff/0x140
[<c0317d70>] ? tty_read+0x0/0xb0
[<c018acdd>] ? sys_read+0x3d/0x70
[<c02c9d84>] ? trace_hardirqs_on_thunk+0xc/0x10
[<c0102e48>] sysenter_do_call+0x12/0x36
rtorrent D 00000000 4084 2543 2542
f6325d88 00000046 00000046 00000000 00000000 f6b894a0 f6b8972c d79a38dd
00000af9 00000000 f6b8972c c020375e 00000202 00000202 f670a014 f6b894a0
f6325dd0 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000000
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01e266a>] reiserfs_readdir_dentry+0x5a/0x740
[<c018b46f>] ? file_move+0x1f/0x50
[<c0197a80>] ? filldir64+0x0/0xf0
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c044ae07>] ? _spin_unlock+0x27/0x50
[<c018b48e>] ? file_move+0x3e/0x50
[<c0173f9d>] ? might_fault+0x9d/0xa0
[<c0197d6f>] ? vfs_readdir+0x4f/0xa0
[<c0449013>] ? __mutex_lock_common+0x2f3/0x430
[<c0197d6f>] ? vfs_readdir+0x4f/0xa0
[<c01e2d67>] reiserfs_readdir+0x17/0x20
[<c0197d9e>] vfs_readdir+0x7e/0xa0
[<c0197a80>] ? filldir64+0x0/0xf0
[<c0197e29>] sys_getdents64+0x69/0xb0
[<c0102e48>] sysenter_do_call+0x12/0x36
bash D 00000000 5788 2546 2529
f585dcd8 00000046 00000046 00000000 00000000 f6b8bde0 f6b8c06c 188b381a
00000b1b 00000000 f6b8c06c c020375e 00000202 00000202 f670a014 f6b8bde0
f585dd20 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000000
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01db389>] reiserfs_lookup+0x59/0x170
[<c019b923>] ? d_alloc+0x123/0x1b0
[<c0193cb1>] do_lookup+0x191/0x1c0
[<c0194305>] __link_path_walk+0x625/0xcc0
[<c0194b53>] path_walk+0x53/0xb0
[<c0194c8b>] do_path_lookup+0x8b/0x170
[<c01958ea>] user_path_at+0x4a/0x80
[<c018dda4>] ? cp_new_stat64+0xe4/0x100
[<c018e471>] vfs_stat_fd+0x21/0x50
[<c018e571>] vfs_stat+0x11/0x20
[<c018e594>] sys_stat64+0x14/0x30
[<c02c9d84>] ? trace_hardirqs_on_thunk+0xc/0x10
[<c0102e48>] sysenter_do_call+0x12/0x36
dbench D 00000000 4816 20778 1
f44bbd1c 00000046 00000046 00000000 00000000 ce105280 ce10550c d12c8ac7
00000af8 00000000 ce10550c c020375e 00000202 00000202 f670a014 ce105280
f44bbd64 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766a00
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fb8c5>] flush_commit_list+0x125/0x7c0
[<c018570d>] ? check_object+0x13d/0x200
[<c01fba72>] flush_commit_list+0x2d2/0x7c0
[<c0121ab0>] ? __wake_up+0x40/0x50
[<c01fde19>] do_journal_end+0xdc9/0xf50
[<c0133420>] ? process_timeout+0x0/0x10
[<c01fdffd>] journal_end_sync+0x5d/0x80
[<c01feadd>] reiserfs_commit_for_inode+0x1bd/0x1e0
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01e1e68>] reiserfs_sync_file+0x38/0x90
[<c01a905d>] vfs_fsync+0x6d/0xe0
[<c01a90fd>] do_fsync+0x2d/0x50
[<c01a914d>] sys_fsync+0xd/0x10
[<c0102e48>] sysenter_do_call+0x12/0x36
dbench D 00000000 4660 20779 1
f44b3de4 00000046 00000046 00000000 00000000 ce106720 ce1069ac d12c836c
00000af8 00000000 ce1069ac c020375e 00000202 00000202 f670a014 ce106720
f44b3e2c c0448e3a 00000000 00000002 00000000 c020375e 00000000 00767400
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fb8c5>] flush_commit_list+0x125/0x7c0
[<c04480b8>] ? __schedule+0x2f8/0x4f0
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01fba72>] flush_commit_list+0x2d2/0x7c0
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0133420>] ? process_timeout+0x0/0x10
[<c0449203>] ? mutex_lock_nested+0x33/0x40
[<c01fe9c1>] reiserfs_commit_for_inode+0xa1/0x1e0
[<c0449203>] ? mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01e1e68>] reiserfs_sync_file+0x38/0x90
[<c01a905d>] vfs_fsync+0x6d/0xe0
[<c01a90fd>] do_fsync+0x2d/0x50
[<c01a914d>] sys_fsync+0xd/0x10
[<c0102e48>] sysenter_do_call+0x12/0x36
dbench D 00000000 4724 20780 1
f6389de4 00000046 00000046 00000000 00000000 ce102940 ce102bcc d12c7533
00000af8 00000000 ce102bcc c020375e 00000202 00000202 f670a014 ce102940
f6389e2c c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766280
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fb8c5>] flush_commit_list+0x125/0x7c0
[<c0169c29>] ? pagevec_lookup_tag+0x29/0x40
[<c0168241>] ? write_cache_pages+0xd1/0x350
[<c01fba72>] flush_commit_list+0x2d2/0x7c0
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449013>] ? __mutex_lock_common+0x2f3/0x430
[<c01fe9c1>] reiserfs_commit_for_inode+0xa1/0x1e0
[<c0449203>] ? mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01e1e68>] reiserfs_sync_file+0x38/0x90
[<c01a905d>] vfs_fsync+0x6d/0xe0
[<c01a90fd>] do_fsync+0x2d/0x50
[<c01a914d>] sys_fsync+0xd/0x10
[<c0102e48>] sysenter_do_call+0x12/0x36
dbench D 00000000 4944 20781 1
f589dc84 00000046 00000046 00000000 00000000 ce1014a0 ce10172c d0f2ddbd
00000af8 00000000 ce10172c c020375e 00000202 00000202 f670a014 ce1014a0
f589dccc c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766280
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fe08b>] do_journal_begin_r+0x6b/0x320
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01fe400>] journal_begin+0x80/0x110
[<c0449203>] ? mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01e9686>] reiserfs_dirty_inode+0x36/0xb0
[<c01a0f7a>] ? mnt_want_write+0x1a/0xc0
[<c01a613c>] __mark_inode_dirty+0x2c/0x180
[<c019c6f5>] file_update_time+0xc5/0xe0
[<c01634eb>] __generic_file_aio_write_nolock+0x18b/0x520
[<c0163980>] ? generic_file_aio_write+0x50/0xd0
[<c0163993>] generic_file_aio_write+0x63/0xd0
[<c014e52b>] ? trace_hardirqs_on+0xb/0x10
[<c018a331>] do_sync_write+0xd1/0x110
[<c013dab0>] ? autoremove_wake_function+0x0/0x40
[<c01e23ba>] reiserfs_file_write+0x8a/0xa0
[<c018a94c>] vfs_write+0x9c/0x140
[<c02ca436>] ? copy_to_user+0x36/0x130
[<c01e2330>] ? reiserfs_file_write+0x0/0xa0
[<c018aa6b>] sys_pwrite64+0x7b/0x80
[<c0102e48>] sysenter_do_call+0x12/0x36
dbench D 00000000 4832 20782 1
f4617a64 00000046 00000046 00000000 00000000 ce103de0 ce10406c d12ca393
00000af8 00000000 ce10406c c020375e 00000202 00000202 f670a014 ce103de0
f4617aac c0448e3a 00000000 00000002 00000000 c020375e 00000000 00767900
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fe08b>] do_journal_begin_r+0x6b/0x320
[<c01fe400>] journal_begin+0x80/0x110
[<c01fe4c2>] ? reiserfs_persistent_transaction+0x32/0x90
[<c01fe4d1>] reiserfs_persistent_transaction+0x41/0x90
[<c01df460>] reiserfs_get_block+0x1e0/0x14f0
[<c018dbbe>] ? inode_get_bytes+0x1e/0x50
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c018570d>] ? check_object+0x13d/0x200
[<c01ab2c3>] ? create_empty_buffers+0x33/0xa0
[<c01ad424>] __block_prepare_write+0x1a4/0x380
[<c0161184>] ? add_to_page_cache_lru+0x74/0x80
[<c01ad7a8>] block_write_begin+0x48/0xe0
[<c01df280>] ? reiserfs_get_block+0x0/0x14f0
[<c01ddfbc>] reiserfs_write_begin+0xec/0x200
[<c01df280>] ? reiserfs_get_block+0x0/0x14f0
[<c01618f5>] generic_file_buffered_write+0x155/0x2c0
[<c0163520>] __generic_file_aio_write_nolock+0x1c0/0x520
[<c0163993>] generic_file_aio_write+0x63/0xd0
[<c018a331>] do_sync_write+0xd1/0x110
[<c0173f52>] ? might_fault+0x52/0xa0
[<c013dab0>] ? autoremove_wake_function+0x0/0x40
[<c01e23ba>] reiserfs_file_write+0x8a/0xa0
[<c018a94c>] vfs_write+0x9c/0x140
[<c02ca436>] ? copy_to_user+0x36/0x130
[<c01e2330>] ? reiserfs_file_write+0x0/0xa0
[<c018aa6b>] sys_pwrite64+0x7b/0x80
[<c0102e48>] sysenter_do_call+0x12/0x36
dbench D 00000000 4764 20783 1
f5833e48 00000046 00000046 00000000 00000000 f73de720 f73de9ac d12c96e7
00000af8 00000000 f73de9ac c020375e 00000202 00000202 f670a014 f73de720
f5833e90 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766280
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fb8c5>] flush_commit_list+0x125/0x7c0
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449013>] ? __mutex_lock_common+0x2f3/0x430
[<c01fe9c1>] reiserfs_commit_for_inode+0xa1/0x1e0
[<c0449203>] ? mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01e1e68>] reiserfs_sync_file+0x38/0x90
[<c01a905d>] vfs_fsync+0x6d/0xe0
[<c01a90fd>] do_fsync+0x2d/0x50
[<c01a914d>] sys_fsync+0xd/0x10
[<c0102e48>] sysenter_do_call+0x12/0x36
dbench D 00000000 4600 20785 1
f460bd1c 00000046 00000046 00000000 00000000 f6b88000 f6b8828c d12c9d18
00000af8 00000000 f6b8828c c020375e 00000202 00000202 f670a014 f6b88000
f460bd64 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766000
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fb8c5>] flush_commit_list+0x125/0x7c0
[<c01fba72>] flush_commit_list+0x2d2/0x7c0
[<c0121ab0>] ? __wake_up+0x40/0x50
[<c01fde19>] do_journal_end+0xdc9/0xf50
[<c0133420>] ? process_timeout+0x0/0x10
[<c01fdffd>] journal_end_sync+0x5d/0x80
[<c01feadd>] reiserfs_commit_for_inode+0x1bd/0x1e0
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01e1e68>] reiserfs_sync_file+0x38/0x90
[<c01a905d>] vfs_fsync+0x6d/0xe0
[<c01a90fd>] do_fsync+0x2d/0x50
[<c01a914d>] sys_fsync+0xd/0x10
[<c0102e48>] sysenter_do_call+0x12/0x36
dbench D 00000000 4548 20786 1
f5903d1c 00000046 00000046 00000000 00000000 f6960000 f696028c d12fcd2a
00000af8 00000000 f696028c c020375e 00000202 00000202 f670a014 f6960000
f5903d64 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766280
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fb8c5>] flush_commit_list+0x125/0x7c0
[<c018570d>] ? check_object+0x13d/0x200
[<c01fba72>] flush_commit_list+0x2d2/0x7c0
[<c0121ab0>] ? __wake_up+0x40/0x50
[<c01fde19>] do_journal_end+0xdc9/0xf50
[<c0133420>] ? process_timeout+0x0/0x10
[<c01fdffd>] journal_end_sync+0x5d/0x80
[<c01feadd>] reiserfs_commit_for_inode+0x1bd/0x1e0
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01e1e68>] reiserfs_sync_file+0x38/0x90
[<c01a905d>] vfs_fsync+0x6d/0xe0
[<c01a90fd>] do_fsync+0x2d/0x50
[<c01a914d>] sys_fsync+0xd/0x10
[<c0102e48>] sysenter_do_call+0x12/0x36
dbench D 00000000 4704 20787 1
f4557d60 00000046 00000046 00000000 00000000 f69614a0 f696172c d12c7cad
00000af8 00000000 f696172c c020375e 00000202 00000202 f670a014 f69614a0
f4557da8 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00767b80
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fb8c5>] flush_commit_list+0x125/0x7c0
[<c01862b1>] ? __slab_alloc+0xc1/0x5f0
[<c02c4f65>] ? __prop_inc_single+0x65/0x80
[<c01fbfc3>] get_list_bitmap+0x63/0xb0
[<c01fdd5f>] do_journal_end+0xd0f/0xf50
[<c0133420>] ? process_timeout+0x0/0x10
[<c01fdffd>] journal_end_sync+0x5d/0x80
[<c01feadd>] reiserfs_commit_for_inode+0x1bd/0x1e0
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01e1e68>] reiserfs_sync_file+0x38/0x90
[<c01a905d>] vfs_fsync+0x6d/0xe0
[<c01a90fd>] do_fsync+0x2d/0x50
[<c01a914d>] sys_fsync+0xd/0x10
[<c0102e48>] sysenter_do_call+0x12/0x36
pdflush D 00000000 6080 20790 2
f09bfe3c 00000046 00000046 00000000 00000000 f7088000 f708828c d12caa96
00000af8 00000000 f708828c c020375e 00000202 00000202 f670a014 f7088000
f09bfe84 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766c80
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01fe08b>] do_journal_begin_r+0x6b/0x320
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01fe400>] journal_begin+0x80/0x110
[<c0449203>] ? mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c01e95fe>] reiserfs_sync_fs+0x3e/0x80
[<c0449203>] ? mutex_lock_nested+0x33/0x40
[<c018c1df>] ? lock_super+0x1f/0x30
[<c01e964d>] reiserfs_write_super+0xd/0x10
[<c018c3d4>] sync_supers+0x84/0xc0
[<c0169330>] ? pdflush+0x0/0x1c0
[<c016860b>] wb_kupdate+0x2b/0x100
[<c014e52b>] ? trace_hardirqs_on+0xb/0x10
[<c0169330>] ? pdflush+0x0/0x1c0
[<c01693f8>] pdflush+0xc8/0x1c0
[<c01685e0>] ? wb_kupdate+0x0/0x100
[<c013d86a>] kthread+0x3a/0x70
[<c013d830>] ? kthread+0x0/0x70
[<c0103677>] kernel_thread_helper+0x7/0x10
git-daemon D 00000000 6696 20791 2273
f3705ba8 00000046 00000046 00000000 00000000 f66fe720 f66fe9ac 4a31216d
00000afb 00000000 f66fe9ac c020375e 00000202 00000202 f670a014 f66fe720
f3705bf0 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000000
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01df2e6>] reiserfs_get_block+0x66/0x14f0
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c018440c>] ? slab_pad_check+0x3c/0x120
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c01666ee>] ? get_page_from_freelist+0x41e/0x530
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c016108e>] ? add_to_page_cache_locked+0x5e/0xe0
[<c01b32f0>] do_mpage_readpage+0x340/0x490
[<c01610e9>] ? add_to_page_cache_locked+0xb9/0xe0
[<c016a415>] ? __lru_cache_add+0x65/0xa0
[<c0161184>] ? add_to_page_cache_lru+0x74/0x80
[<c01b352e>] mpage_readpages+0x9e/0xe0
[<c01df280>] ? reiserfs_get_block+0x0/0x14f0
[<c01dc710>] ? reiserfs_readpages+0x0/0x20
[<c01dc729>] reiserfs_readpages+0x19/0x20
[<c01df280>] ? reiserfs_get_block+0x0/0x14f0
[<c01696ff>] __do_page_cache_readahead+0x13f/0x200
[<c016980e>] do_page_cache_readahead+0x4e/0x70
[<c0162caf>] filemap_fault+0x30f/0x430
[<c0175f6d>] __do_fault+0x3d/0x360
[<c0176aa0>] handle_mm_fault+0x100/0x590
[<c014159d>] ? down_read_trylock+0x5d/0x70
[<c011aff1>] do_page_fault+0x121/0x280
[<c011aed0>] ? do_page_fault+0x0/0x280
[<c044b40a>] error_code+0x6a/0x70
[<c011aed0>] ? do_page_fault+0x0/0x280
git-daemon D f63ede14 6888 20792 2273
f63ede20 00000046 00000000 f63ede14 00000046 f66fa940 f66fabcc 977b8ab0
00000afe 00000000 f66fabcc f63ede14 c014e52b c057e5a0 000003ae f63ede70
f63ede28 c04482c2 f63ede34 c04482f4 f63ede68 f63ede3c c0160d95 f63ede5c
Call Trace:
[<c014e52b>] ? trace_hardirqs_on+0xb/0x10
[<c04482c2>] schedule+0x12/0x30
[<c04482f4>] io_schedule+0x14/0x20
[<c0160d95>] sync_page+0x55/0x60
[<c0448881>] __wait_on_bit_lock+0x41/0x80
[<c0160d40>] ? sync_page+0x0/0x60
[<c0160d18>] __lock_page+0x88/0xa0
[<c013daf0>] ? wake_bit_function+0x0/0x60
[<c0160eb9>] find_lock_page+0x39/0x70
[<c0162ad7>] filemap_fault+0x137/0x430
[<c0175f6d>] __do_fault+0x3d/0x360
[<c0176aa0>] handle_mm_fault+0x100/0x590
[<c014159d>] ? down_read_trylock+0x5d/0x70
[<c011aff1>] do_page_fault+0x121/0x280
[<c011aed0>] ? do_page_fault+0x0/0x280
[<c044b40a>] error_code+0x6a/0x70
[<c011aed0>] ? do_page_fault+0x0/0x280
fcron D 00000000 7168 20797 2257
f5a0fd80 00000046 00000046 00000000 00000000 f6952940 f6952bcc df71112a
00000b25 00000000 f6952bcc c020375e 00000202 00000202 f670a014 f6952940
f5a0fdc8 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000000
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01db389>] reiserfs_lookup+0x59/0x170
[<c019b923>] ? d_alloc+0x123/0x1b0
[<c019275f>] __lookup_hash+0xcf/0x130
[<c01927e7>] lookup_hash+0x27/0x30
[<c0195cb5>] do_filp_open+0x245/0x7a0
[<c019ed68>] ? alloc_fd+0x28/0xf0
[<c044ae07>] ? _spin_unlock+0x27/0x50
[<c019ee18>] ? alloc_fd+0xd8/0xf0
[<c0188879>] do_sys_open+0x59/0xf0
[<c0188979>] sys_open+0x29/0x40
[<c0102e48>] sysenter_do_call+0x12/0x36
Sched Debug Version: v0.09, 2.6.29-10247-g8e2c4f2-dirty #2
now at 12468457.474391 msecs
.jiffies : 1216523
.sysctl_sched_latency : 20.000000
.sysctl_sched_min_granularity : 4.000000
.sysctl_sched_wakeup_granularity : 5.000000
.sysctl_sched_child_runs_first : 0.000001
.sysctl_sched_features : 113917

cpu#0, 1830.032 MHz
.nr_running : 1
.load : 1024
.nr_switches : 6163991
.nr_load_updates : 582498
.nr_uninterruptible : 21
.next_balance : 0.000000
.curr->pid : 20784
.clock : 12465239.317891
.cpu_load[0] : 1024
.cpu_load[1] : 1024
.cpu_load[2] : 1024
.cpu_load[3] : 1024
.cpu_load[4] : 1024
.yld_count : 0
.sched_switch : 0
.sched_count : 778698744
.sched_goidle : 307156
.ttwu_count : 0
.ttwu_local : 0
.bkl_count : 275

cfs_rq[0]:
.exec_clock : 1507714.391289
.MIN_vruntime : 0.000001
.min_vruntime : 1444293.051451
.max_vruntime : 0.000001
.spread : 0.000000
.spread0 : 0.000000
.nr_running : 1
.load : 1024
.nr_spread_over : 35

rt_rq[0]:
.rt_nr_running : 0
.rt_throttled : 0
.rt_time : 0.000000
.rt_runtime : 950.000000

runnable tasks:
task PID tree-key switches prio exec-runtime sum-exec sum-sleep
----------------------------------------------------------------------------------------------------------
R dbench 20784 1444293.051451 4569 120 1444293.051451 401016.637062 16471.092414

INFO: task svscan:2223 blocked for more than 480 seconds.
"echo 0 > /proc/sys/kernel/hung_task_timeout_secs" disables this message.
svscan D 00000000 5384 2223 1
f731dd8c 00000046 00000046 00000000 00000000 f6740000 f674028c eaa8d98d
00000af9 00000000 f674028c c020375e 00000202 00000202 f670a014 f6740000
f731ddd4 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00740000
Call Trace:
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0448e3a>] __mutex_lock_common+0x11a/0x430
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c0449203>] mutex_lock_nested+0x33/0x40
[<c020375e>] ? reiserfs_write_lock+0x1e/0x30
[<c020375e>] reiserfs_write_lock+0x1e/0x30
[<c01e266a>] reiserfs_readdir_dentry+0x5a/0x740
[<c018b46f>] ? file_move+0x1f/0x50
[<c0197b70>] ? filldir+0x0/0xd0
[<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
[<c044ae07>] ? _spin_unlock+0x27/0x50
[<c018b48e>] ? file_move+0x3e/0x50
[<c0173f9d>] ? might_fault+0x9d/0xa0
[<c0197d6f>] ? vfs_readdir+0x4f/0xa0
[<c0449013>] ? __mutex_lock_common+0x2f3/0x430
[<c0197d6f>] ? vfs_readdir+0x4f/0xa0
[<c01e2d67>] reiserfs_readdir+0x17/0x20
[<c0197d9e>] vfs_readdir+0x7e/0xa0
[<c0197b70>] ? filldir+0x0/0xd0
[<c0197ed8>] sys_getdents+0x68/0xb0
[<c0102e48>] sysenter_do_call+0x12/0x36
INFO: lockdep is turned off.

2009-04-08 00:43:18

by Frederic Weisbecker

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL

On Wed, Apr 08, 2009 at 02:19:50AM +0400, Alexander Beregalov wrote:
> On Tue, Apr 07, 2009 at 11:57:22PM +0200, Frederic Weisbecker wrote:
> > On Wed, Apr 08, 2009 at 01:34:07AM +0400, Alexander Beregalov wrote:
> > > Hi Frederic
> > >
> > > I've applied your patch.
> > > This is UP system.
> > > This warning seems related to your patch.
> > >
> > >
> > > [ INFO: possible circular locking dependency detected ]
> > > 2.6.29-10247-g8e2c4f2-dirty #2
> > > -------------------------------------------------------
> > > udevadm/1239 is trying to acquire lock:
> > > (&mm->mmap_sem){++++++}, at: [<c0173f52>] might_fault+0x52/0xa0
> > >
> > > but task is already holding lock:
> > > (sysfs_mutex){+.+.+.}, at: [<c01ccd36>] sysfs_readdir+0x56/0x200
> >
> >
> > Ah, I've never hit it. I'm not sure this is related to my patch
> > but it could be.
> > Could you send me your config?
> [Sent in private mail]
>
> This should be related.
> System is hung after dbench testing for some time.
>



Ok, I can reproduce these hangs with dbench.
I will investigate on that and try to fix it soon.

Thanks a lot for this report,
Frederic.



> SysRq : Emergency Sync
> SysRq : Show Blocked State
> task PC stack pid father
> events/0 D 00000000 5672 6 2
> f7067ee8 00000046 00000046 00000000 00000000 f7056720 f70569ac b6acece3
> 00000b44 00000000 f70569ac c018c1df 00000202 00000202 f6982a60 f7056720
> f7067f30 c0448e3a 00000000 00000002 00000000 c018c1df 00000000 0018c3b2
> Call Trace:
> [<c018c1df>] ? lock_super+0x1f/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c018c1df>] ? lock_super+0x1f/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c018c1df>] ? lock_super+0x1f/0x30
> [<c018c1df>] lock_super+0x1f/0x30
> [<c018c3b9>] sync_supers+0x69/0xc0
> [<c01a922c>] do_sync+0x1c/0x60
> [<c013a0e6>] ? worker_thread+0xe6/0x250
> [<c01a927d>] do_sync_work+0xd/0x20
> [<c013a148>] worker_thread+0x148/0x250
> [<c013a0e6>] ? worker_thread+0xe6/0x250
> [<c01a9270>] ? do_sync_work+0x0/0x20
> [<c013dab0>] ? autoremove_wake_function+0x0/0x40
> [<c013a000>] ? worker_thread+0x0/0x250
> [<c013d86a>] kthread+0x3a/0x70
> [<c013d830>] ? kthread+0x0/0x70
> [<c0103677>] kernel_thread_helper+0x7/0x10
> metalog D 00000000 4936 2183 1
> f7161cec 00000046 00000046 00000000 00000000 f6b8d280 f6b8d50c 1eba9be4
> 00000b44 00000000 f6b8d50c c020375e 00000202 00000202 f670a014 f6b8d280
> f7161d34 c0448e3a 00000000 00000002 00000000 c020375e 00000000 0018440c
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01e9674>] reiserfs_dirty_inode+0x24/0xb0
> [<c01a0f7a>] ? mnt_want_write+0x1a/0xc0
> [<c01a613c>] __mark_inode_dirty+0x2c/0x180
> [<c019c6f5>] file_update_time+0xc5/0xe0
> [<c01634eb>] __generic_file_aio_write_nolock+0x18b/0x520
> [<c0163980>] ? generic_file_aio_write+0x50/0xd0
> [<c0163993>] generic_file_aio_write+0x63/0xd0
> [<c018a331>] do_sync_write+0xd1/0x110
> [<c013dab0>] ? autoremove_wake_function+0x0/0x40
> [<c01e23ba>] reiserfs_file_write+0x8a/0xa0
> [<c018a94c>] vfs_write+0x9c/0x140
> [<c01e2330>] ? reiserfs_file_write+0x0/0xa0
> [<c018aaad>] sys_write+0x3d/0x70
> [<c0102e48>] sysenter_do_call+0x12/0x36
> svscan D 00000000 5384 2223 1
> f731dd8c 00000046 00000046 00000000 00000000 f6740000 f674028c eaa8d98d
> 00000af9 00000000 f674028c c020375e 00000202 00000202 f670a014 f6740000
> f731ddd4 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00740000
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01e266a>] reiserfs_readdir_dentry+0x5a/0x740
> [<c018b46f>] ? file_move+0x1f/0x50
> [<c0197b70>] ? filldir+0x0/0xd0
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c044ae07>] ? _spin_unlock+0x27/0x50
> [<c018b48e>] ? file_move+0x3e/0x50
> [<c0173f9d>] ? might_fault+0x9d/0xa0
> [<c0197d6f>] ? vfs_readdir+0x4f/0xa0
> [<c0449013>] ? __mutex_lock_common+0x2f3/0x430
> [<c0197d6f>] ? vfs_readdir+0x4f/0xa0
> [<c01e2d67>] reiserfs_readdir+0x17/0x20
> [<c0197d9e>] vfs_readdir+0x7e/0xa0
> [<c0197b70>] ? filldir+0x0/0xd0
> [<c0197ed8>] sys_getdents+0x68/0xb0
> [<c0102e48>] sysenter_do_call+0x12/0x36
> multilog D 00000000 4800 2236 2228
> f731bcec 00000046 00000046 00000000 00000000 f6ba8000 f6ba828c 50105c29
> 00000b26 00000000 f6ba828c c020375e 00000202 00000202 f670a014 f6ba8000
> f731bd34 c0448e3a 00000000 00000002 00000000 c020375e 00000000 0014ce9d
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01e9674>] reiserfs_dirty_inode+0x24/0xb0
> [<c01a0f7a>] ? mnt_want_write+0x1a/0xc0
> [<c01a613c>] __mark_inode_dirty+0x2c/0x180
> [<c019c6f5>] file_update_time+0xc5/0xe0
> [<c01634eb>] __generic_file_aio_write_nolock+0x18b/0x520
> [<c0163980>] ? generic_file_aio_write+0x50/0xd0
> [<c0163993>] generic_file_aio_write+0x63/0xd0
> [<c018a331>] do_sync_write+0xd1/0x110
> [<c013dab0>] ? autoremove_wake_function+0x0/0x40
> [<c01b42ba>] ? dnotify_parent+0x2a/0x80
> [<c01e23ba>] reiserfs_file_write+0x8a/0xa0
> [<c018a94c>] vfs_write+0x9c/0x140
> [<c0173f52>] ? might_fault+0x52/0xa0
> [<c01e2330>] ? reiserfs_file_write+0x0/0xa0
> [<c018aaad>] sys_write+0x3d/0x70
> [<c0102e48>] sysenter_do_call+0x12/0x36
> agetty D 00000000 5432 2441 1
> f73cbba8 00000046 00000046 00000000 00000000 f73dd280 f73dd50c 6a069cd5
> 00000b33 00000000 f73dd50c c020375e 00000202 00000202 f670a014 f73dd280
> f73cbbf0 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000246
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01df2e6>] reiserfs_get_block+0x66/0x14f0
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014e52b>] ? trace_hardirqs_on+0xb/0x10
> [<c044b07d>] ? _spin_unlock_irq+0x2d/0x50
> [<c0124b02>] ? finish_task_switch+0x62/0xb0
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c0124aa0>] ? finish_task_switch+0x0/0xb0
> [<c044b2df>] ? __reacquire_kernel_lock+0x2f/0x40
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c0332d78>] ? serial8250_tx_empty+0x18/0x50
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c016108e>] ? add_to_page_cache_locked+0x5e/0xe0
> [<c01b32f0>] do_mpage_readpage+0x340/0x490
> [<c01610e9>] ? add_to_page_cache_locked+0xb9/0xe0
> [<c016a415>] ? __lru_cache_add+0x65/0xa0
> [<c0161184>] ? add_to_page_cache_lru+0x74/0x80
> [<c01b352e>] mpage_readpages+0x9e/0xe0
> [<c01df280>] ? reiserfs_get_block+0x0/0x14f0
> [<c04480b8>] ? __schedule+0x2f8/0x4f0
> [<c01dc710>] ? reiserfs_readpages+0x0/0x20
> [<c01dc729>] reiserfs_readpages+0x19/0x20
> [<c01df280>] ? reiserfs_get_block+0x0/0x14f0
> [<c01696ff>] __do_page_cache_readahead+0x13f/0x200
> [<c016980e>] do_page_cache_readahead+0x4e/0x70
> [<c0162caf>] filemap_fault+0x30f/0x430
> [<c0175f6d>] __do_fault+0x3d/0x360
> [<c01b42ba>] ? dnotify_parent+0x2a/0x80
> [<c0176aa0>] handle_mm_fault+0x100/0x590
> [<c014159d>] ? down_read_trylock+0x5d/0x70
> [<c011aff1>] do_page_fault+0x121/0x280
> [<c011aed0>] ? do_page_fault+0x0/0x280
> [<c044b40a>] error_code+0x6a/0x70
> [<c011aed0>] ? do_page_fault+0x0/0x280
> screen D 00000000 5244 2529 2528
> f612fd90 00000046 00000046 00000000 00000000 f6babde0 f6bac06c 33c73fab
> 00000b1c 00000000 f6bac06c c0193bf3 00000202 00000202 f6f9615c f6babde0
> f612fdd8 c0448e3a 00000000 00000002 00000000 c0193bf3 00000000 0012fed0
> Call Trace:
> [<c0193bf3>] ? do_lookup+0xd3/0x1c0
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c0193bf3>] ? do_lookup+0xd3/0x1c0
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c0193bf3>] ? do_lookup+0xd3/0x1c0
> [<c0193bf3>] do_lookup+0xd3/0x1c0
> [<c0193e49>] __link_path_walk+0x169/0xcc0
> [<c0194b53>] path_walk+0x53/0xb0
> [<c0194c8b>] do_path_lookup+0x8b/0x170
> [<c01958ea>] user_path_at+0x4a/0x80
> [<c01b42ba>] ? dnotify_parent+0x2a/0x80
> [<c018e471>] vfs_stat_fd+0x21/0x50
> [<c018e571>] vfs_stat+0x11/0x20
> [<c018e594>] sys_stat64+0x14/0x30
> [<c018abdf>] ? vfs_read+0xff/0x140
> [<c0317d70>] ? tty_read+0x0/0xb0
> [<c018acdd>] ? sys_read+0x3d/0x70
> [<c02c9d84>] ? trace_hardirqs_on_thunk+0xc/0x10
> [<c0102e48>] sysenter_do_call+0x12/0x36
> rtorrent D 00000000 4084 2543 2542
> f6325d88 00000046 00000046 00000000 00000000 f6b894a0 f6b8972c d79a38dd
> 00000af9 00000000 f6b8972c c020375e 00000202 00000202 f670a014 f6b894a0
> f6325dd0 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000000
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01e266a>] reiserfs_readdir_dentry+0x5a/0x740
> [<c018b46f>] ? file_move+0x1f/0x50
> [<c0197a80>] ? filldir64+0x0/0xf0
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c044ae07>] ? _spin_unlock+0x27/0x50
> [<c018b48e>] ? file_move+0x3e/0x50
> [<c0173f9d>] ? might_fault+0x9d/0xa0
> [<c0197d6f>] ? vfs_readdir+0x4f/0xa0
> [<c0449013>] ? __mutex_lock_common+0x2f3/0x430
> [<c0197d6f>] ? vfs_readdir+0x4f/0xa0
> [<c01e2d67>] reiserfs_readdir+0x17/0x20
> [<c0197d9e>] vfs_readdir+0x7e/0xa0
> [<c0197a80>] ? filldir64+0x0/0xf0
> [<c0197e29>] sys_getdents64+0x69/0xb0
> [<c0102e48>] sysenter_do_call+0x12/0x36
> bash D 00000000 5788 2546 2529
> f585dcd8 00000046 00000046 00000000 00000000 f6b8bde0 f6b8c06c 188b381a
> 00000b1b 00000000 f6b8c06c c020375e 00000202 00000202 f670a014 f6b8bde0
> f585dd20 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000000
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01db389>] reiserfs_lookup+0x59/0x170
> [<c019b923>] ? d_alloc+0x123/0x1b0
> [<c0193cb1>] do_lookup+0x191/0x1c0
> [<c0194305>] __link_path_walk+0x625/0xcc0
> [<c0194b53>] path_walk+0x53/0xb0
> [<c0194c8b>] do_path_lookup+0x8b/0x170
> [<c01958ea>] user_path_at+0x4a/0x80
> [<c018dda4>] ? cp_new_stat64+0xe4/0x100
> [<c018e471>] vfs_stat_fd+0x21/0x50
> [<c018e571>] vfs_stat+0x11/0x20
> [<c018e594>] sys_stat64+0x14/0x30
> [<c02c9d84>] ? trace_hardirqs_on_thunk+0xc/0x10
> [<c0102e48>] sysenter_do_call+0x12/0x36
> dbench D 00000000 4816 20778 1
> f44bbd1c 00000046 00000046 00000000 00000000 ce105280 ce10550c d12c8ac7
> 00000af8 00000000 ce10550c c020375e 00000202 00000202 f670a014 ce105280
> f44bbd64 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766a00
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fb8c5>] flush_commit_list+0x125/0x7c0
> [<c018570d>] ? check_object+0x13d/0x200
> [<c01fba72>] flush_commit_list+0x2d2/0x7c0
> [<c0121ab0>] ? __wake_up+0x40/0x50
> [<c01fde19>] do_journal_end+0xdc9/0xf50
> [<c0133420>] ? process_timeout+0x0/0x10
> [<c01fdffd>] journal_end_sync+0x5d/0x80
> [<c01feadd>] reiserfs_commit_for_inode+0x1bd/0x1e0
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01e1e68>] reiserfs_sync_file+0x38/0x90
> [<c01a905d>] vfs_fsync+0x6d/0xe0
> [<c01a90fd>] do_fsync+0x2d/0x50
> [<c01a914d>] sys_fsync+0xd/0x10
> [<c0102e48>] sysenter_do_call+0x12/0x36
> dbench D 00000000 4660 20779 1
> f44b3de4 00000046 00000046 00000000 00000000 ce106720 ce1069ac d12c836c
> 00000af8 00000000 ce1069ac c020375e 00000202 00000202 f670a014 ce106720
> f44b3e2c c0448e3a 00000000 00000002 00000000 c020375e 00000000 00767400
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fb8c5>] flush_commit_list+0x125/0x7c0
> [<c04480b8>] ? __schedule+0x2f8/0x4f0
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01fba72>] flush_commit_list+0x2d2/0x7c0
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0133420>] ? process_timeout+0x0/0x10
> [<c0449203>] ? mutex_lock_nested+0x33/0x40
> [<c01fe9c1>] reiserfs_commit_for_inode+0xa1/0x1e0
> [<c0449203>] ? mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01e1e68>] reiserfs_sync_file+0x38/0x90
> [<c01a905d>] vfs_fsync+0x6d/0xe0
> [<c01a90fd>] do_fsync+0x2d/0x50
> [<c01a914d>] sys_fsync+0xd/0x10
> [<c0102e48>] sysenter_do_call+0x12/0x36
> dbench D 00000000 4724 20780 1
> f6389de4 00000046 00000046 00000000 00000000 ce102940 ce102bcc d12c7533
> 00000af8 00000000 ce102bcc c020375e 00000202 00000202 f670a014 ce102940
> f6389e2c c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766280
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fb8c5>] flush_commit_list+0x125/0x7c0
> [<c0169c29>] ? pagevec_lookup_tag+0x29/0x40
> [<c0168241>] ? write_cache_pages+0xd1/0x350
> [<c01fba72>] flush_commit_list+0x2d2/0x7c0
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449013>] ? __mutex_lock_common+0x2f3/0x430
> [<c01fe9c1>] reiserfs_commit_for_inode+0xa1/0x1e0
> [<c0449203>] ? mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01e1e68>] reiserfs_sync_file+0x38/0x90
> [<c01a905d>] vfs_fsync+0x6d/0xe0
> [<c01a90fd>] do_fsync+0x2d/0x50
> [<c01a914d>] sys_fsync+0xd/0x10
> [<c0102e48>] sysenter_do_call+0x12/0x36
> dbench D 00000000 4944 20781 1
> f589dc84 00000046 00000046 00000000 00000000 ce1014a0 ce10172c d0f2ddbd
> 00000af8 00000000 ce10172c c020375e 00000202 00000202 f670a014 ce1014a0
> f589dccc c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766280
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fe08b>] do_journal_begin_r+0x6b/0x320
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01fe400>] journal_begin+0x80/0x110
> [<c0449203>] ? mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01e9686>] reiserfs_dirty_inode+0x36/0xb0
> [<c01a0f7a>] ? mnt_want_write+0x1a/0xc0
> [<c01a613c>] __mark_inode_dirty+0x2c/0x180
> [<c019c6f5>] file_update_time+0xc5/0xe0
> [<c01634eb>] __generic_file_aio_write_nolock+0x18b/0x520
> [<c0163980>] ? generic_file_aio_write+0x50/0xd0
> [<c0163993>] generic_file_aio_write+0x63/0xd0
> [<c014e52b>] ? trace_hardirqs_on+0xb/0x10
> [<c018a331>] do_sync_write+0xd1/0x110
> [<c013dab0>] ? autoremove_wake_function+0x0/0x40
> [<c01e23ba>] reiserfs_file_write+0x8a/0xa0
> [<c018a94c>] vfs_write+0x9c/0x140
> [<c02ca436>] ? copy_to_user+0x36/0x130
> [<c01e2330>] ? reiserfs_file_write+0x0/0xa0
> [<c018aa6b>] sys_pwrite64+0x7b/0x80
> [<c0102e48>] sysenter_do_call+0x12/0x36
> dbench D 00000000 4832 20782 1
> f4617a64 00000046 00000046 00000000 00000000 ce103de0 ce10406c d12ca393
> 00000af8 00000000 ce10406c c020375e 00000202 00000202 f670a014 ce103de0
> f4617aac c0448e3a 00000000 00000002 00000000 c020375e 00000000 00767900
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fe08b>] do_journal_begin_r+0x6b/0x320
> [<c01fe400>] journal_begin+0x80/0x110
> [<c01fe4c2>] ? reiserfs_persistent_transaction+0x32/0x90
> [<c01fe4d1>] reiserfs_persistent_transaction+0x41/0x90
> [<c01df460>] reiserfs_get_block+0x1e0/0x14f0
> [<c018dbbe>] ? inode_get_bytes+0x1e/0x50
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c018570d>] ? check_object+0x13d/0x200
> [<c01ab2c3>] ? create_empty_buffers+0x33/0xa0
> [<c01ad424>] __block_prepare_write+0x1a4/0x380
> [<c0161184>] ? add_to_page_cache_lru+0x74/0x80
> [<c01ad7a8>] block_write_begin+0x48/0xe0
> [<c01df280>] ? reiserfs_get_block+0x0/0x14f0
> [<c01ddfbc>] reiserfs_write_begin+0xec/0x200
> [<c01df280>] ? reiserfs_get_block+0x0/0x14f0
> [<c01618f5>] generic_file_buffered_write+0x155/0x2c0
> [<c0163520>] __generic_file_aio_write_nolock+0x1c0/0x520
> [<c0163993>] generic_file_aio_write+0x63/0xd0
> [<c018a331>] do_sync_write+0xd1/0x110
> [<c0173f52>] ? might_fault+0x52/0xa0
> [<c013dab0>] ? autoremove_wake_function+0x0/0x40
> [<c01e23ba>] reiserfs_file_write+0x8a/0xa0
> [<c018a94c>] vfs_write+0x9c/0x140
> [<c02ca436>] ? copy_to_user+0x36/0x130
> [<c01e2330>] ? reiserfs_file_write+0x0/0xa0
> [<c018aa6b>] sys_pwrite64+0x7b/0x80
> [<c0102e48>] sysenter_do_call+0x12/0x36
> dbench D 00000000 4764 20783 1
> f5833e48 00000046 00000046 00000000 00000000 f73de720 f73de9ac d12c96e7
> 00000af8 00000000 f73de9ac c020375e 00000202 00000202 f670a014 f73de720
> f5833e90 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766280
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fb8c5>] flush_commit_list+0x125/0x7c0
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449013>] ? __mutex_lock_common+0x2f3/0x430
> [<c01fe9c1>] reiserfs_commit_for_inode+0xa1/0x1e0
> [<c0449203>] ? mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01e1e68>] reiserfs_sync_file+0x38/0x90
> [<c01a905d>] vfs_fsync+0x6d/0xe0
> [<c01a90fd>] do_fsync+0x2d/0x50
> [<c01a914d>] sys_fsync+0xd/0x10
> [<c0102e48>] sysenter_do_call+0x12/0x36
> dbench D 00000000 4600 20785 1
> f460bd1c 00000046 00000046 00000000 00000000 f6b88000 f6b8828c d12c9d18
> 00000af8 00000000 f6b8828c c020375e 00000202 00000202 f670a014 f6b88000
> f460bd64 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766000
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fb8c5>] flush_commit_list+0x125/0x7c0
> [<c01fba72>] flush_commit_list+0x2d2/0x7c0
> [<c0121ab0>] ? __wake_up+0x40/0x50
> [<c01fde19>] do_journal_end+0xdc9/0xf50
> [<c0133420>] ? process_timeout+0x0/0x10
> [<c01fdffd>] journal_end_sync+0x5d/0x80
> [<c01feadd>] reiserfs_commit_for_inode+0x1bd/0x1e0
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01e1e68>] reiserfs_sync_file+0x38/0x90
> [<c01a905d>] vfs_fsync+0x6d/0xe0
> [<c01a90fd>] do_fsync+0x2d/0x50
> [<c01a914d>] sys_fsync+0xd/0x10
> [<c0102e48>] sysenter_do_call+0x12/0x36
> dbench D 00000000 4548 20786 1
> f5903d1c 00000046 00000046 00000000 00000000 f6960000 f696028c d12fcd2a
> 00000af8 00000000 f696028c c020375e 00000202 00000202 f670a014 f6960000
> f5903d64 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766280
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fb8c5>] flush_commit_list+0x125/0x7c0
> [<c018570d>] ? check_object+0x13d/0x200
> [<c01fba72>] flush_commit_list+0x2d2/0x7c0
> [<c0121ab0>] ? __wake_up+0x40/0x50
> [<c01fde19>] do_journal_end+0xdc9/0xf50
> [<c0133420>] ? process_timeout+0x0/0x10
> [<c01fdffd>] journal_end_sync+0x5d/0x80
> [<c01feadd>] reiserfs_commit_for_inode+0x1bd/0x1e0
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01e1e68>] reiserfs_sync_file+0x38/0x90
> [<c01a905d>] vfs_fsync+0x6d/0xe0
> [<c01a90fd>] do_fsync+0x2d/0x50
> [<c01a914d>] sys_fsync+0xd/0x10
> [<c0102e48>] sysenter_do_call+0x12/0x36
> dbench D 00000000 4704 20787 1
> f4557d60 00000046 00000046 00000000 00000000 f69614a0 f696172c d12c7cad
> 00000af8 00000000 f696172c c020375e 00000202 00000202 f670a014 f69614a0
> f4557da8 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00767b80
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fb8c5>] flush_commit_list+0x125/0x7c0
> [<c01862b1>] ? __slab_alloc+0xc1/0x5f0
> [<c02c4f65>] ? __prop_inc_single+0x65/0x80
> [<c01fbfc3>] get_list_bitmap+0x63/0xb0
> [<c01fdd5f>] do_journal_end+0xd0f/0xf50
> [<c0133420>] ? process_timeout+0x0/0x10
> [<c01fdffd>] journal_end_sync+0x5d/0x80
> [<c01feadd>] reiserfs_commit_for_inode+0x1bd/0x1e0
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01e1e68>] reiserfs_sync_file+0x38/0x90
> [<c01a905d>] vfs_fsync+0x6d/0xe0
> [<c01a90fd>] do_fsync+0x2d/0x50
> [<c01a914d>] sys_fsync+0xd/0x10
> [<c0102e48>] sysenter_do_call+0x12/0x36
> pdflush D 00000000 6080 20790 2
> f09bfe3c 00000046 00000046 00000000 00000000 f7088000 f708828c d12caa96
> 00000af8 00000000 f708828c c020375e 00000202 00000202 f670a014 f7088000
> f09bfe84 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00766c80
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01fe08b>] do_journal_begin_r+0x6b/0x320
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01fe400>] journal_begin+0x80/0x110
> [<c0449203>] ? mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c01e95fe>] reiserfs_sync_fs+0x3e/0x80
> [<c0449203>] ? mutex_lock_nested+0x33/0x40
> [<c018c1df>] ? lock_super+0x1f/0x30
> [<c01e964d>] reiserfs_write_super+0xd/0x10
> [<c018c3d4>] sync_supers+0x84/0xc0
> [<c0169330>] ? pdflush+0x0/0x1c0
> [<c016860b>] wb_kupdate+0x2b/0x100
> [<c014e52b>] ? trace_hardirqs_on+0xb/0x10
> [<c0169330>] ? pdflush+0x0/0x1c0
> [<c01693f8>] pdflush+0xc8/0x1c0
> [<c01685e0>] ? wb_kupdate+0x0/0x100
> [<c013d86a>] kthread+0x3a/0x70
> [<c013d830>] ? kthread+0x0/0x70
> [<c0103677>] kernel_thread_helper+0x7/0x10
> git-daemon D 00000000 6696 20791 2273
> f3705ba8 00000046 00000046 00000000 00000000 f66fe720 f66fe9ac 4a31216d
> 00000afb 00000000 f66fe9ac c020375e 00000202 00000202 f670a014 f66fe720
> f3705bf0 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000000
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01df2e6>] reiserfs_get_block+0x66/0x14f0
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c018440c>] ? slab_pad_check+0x3c/0x120
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c01666ee>] ? get_page_from_freelist+0x41e/0x530
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c016108e>] ? add_to_page_cache_locked+0x5e/0xe0
> [<c01b32f0>] do_mpage_readpage+0x340/0x490
> [<c01610e9>] ? add_to_page_cache_locked+0xb9/0xe0
> [<c016a415>] ? __lru_cache_add+0x65/0xa0
> [<c0161184>] ? add_to_page_cache_lru+0x74/0x80
> [<c01b352e>] mpage_readpages+0x9e/0xe0
> [<c01df280>] ? reiserfs_get_block+0x0/0x14f0
> [<c01dc710>] ? reiserfs_readpages+0x0/0x20
> [<c01dc729>] reiserfs_readpages+0x19/0x20
> [<c01df280>] ? reiserfs_get_block+0x0/0x14f0
> [<c01696ff>] __do_page_cache_readahead+0x13f/0x200
> [<c016980e>] do_page_cache_readahead+0x4e/0x70
> [<c0162caf>] filemap_fault+0x30f/0x430
> [<c0175f6d>] __do_fault+0x3d/0x360
> [<c0176aa0>] handle_mm_fault+0x100/0x590
> [<c014159d>] ? down_read_trylock+0x5d/0x70
> [<c011aff1>] do_page_fault+0x121/0x280
> [<c011aed0>] ? do_page_fault+0x0/0x280
> [<c044b40a>] error_code+0x6a/0x70
> [<c011aed0>] ? do_page_fault+0x0/0x280
> git-daemon D f63ede14 6888 20792 2273
> f63ede20 00000046 00000000 f63ede14 00000046 f66fa940 f66fabcc 977b8ab0
> 00000afe 00000000 f66fabcc f63ede14 c014e52b c057e5a0 000003ae f63ede70
> f63ede28 c04482c2 f63ede34 c04482f4 f63ede68 f63ede3c c0160d95 f63ede5c
> Call Trace:
> [<c014e52b>] ? trace_hardirqs_on+0xb/0x10
> [<c04482c2>] schedule+0x12/0x30
> [<c04482f4>] io_schedule+0x14/0x20
> [<c0160d95>] sync_page+0x55/0x60
> [<c0448881>] __wait_on_bit_lock+0x41/0x80
> [<c0160d40>] ? sync_page+0x0/0x60
> [<c0160d18>] __lock_page+0x88/0xa0
> [<c013daf0>] ? wake_bit_function+0x0/0x60
> [<c0160eb9>] find_lock_page+0x39/0x70
> [<c0162ad7>] filemap_fault+0x137/0x430
> [<c0175f6d>] __do_fault+0x3d/0x360
> [<c0176aa0>] handle_mm_fault+0x100/0x590
> [<c014159d>] ? down_read_trylock+0x5d/0x70
> [<c011aff1>] do_page_fault+0x121/0x280
> [<c011aed0>] ? do_page_fault+0x0/0x280
> [<c044b40a>] error_code+0x6a/0x70
> [<c011aed0>] ? do_page_fault+0x0/0x280
> fcron D 00000000 7168 20797 2257
> f5a0fd80 00000046 00000046 00000000 00000000 f6952940 f6952bcc df71112a
> 00000b25 00000000 f6952bcc c020375e 00000202 00000202 f670a014 f6952940
> f5a0fdc8 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00000000
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01db389>] reiserfs_lookup+0x59/0x170
> [<c019b923>] ? d_alloc+0x123/0x1b0
> [<c019275f>] __lookup_hash+0xcf/0x130
> [<c01927e7>] lookup_hash+0x27/0x30
> [<c0195cb5>] do_filp_open+0x245/0x7a0
> [<c019ed68>] ? alloc_fd+0x28/0xf0
> [<c044ae07>] ? _spin_unlock+0x27/0x50
> [<c019ee18>] ? alloc_fd+0xd8/0xf0
> [<c0188879>] do_sys_open+0x59/0xf0
> [<c0188979>] sys_open+0x29/0x40
> [<c0102e48>] sysenter_do_call+0x12/0x36
> Sched Debug Version: v0.09, 2.6.29-10247-g8e2c4f2-dirty #2
> now at 12468457.474391 msecs
> .jiffies : 1216523
> .sysctl_sched_latency : 20.000000
> .sysctl_sched_min_granularity : 4.000000
> .sysctl_sched_wakeup_granularity : 5.000000
> .sysctl_sched_child_runs_first : 0.000001
> .sysctl_sched_features : 113917
>
> cpu#0, 1830.032 MHz
> .nr_running : 1
> .load : 1024
> .nr_switches : 6163991
> .nr_load_updates : 582498
> .nr_uninterruptible : 21
> .next_balance : 0.000000
> .curr->pid : 20784
> .clock : 12465239.317891
> .cpu_load[0] : 1024
> .cpu_load[1] : 1024
> .cpu_load[2] : 1024
> .cpu_load[3] : 1024
> .cpu_load[4] : 1024
> .yld_count : 0
> .sched_switch : 0
> .sched_count : 778698744
> .sched_goidle : 307156
> .ttwu_count : 0
> .ttwu_local : 0
> .bkl_count : 275
>
> cfs_rq[0]:
> .exec_clock : 1507714.391289
> .MIN_vruntime : 0.000001
> .min_vruntime : 1444293.051451
> .max_vruntime : 0.000001
> .spread : 0.000000
> .spread0 : 0.000000
> .nr_running : 1
> .load : 1024
> .nr_spread_over : 35
>
> rt_rq[0]:
> .rt_nr_running : 0
> .rt_throttled : 0
> .rt_time : 0.000000
> .rt_runtime : 950.000000
>
> runnable tasks:
> task PID tree-key switches prio exec-runtime sum-exec sum-sleep
> ----------------------------------------------------------------------------------------------------------
> R dbench 20784 1444293.051451 4569 120 1444293.051451 401016.637062 16471.092414
>
> INFO: task svscan:2223 blocked for more than 480 seconds.
> "echo 0 > /proc/sys/kernel/hung_task_timeout_secs" disables this message.
> svscan D 00000000 5384 2223 1
> f731dd8c 00000046 00000046 00000000 00000000 f6740000 f674028c eaa8d98d
> 00000af9 00000000 f674028c c020375e 00000202 00000202 f670a014 f6740000
> f731ddd4 c0448e3a 00000000 00000002 00000000 c020375e 00000000 00740000
> Call Trace:
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0448e3a>] __mutex_lock_common+0x11a/0x430
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c0449203>] mutex_lock_nested+0x33/0x40
> [<c020375e>] ? reiserfs_write_lock+0x1e/0x30
> [<c020375e>] reiserfs_write_lock+0x1e/0x30
> [<c01e266a>] reiserfs_readdir_dentry+0x5a/0x740
> [<c018b46f>] ? file_move+0x1f/0x50
> [<c0197b70>] ? filldir+0x0/0xd0
> [<c014ce9d>] ? print_lock_contention_bug+0x1d/0x110
> [<c044ae07>] ? _spin_unlock+0x27/0x50
> [<c018b48e>] ? file_move+0x3e/0x50
> [<c0173f9d>] ? might_fault+0x9d/0xa0
> [<c0197d6f>] ? vfs_readdir+0x4f/0xa0
> [<c0449013>] ? __mutex_lock_common+0x2f3/0x430
> [<c0197d6f>] ? vfs_readdir+0x4f/0xa0
> [<c01e2d67>] reiserfs_readdir+0x17/0x20
> [<c0197d9e>] vfs_readdir+0x7e/0xa0
> [<c0197b70>] ? filldir+0x0/0xd0
> [<c0197ed8>] sys_getdents+0x68/0xb0
> [<c0102e48>] sysenter_do_call+0x12/0x36
> INFO: lockdep is turned off.

2009-04-09 15:15:47

by Andi Kleen

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL

Frederic Weisbecker <[email protected]> writes:
>
> So the new lock that replaces the bkl here is a per superblock mutex

You have to be very careful with this. Mutexes can be slower than
spinlocks (and the new BKL is a spinlock) in some situations, they
typically schedule much more etc., which can be costly. So this might
have actually made it slower if you're unlucky. There were some
experimental changes recently to make mutexes more behave like
spinlocks, but I don't think they're fully performance equivalent.

Better would be to use spinlocks if possible. I guess you just would
need to find all sleep points and wrap them with lock dropping?

> After this patch, reiserfs suffers from a slight performance regression (for now).

That might be related to the scheduling behaviour. Watch the reschedule
counters in vmstat 1

-Andi

--
[email protected] -- Speaking for myself only.

2009-04-09 15:41:31

by Linus Torvalds

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL



On Thu, 9 Apr 2009, Andi Kleen wrote:
>
> You have to be very careful with this. Mutexes can be slower than
> spinlocks (and the new BKL is a spinlock) in some situations, they
> typically schedule much more etc., which can be costly.

Actually, with the new adaptive spinning, that basically shouldn't be true
any more. Or rather, you should need some really bad/unlucky situation for
it to scheduler more than necessary, and if the locker _acts_ like a
spinlock (ie it doesn't block while holding the lock), performance should
approach a spinlock.

That said, there are definitely reasons why a mutex can be slower than the
BKL, and the whole "BKL gets implicitly dropped at sleep time" is very
high on that list of reasons. The sleeping patterns can be _very_
different with a mutex than with a BKL.

> Better would be to use spinlocks if possible. I guess you just would
> need to find all sleep points and wrap them with lock dropping?

I do agree that a filesystem should try to avoid sleeping locks if at all
possible, especially on the paths that the VM uses for writeback. But on
the other hand, I think the issue with reiserfs is just the bad latencies
that the BKL can cause, and then it doesn't matter.

Linus

2009-04-09 18:40:45

by Ingo Molnar

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL


* Linus Torvalds <[email protected]> wrote:

> > Better would be to use spinlocks if possible. I guess you just would
> > need to find all sleep points and wrap them with lock dropping?
>
> I do agree that a filesystem should try to avoid sleeping locks if at
> all possible, especially on the paths that the VM uses for writeback.
> But on the other hand, I think the issue with reiserfs is just the bad
> latencies that the BKL can cause, and then it doesn't matter.

The main motivator is the tip:core/kill-the-BKL tree: we are working on
removing the BKL from all of the kernel, once and forever. We are
actually quite close to that end goal: reiser3 was the last big
stumbling block and it's great that Frederic is tackling that.

Using a mutex seems like the sane choice here. I'd advocate spinlocks
for a new filesystem any day (but even there it's a fine choice to have
a mutex, if top of the line scalability is not an issue).

But for a legacy filesystem like reiser3, which depended on the BKL
auto-dropping on schedule() it would be rather fragile to use spinlocks,
and it would take forever to validate the result. Just one codepath
missed with having some rare scheduling possibility and we'd have a
kernel crash down the road.

Ingo

2009-04-09 19:34:26

by Andi Kleen

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL

> Using a mutex seems like the sane choice here. I'd advocate spinlocks
> for a new filesystem any day (but even there it's a fine choice to have
> a mutex, if top of the line scalability is not an issue).
>
> But for a legacy filesystem like reiser3, which depended on the BKL

reiser3 is much more widely used in the user base than a lot of "non
legacy" file systems. It's very likely it has significantly
more users than ext4 for example. Remember that it was the default file system
for a major distribution until very recently. I also got a few
reiser3 fs still around, it tended to perform very well
on kernel hacker workloads.

Given all that I think the current performance penalties Frederic reports
are not acceptable. Dropping BKL is not a cause in itself, but should
just improve performance.

> auto-dropping on schedule() it would be rather fragile to use spinlocks,
> and it would take forever to validate the result.

Not convinced it would be that hard. It could be probably done with some
straight forward static code analysis. And after that since there's
not much development going anymore it's unlikely to break again.

-Andi

--
[email protected] -- Speaking for myself only.

2009-04-09 20:05:50

by Frederic Weisbecker

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL

On Thu, Apr 09, 2009 at 09:36:35PM +0200, Andi Kleen wrote:
> > Using a mutex seems like the sane choice here. I'd advocate spinlocks
> > for a new filesystem any day (but even there it's a fine choice to have
> > a mutex, if top of the line scalability is not an issue).
> >
> > But for a legacy filesystem like reiser3, which depended on the BKL
>
> reiser3 is much more widely used in the user base than a lot of "non
> legacy" file systems. It's very likely it has significantly
> more users than ext4 for example. Remember that it was the default file system
> for a major distribution until very recently. I also got a few
> reiser3 fs still around, it tended to perform very well
> on kernel hacker workloads.
>
> Given all that I think the current performance penalties Frederic reports
> are not acceptable. Dropping BKL is not a cause in itself, but should
> just improve performance.


It's not only about performances, latency is also a fair goal here.
Moreover, reiserfs is not the only one concerned while dropping its
Bkl, this is one big source of contention less for the rest of the system.

And concerning performance, I'm working on it :-)

Frederic.


> > auto-dropping on schedule() it would be rather fragile to use spinlocks,
> > and it would take forever to validate the result.
>
> Not convinced it would be that hard. It could be probably done with some
> straight forward static code analysis. And after that since there's
> not much development going anymore it's unlikely to break again.
>
> -Andi
>
> --
> [email protected] -- Speaking for myself only.

2009-04-09 21:08:32

by Ingo Molnar

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL


* Andi Kleen <[email protected]> wrote:

> > After this patch, reiserfs suffers from a slight performance
> > regression (for now).
>
> That might be related to the scheduling behaviour. Watch the
> reschedule counters in vmstat 1

FYI, Frederic is one of the main upstream tracing developers, and i
can assure you that when working on scalability he watches something
far more sophisticated than "vmstat 1" ;-)

See the commit below for example.

Thanks,

Ingo

--------------------->
>From 8f350bb74293dded6c51870c34a74645b054caba Mon Sep 17 00:00:00 2001
From: Frederic Weisbecker <[email protected]>
Date: Mon, 6 Apr 2009 01:49:33 +0200
Subject: [PATCH] tracing/lockdep: report the time waited for a lock

While trying to optimize the new lock on reiserfs to replace
the bkl, I find the lock tracing very useful though it lacks
something important for performance (and latency) instrumentation:
the time a task waits for a lock.

That's what this patch implements:

bash-4816 [000] 202.652815: lock_contended: lock_contended: &sb->s_type->i_mutex_key
bash-4816 [000] 202.652819: lock_acquired: &rq->lock (0.000 us)
<...>-4787 [000] 202.652825: lock_acquired: &rq->lock (0.000 us)
<...>-4787 [000] 202.652829: lock_acquired: &rq->lock (0.000 us)
bash-4816 [000] 202.652833: lock_acquired: &sb->s_type->i_mutex_key (16.005 us)

As shown above, the "lock acquired" field is followed by the time
it has been waiting for the lock. Usually, a lock contended entry
is followed by a near lock_acquired entry with a non-zero time waited.

Signed-off-by: Frederic Weisbecker <[email protected]>
Acked-by: Peter Zijlstra <[email protected]>
Cc: Steven Rostedt <[email protected]>
LKML-Reference: <[email protected]>
Signed-off-by: Ingo Molnar <[email protected]>
---
include/trace/lockdep_event_types.h | 23 ++++++++++++++++++-----
kernel/lockdep.c | 8 ++++----
2 files changed, 22 insertions(+), 9 deletions(-)

diff --git a/include/trace/lockdep_event_types.h b/include/trace/lockdep_event_types.h
index adccfcd..863f1e4 100644
--- a/include/trace/lockdep_event_types.h
+++ b/include/trace/lockdep_event_types.h
@@ -32,11 +32,24 @@ TRACE_FORMAT(lock_contended,
TP_FMT("%s", lock->name)
);

-TRACE_FORMAT(lock_acquired,
- TP_PROTO(struct lockdep_map *lock, unsigned long ip),
- TP_ARGS(lock, ip),
- TP_FMT("%s", lock->name)
- );
+TRACE_EVENT(lock_acquired,
+ TP_PROTO(struct lockdep_map *lock, unsigned long ip, s64 waittime),
+
+ TP_ARGS(lock, ip, waittime),
+
+ TP_STRUCT__entry(
+ __field(const char *, name)
+ __field(unsigned long, wait_usec)
+ __field(unsigned long, wait_nsec_rem)
+ ),
+ TP_fast_assign(
+ __entry->name = lock->name;
+ __entry->wait_nsec_rem = do_div(waittime, NSEC_PER_USEC);
+ __entry->wait_usec = (unsigned long) waittime;
+ ),
+ TP_printk("%s (%lu.%03lu us)", __entry->name, __entry->wait_usec,
+ __entry->wait_nsec_rem)
+);

#endif
#endif
diff --git a/kernel/lockdep.c b/kernel/lockdep.c
index b0f0118..c4582a6 100644
--- a/kernel/lockdep.c
+++ b/kernel/lockdep.c
@@ -3061,6 +3061,8 @@ found_it:
put_lock_stats(stats);
}

+DEFINE_TRACE(lock_acquired);
+
static void
__lock_acquired(struct lockdep_map *lock, unsigned long ip)
{
@@ -3099,6 +3101,8 @@ found_it:
hlock->holdtime_stamp = now;
}

+ trace_lock_acquired(lock, ip, waittime);
+
stats = get_lock_stats(hlock_class(hlock));
if (waittime) {
if (hlock->read)
@@ -3137,14 +3141,10 @@ void lock_contended(struct lockdep_map *lock, unsigned long ip)
}
EXPORT_SYMBOL_GPL(lock_contended);

-DEFINE_TRACE(lock_acquired);
-
void lock_acquired(struct lockdep_map *lock, unsigned long ip)
{
unsigned long flags;

- trace_lock_acquired(lock, ip);
-
if (unlikely(!lock_stat))
return;

2009-04-09 21:18:14

by Ingo Molnar

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL


* Andi Kleen <[email protected]> wrote:

> > Using a mutex seems like the sane choice here. I'd advocate spinlocks
> > for a new filesystem any day (but even there it's a fine choice to have
> > a mutex, if top of the line scalability is not an issue).
> >
> > But for a legacy filesystem like reiser3, which depended on the BKL
>
> reiser3 is much more widely used in the user base than a lot of
> "non legacy" file systems. It's very likely it has significantly
> more users than ext4 for example. Remember that it was the default
> file system for a major distribution until very recently. [...]

( Drop the condescending tone please - i very much know that SuSE
installed reiser3 by default for years. It is still a legacy
filesystem and no new development has gone into it for years. )

> [...] I also got a few reiser3 fs still around, it tended to
> perform very well on kernel hacker workloads.

Then i am sure you must like this patch: it introduces a per
superblock lock, splitting up the big BKL serialization. You
totally failed to even acknowledge that advantage, maybe you
missed that aspect?

For example, if you have /home and / on separate reiser3
filesystems, you could see as much as a 200% jump in performance
straight away on certain workloads, on a dual-core box.

That big BKL overhead is a real reiser3 scalability problem -
especially on reiser3 using servers which are likely to have several
filesystems on the same box.

Frederic reported a slight drop in single-threaded performance,
to be expected from a work in progress patch.

Ingo

2009-04-10 00:40:12

by Bron Gondwana

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL

On Thu, Apr 09, 2009 at 11:17:33PM +0200, Ingo Molnar wrote:
>
> * Andi Kleen <[email protected]> wrote:
>
> > > Using a mutex seems like the sane choice here. I'd advocate spinlocks
> > > for a new filesystem any day (but even there it's a fine choice to have
> > > a mutex, if top of the line scalability is not an issue).
> > >
> > > But for a legacy filesystem like reiser3, which depended on the BKL
> >
> > reiser3 is much more widely used in the user base than a lot of
> > "non legacy" file systems. It's very likely it has significantly
> > more users than ext4 for example. Remember that it was the default
> > file system for a major distribution until very recently. [...]
>
> ( Drop the condescending tone please - i very much know that SuSE
> installed reiser3 by default for years. It is still a legacy
> filesystem and no new development has gone into it for years. )

legacy (adj) — A pejorative term used in the computer industry meaning
"it works"

http://homepages.tesco.net/~J.deBoynePollard/FGA/legacy-is-not-a-pejorative.html

It's by far the best of all the currently available and stable
filesystems for our workload (big honking Cyrus IMAP boxes)

There have been bugfixes and occasional bits and pieces over
the years. We applied a couple of patches for a while until
they were accepted upstream only a couple of years ago.

It doesn't get much new development because, gee, "it works".
Some people like their filesystem to keep just working in a
predictable way.

I would be very concerned if people though it was OK to break
it just because these shiny new ext4 and btrfs filesystem which
are _NOT_YET_READY_ had replaced it.

Ta.

> > [...] I also got a few reiser3 fs still around, it tended to
> > perform very well on kernel hacker workloads.
>
> Then i am sure you must like this patch: it introduces a per
> superblock lock, splitting up the big BKL serialization. You
> totally failed to even acknowledge that advantage, maybe you
> missed that aspect?
>
> For example, if you have /home and / on separate reiser3
> filesystems, you could see as much as a 200% jump in performance
> straight away on certain workloads, on a dual-core box.
>
> That big BKL overhead is a real reiser3 scalability problem -
> especially on reiser3 using servers which are likely to have several
> filesystems on the same box.

Yes - I'm certainly interested in that.

That said, we have a box with 83 reiserfs partitions on it, and
which is constrained by IO (main servers really don't need much
CPU). Performance is pretty good even now.

So - I'm interested in this patch series, but not at the expense
of making reiserfs any less stable. Our customers, funnily enough,
like it when our service is stable!

Thanks,

Bron.

2009-04-10 13:08:15

by Ingo Molnar

[permalink] [raw]
Subject: Re: [PATCH] reiserfs: kill-the-BKL


* Bron Gondwana <[email protected]> wrote:

> > For example, if you have /home and / on separate reiser3
> > filesystems, you could see as much as a 200% jump in performance
> > straight away on certain workloads, on a dual-core box.
> >
> > That big BKL overhead is a real reiser3 scalability problem -
> > especially on reiser3 using servers which are likely to have
> > several filesystems on the same box.
>
> Yes - I'm certainly interested in that.
>
> That said, we have a box with 83 reiserfs partitions on it, and
> which is constrained by IO (main servers really don't need much
> CPU). Performance is pretty good even now.
>
> So - I'm interested in this patch series, but not at the expense
> of making reiserfs any less stable. Our customers, funnily
> enough, like it when our service is stable!

Definitely so :-)

Ingo