diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 1da7dfb..2e6bb10 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -1818,6 +1818,24 @@ include_dir 'conf.d'
        </listitem>
       </varlistentry>
 
+     <variablelist>
+      <varlistentry id="guc-bgwriter-flush-to-disk" xreflabel="bgwriter_flush_to_disk">
+       <term><varname>bgwriter_flush_to_disk</varname> (<type>bool</type>)
+       <indexterm>
+        <primary><varname>bgwriter_flush_to_disk</> configuration parameter</primary>
+       </indexterm>
+       </term>
+       <listitem>
+        <para>
+         When the bgwriter writes data, hint the underlying OS that the data
+         must be sent to disk as soon as possible.  This may help smoothing
+         disk IO writes and avoid a stall when an fsync is issued by a
+         checkpoint, but it may also reduce average performance.
+         This setting may have no effect on some platforms.
+        </para>
+       </listitem>
+      </varlistentry>
+
       <varlistentry id="guc-bgwriter-lru-maxpages" xreflabel="bgwriter_lru_maxpages">
        <term><varname>bgwriter_lru_maxpages</varname> (<type>integer</type>)
        <indexterm>
@@ -2495,6 +2513,23 @@ include_dir 'conf.d'
       </listitem>
      </varlistentry>
 
+     <varlistentry id="guc-checkpoint-flush-to-disk" xreflabel="checkpoint_flush_to_disk">
+      <term><varname>checkpoint_flush_to_disk</varname> (<type>bool</type>)
+      <indexterm>
+       <primary><varname>checkpoint_flush_to_disk</> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        When writing data for a checkpoint, hint the underlying OS that the
+        data must be sent to disk as soon as possible.  This may help smoothing
+        disk IO writes and avoid a stall when fsync is issued at the end of
+        the checkpoint, but it may also reduce average performance.
+        This setting may have no effect on some platforms.
+       </para>
+      </listitem>
+     </varlistentry>
+
      <varlistentry id="guc-min-wal-size" xreflabel="min_wal_size">
       <term><varname>min_wal_size</varname> (<type>integer</type>)
       <indexterm>
diff --git a/doc/src/sgml/wal.sgml b/doc/src/sgml/wal.sgml
index f4083c3..cdbdca9 100644
--- a/doc/src/sgml/wal.sgml
+++ b/doc/src/sgml/wal.sgml
@@ -546,6 +546,15 @@
   </para>
 
   <para>
+   On Linux and POSIX platforms, <xref linkend="guc-checkpoint-flush-to-disk">
+   allows to hint the OS that pages written on checkpoints must be flushed
+   to disk quickly.  Otherwise, these pages may be kept in cache for some time,
+   inducing a stall later when <literal>fsync</> is called to actually
+   complete the checkpoint.  This setting helps to reduce transaction latency,
+   but it also has an adverse effect on the average transaction rate.
+  </para>
+
+  <para>
    The number of WAL segment files in <filename>pg_xlog</> directory depends on
    <varname>min_wal_size</>, <varname>max_wal_size</> and
    the amount of WAL generated in previous checkpoint cycles. When old log
diff --git a/src/backend/access/heap/rewriteheap.c b/src/backend/access/heap/rewriteheap.c
index bcce3e3..f565dc4 100644
--- a/src/backend/access/heap/rewriteheap.c
+++ b/src/backend/access/heap/rewriteheap.c
@@ -918,7 +918,7 @@ logical_heap_rewrite_flush_mappings(RewriteState state)
 		 * Note that we deviate from the usual WAL coding practices here,
 		 * check the above "Logical rewrite support" comment for reasoning.
 		 */
-		written = FileWrite(src->vfd, waldata_start, len);
+		written = FileWrite(src->vfd, waldata_start, len, false, NULL);
 		if (written != len)
 			ereport(ERROR,
 					(errcode_for_file_access(),
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 9431ab5..49ec258 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -203,7 +203,7 @@ btbuildempty(PG_FUNCTION_ARGS)
 	/* Write the page.  If archiving/streaming, XLOG it. */
 	PageSetChecksumInplace(metapage, BTREE_METAPAGE);
 	smgrwrite(index->rd_smgr, INIT_FORKNUM, BTREE_METAPAGE,
-			  (char *) metapage, true);
+			  (char *) metapage, true, false, NULL);
 	if (XLogIsNeeded())
 		log_newpage(&index->rd_smgr->smgr_rnode.node, INIT_FORKNUM,
 					BTREE_METAPAGE, metapage, false);
diff --git a/src/backend/access/nbtree/nbtsort.c b/src/backend/access/nbtree/nbtsort.c
index f95f67a..ea7a45d 100644
--- a/src/backend/access/nbtree/nbtsort.c
+++ b/src/backend/access/nbtree/nbtsort.c
@@ -315,7 +315,7 @@ _bt_blwritepage(BTWriteState *wstate, Page page, BlockNumber blkno)
 	{
 		/* overwriting a block we zero-filled before */
 		smgrwrite(wstate->index->rd_smgr, MAIN_FORKNUM, blkno,
-				  (char *) page, true);
+				  (char *) page, true, false, NULL);
 	}
 
 	pfree(page);
