linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/1] Fix false-negative error reporting from fsync/fdatasync
@ 2015-09-15  9:46 Junichi Nomura
  2015-09-15  9:49 ` Test program: check if fsync() can detect I/O error (1/2) Junichi Nomura
  2015-09-15  9:54 ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Junichi Nomura
  0 siblings, 2 replies; 13+ messages in thread
From: Junichi Nomura @ 2015-09-15  9:46 UTC (permalink / raw)
  To: linux-kernel, linux-fsdevel, linux-mm
  Cc: akpm, andi, fengguang.wu, tony.luck, liwanp, david, Tejun Heo,
	Naoya Horiguchi

Applications use fsync/fdatasync to make sure data is written back to
storage. It is expected that those system calls return error if
writeback has failed (e.g. disk/transport failure, memory failure..)

However if admins run a command such as sync or fsfreeze along side,
fsync/fdatasync may return success even if writeback has failed.
That could lead to data corruption.

This patch is a minimal fix for the problem.
-- 
Jun'ichi Nomura, NEC Corporation

^ permalink raw reply	[flat|nested] 13+ messages in thread

* Test program: check if fsync() can detect I/O error (1/2)
  2015-09-15  9:46 [PATCH 0/1] Fix false-negative error reporting from fsync/fdatasync Junichi Nomura
@ 2015-09-15  9:49 ` Junichi Nomura
  2015-09-15  9:52   ` Test program: check if fsync() can detect I/O error (2/2) Junichi Nomura
  2015-09-15  9:54 ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Junichi Nomura
  1 sibling, 1 reply; 13+ messages in thread
From: Junichi Nomura @ 2015-09-15  9:49 UTC (permalink / raw)
  To: linux-kernel, linux-fsdevel, linux-mm
  Cc: akpm, andi, fengguang.wu, tony.luck, liwanp, david, Tejun Heo,
	Naoya Horiguchi

> However if admins run a command such as sync or fsfreeze along side,
> fsync/fdatasync may return success even if writeback has failed.
> That could lead to data corruption.

For reproducing the problem, compile the attached C program (iogen.c)
and run with 'runtest.sh' script in the next mail:
  # gcc -o iogen iogen.c
  # bash ./runtest.sh

"iogen" does write(), fsync() and checks if on-disk data is same
as application's buffer after successful fsync.
"runtest.sh" injects failure for the file being written by "iogen".
(You need to enable CONFIG_HWPOISON_INJECT=m for the memory error
 injection to work.)

Without the patch, fsync returns success even though data is not on
disk.

  TEST: ext4 / ioerr / sync-command
  (iogen): inject
  (admin): Injecting I/O error
  (admin): Calling sync(2)
  (iogen): remove
  FAIL: corruption!
  DIFF 00000200: de de de de de de de de  | 00 00 00 00 00 00 00 00
  ...

With the patch, fsync detects error correctly.

  TEST: ext4 / ioerr / sync-command
  (iogen): inject
  (admin): Injecting I/O error
  (admin): Calling sync(2)
  INFO: App fsync: Input/output error
  (iogen): remove
  PASS: detected error right
  (iogen): end

-- cut here --
#define _GNU_SOURCE
#include <stdio.h>
#include <string.h>
#include <stdlib.h>
#include <unistd.h>
#include <sys/types.h>
#include <sys/stat.h>
#include <fcntl.h>

unsigned char *app_buf;
unsigned char *ondisk_data;
char *testfile;
size_t buflen;
int fd;
int rfd;

void dumpdiff(unsigned char *buf1, unsigned char *buf2, int len)
{
	int i, j;
	for(i = 0; i < len; i += 8) {
		if (!memcmp(&buf1[i], &buf2[i], 8))
			continue;
		fprintf(stderr, "DIFF %08x: ", i);
		for(j = 0; j < 8 && i + j < len; j++)
			fprintf(stderr, "%02x ", buf1[i]);
                fprintf(stderr, " | ");
		for(j = 0; j < 8 && i + j < len; j++)
			fprintf(stderr, "%02x ", buf2[i]);
                fprintf(stderr, "\n");
        }
}

void notify_injector(char *str)
{
        if (str)
                fprintf(stderr, "(iogen): %s\n", str);
        write(1, "\n", 2);
        sleep(1);
}

void open_fds(void)
{
	fd = open(testfile, O_RDWR);
	if (fd < 0) {
		perror("????: App open");
		exit(1);
	}
	rfd = open(testfile, O_RDONLY|O_DIRECT); /* for verification */
	if (rfd < 0) {
		perror("????: App open rfd");
		exit(1);
	}
}

void init_fd_status(void)
{
	int r;

	r = fsync(fd); /* flush and clean */
	if (r) {
		perror("????: App fsync0");
		exit(1);
	}
	r = pread(fd, app_buf, buflen, 0); /* stage onto cache */
	if (r != buflen) {
		perror("????: App read1");
		exit(1);
	}
}

void close_fds(void)
{
	int r;

	r = close(rfd);
	if (r)
		perror("????: App close read fd");
	r = close(fd);
	if (r)
		perror("????: App close write fd");
}

void write_data(int cnt)
{
	int r;

	memset(app_buf, cnt, buflen);
	r = pwrite(fd, app_buf, buflen, 0);
	if (r != buflen)
		perror("????: App write1");
}

int sync_data(void)
{
	int r, r2;

	r = fsync(fd);
	if (r)
		perror("INFO: App fsync");
	r2 = fsync(fd);
	if (r2)
		perror("????: App fsync (redo)");

	return r;
}

void read_data_direct(void)
{
	int r;

	r = pread(rfd, ondisk_data, buflen, 0);
	if (r != buflen) {
		perror("????: App direct read");
		r = pread(rfd, ondisk_data, buflen, 0);
		if (r != buflen)
			perror("FAIL: App direct read (retry)");
	}
}

void check_data(int fsync_result)
{
	int r;

	r = memcmp(app_buf, ondisk_data, buflen);
	if (r) {
		/* data is different */
		if (fsync_result == 0) {
			fprintf(stderr, "FAIL: corruption!\n");
			dumpdiff(app_buf, ondisk_data, buflen);
		} else
			fprintf(stderr, "PASS: detected error right\n");
	} else {
		/* data is same */
		if (fsync_result == 0)
			fprintf(stderr, "PASS: no error, data is ok\n");
		else
			fprintf(stderr, "????: sync failed, data is ok\n");
	}
}

void cleanup_data(int cnt)
{
	int r;

	/* write-fsync-read without error injection */
	memset(app_buf, cnt, buflen);
	r = pwrite(fd, app_buf, buflen, 0);
	if (r != buflen)
		perror("BUG : App write (w/o failure)");
	r = fsync(fd);
	if (r)
		perror("BUG : App fsync (w/o failure)");
	r = pread(rfd, ondisk_data, buflen, 0);
	if (r != buflen)
		perror("BUG : App read (w/o failure)");
	r = memcmp(app_buf, ondisk_data, buflen);
	if (r)
		fprintf(stderr, "BUG : memcmp failed\n");
}

/*
 * Do this:
 *   1) write
 *   2) inject failure
 *   3) fsync (should return error)
 *   4) remove failure
 *   5) check on-disk data (using direct read)
 */
void runtest(void)
{
	int fsync_result;

	notify_injector("start");
	open_fds();
	init_fd_status();

	write_data(0xde);
	notify_injector("inject");
	fsync_result = sync_data();
	notify_injector("remove");

	/* re-read and compare */
	read_data_direct();
	check_data(fsync_result);

	cleanup_data(0);
	close_fds();
	notify_injector("end");
}

int main(int argc, char **argv)
{
	testfile = argv[1];
	buflen = atoi(argv[2]);

	app_buf = malloc(buflen);
	if (!app_buf)
		exit(1);
	if (posix_memalign((void **) &ondisk_data, 4096, buflen))
		exit(1);

	runtest();

	free(app_buf);
	free(ondisk_data);

	return 0;
}

^ permalink raw reply	[flat|nested] 13+ messages in thread

* Test program: check if fsync() can detect I/O error (2/2)
  2015-09-15  9:49 ` Test program: check if fsync() can detect I/O error (1/2) Junichi Nomura