diff --git a/src/backend/access/spgist/spginsert.c b/src/backend/access/spgist/spginsert.c
index bceee8d..b700efb 100644
--- a/src/backend/access/spgist/spginsert.c
+++ b/src/backend/access/spgist/spginsert.c
@@ -170,7 +170,7 @@ spgbuildempty(PG_FUNCTION_ARGS)
 	/* Write the page.  If archiving/streaming, XLOG it. */
 	PageSetChecksumInplace(page, SPGIST_METAPAGE_BLKNO);
 	smgrwrite(index->rd_smgr, INIT_FORKNUM, SPGIST_METAPAGE_BLKNO,
-			  (char *) page, true);
+			  (char *) page, true, false, NULL);
 	if (XLogIsNeeded())
 		log_newpage(&index->rd_smgr->smgr_rnode.node, INIT_FORKNUM,
 					SPGIST_METAPAGE_BLKNO, page, false);
@@ -180,7 +180,7 @@ spgbuildempty(PG_FUNCTION_ARGS)
 
 	PageSetChecksumInplace(page, SPGIST_ROOT_BLKNO);
 	smgrwrite(index->rd_smgr, INIT_FORKNUM, SPGIST_ROOT_BLKNO,
-			  (char *) page, true);
+			  (char *) page, true, false, NULL);
 	if (XLogIsNeeded())
 		log_newpage(&index->rd_smgr->smgr_rnode.node, INIT_FORKNUM,
 					SPGIST_ROOT_BLKNO, page, true);
@@ -190,7 +190,7 @@ spgbuildempty(PG_FUNCTION_ARGS)
 
 	PageSetChecksumInplace(page, SPGIST_NULL_BLKNO);
 	smgrwrite(index->rd_smgr, INIT_FORKNUM, SPGIST_NULL_BLKNO,
-			  (char *) page, true);
+			  (char *) page, true, false, NULL);
 	if (XLogIsNeeded())
 		log_newpage(&index->rd_smgr->smgr_rnode.node, INIT_FORKNUM,
 					SPGIST_NULL_BLKNO, page, true);
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index 0dce6a8..d962c3a 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -663,7 +663,7 @@ ImmediateCheckpointRequested(void)
  * fraction between 0.0 meaning none, and 1.0 meaning all done.
  */
 void
-CheckpointWriteDelay(int flags, double progress)
+CheckpointWriteDelay(int flags, double progress, FileFlushContext * context)
 {
 	static int	absorb_counter = WRITES_PER_ABSORB;
 
@@ -693,6 +693,14 @@ CheckpointWriteDelay(int flags, double progress)
 
 		CheckArchiveTimeout();
 
+		/* Before sleeping, sync written blocks
+		 */
+		if (checkpoint_flush_to_disk && context->ncalls != 0)
+		{
+			PerformFileFlush(context);
+			ResetFileFlushContext(context);
+		}
+
 		/*
 		 * Report interim activity statistics to the stats collector.
 		 */
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index cc973b5..b341bf7 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -80,6 +80,10 @@ bool		zero_damaged_pages = false;
 int			bgwriter_lru_maxpages = 100;
 double		bgwriter_lru_multiplier = 2.0;
 bool		track_io_timing = false;
+/* hint to move writes to high priority */
+bool		checkpoint_flush_to_disk = false;
+bool		bgwriter_flush_to_disk = false;
+int			checkpoint_sort_size = 1024 * 1024;
 
 /*
  * How many buffers PrefetchBuffer callers should try to stay ahead of their
@@ -396,7 +400,8 @@ static bool PinBuffer(volatile BufferDesc *buf, BufferAccessStrategy strategy);
 static void PinBuffer_Locked(volatile BufferDesc *buf);
 static void UnpinBuffer(volatile BufferDesc *buf, bool fixOwner);
 static void BufferSync(int flags);
-static int	SyncOneBuffer(int buf_id, bool skip_recently_used);
+static int	SyncOneBuffer(int buf_id, bool skip_recently_used,
+						  bool flush_to_disk, FileFlushContext *context);
 static void WaitIO(volatile BufferDesc *buf);
 static bool StartBufferIO(volatile BufferDesc *buf, bool forInput);
 static void TerminateBufferIO(volatile BufferDesc *buf, bool clear_dirty,
@@ -409,7 +414,8 @@ static volatile BufferDesc *BufferAlloc(SMgrRelation smgr,
 			BlockNumber blockNum,
 			BufferAccessStrategy strategy,
 			bool *foundPtr);
-static void FlushBuffer(volatile BufferDesc *buf, SMgrRelation reln);
+static void FlushBuffer(volatile BufferDesc *buf, SMgrRelation reln,
+						bool flush_to_disk, FileFlushContext *context);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -1018,7 +1024,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 												smgr->smgr_rnode.node.dbNode,
 											  smgr->smgr_rnode.node.relNode);
 
-				FlushBuffer(buf, NULL);
+				FlushBuffer(buf, NULL, false, NULL);
 				LWLockRelease(buf->content_lock);
 
 				TRACE_POSTGRESQL_BUFFER_WRITE_DIRTY_DONE(forkNum, blockNum,
@@ -1561,6 +1567,53 @@ UnpinBuffer(volatile BufferDesc *buf, bool fixOwner)
 	}
 }
 
+/* Array of buffer ids of all buffers to checkpoint.
+ */
+static int * CheckpointBufferIds = NULL;
+
+/* Compare checkpoint buffers
+ */
+static int bufcmp(const int * pa, const int * pb)
+{
+	BufferDesc
+		*a = GetBufferDescriptor(*pa),
+		*b = GetBufferDescriptor(*pb);
+
+	/* tag: rnode, forkNum (different files), blockNum
+	 * rnode: { spcNode, dbNode (ignore: this is a directory), relNode }
+	 * spcNode: table space oid, not that there are at least two
+	 * (pg_global and pg_default).
+	 */
+	/* first, compare table space (hmmm...) */
+	if (a->tag.rnode.spcNode < b->tag.rnode.spcNode)
+		return -1;
+	else if (a->tag.rnode.spcNode > b->tag.rnode.spcNode)
+		return 1;
+	/* same table space, compare relation */
+	else if (a->tag.rnode.relNode < b->tag.rnode.relNode)
+		return -1;
+	else if (a->tag.rnode.relNode > b->tag.rnode.relNode)
+		return 1;
+	/* same relation, compare fork */
+	else if (a->tag.forkNum < b->tag.forkNum)
+		return -1;
+	else if (a->tag.forkNum > b->tag.forkNum)
+		return 1;
+	/* same relation/fork, so same file, try block number */
+	else if (a->tag.blockNum < b->tag.blockNum)
+		return -1;
+	else /* should not be the same block... */
+		return 1;
+}
+
+static void AllocateCheckpointBufferIds(void)
+{
+	/* safe worst case allocation, all buffers belong to the checkpoint...
+	 * that is pretty unlikely.
+	 */
+	CheckpointBufferIds = (int *) palloc(sizeof(int) * NBuffers);
+}
+
 /*
  * BufferSync -- Write out all dirty buffers in the pool.
  *
@@ -1575,10 +1628,17 @@ static void
 BufferSync(int flags)
 {
 	int			buf_id;
-	int			num_to_scan;
 	int			num_to_write;
 	int			num_written;
+	int			i;
 	int			mask = BM_DIRTY;
+	FileFlushContext context;
+
+	ResetFileFlushContext(&context);
+
+	// lazy, to be really called by CheckpointerMain
+	if (CheckpointBufferIds == NULL)
+		AllocateCheckpointBufferIds();
 
 	/* Make sure we can handle the pin inside SyncOneBuffer */
 	ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
@@ -1622,6 +1682,7 @@ BufferSync(int flags)
 		if ((bufHdr->flags & mask) == mask)
 		{
 			bufHdr->flags |= BM_CHECKPOINT_NEEDED;
+			CheckpointBufferIds[num_to_write] = buf_id;
 			num_to_write++;
 		}
 
@@ -1633,19 +1694,47 @@ BufferSync(int flags)
 
 	TRACE_POSTGRESQL_BUFFER_SYNC_START(NBuffers, num_to_write);
 
+	/* Sort buffer ids by chunks to help find sequential writes.
+	 * Note: buffers are not locked in anyway, but that does not matter,
+	 * this sorting is really advisory, if some buffer changes status during
+	 * this pass it will be filtered out later.  The only necessary property
+	 * is that marked buffers do not move elsewhere.  Also, qsort implementation
+	 * should be resilient to occasional contradictions (cmp(a,b) != -cmp(b,a))
+	 * because of these possible concurrent changes.
+	 */
+	if (checkpoint_sort_size > 1)
+	{
+		int i;
+
+		// debug...
+		ereport(WARNING,
+				(errcode(ERRCODE_WARNING),
+				 errmsg("Checkpoint: sorting %d buffers (%d chunks of size %d)",
+						num_to_write,
+						(checkpoint_sort_size+num_to_write-1) /
+						  checkpoint_sort_size,
+						checkpoint_sort_size)));
+
+		// hmmm... should it equalize on the number of chunks?
+		for (i = 0; i < num_to_write; i += checkpoint_sort_size)
+			qsort(CheckpointBufferIds + i,
+				  (i + checkpoint_sort_size <= num_to_write ?
+				   checkpoint_sort_size : num_to_write - i),
+				  sizeof(int),
+				  (int(*)(const void *, const void *)) bufcmp);
+	}
+
 	/*
-	 * Loop over all buffers again, and write the ones (still) marked with
-	 * BM_CHECKPOINT_NEEDED.  In this loop, we start at the clock sweep point
-	 * since we might as well dump soon-to-be-recycled buffers first.
+	 * Loop over buffers again, and write the ones (still) marked with
+	 * BM_CHECKPOINT_NEEDED.
 	 *
-	 * Note that we don't read the buffer alloc count here --- that should be
-	 * left untouched till the next BgBufferSync() call.
+	 * TODO: do something clever about table spaces...
+	 * scan them in parallel with multiple threads?
 	 */
-	buf_id = StrategySyncStart(NULL, NULL);
-	num_to_scan = NBuffers;
 	num_written = 0;
-	while (num_to_scan-- > 0)
+	for (i = 0; i < num_to_write; i++)
 	{
+		int buf_id = CheckpointBufferIds[i];
 		volatile BufferDesc *bufHdr = GetBufferDescriptor(buf_id);
 
 		/*
@@ -1662,38 +1751,31 @@ BufferSync(int flags)
 		 */
 		if (bufHdr->flags & BM_CHECKPOINT_NEEDED)
 		{
-			if (SyncOneBuffer(buf_id, false) & BUF_WRITTEN)
+			if (SyncOneBuffer(buf_id, false, checkpoint_flush_to_disk, &context) & BUF_WRITTEN)
 			{
 				TRACE_POSTGRESQL_BUFFER_SYNC_WRITTEN(buf_id);
 				BgWriterStats.m_buf_written_checkpoints++;
 				num_written++;
 
 				/*
-				 * We know there are at most num_to_write buffers with
-				 * BM_CHECKPOINT_NEEDED set; so we can stop scanning if
-				 * num_written reaches num_to_write.
-				 *
-				 * Note that num_written doesn't include buffers written by
-				 * other backends, or by the bgwriter cleaning scan. That
-				 * means that the estimate of how much progress we've made is
-				 * conservative, and also that this test will often fail to
-				 * trigger.  But it seems worth making anyway.
-				 */
-				if (num_written >= num_to_write)
-					break;
-
-				/*
 				 * Sleep to throttle our I/O rate.
 				 */
-				CheckpointWriteDelay(flags, (double) num_written / num_to_write);
+				CheckpointWriteDelay(flags, (double) num_written / num_to_write, &context);
 			}
 		}
-
-		if (++buf_id >= NBuffers)
-			buf_id = 0;
 	}
 
 	/*
+	 * Loop over all buffers again, and write the ones (still) marked with
+	 * BM_CHECKPOINT_NEEDED.  In this loop, we start at the clock sweep point
+	 * since we might as well dump soon-to-be-recycled buffers first.
+	 *
+	 * Note that we don't read the buffer alloc count here --- that should be
+	 * left untouched till the next BgBufferSync() call.
+	 */
+	/* OLD CODE REMOVED */
+
+	/*
 	 * Update checkpoint statistics. As noted above, this doesn't include
 	 * buffers written by other backends or bgwriter scan.
 	 */
@@ -1757,6 +1839,8 @@ BgBufferSync(void)
 	long		new_strategy_delta;
 	uint32		new_recent_alloc;
 
+	FileFlushContext context;
+
 	/*
 	 * Find out where the freelist clock sweep currently is, and how many
 	 * buffer allocations have happened since our last call.
@@ -1935,11 +2019,13 @@ BgBufferSync(void)
 	num_to_scan = bufs_to_lap;
 	num_written = 0;
 	reusable_buffers = reusable_buffers_est;
+	ResetFileFlushContext(&context);
 
 	/* Execute the LRU scan */
 	while (num_to_scan > 0 && reusable_buffers < upcoming_alloc_est)
 	{
-		int			buffer_state = SyncOneBuffer(next_to_clean, true);
+		int	buffer_state =
+			SyncOneBuffer(next_to_clean, true, bgwriter_flush_to_disk, &context);
 
 		if (++next_to_clean >= NBuffers)
 		{
@@ -1963,6 +2049,9 @@ BgBufferSync(void)
 
 	BgWriterStats.m_buf_written_clean += num_written;
 
+	PerformFileFlush(&context);
+	ResetFileFlushContext(&context);
+
 #ifdef BGW_DEBUG
 	elog(DEBUG1, "bgwriter: recent_alloc=%u smoothed=%.2f delta=%ld ahead=%d density=%.2f reusable_est=%d upcoming_est=%d scanned=%d wrote=%d reusable=%d",
 		 recent_alloc, smoothed_alloc, strategy_delta, bufs_ahead,
@@ -2016,7 +2105,8 @@ BgBufferSync(void)
  * Note: caller must have done ResourceOwnerEnlargeBuffers.
  */
 static int
-SyncOneBuffer(int buf_id, bool skip_recently_used)
+SyncOneBuffer(int buf_id, bool skip_recently_used, bool flush_to_disk,
+	FileFlushContext * context)
 {
 	volatile BufferDesc *bufHdr = GetBufferDescriptor(buf_id);
 	int			result = 0;
@@ -2057,7 +2147,7 @@ SyncOneBuffer(int buf_id, bool skip_recently_used)
 	PinBuffer_Locked(bufHdr);
 	LWLockAcquire(bufHdr->content_lock, LW_SHARED);
 
-	FlushBuffer(bufHdr, NULL);
+	FlushBuffer(bufHdr, NULL, flush_to_disk, context);
 
 	LWLockRelease(bufHdr->content_lock);
 	UnpinBuffer(bufHdr, true);
@@ -2319,9 +2409,13 @@ BufferGetTag(Buffer buffer, RelFileNode *rnode, ForkNumber *forknum,
  *
  * If the caller has an smgr reference for the buffer's relation, pass it
  * as the second parameter.  If not, pass NULL.
+ *
+ * The third parameter tries to hint the OS that a high priority write is meant,
+ * possibly because io-throttling is already managed elsewhere.
  */
 static void
-FlushBuffer(volatile BufferDesc *buf, SMgrRelation reln)
+FlushBuffer(volatile BufferDesc *buf, SMgrRelation reln, bool flush_to_disk,
+	FileFlushContext * context)
 {
 	XLogRecPtr	recptr;
 	ErrorContextCallback errcallback;
@@ -2410,7 +2504,9 @@ FlushBuffer(volatile BufferDesc *buf, SMgrRelation reln)
 			  buf->tag.forkNum,
 			  buf->tag.blockNum,
 			  bufToWrite,
-			  false);
+			  false,
+			  flush_to_disk,
+			  context);
 
 	if (track_io_timing)
 	{
@@ -2830,7 +2926,9 @@ FlushRelationBuffers(Relation rel)
 						  bufHdr->tag.forkNum,
 						  bufHdr->tag.blockNum,
 						  localpage,
-						  false);
+						  false,
+						  false,
+						  NULL);
 
 				bufHdr->flags &= ~(BM_DIRTY | BM_JUST_DIRTIED);
 
@@ -2864,7 +2962,7 @@ FlushRelationBuffers(Relation rel)
 		{
 			PinBuffer_Locked(bufHdr);
 			LWLockAcquire(bufHdr->content_lock, LW_SHARED);
-			FlushBuffer(bufHdr, rel->rd_smgr);
+			FlushBuffer(bufHdr, rel->rd_smgr, false, NULL);
 			LWLockRelease(bufHdr->content_lock);
 			UnpinBuffer(bufHdr, true);
 		}
@@ -2916,7 +3014,7 @@ FlushDatabaseBuffers(Oid dbid)
 		{
 			PinBuffer_Locked(bufHdr);
 			LWLockAcquire(bufHdr->content_lock, LW_SHARED);
-			FlushBuffer(bufHdr, NULL);
+			FlushBuffer(bufHdr, NULL, false, NULL);
 			LWLockRelease(bufHdr->content_lock);
 			UnpinBuffer(bufHdr, true);
 		}
diff --git a/src/backend/storage/buffer/localbuf.c b/src/backend/storage/buffer/localbuf.c
index 3144afe..114a0a6 100644
--- a/src/backend/storage/buffer/localbuf.c
+++ b/src/backend/storage/buffer/localbuf.c
@@ -208,7 +208,9 @@ LocalBufferAlloc(SMgrRelation smgr, ForkNumber forkNum, BlockNumber blockNum,
 				  bufHdr->tag.forkNum,
 				  bufHdr->tag.blockNum,
 				  localpage,
-				  false);
+				  false,
+				  false,
+				  NULL);
 
 		/* Mark not-dirty now in case we error out below */
 		bufHdr->flags &= ~BM_DIRTY;
diff --git a/src/backend/storage/file/buffile.c b/src/backend/storage/file/buffile.c
index ea4d689..fb3b383 100644
--- a/src/backend/storage/file/buffile.c
+++ b/src/backend/storage/file/buffile.c
@@ -317,7 +317,7 @@ BufFileDumpBuffer(BufFile *file)
 				return;			/* seek failed, give up */
 			file->offsets[file->curFile] = file->curOffset;
 		}
-		bytestowrite = FileWrite(thisfile, file->buffer + wpos, bytestowrite);
+		bytestowrite = FileWrite(thisfile, file->buffer + wpos, bytestowrite, false, NULL);
 		if (bytestowrite <= 0)
 			return;				/* failed to write */
 		file->offsets[file->curFile] += bytestowrite;
diff --git a/src/backend/storage/file/fd.c b/src/backend/storage/file/fd.c
index 1ba4946..bb28aec 100644
--- a/src/backend/storage/file/fd.c
+++ b/src/backend/storage/file/fd.c
@@ -1344,8 +1344,95 @@ retry:
 	return returnCode;
 }
 
+#if defined(HAVE_SYNC_FILE_RANGE) || defined(HAVE_POSIX_FADVISE)
+
+void
+ResetFileFlushContext(FileFlushContext * context)
+{
+	context->fd = 0;
+	context->ncalls = 0;
+	context->offset = 0;
+	context->nbytes = 0;
+	context->filename = NULL;
+}
+
+void
+PerformFileFlush(FileFlushContext * context)
+{
+	if (context->ncalls != 0)
+	{
+		int rc;
+
+#if defined(HAVE_SYNC_FILE_RANGE)
+
+		/* Linux: tell the memory manager to move these blocks to io so
+		 * that they are considered for being actually written to disk.
+		 */
+		rc = sync_file_range(context->fd, context->offset, context->nbytes,
+							 SYNC_FILE_RANGE_WRITE);
+
+#elif defined(HAVE_POSIX_FADVISE)
+
+		/* Others: say that data should not be kept in memory...
+		 * This is not exactly what we want to say, because we want to write
+		 * the data for durability but we may need it later nevertheless.
+		 * It seems that Linux would free the memory *if* the data has
+		 * already been written do disk, else it is ignored.
+		 * For FreeBSD this may have the desired effect of moving the
+		 * data to the io layer.
+		 */
+		rc = posix_fadvise(context->fd, context->offset, context->nbytes,
+						   POSIX_FADV_DONTNEED);
+
+#endif
+
+		if (rc < 0)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not flush block " INT64_FORMAT
+							" on " INT64_FORMAT " blocks in file \"%s\": %m",
+							context->offset / BLCKSZ,
+							context->nbytes / BLCKSZ,
+							context->filename)));
+	}
+}
+
+void
+FileAsynchronousFlush(FileFlushContext * context,
+					  int fd, off_t offset, off_t nbytes, char * filename)
+{
+	if (context->ncalls != 0 && context->fd == fd)
+	{
+		/* Same file: merge current flush with previous ones */
+		off_t new_offset = offset < context->offset? offset: context->offset;
+
+		context->nbytes =
+			(context->offset + context->nbytes > offset + nbytes ?
+			 context->offset + context->nbytes : offset + nbytes) -
+			new_offset;
+		context->offset = new_offset;
+		context->ncalls ++;
+	}
+	else
+	{
+		/* file has changed; actually flush previous file before restarting
+		 * to accumulate flushes
+		 */
+		PerformFileFlush(context);
+
+		context->fd = fd;
+		context->ncalls = 1;
+		context->offset = offset;
+		context->nbytes = nbytes;
+		context->filename = filename;
+	}
+}
+
+#endif /* HAVE_SYNC_FILE_RANGE || HAVE_POSIX_FADVISE */
+
 int