@ 2015-09-15  9:52   ` Junichi Nomura
  0 siblings, 0 replies; 13+ messages in thread
From: Junichi Nomura @ 2015-09-15  9:52 UTC (permalink / raw)
  To: linux-kernel, linux-fsdevel, linux-mm
  Cc: akpm, andi, fengguang.wu, tony.luck, liwanp, david, Tejun Heo,
	Naoya Horiguchi

On 09/15/15 17:39, Jun'ichi Nomura wrote:
>> However if admins run a command such as sync or fsfreeze along side,
>> fsync/fdatasync may return success even if writeback has failed.
>> That could lead to data corruption.
>
> For reproducing the problem, compile the attached C program (iogen.c)
> and run with 'runtest.sh' script in the next mail:
>   # gcc -o iogen iogen.c
>   # bash ./runtest.sh

-- cut here --
#!/bin/bash

# preparation for hwpoison injection
export KERNEL_SRC=/lib/modules/$(uname -r)/build
[ -d "$KERNEL_SRC" ] || exit 1 # no kernel source given
make vm -C $KERNEL_SRC/tools || exit 1 # tools/vm failed to build
pagetypes=$KERNEL_SRC/tools/vm/page-types
[ -x $pagetypes ] || exit 1
modprobe hwpoison-inject

# -------------------------------------------------------------------
fstype=ext4

# file name of loopback image
loopfile=test.img
imgsize=16M
lodev=/dev/loop0

# filesystem to use
mkfs=mkfs.$fstype

# device-mapper map name
testmap=testmap

# file name to store device-mapper table data
mapok=testmap.ok
maperr=testmap.err

# mount point and file name used for testing
testdir=/mnt/test
testfile=$testdir/x

# test file size
filesize=16384

# -------------------------------------------------------------------
# Set up
#

endtest() {
	sleep 3
	umount $testdir
	dmsetup remove $testmap
	losetup -d $lodev
	exit
}

# Create loopback device for testing
dd if=/dev/zero of=$loopfile bs=$imgsize count=1
losetup $lodev $loopfile || endtest
if [ ! -b $lodev ]; then
	endtest
fi


# Layer DM device for error injection
echo "0 $(blockdev --getsz $lodev) linear $lodev 0" | dmsetup create $testmap
dmsetup table $testmap > $mapok || endtest
if [ ! -b /dev/mapper/$testmap ]; then
	endtest
fi

# Mount and create target file
mkdir -p $testdir
$mkfs /dev/mapper/$testmap
mount /dev/mapper/$testmap $testdir || endtest
dd if=/dev/zero of=$testfile bs=$filesize count=1 oflag=direct || endtest

# Find physical location of the target file
find_location() {
	# pick up physical block number of file offset 0
	filefrag -v $1 | \
		awk '$1 == "0" {print $3} $1 == "0:" {print $4}' | \
		sed 's/\.//g'
}
filefrag -v $testfile
block=$(find_location $testfile)
if [ -z "$block" ]; then
	endtest
fi
blocksize=$(stat -c %s -f $testfile)
secsize=512
sector=$((block * blocksize / secsize + 1))

# Create error mapping: inject error at $sector
next=$((sector + 1))
total=$(blockdev --getsz $lodev)
remainder=$((total - next))
cat <<EOF > $maperr
0 $sector linear $lodev 0
$sector 1 error
$next $remainder linear $lodev $next
EOF

map_replace() {
	cat $1 | dmsetup load $testmap
	dmsetup suspend --nolockfs $testmap
	dmsetup resume $testmap
}

inject_memory_error() {
	local pfn=0x$($pagetypes -f $testfile -Nl | grep ^1$'\t' | cut -f2)
	[ "$pfn" = 0x ] && return 1 # target pfn not found
	$pagetypes -a $pfn -X -N
}

# -------------------------------------------------------------------
# Test
#

msg() {
	echo $* > /dev/kmsg
	echo $*
}

injector_ioerr_nop() {
	# start
	read x
	msg "TEST: $fstype / ioerr / (no admin action)"

	# inject
	read x
	msg "(admin): Injecting I/O error"
	map_replace $maperr
	msg "(admin): Do nothing"

	# remove
	read x
	map_replace $mapok

	# end
	read x
	umount /dev/mapper/$testmap || endtest
	mount /dev/mapper/$testmap $testdir || endtest
}

injector_ioerr_synccmd() {
	# start
	read x
	msg "TEST: $fstype / ioerr / sync-command"

	# inject
	read x
	msg "(admin): Injecting I/O error"
	map_replace $maperr
	msg "(admin): Calling sync(2)"
	sync

	# remove
	read x
	map_replace $mapok

	# end
	read x
	umount /dev/mapper/$testmap || endtest
	mount /dev/mapper/$testmap $testdir || endtest
}

injector_hwpoison_synccmd() {
	# start
	read x
	msg "TEST: $fstype / memory-error / sync-command"

	# inject
	read x
	msg "(admin): Injecting memory error"
	inject_memory_error
	msg "(admin): Calling sync(2)"
	sync

	# remove
	read x

	# end
	read x
	umount /dev/mapper/$testmap || endtest
	mount /dev/mapper/$testmap $testdir || endtest
}

msg '============'
./iogen $testfile $filesize | injector_ioerr_nop
msg '============'
./iogen $testfile $filesize | injector_ioerr_synccmd
msg '============'
./iogen $testfile $filesize | injector_hwpoison_synccmd

# -------------------------------------------------------------------
# Clean up
#
endtest

^ permalink raw reply	[flat|nested] 13+ messages in thread

* [PATCH 1/1] fs: global sync to not clear error status of individual inodes
  2015-09-15  9:46 [PATCH 0/1] Fix false-negative error reporting from fsync/fdatasync Junichi Nomura
  2015-09-15  9:49 ` Test program: check if fsync() can detect I/O error (1/2) Junichi Nomura