-FileWrite(File file, char *buffer, int amount)
+FileWrite(File file, char *buffer, int amount, bool flush_to_disk,
+		  FileFlushContext * context)
 {
 	int			returnCode;
 
@@ -1395,6 +1482,28 @@ retry:
 
 	if (returnCode >= 0)
 	{
+
+#if defined(HAVE_SYNC_FILE_RANGE) || defined(HAVE_POSIX_FADVISE)
+
+		/*
+		 * Calling "write" tells the OS that pg wants to write some page to disk,
+		 * however when it is really done is chosen by the OS.
+		 * Depending on other disk activities this may be delayed significantly,
+		 * maybe up to an "fsync" call, which could induce an IO write surge.
+		 * When checkpointing pg is doing its own throttling and the result
+		 * should really be written to disk with high priority, so as to meet
+		 * the completion target.
+		 * This call hints that such write have a higher priority.
+		 */
+		if (flush_to_disk && returnCode == amount && errno == 0)
+		{
+			FileAsynchronousFlush(context,
+								  VfdCache[file].fd, VfdCache[file].seekPos,
+								  amount, VfdCache[file].fileName);
+		}
+
+#endif /* HAVE_SYNC_FILE_RANGE || HAVE_POSIX_FADVISE */
+
 		VfdCache[file].seekPos += returnCode;
 
 		/* maintain fileSize and temporary_files_size if it's a temp file */
diff --git a/src/backend/storage/smgr/md.c b/src/backend/storage/smgr/md.c
index 42a43bb..dbf057f 100644
--- a/src/backend/storage/smgr/md.c
+++ b/src/backend/storage/smgr/md.c
@@ -531,7 +531,7 @@ mdextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 				 errmsg("could not seek to block %u in file \"%s\": %m",
 						blocknum, FilePathName(v->mdfd_vfd))));
 
-	if ((nbytes = FileWrite(v->mdfd_vfd, buffer, BLCKSZ)) != BLCKSZ)
+	if ((nbytes = FileWrite(v->mdfd_vfd, buffer, BLCKSZ, false, NULL)) != BLCKSZ)
 	{
 		if (nbytes < 0)
 			ereport(ERROR,
@@ -738,7 +738,8 @@ mdread(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  */
 void
 mdwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
-		char *buffer, bool skipFsync)
+		char *buffer, bool skipFsync, bool flush_to_disk,
+		FileFlushContext * context)
 {
 	off_t		seekpos;
 	int			nbytes;
@@ -767,7 +768,7 @@ mdwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 				 errmsg("could not seek to block %u in file \"%s\": %m",
 						blocknum, FilePathName(v->mdfd_vfd))));
 
-	nbytes = FileWrite(v->mdfd_vfd, buffer, BLCKSZ);
+	nbytes = FileWrite(v->mdfd_vfd, buffer, BLCKSZ, flush_to_disk, context);
 
 	TRACE_POSTGRESQL_SMGR_MD_WRITE_DONE(forknum, blocknum,
 										reln->smgr_rnode.node.spcNode,
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 244b4ea..2db3cd3 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -52,7 +52,8 @@ typedef struct f_smgr
 	void		(*smgr_read) (SMgrRelation reln, ForkNumber forknum,
 										  BlockNumber blocknum, char *buffer);
 	void		(*smgr_write) (SMgrRelation reln, ForkNumber forknum,
-						 BlockNumber blocknum, char *buffer, bool skipFsync);
+						   BlockNumber blocknum, char *buffer, bool skipFsync,
+							   bool flush_to_disk, FileFlushContext *context);
 	BlockNumber (*smgr_nblocks) (SMgrRelation reln, ForkNumber forknum);
 	void		(*smgr_truncate) (SMgrRelation reln, ForkNumber forknum,
 											  BlockNumber nblocks);