@ 2015-09-15  9:54 ` Junichi Nomura
  2015-09-15 14:37   ` Andi Kleen
  2015-09-15 15:20   ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Tejun Heo
  1 sibling, 2 replies; 13+ messages in thread
From: Junichi Nomura @ 2015-09-15  9:54 UTC (permalink / raw)
  To: linux-kernel, linux-fsdevel, linux-mm
  Cc: akpm, andi, fengguang.wu, tony.luck, liwanp, david, Tejun Heo,
	Naoya Horiguchi

filemap_fdatawait() is a function to wait for on-going writeback
to complete but also consume and clear error status of the mapping
set during writeback.
The latter functionality is critical for applications to detect
writeback error with system calls like fsync(2)/fdatasync(2).

However filemap_fdatawait() is also used by sync(2) or FIFREEZE
ioctl, which don't check error status of individual mappings.

As a result, fsync() may not be able to detect writeback error
if events happen in the following order:

   Application                    System admin
   ----------------------------------------------------------
   write data on page cache
                                  Run sync command
                                  writeback completes with error
                                  filemap_fdatawait() clears error
   fsync returns success
   (but the data is not on disk)

This patch adds filemap_fdatawait_keep_errors() for call sites where
writeback error is not handled so that they don't clear error status.

Signed-off-by: Jun'ichi Nomura <j-nomura@ce.jp.nec.com>
---
 fs/fs-writeback.c  |  8 +++++++-
 fs/sync.c          |  2 +-
 include/linux/fs.h |  1 +
 mm/filemap.c       | 35 ++++++++++++++++++++++++++++++++---
 4 files changed, 41 insertions(+), 5 deletions(-)

diff --git a/fs/fs-writeback.c b/fs/fs-writeback.c
index 587ac08..df52aad 100644
--- a/fs/fs-writeback.c
+++ b/fs/fs-writeback.c
@@ -2121,7 +2121,13 @@ static void wait_sb_inodes(struct super_block *sb)
 		iput(old_inode);
 		old_inode = inode;
 
-		filemap_fdatawait(mapping);
+		/*
+		 * Wait for on-going writeback to complete
+		 * but not consume error status on this mapping.
+		 * Otherwise application may fail to catch writeback error
+		 * using fsync(2).
+		 */
+		filemap_fdatawait_keep_errors(mapping);
 
 		cond_resched();
 
diff --git a/fs/sync.c b/fs/sync.c
index fbc98ee..e2b7a77 100644
--- a/fs/sync.c
+++ b/fs/sync.c
@@ -86,7 +86,7 @@ static void fdatawrite_one_bdev(struct block_device *bdev, void *arg)
 
 static void fdatawait_one_bdev(struct block_device *bdev, void *arg)
 {
-	filemap_fdatawait(bdev->bd_inode->i_mapping);
+	filemap_fdatawait_keep_errors(bdev->bd_inode->i_mapping);
 }
 
 /*
diff --git a/include/linux/fs.h b/include/linux/fs.h
index 72d8a84..9355f37 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -2422,6 +2422,7 @@ extern int write_inode_now(struct inode *, int);
 extern int filemap_fdatawrite(struct address_space *);
 extern int filemap_flush(struct address_space *);
 extern int filemap_fdatawait(struct address_space *);
+extern void filemap_fdatawait_keep_errors(struct address_space *);
 extern int filemap_fdatawait_range(struct address_space *, loff_t lstart,
 				   loff_t lend);
 extern int filemap_write_and_wait(struct address_space *mapping);
diff --git a/mm/filemap.c b/mm/filemap.c
index 72940fb..059050a 100644
--- a/mm/filemap.c
+++ b/mm/filemap.c
@@ -340,14 +340,14 @@ EXPORT_SYMBOL(filemap_flush);
  * Walk the list of under-writeback pages of the given address space
  * in the given range and wait for all of them.
  */
-int filemap_fdatawait_range(struct address_space *mapping, loff_t start_byte,
-			    loff_t end_byte)
+static int __filemap_fdatawait_range(struct address_space *mapping,
+				     loff_t start_byte, loff_t end_byte)
 {
 	pgoff_t index = start_byte >> PAGE_CACHE_SHIFT;
 	pgoff_t end = end_byte >> PAGE_CACHE_SHIFT;
 	struct pagevec pvec;
 	int nr_pages;
-	int ret2, ret = 0;
+	int ret = 0;
 
 	if (end_byte < start_byte)
 		goto out;
@@ -374,6 +374,15 @@ int filemap_fdatawait_range(struct address_space *mapping, loff_t start_byte,
 		cond_resched();
 	}
 out:
+	return ret;
+}
+
+int filemap_fdatawait_range(struct address_space *mapping, loff_t start_byte,
+			    loff_t end_byte)
+{
+	int ret, ret2;
+
+	ret = __filemap_fdatawait_range(mapping, start_byte, end_byte);
 	ret2 = filemap_check_errors(mapping);
 	if (!ret)
 		ret = ret2;
@@ -382,6 +391,26 @@ out:
 }
 EXPORT_SYMBOL(filemap_fdatawait_range);
 
+/*
+ * As filemap_check_errors() consumes and clears error status of mapping,
+ * filemap_fdatawait() should be used only when the caller is responsible
+ * for handling the error.
+ *
+ * Use filemap_fdatawait_keep_errors() if callers just want to wait for
+ * witeback and don't handle errors themselves.
+ * Expected call sites are system-wide / filesystem-wide data flushers:
+ * e.g. sync(2), fsfreeze(8)
+ */
+void filemap_fdatawait_keep_errors(struct address_space *mapping)
+{
+	loff_t i_size = i_size_read(mapping->host);
+
+	if (i_size == 0)
+		return;
+
+	__filemap_fdatawait_range(mapping, 0, i_size - 1);
+}
+
 /**
  * filemap_fdatawait - wait for all under-writeback pages to complete
  * @mapping: address space structure to wait for
-- 
2.1.0

^ permalink raw reply related	[flat|nested] 13+ messages in thread

* Re: [PATCH 1/1] fs: global sync to not clear error status of individual inodes
  2015-09-15  9:54 ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Junichi Nomura
@ 2015-09-15 14:37   ` Andi Kleen
  2015-09-15 22:02     ` Andrew Morton
  2015-09-15 15:20   ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Tejun Heo
  1 sibling, 1 reply; 13+ messages in thread
From: Andi Kleen @ 2015-09-15 14:37 UTC (permalink / raw)
  To: Junichi Nomura
  Cc: linux-kernel, linux-fsdevel, linux-mm, akpm, andi, fengguang.wu,
	tony.luck, liwanp, david, Tejun Heo, Naoya Horiguchi

> This patch adds filemap_fdatawait_keep_errors() for call sites where
> writeback error is not handled so that they don't clear error status.

Patch looks good to me. 

Acked-by: Andi Kleen <ak@linux.intel.com>

-Andi

^ permalink raw reply	[flat|nested] 13+ messages in thread

* Re: [PATCH 1/1] fs: global sync to not clear error status of individual inodes
  2015-09-15  9:54 ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Junichi Nomura
  2015-09-15 14:37   ` Andi Kleen
@ 2015-09-15 15:20   ` Tejun Heo
  2015-09-15 16:52     ` Andi Kleen
  2015-09-16  0:59     ` Junichi Nomura
  1 sibling, 2 replies; 13+ messages in thread
From: Tejun Heo @ 2015-09-15 15:20 UTC (permalink / raw)
  To: Junichi Nomura
  Cc: linux-kernel, linux-fsdevel, linux-mm, akpm, andi, fengguang.wu,
	tony.luck, liwanp, david, Naoya Horiguchi

Hello, Junichi.

On Tue, Sep 15, 2015 at 09:54:13AM +0000, Junichi Nomura wrote:
> filemap_fdatawait() is a function to wait for on-going writeback
> to complete but also consume and clear error status of the mapping
> set during writeback.
> The latter functionality is critical for applications to detect
> writeback error with system calls like fsync(2)/fdatasync(2).
> 
> However filemap_fdatawait() is also used by sync(2) or FIFREEZE
> ioctl, which don't check error status of individual mappings.
> 
> As a result, fsync() may not be able to detect writeback error
> if events happen in the following order:
> 
>    Application                    System admin
>    ----------------------------------------------------------
>    write data on page cache
>                                   Run sync command
>                                   writeback completes with error
>                                   filemap_fdatawait() clears error
>    fsync returns success
>    (but the data is not on disk)
> 
> This patch adds filemap_fdatawait_keep_errors() for call sites where
> writeback error is not handled so that they don't clear error status.

Is this an actual problem?  Write errors usually indicate that the
underlying device is completely hosed and the kernel tends to make a
lot of noise throughout the different layers and it often pretty
quickly leads to failures of metadata IOs which which results in
damage-control actions like RO remounts, so in most cases the
specifics of failure handling don't end up mattering all that much.

That said, no reason to not improve upon it.

> @@ -2121,7 +2121,13 @@ static void wait_sb_inodes(struct super_block *sb)
>  		iput(old_inode);
>  		old_inode = inode;
>  
> -		filemap_fdatawait(mapping);
> +		/*
> +		 * Wait for on-going writeback to complete
> +		 * but not consume error status on this mapping.
                       ^don't

> +		 * Otherwise application may fail to catch writeback error

                   mapping; otherwise,

> +		 * using fsync(2).
> +		 */

Can you please re-flow the comment so that it's filling up to, say, 72
or 76 or whatever column?

> +		filemap_fdatawait_keep_errors(mapping);
>  
>  		cond_resched();
>  
> diff --git a/fs/sync.c b/fs/sync.c
> index fbc98ee..e2b7a77 100644
> --- a/fs/sync.c
> +++ b/fs/sync.c
> @@ -86,7 +86,7 @@ static void fdatawrite_one_bdev(struct block_device *bdev, void *arg)
>  
>  static void fdatawait_one_bdev(struct block_device *bdev, void *arg)
>  {
> -	filemap_fdatawait(bdev->bd_inode->i_mapping);
> +	filemap_fdatawait_keep_errors(bdev->bd_inode->i_mapping);

Maybe it'd be better to describe what's going on in detail in the
function comment of filemat_fdatawait_keep_errors() and refer to that
from its callers?

> @@ -382,6 +391,26 @@ out:
>  }
>  EXPORT_SYMBOL(filemap_fdatawait_range);
>  
> +/*
> + * As filemap_check_errors() consumes and clears error status of mapping,
> + * filemap_fdatawait() should be used only when the caller is responsible
> + * for handling the error.

Please make this a proper function comment.

> + *
> + * Use filemap_fdatawait_keep_errors() if callers just want to wait for
> + * witeback and don't handle errors themselves.
      writeback

> + * Expected call sites are system-wide / filesystem-wide data flushers:
> + * e.g. sync(2), fsfreeze(8)

Ditto, please either break up paragraphs or reflow to column
consistently.

> + */
> +void filemap_fdatawait_keep_errors(struct address_space *mapping)
> +{
> +	loff_t i_size = i_size_read(mapping->host);
> +
> +	if (i_size == 0)
> +		return;
> +
> +	__filemap_fdatawait_range(mapping, 0, i_size - 1);
> +}

Generally looks good to me.  We may end up deferring writeback error
from way earlier to later fsync callers but given that sync(2) doesn't
even have a return value, it looks like that's the best we can do.

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 13+ messages in thread

* Re: [PATCH 1/1] fs: global sync to not clear error status of individual inodes
  2015-09-15 15:20   ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Tejun Heo
@ 2015-09-15 16:52     ` Andi Kleen
  2015-09-16  0:59     ` Junichi Nomura
  1 sibling, 0 replies; 13+ messages in thread
From: Andi Kleen @ 2015-09-15 16:52 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Junichi Nomura, linux-kernel, linux-fsdevel, linux-mm, akpm,
	andi, fengguang.wu, tony.luck, liwanp, david, Naoya Horiguchi

> Is this an actual problem?  Write errors usually indicate that the

There are transaction systems that need to track errors at the level
of the individual IO.  So yes we should guarantee that if a particular
sync failed an error is always returned.

-Andi

^ permalink raw reply	[flat|nested] 13+ messages in thread