@@ -643,10 +644,11 @@ smgrread(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  */
 void
 smgrwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
-		  char *buffer, bool skipFsync)
+		  char *buffer, bool skipFsync, bool flush_to_disk,
+		  FileFlushContext * context)
 {
 	(*(smgrsw[reln->smgr_which].smgr_write)) (reln, forknum, blocknum,
-											  buffer, skipFsync);
+									buffer, skipFsync, flush_to_disk, context);
 }
 
 /*
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index b3c9f14..c8706ba 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -158,6 +158,7 @@ static bool check_bonjour(bool *newval, void **extra, GucSource source);
 static bool check_ssl(bool *newval, void **extra, GucSource source);
 static bool check_stage_log_stats(bool *newval, void **extra, GucSource source);
 static bool check_log_stats(bool *newval, void **extra, GucSource source);
+static bool check_flush_to_disk(bool *newval, void **extra, GucSource source);
 static bool check_canonical_path(char **newval, void **extra, GucSource source);
 static bool check_timezone_abbreviations(char **newval, void **extra, GucSource source);
 static void assign_timezone_abbreviations(const char *newval, void *extra);
@@ -569,6 +570,8 @@ const char *const config_group_names[] =
 	gettext_noop("Write-Ahead Log / Checkpoints"),
 	/* WAL_ARCHIVING */
 	gettext_noop("Write-Ahead Log / Archiving"),