* Re: [PATCH 1/1] fs: global sync to not clear error status of individual inodes
  2015-09-15 14:37   ` Andi Kleen
@ 2015-09-15 22:02     ` Andrew Morton
  2015-09-16  0:45       ` Junichi Nomura
  0 siblings, 1 reply; 13+ messages in thread
From: Andrew Morton @ 2015-09-15 22:02 UTC (permalink / raw)
  To: Andi Kleen
  Cc: Junichi Nomura, linux-kernel, linux-fsdevel, linux-mm,
	fengguang.wu, tony.luck, liwanp, david, Tejun Heo,
	Naoya Horiguchi

On Tue, 15 Sep 2015 16:37:24 +0200 Andi Kleen <andi@firstfloor.org> wrote:

> > This patch adds filemap_fdatawait_keep_errors() for call sites where
> > writeback error is not handled so that they don't clear error status.
> 
> Patch looks good to me. 
> 

Me too.

It would be nice to capture the test case(s) somewhere permanent. 
Possibly in tools/testing/selftests, but selftests is more for peculiar
linux-specific things.  LTP or xfstests would be a better place.

^ permalink raw reply	[flat|nested] 13+ messages in thread

* Re: [PATCH 1/1] fs: global sync to not clear error status of individual inodes
  2015-09-15 22:02     ` Andrew Morton
@ 2015-09-16  0:45       ` Junichi Nomura
  2015-09-16 10:45         ` xfstests: test data-writeback error detection with fsync Junichi Nomura
  0 siblings, 1 reply; 13+ messages in thread
From: Junichi Nomura @ 2015-09-16  0:45 UTC (permalink / raw)
  To: Andrew Morton, Andi Kleen
  Cc: linux-kernel, linux-fsdevel, linux-mm, fengguang.wu, tony.luck,
	david, Tejun Heo, Naoya Horiguchi

On 09/16/15 07:02, Andrew Morton wrote:
> It would be nice to capture the test case(s) somewhere permanent. 
> Possibly in tools/testing/selftests, but selftests is more for peculiar
> linux-specific things.  LTP or xfstests would be a better place.

I'll check xfstests if I can adapt the test case for its framework.

-- 
Jun'ichi Nomura, NEC Corporation

^ permalink raw reply	[flat|nested] 13+ messages in thread

* Re: [PATCH 1/1] fs: global sync to not clear error status of individual inodes
  2015-09-15 15:20   ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Tejun Heo
  2015-09-15 16:52     ` Andi Kleen
@ 2015-09-16  0:59     ` Junichi Nomura
  2015-09-16  8:39       ` [PATCH v2] " Junichi Nomura
  1 sibling, 1 reply; 13+ messages in thread
From: Junichi Nomura @ 2015-09-16  0:59 UTC (permalink / raw)
  To: Tejun Heo
  Cc: linux-kernel, linux-fsdevel, linux-mm, akpm, andi, fengguang.wu,
	tony.luck, david, Naoya Horiguchi

On 09/16/15 00:20, Tejun Heo wrote:
>> @@ -2121,7 +2121,13 @@ static void wait_sb_inodes(struct super_block *sb)
>>  		iput(old_inode);
>>  		old_inode = inode;
>>  
>> -		filemap_fdatawait(mapping);
>> +		/*
>> +		 * Wait for on-going writeback to complete
>> +		 * but not consume error status on this mapping.
>                        ^don't
> 
>> +		 * Otherwise application may fail to catch writeback error
> 
>                    mapping; otherwise,
> 
>> +		 * using fsync(2).
>> +		 */
> 
> Can you please re-flow the comment so that it's filling up to, say, 72
> or 76 or whatever column?

I'll fix them.

>> -	filemap_fdatawait(bdev->bd_inode->i_mapping);
>> +	filemap_fdatawait_keep_errors(bdev->bd_inode->i_mapping);
> 
> Maybe it'd be better to describe what's going on in detail in the
> function comment of filemat_fdatawait_keep_errors() and refer to that
> from its callers?

Thanks, that seems better.
I'll also extend function comments of filemap_fdatawait so that the
difference becomes clear.

-- 
Jun'ichi Nomura, NEC Corporation

^ permalink raw reply	[flat|nested] 13+ messages in thread

* [PATCH v2] fs: global sync to not clear error status of individual inodes
  2015-09-16  0:59     ` Junichi Nomura
@ 2015-09-16  8:39       ` Junichi Nomura
  2015-09-16 17:47         ` Tejun Heo
  0 siblings, 1 reply; 13+ messages in thread
From: Junichi Nomura @ 2015-09-16  8:39 UTC (permalink / raw)
  To: linux-kernel, linux-fsdevel, linux-mm
  Cc: Tejun Heo, akpm, andi, fengguang.wu, tony.luck, david, Naoya Horiguchi

filemap_fdatawait() is a function to wait for on-going writeback
to complete but also consume and clear error status of the mapping
set during writeback.
The latter functionality is critical for applications to detect
writeback error with system calls like fsync(2)/fdatasync(2).

However filemap_fdatawait() is also used by sync(2) or FIFREEZE
ioctl, which don't check error status of individual mappings.

As a result, fsync() may not be able to detect writeback error
if events happen in the following order:

   Application                    System admin
   ----------------------------------------------------------
   write data on page cache
                                  Run sync command
                                  writeback completes with error
                                  filemap_fdatawait() clears error
   fsync returns success
   (but the data is not on disk)

This patch adds filemap_fdatawait_keep_errors() for call sites where
writeback error is not handled so that they don't clear error status.

Signed-off-by: Jun'ichi Nomura <j-nomura@ce.jp.nec.com>
Acked-by: Andi Kleen <ak@linux.intel.com>
---
v2:
  - Fixed comments based on Tejun's suggestions. No code changes.

v1 and test cases:
  - https://lkml.org/lkml/2015/9/15/335

---
 fs/fs-writeback.c  |  7 +++++-
 fs/sync.c          |  7 +++++-
 include/linux/fs.h |  1 +
 mm/filemap.c       | 67 +++++++++++++++++++++++++++++++++++++++++++-----------
 4 files changed, 67 insertions(+), 15 deletions(-)

diff --git a/fs/fs-writeback.c b/fs/fs-writeback.c
index 587ac08..f252a00 100644
--- a/fs/fs-writeback.c
+++ b/fs/fs-writeback.c
@@ -2121,7 +2121,12 @@ static void wait_sb_inodes(struct super_block *sb)
 		iput(old_inode);
 		old_inode = inode;
 
-		filemap_fdatawait(mapping);
+		/*
+		 * We keep the error status of individual mapping so that
+		 * applications can catch the writeback error using fsync(2).
+		 * See filemap_fdatawait_keep_errors() for details.
+		 */
+		filemap_fdatawait_keep_errors(mapping);
 
 		cond_resched();
 
diff --git a/fs/sync.c b/fs/sync.c
index fbc98ee..4ec430a 100644
--- a/fs/sync.c
+++ b/fs/sync.c
@@ -86,7 +86,12 @@ static void fdatawrite_one_bdev(struct block_device *bdev, void *arg)
 
 static void fdatawait_one_bdev(struct block_device *bdev, void *arg)
 {
-	filemap_fdatawait(bdev->bd_inode->i_mapping);
+	/*
+	 * We keep the error status of individual mapping so that
+	 * applications can catch the writeback error using fsync(2).
+	 * See filemap_fdatawait_keep_errors() for details.
+	 */
+	filemap_fdatawait_keep_errors(bdev->bd_inode->i_mapping);
 }
 
 /*
diff --git a/include/linux/fs.h b/include/linux/fs.h
index 72d8a84..9355f37 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -2422,6 +2422,7 @@ extern int write_inode_now(struct inode *, int);
 extern int filemap_fdatawrite(struct address_space *);
 extern int filemap_flush(struct address_space *);
 extern int filemap_fdatawait(struct address_space *);
+extern void filemap_fdatawait_keep_errors(struct address_space *);
 extern int filemap_fdatawait_range(struct address_space *, loff_t lstart,
 				   loff_t lend);
 extern int filemap_write_and_wait(struct address_space *mapping);
diff --git a/mm/filemap.c b/mm/filemap.c
index 72940fb..40e0af9 100644
--- a/mm/filemap.c
+++ b/mm/filemap.c
@@ -331,23 +331,14 @@ int filemap_flush(struct address_space *mapping)
 }
 EXPORT_SYMBOL(filemap_flush);
 
-/**
- * filemap_fdatawait_range - wait for writeback to complete
- * @mapping:		address space structure to wait for
- * @start_byte:		offset in bytes where the range starts
- * @end_byte:		offset in bytes where the range ends (inclusive)
- *
- * Walk the list of under-writeback pages of the given address space
- * in the given range and wait for all of them.
- */
-int filemap_fdatawait_range(struct address_space *mapping, loff_t start_byte,
-			    loff_t end_byte)
+static int __filemap_fdatawait_range(struct address_space *mapping,
+				     loff_t start_byte, loff_t end_byte)
 {
 	pgoff_t index = start_byte >> PAGE_CACHE_SHIFT;
 	pgoff_t end = end_byte >> PAGE_CACHE_SHIFT;
 	struct pagevec pvec;
 	int nr_pages;
-	int ret2, ret = 0;
+	int ret = 0;
 
 	if (end_byte < start_byte)
 		goto out;
@@ -374,6 +365,29 @@ int filemap_fdatawait_range(struct address_space *mapping, loff_t start_byte,
 		cond_resched();
 	}
 out:
+	return ret;
+}
+
+/**
+ * filemap_fdatawait_range - wait for writeback to complete
+ * @mapping:		address space structure to wait for
+ * @start_byte:		offset in bytes where the range starts
+ * @end_byte:		offset in bytes where the range ends (inclusive)
+ *
+ * Walk the list of under-writeback pages of the given address space
+ * in the given range and wait for all of them.  Check error status of
+ * the address space and return it.
+ *
+ * Since the error status of the address space is cleared by this function,
+ * callers are responsible for checking the return value and handling and/or
+ * reporting the error.
+ */
+int filemap_fdatawait_range(struct address_space *mapping, loff_t start_byte,
+			    loff_t end_byte)
+{
+	int ret, ret2;
+
+	ret = __filemap_fdatawait_range(mapping, start_byte, end_byte);
 	ret2 = filemap_check_errors(mapping);
 	if (!ret)
 		ret = ret2;
@@ -383,11 +397,38 @@ out:
 EXPORT_SYMBOL(filemap_fdatawait_range);
 
 /**
+ * filemap_fdatawait_keep_errors - wait for writeback without clearing errors
+ * @mapping: address space structure to wait for
+ *
+ * Walk the list of under-writeback pages of the given address space
+ * and wait for all of them.  Unlike filemap_fdatawait(), this function
+ * does not clear error status of the address space.
+ *
+ * Use this function if callers don't handle errors themselves.  Expected
+ * call sites are system-wide / filesystem-wide data flushers: e.g. sync(2),
+ * fsfreeze(8)
+ */
+void filemap_fdatawait_keep_errors(struct address_space *mapping)
+{
+	loff_t i_size = i_size_read(mapping->host);
+
+	if (i_size == 0)
+		return;
+
+	__filemap_fdatawait_range(mapping, 0, i_size - 1);
+}
+
+/**
  * filemap_fdatawait - wait for all under-writeback pages to complete
  * @mapping: address space structure to wait for
  *
  * Walk the list of under-writeback pages of the given address space
- * and wait for all of them.
+ * and wait for all of them.  Check error status of the address space
+ * and return it.
+ *
+ * Since the error status of the address space is cleared by this function,
+ * callers are responsible for checking the return value and handling and/or
+ * reporting the error.
  */
 int filemap_fdatawait(struct address_space *mapping)
 {
-- 
2.1.0

^ permalink raw reply related	[flat|nested] 13+ messages in thread

* xfstests: test data-writeback error detection with fsync
  2015-09-16  0:45       ` Junichi Nomura
@ 2015-09-16 10:45         ` Junichi Nomura
  0 siblings, 0 replies; 13+ messages in thread
From: Junichi Nomura @ 2015-09-16 10:45 UTC (permalink / raw)
  To: linux-kernel, linux-fsdevel, linux-mm
  Cc: Andrew Morton, Andi Kleen, fengguang.wu, tony.luck, david,
	Tejun Heo, Naoya Horiguchi

On 09/16/15 07:02, Andrew Morton wrote:
> It would be nice to capture the test case(s) somewhere permanent. 
> Possibly in tools/testing/selftests, but selftests is more for
> peculiar
> linux-specific things.  LTP or xfstests would be a better place.

This is a xfstests version of my test case.
(Device failure portion only. Memory failure will need additional code.)

I used '9999' in this proposal temporarily but if I should other number,
I'll fix that.

---
 common/dm_error       |   96 ++++++++++++++++++++++++++++++++++++++++++
 common/rc             |   16 +++++++
 tests/shared/9999     |  113 ++++++++++++++++++++++++++++++++++++++++++++++++++
 tests/shared/9999.out |   18 +++++++
 tests/shared/group    |    1 
 5 files changed, 244 insertions(+)

diff --git a/common/dm_error b/common/dm_error
new file mode 100644
index 0000000..f6c926f
--- /dev/null
+++ b/common/dm_error
@@ -0,0 +1,96 @@
+##/bin/bash
+#
+# Copyright (c) 2015 NEC Corporation.  All Rights Reserved.
+#
+# This program is free software; you can redistribute it and/or
+# modify it under the terms of the GNU General Public License as
+# published by the Free Software Foundation.
+#
+# This program is distributed in the hope that it would be useful,
+# but WITHOUT ANY WARRANTY; without even the implied warranty of
+# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+# GNU General Public License for more details.
+#
+# You should have received a copy of the GNU General Public License
+# along with this program; if not, write the Free Software Foundation,
+# Inc.,  51 Franklin St, Fifth Floor, Boston, MA  02110-1301  USA
+#
+#
+# common functions for setting up and tearing down a dm error device
+
+# device-mapper map name
+DM_ERR_MAPNAME=xfstests-dm-error
+
+# temporary file names for storing device-mapper table data
+DM_ERR_NORMAL_MAP=$RESULT_DIR/$DM_ERR_MAPNAME.ok
+DM_ERR_ERROR_MAP=$RESULT_DIR/$DM_ERR_MAPNAME.err
+
+_init_dm_error() {
+	# Layer DM device for error injection
+	echo "0 $(blockdev --getsz $SCRATCH_DEV) linear $SCRATCH_DEV 0" | \
+		$DMSETUP_PROG create $DM_ERR_MAPNAME || \
+		_fatal "failed to create dm linear device"
+	$DMSETUP_PROG table $DM_ERR_MAPNAME > $DM_ERR_NORMAL_MAP
+}
+
+_prepare_dm_error_table_for_file() {
+	local file=$1
+	local offset=$2
+	local len=$3
+
+	# Find physical location of the target file
+	find_location() {
+		# pick up physical block number of file offset 0
+		$FILEFRAG_PROG -v $1 | \
+			awk '$1 == "0" {print $3} $1 == "0:" {print $4}' | \
+			sed 's/\.//g'
+	}
+	local block=$(find_location $file)
+	if [ -z "$block" ]; then
+		_fatal "failed to find physical block for $file"
+	fi
+	local blocksize=$(stat -c %s -f $file)
+	local secsize=512
+	local sector=$((block * blocksize / secsize + offset))
+
+	# Create error mapping: inject error at $sector
+	local next=$((sector + len))
+	local total=$(blockdev --getsz $SCRATCH_DEV)
+	local remainder=$((total - next))
+
+	# Generate error mapping
+	echo "0 $sector linear $SCRATCH_DEV 0" > $DM_ERR_ERROR_MAP
+	echo "$sector $len error" >> $DM_ERR_ERROR_MAP
+	echo "$next $remainder linear $SCRATCH_DEV $next" >> $DM_ERR_ERROR_MAP
+}
+
+_load_dm_error_table() {
+	cat $DM_ERR_ERROR_MAP | $DMSETUP_PROG load $DM_ERR_MAPNAME || \
+		_fatal "failed to load dm error table"
+	$DMSETUP_PROG suspend --nolockfs $DM_ERR_MAPNAME || \
+		_fatal "failed to suspend dm device"
+	$DMSETUP_PROG resume $DM_ERR_MAPNAME || \
+		_fatal "failed to suspend dm device"
+}
+_unload_dm_error_table() {
+	cat $DM_ERR_NORMAL_MAP | $DMSETUP_PROG load $DM_ERR_MAPNAME || \
+		_fatal "failed to re-load normal dm table"
+	$DMSETUP_PROG suspend --nolockfs $DM_ERR_MAPNAME || \
+		_fatal "failed to suspend dm device"
+	$DMSETUP_PROG resume $DM_ERR_MAPNAME || \
+		_fatal "failed to suspend dm device"
+}
+
+_mount_dm_error() {
+	mount -t $FSTYP $MOUNT_OPTIONS /dev/mapper/$DM_ERR_MAPNAME $SCRATCH_MNT
+}
+
+_unmount_dm_error() {
+	$UMOUNT_PROG $SCRATCH_MNT
+}
+
+_cleanup_dm_error() {
+	_unmount_dm_error
+	$DMSETUP_PROG remove $DM_ERR_MAPNAME
+	rm -f $DM_ERR_NORMAL_MAP $DM_ERR_ERROR_MAP
+}
diff --git a/common/rc b/common/rc
index 70d2fa8..a4478f6 100644
--- a/common/rc
+++ b/common/rc
@@ -1337,6 +1337,22 @@ _require_sane_bdev_flush()
 	fi
 }
 
+# this test requires the device mapper error target
+#
+_require_dm_error()
+{
+	_require_block_device $SCRATCH_DEV
+	_require_command "$DMSETUP_PROG" dmsetup
+	# Use filefrag to find location to inject failure
+	_require_command "$FILEFRAG_PROG" filefrag
+
+	modprobe dm-mod >/dev/null 2>&1
+	$DMSETUP_PROG targets | grep error >/dev/null 2>&1
+	if [ $? -ne 0 ]; then
+		_notrun "This test requires dm error support"
+	fi
+}
+
 # this test requires the device mapper flakey target
 #
 _require_dm_flakey()
diff --git a/tests/shared/9999 b/tests/shared/9999
new file mode 100755
index 0000000..9e66f77
--- /dev/null
+++ b/tests/shared/9999
@@ -0,0 +1,113 @@
+#!/bin/bash
+# FS QA Test No. 9999
+#
+# Overwrite blocks on buffer, inject sector error using device-mapper,
+# run sync, and then fsync the file.
+# Verify if fsync could detect the error.
+#
+#-----------------------------------------------------------------------
+# Copyright (C) 2015 NEC Corporation. All Rights Reserved.
+# Author: Jun'ichi Nomura <j-nomura@ce.jp.nec.com>
+#
+# This program is free software; you can redistribute it and/or
+# modify it under the terms of the GNU General Public License as
+# published by the Free Software Foundation.
+#
+# This program is distributed in the hope that it would be useful,
+# but WITHOUT ANY WARRANTY; without even the implied warranty of
+# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+# GNU General Public License for more details.
+#
+# You should have received a copy of the GNU General Public License
+# along with this program; if not, write the Free Software Foundation,
+# Inc.,  51 Franklin St, Fifth Floor, Boston, MA  02110-1301  USA
+#-----------------------------------------------------------------------
+#
+
+seq=`basename $0`
+seqres=$RESULT_DIR/$seq
+echo "QA output created by $seq"
+
+here=`pwd`
+status=1        # failure is the default!
+
+_cleanup() {
+	_cleanup_dm_error
+}
+trap "_cleanup; exit \$status" 0 1 2 3 15
+
+# get standard environment, filters and checks
+. ./common/rc
+. ./common/filter
+. ./common/dm_error
+
+# real QA test starts here
+_supported_fs ext4 ext3 ext2 xfs
+_supported_os Linux
+_need_to_be_root
+_require_scratch
+_require_dm_error
+
+rm -f $seqres.full
+
+_scratch_mkfs >> $seqres.full 2>&1
+
+# test file name, size and filling patterns
+testfile=$SCRATCH_MNT/x
+filesize=16384
+pat1=0x11
+pat2=0xaa
+
+_init_dm_error
+_mount_dm_error
+
+echo "Create testfile"
+$XFS_IO_PROG -f \
+	-c "pwrite -S $pat1 0 $filesize" -c "fsync" \
+	$testfile | _filter_xfs_io
+$XFS_IO_PROG -f \
+	-c "pwrite -S $pat2 0 $filesize" -c "fsync" \
+	$testfile.expected | _filter_xfs_io
+_prepare_dm_error_table_for_file $testfile 0 1
+
+echo "Buffered write on the file"
+$XFS_IO_PROG -c "pwrite -S $pat2 0 $filesize" $testfile | _filter_xfs_io
+
+echo "Inject device error"
+_load_dm_error_table
+
+# Running 'sync' while written data is on buffer. This should start
+# writeback and wait for completion.  Beause of the injected failure,
+# the file is marked with AS_EIO.
+echo "Execute sync command"
+sync
+
+# fsync() should get error return.
+echo "Do fsync on the file (should fail)"
+$XFS_IO_PROG -c "fsync" $testfile | _filter_xfs_io
+
+echo "Remove injected device error"
+_unload_dm_error_table
+_unmount_dm_error
+_mount_dm_error
+
+cmp $testfile $testfile.expected >> $seqres.full 2>&1
+if [ $? -ne 0 ]; then
+	echo "Data was not written to disk"
+fi
+echo "Expected contents of the file if error was not injected:" >> $seqres.full
+od -t x1 $testfile.expected >> $seqres.full
+echo "Actual contents of the file:" >> $seqres.full
+od -t x1 $testfile >> $seqres.full
+
+echo "Retry write and fsync"
+$XFS_IO_PROG -f \
+	-c "pwrite -S $pat2 0 $filesize" -c "fsync" \
+	$testfile | _filter_xfs_io
+
+cmp $testfile $testfile.expected
+echo "Contents of the file after retry:" >> $seqres.full
+od -t x1 $testfile >> $seqres.full
+
+status=0
+exit
diff --git a/tests/shared/9999.out b/tests/shared/9999.out
new file mode 100644
index 0000000..236e913
--- /dev/null
+++ b/tests/shared/9999.out
@@ -0,0 +1,18 @@
+QA output created by 9999
+Create testfile
+wrote 16384/16384 bytes at offset 0
+XXX Bytes, X ops; XX:XX:XX.X (XXX YYY/sec and XXX ops/sec)
+wrote 16384/16384 bytes at offset 0
+XXX Bytes, X ops; XX:XX:XX.X (XXX YYY/sec and XXX ops/sec)
+Buffered write on the file
+wrote 16384/16384 bytes at offset 0
+XXX Bytes, X ops; XX:XX:XX.X (XXX YYY/sec and XXX ops/sec)
+Inject device error
+Execute sync command
+Do fsync on the file (should fail)
+fsync: Input/output error
+Remove injected device error
+Data was not written to disk
+Retry write and fsync
+wrote 16384/16384 bytes at offset 0
+XXX Bytes, X ops; XX:XX:XX.X (XXX YYY/sec and XXX ops/sec)
diff --git a/tests/shared/group b/tests/shared/group
index 00d42c8..f196b71 100644
--- a/tests/shared/group
+++ b/tests/shared/group
@@ -11,3 +11,4 @@
 272 auto enospc rw
 289 auto quick
 298 auto trim
+9999 auto quick data

^ permalink raw reply related	[flat|nested] 13+ messages in thread

* Re: [PATCH v2] fs: global sync to not clear error status of individual inodes
  2015-09-16  8:39       ` [PATCH v2] " Junichi Nomura
@ 2015-09-16 17:47         ` Tejun Heo
  0 siblings, 0 replies; 13+ messages in thread
From: Tejun Heo @ 2015-09-16 17:47 UTC (permalink / raw)
  To: Junichi Nomura
  Cc: linux-kernel, linux-fsdevel, linux-mm, akpm, andi, fengguang.wu,
	tony.luck, david, Naoya Horiguchi

On Wed, Sep 16, 2015 at 08:39:09AM +0000, Junichi Nomura wrote:
> filemap_fdatawait() is a function to wait for on-going writeback
> to complete but also consume and clear error status of the mapping
> set during writeback.
> The latter functionality is critical for applications to detect
> writeback error with system calls like fsync(2)/fdatasync(2).
> 
> However filemap_fdatawait() is also used by sync(2) or FIFREEZE
> ioctl, which don't check error status of individual mappings.
> 
> As a result, fsync() may not be able to detect writeback error
> if events happen in the following order:
> 
>    Application                    System admin
>    ----------------------------------------------------------
>    write data on page cache
>                                   Run sync command
>                                   writeback completes with error
>                                   filemap_fdatawait() clears error
>    fsync returns success
>    (but the data is not on disk)
> 
> This patch adds filemap_fdatawait_keep_errors() for call sites where
> writeback error is not handled so that they don't clear error status.
> 
> Signed-off-by: Jun'ichi Nomura <j-nomura@ce.jp.nec.com>
> Acked-by: Andi Kleen <ak@linux.intel.com>

Reviewed-by: Tejun Heo <tj@kernel.org>

Thanks.

-- 
tejun

^ permalink raw reply	[flat|nested] 13+ messages in thread

end of thread, other threads:[~2015-09-16 17:47 UTC | newest]

Thread overview: 13+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-09-15  9:46 [PATCH 0/1] Fix false-negative error reporting from fsync/fdatasync Junichi Nomura
2015-09-15  9:49 ` Test program: check if fsync() can detect I/O error (1/2) Junichi Nomura
2015-09-15  9:52   ` Test program: check if fsync() can detect I/O error (2/2) Junichi Nomura
2015-09-15  9:54 ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Junichi Nomura
2015-09-15 14:37   ` Andi Kleen
2015-09-15 22:02     ` Andrew Morton
2015-09-16  0:45       ` Junichi Nomura
2015-09-16 10:45         ` xfstests: test data-writeback error detection with fsync Junichi Nomura
2015-09-15 15:20   ` [PATCH 1/1] fs: global sync to not clear error status of individual inodes Tejun Heo
2015-09-15 16:52     ` Andi Kleen
2015-09-16  0:59     ` Junichi Nomura
2015-09-16  8:39       ` [PATCH v2] " Junichi Nomura
2015-09-16 17:47         ` Tejun Heo

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).