+	/* BGWRITER */
+	gettext_noop("Background Writer"),
 	/* REPLICATION */
 	gettext_noop("Replication"),
 	/* REPLICATION_SENDING */
@@ -1009,6 +1012,27 @@ static struct config_bool ConfigureNamesBool[] =
 		false,
 		NULL, NULL, NULL
 	},
+
+	{
+		{"checkpoint_flush_to_disk", PGC_SIGHUP, WAL_CHECKPOINTS,
+			gettext_noop("Hint that checkpoint's writes are high priority."),
+			NULL
+		},
+		&checkpoint_flush_to_disk,
+		false,
+		check_flush_to_disk, NULL, NULL
+	},
+
+	{
+		{"bgwriter_flush_to_disk", PGC_SIGHUP, BGWRITER,
+			gettext_noop("Hint that bgwriter's writes are high priority."),
+			NULL
+		},
+		&bgwriter_flush_to_disk,
+		false,
+		check_flush_to_disk, NULL, NULL
+	},
+
 	{
 		{"log_connections", PGC_SU_BACKEND, LOGGING_WHAT,
 			gettext_noop("Logs each successful connection."),
@@ -2205,6 +2229,16 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
+		{"checkpoint_sort_size", PGC_SIGHUP, WAL_CHECKPOINTS,
+		 gettext_noop("Sort chunks of pages before writing them, ...."),
+		 NULL
+		},
+		&checkpoint_sort_size,
+		1024*1024, 0, INT_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"wal_buffers", PGC_POSTMASTER, WAL_SETTINGS,
 			gettext_noop("Sets the number of disk-page buffers in shared memory for WAL."),
 			NULL,
@@ -9761,6 +9795,22 @@ check_log_stats(bool *newval, void **extra, GucSource source)
 }
 
 static bool
+check_flush_to_disk(bool *newval, void **extra, GucSource source)
+{
+/* This test must be consistent with the one in FileWrite (storage/file/fd.c)
+ */
+#if ! (defined(HAVE_SYNC_FILE_RANGE) || defined(HAVE_POSIX_FADVISE))
+	/* just warn if it has no effect */
+	ereport(WARNING,
+			(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+			 errmsg("Setting \"checkpoint_flush_to_disk\" or "
+					"\"bgwriter_flush_to_disk\" has no effect "
+					"on this platform.")));
+#endif /* HAVE_SYNC_FILE_RANGE */
+	return true;
+}
+
+static bool
 check_canonical_path(char **newval, void **extra, GucSource source)
 {
 	/*
diff --git a/src/include/postmaster/bgwriter.h b/src/include/postmaster/bgwriter.h
index a49c208..c483832 100644
--- a/src/include/postmaster/bgwriter.h
+++ b/src/include/postmaster/bgwriter.h
@@ -29,7 +29,10 @@ extern void BackgroundWriterMain(void) pg_attribute_noreturn();
 extern void CheckpointerMain(void) pg_attribute_noreturn();
 
 extern void RequestCheckpoint(int flags);
-extern void CheckpointWriteDelay(int flags, double progress);
+struct FileFlushContext;
+typedef struct FileFlushContext FileFlushContext;
+extern void CheckpointWriteDelay(int flags, double progress,
+								 FileFlushContext * context);
 
 extern bool ForwardFsyncRequest(RelFileNode rnode, ForkNumber forknum,
 					BlockNumber segno);
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ec0a254..2bf0cf8 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -54,6 +54,9 @@ extern int	bgwriter_lru_maxpages;
 extern double bgwriter_lru_multiplier;
 extern bool track_io_timing;
 extern int	target_prefetch_pages;
+extern bool checkpoint_flush_to_disk;
+extern bool bgwriter_flush_to_disk;
+extern int  checkpoint_sort_size;
 
 /* in buf_init.c */
 extern PGDLLIMPORT char *BufferBlocks;
diff --git a/src/include/storage/fd.h b/src/include/storage/fd.h
index 7eabe09..150c283 100644
--- a/src/include/storage/fd.h
+++ b/src/include/storage/fd.h
@@ -59,6 +59,22 @@ extern int	max_files_per_process;
  */
 extern int	max_safe_fds;
 
+/* FileFlushContext:
+ * This structure is used to accumulate several flush requests on a file
+ * into a larger flush request.
+ * - fd: file descriptor of the file
+ * - ncalls: number of flushes merged together
+ * - offset: starting offset (minimum of all offset)
+ * - nbytes: size (minimum extent to cover all flushed data
+ * - filename: filename of fd for error messages
+ */
+typedef struct FileFlushContext{
+	int fd;
+	int ncalls;
+	off_t offset;
+	off_t nbytes;
+	char * filename;
+} FileFlushContext;
 
 /*
  * prototypes for functions in fd.c
@@ -70,7 +86,12 @@ extern File OpenTemporaryFile(bool interXact);
 extern void FileClose(File file);
 extern int	FilePrefetch(File file, off_t offset, int amount);
 extern int	FileRead(File file, char *buffer, int amount);
-extern int	FileWrite(File file, char *buffer, int amount);
+extern void ResetFileFlushContext(FileFlushContext * context);
+extern void PerformFileFlush(FileFlushContext * context);
+extern void FileAsynchronousFlush(FileFlushContext * context,
+				int fd, off_t offset, off_t nbytes, char * filename);
+extern int	FileWrite(File file, char *buffer, int amount, bool flush_to_disk,
+	FileFlushContext * context);
 extern int	FileSync(File file);
 extern off_t FileSeek(File file, off_t offset, int whence);
 extern int	FileTruncate(File file, off_t offset);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index 69a624f..da0e929 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -94,8 +94,11 @@ extern void smgrprefetch(SMgrRelation reln, ForkNumber forknum,
 			 BlockNumber blocknum);
 extern void smgrread(SMgrRelation reln, ForkNumber forknum,
 		 BlockNumber blocknum, char *buffer);
+struct FileFlushContext;
+typedef struct FileFlushContext FileFlushContext;
 extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
-		  BlockNumber blocknum, char *buffer, bool skipFsync);
+					  BlockNumber blocknum, char *buffer, bool skipFsync,
+					  bool flush_to_disk, FileFlushContext * context);
 extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber forknum,
 			 BlockNumber nblocks);
@@ -120,8 +123,9 @@ extern void mdprefetch(SMgrRelation reln, ForkNumber forknum,
 		   BlockNumber blocknum);
 extern void mdread(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	   char *buffer);
-extern void mdwrite(SMgrRelation reln, ForkNumber forknum,
-		BlockNumber blocknum, char *buffer, bool skipFsync);
+extern void mdwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
+					char *buffer, bool skipFsync, bool flush_to_disk,
+					FileFlushContext * context);
 extern BlockNumber mdnblocks(SMgrRelation reln, ForkNumber forknum);
 extern void mdtruncate(SMgrRelation reln, ForkNumber forknum,
 		   BlockNumber nblocks);
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index 7a58ddb..b69af2d 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -68,6 +68,7 @@ enum config_group
 	WAL_SETTINGS,
 	WAL_CHECKPOINTS,
 	WAL_ARCHIVING,
+	BGWRITER,
 	REPLICATION,
 	REPLICATION_SENDING,
 	REPLICATION_MASTER,
