diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 967de73596..56eb55bab7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -7450,7 +7450,7 @@ COPY postgres_log FROM '/full/path/to/logfile.csv' WITH csv;
       <listitem>
        <para>
         Enables timing of database I/O calls.  This parameter is off by
-        default, because it will repeatedly query the operating system for
+        default, as it will repeatedly query the operating system for
         the current time, which may cause significant overhead on some
         platforms.  You can use the <xref linkend="pgtesttiming"/> tool to
         measure the overhead of timing on your system.
@@ -7464,6 +7464,27 @@ COPY postgres_log FROM '/full/path/to/logfile.csv' WITH csv;
       </listitem>
      </varlistentry>
 
+     <varlistentry id="guc-track-wal-io-timing" xreflabel="track_wal_io_timing">
+      <term><varname>track_wal_io_timing</varname> (<type>boolean</type>)
+      <indexterm>
+       <primary><varname>track_wal_io_timing</varname> configuration parameter</primary>
+      </indexterm>
+      </term>
+      <listitem>
+       <para>
+        Enables timing of WAL I/O calls. This parameter is off by default,
+        as it will repeatedly query the operating system for
+        the current time, which may cause significant overhead on some
+        platforms.  You can use the <xref linkend="pgtesttiming"/> tool to
+        measure the overhead of timing on your system.
+        I/O timing information is
+        displayed in <link linkend="monitoring-pg-stat-wal-view">
+        <structname>pg_stat_wal</structname></link>.  Only superusers can
+        change this setting.
+       </para>
+      </listitem>
+     </varlistentry>
+
      <varlistentry id="guc-track-functions" xreflabel="track_functions">
       <term><varname>track_functions</varname> (<type>enum</type>)
       <indexterm>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3513e127b7..1520cef505 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -3487,6 +3487,62 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       </para></entry>
      </row>
 
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>wal_write</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of times WAL buffers were written out to disk via
+       <function>XLogWrite</function>, which is invoked during an
+       <function>XLogFlush</function> request (see <xref linkend="wal-configuration"/>)
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>wal_sync</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of times WAL files were synced to disk via 
+       <function>issue_xlog_fsync</function>, which is invoked during an 
+       <function>XLogFlush</function> request (see <xref linkend="wal-configuration"/>)
+       while <xref linkend="guc-wal-sync-method"/> was set to one of the 
+       "sync at commit" options (i.e., <literal>fdatasync</literal>, 
+       <literal>fsync</literal>, or <literal>fsync_writethrough</literal>).
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>wal_write_time</structfield> <type>double precision</type>
+      </para>
+      <para>
+       Total amount of time spent writing WAL buffers were written out to disk via
+       <function>XLogWrite</function>, which is invoked during an
+       <function>XLogFlush</function> request (see <xref linkend="wal-configuration"/>),
+       excluding sync time unless 
+       <xref linkend="guc-wal-sync-method"/> is either <literal>open_datasync</literal> or 
+       <literal>open_sync</literal>. Units are in milliseconds with microsecond resolution.
+       This is zero when <xref linkend="guc-track-wal-io-timing"/> is disabled.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>wal_sync_time</structfield> <type>double precision</type>
+      </para>
+      <para>
+       Total amount of time spent syncing WAL files to disk via
+       <function>issue_xlog_fsync</function>, which is invoked during an 
+       <function>XLogFlush</function> request (see <xref linkend="wal-configuration"/>)
+       while <xref linkend="guc-wal-sync-method"/> was set to one of the 
+       "sync at commit" options (i.e., <literal>fdatasync</literal>, 
+       <literal>fsync</literal>, or <literal>fsync_writethrough</literal>).
+       Units are in milliseconds with microsecond resolution.
+       This is zero when <xref linkend="guc-track-wal-io-timing"/> is disabled.
+      </para></entry>
+     </row>
+
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
        <structfield>stats_reset</structfield> <type>timestamp with time zone</type>
diff --git a/doc/src/sgml/wal.sgml b/doc/src/sgml/wal.sgml
index f75527f764..06e4b37012 100644
--- a/doc/src/sgml/wal.sgml
+++ b/doc/src/sgml/wal.sgml
@@ -663,7 +663,9 @@
    the <acronym>WAL</acronym> buffers in shared memory. If there is no
    space for the new record, <function>XLogInsertRecord</function> will have
    to write (move to kernel cache) a few filled <acronym>WAL</acronym>
-   buffers. This is undesirable because <function>XLogInsertRecord</function>
+   buffers (the tally of this event is reported in 
+   <literal>wal_buffers_full</literal> in <xref linkend="pg-stat-wal-view"/>).
+   This is undesirable because <function>XLogInsertRecord</function> 
    is used on every database low level modification (for example, row
    insertion) at a time when an exclusive lock is held on affected
    data pages, so the operation needs to be as fast as possible.  What
@@ -672,8 +674,12 @@
    time. Normally, <acronym>WAL</acronym> buffers should be written
    and flushed by an <function>XLogFlush</function> request, which is
    made, for the most part, at transaction commit time to ensure that
-   transaction records are flushed to permanent storage. On systems
-   with high log output, <function>XLogFlush</function> requests might
+   transaction records are flushed to permanent storage. 
+   <function>XLogFlush</function> calls <function>XLogWrite</function> to write 
+   and <function>issue_xlog_fsync</function> to flush them, which are counted as 
+   <literal>wal_write</literal> and <literal>wal_sync</literal> in 
+   <xref linkend="pg-stat-wal-view"/>. On systems with high log output, 
+   <function>XLogFlush</function> requests might
    not occur often enough to prevent <function>XLogInsertRecord</function>
    from having to do writes.  On such systems
    one should increase the number of <acronym>WAL</acronym> buffers by
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index fe56324439..24c3dd32f8 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -110,6 +110,7 @@ int			CommitDelay = 0;	/* precommit delay in microseconds */
 int			CommitSiblings = 5; /* # concurrent xacts needed to sleep */
 int			wal_retrieve_retry_interval = 5000;
 int			max_slot_wal_keep_size_mb = -1;
+bool		track_wal_io_timing = false;
 
 #ifdef WAL_DEBUG
 bool		XLOG_DEBUG = false;
@@ -2533,6 +2534,7 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible)
 			Size		nbytes;
 			Size		nleft;
 			int			written;
+			instr_time	start;
 
 			/* OK to write the page(s) */
 			from = XLogCtl->pages + startidx * (Size) XLOG_BLCKSZ;
@@ -2541,9 +2543,30 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible)
 			do
 			{
 				errno = 0;
+
+				/* Measure I/O timing to write WAL data */
+				if (track_wal_io_timing)
+					INSTR_TIME_SET_CURRENT(start);
+
 				pgstat_report_wait_start(WAIT_EVENT_WAL_WRITE);
 				written = pg_pwrite(openLogFile, from, nleft, startoffset);
 				pgstat_report_wait_end();
+
+				/*
+				 * Increment the I/O timing and the number of times WAL data
+				 * were written out to disk.
+				 */
+				if (track_wal_io_timing)
+				{
+					instr_time	duration;
+
+					INSTR_TIME_SET_CURRENT(duration);
+					INSTR_TIME_SUBTRACT(duration, start);
+					WalStats.m_wal_write_time += INSTR_TIME_GET_MICROSEC(duration);
+				}
+
+				WalStats.m_wal_write++;
+
 				if (written <= 0)
 				{
 					char		xlogfname[MAXFNAMELEN];
@@ -10526,6 +10549,20 @@ void
 issue_xlog_fsync(int fd, XLogSegNo segno)
 {
 	char	   *msg = NULL;
+	instr_time	start;
+
+	/*
+	 * Quick exit if fsync is disabled or write() has already synced the WAL
+	 * file.
+	 */
+	if (!enableFsync ||
+		sync_method == SYNC_METHOD_OPEN ||
+		sync_method == SYNC_METHOD_OPEN_DSYNC)
+		return;
+
+	/* Measure I/O timing to sync the WAL file */
+	if (track_wal_io_timing)
+		INSTR_TIME_SET_CURRENT(start);
 
 	pgstat_report_wait_start(WAIT_EVENT_WAL_SYNC);
 	switch (sync_method)
@@ -10548,7 +10585,8 @@ issue_xlog_fsync(int fd, XLogSegNo segno)
 #endif
 		case SYNC_METHOD_OPEN:
 		case SYNC_METHOD_OPEN_DSYNC:
-			/* write synced it already */
+			/* not reachable */
+			Assert(false);
 			break;
 		default:
 			elog(PANIC, "unrecognized wal_sync_method: %d", sync_method);
@@ -10570,6 +10608,20 @@ issue_xlog_fsync(int fd, XLogSegNo segno)
 	}
 
 	pgstat_report_wait_end();
+
+	/*
+	 * Increment the I/O timing and the number of times WAL files were synced.
+	 */
+	if (track_wal_io_timing)
+	{
+		instr_time	duration;
+
+		INSTR_TIME_SET_CURRENT(duration);
+		INSTR_TIME_SUBTRACT(duration, start);
+		WalStats.m_wal_sync_time += INSTR_TIME_GET_MICROSEC(duration);
+	}
+
+	WalStats.m_wal_sync++;
 }
 
 /*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index fc94a73a54..51ba1b5826 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1005,6 +1005,10 @@ CREATE VIEW pg_stat_wal AS
         w.wal_fpi,
         w.wal_bytes,
         w.wal_buffers_full,
+        w.wal_write,
+        w.wal_sync,
+        w.wal_write_time,
+        w.wal_sync_time,
         w.stats_reset
     FROM pg_stat_get_wal() w;
 
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index 0bbeece19d..3894f4a270 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -505,7 +505,7 @@ CheckpointerMain(void)
 		pgstat_send_bgwriter();
 
 		/* Send WAL statistics to the stats collector. */
-		pgstat_send_wal();
+		pgstat_report_wal();
 
 		/*
 		 * If any checkpoint flags have been set, redo the loop to handle the
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index f75b52719d..cf1d3ea366 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -146,8 +146,8 @@ PgStat_MsgWal WalStats;
 
 /*
  * WAL usage counters saved from pgWALUsage at the previous call to
- * pgstat_send_wal(). This is used to calculate how much WAL usage
- * happens between pgstat_send_wal() calls, by substracting
+ * pgstat_report_wal(). This is used to calculate how much WAL usage
+ * happens between pgstat_report_wal() calls, by substracting
  * the previous counters from the current ones.
  */
 static WalUsage prevWalUsage;
@@ -975,7 +975,7 @@ pgstat_report_stat(bool disconnect)
 	pgstat_send_funcstats();
 
 	/* Send WAL statistics */
-	pgstat_send_wal();
+	pgstat_report_wal();
 
 	/* Finally send SLRU statistics */
 	pgstat_send_slru();
@@ -3118,7 +3118,7 @@ pgstat_initialize(void)
 	}
 
 	/*
-	 * Initialize prevWalUsage with pgWalUsage so that pgstat_send_wal() can
+	 * Initialize prevWalUsage with pgWalUsage so that pgstat_report_wal() can
 	 * calculate how much pgWalUsage counters are increased by substracting
 	 * prevWalUsage from pgWalUsage.
 	 */
@@ -4667,17 +4667,17 @@ pgstat_send_bgwriter(void)
 }
 
 /* ----------
- * pgstat_send_wal() -
+ * pgstat_report_wal() -
  *
- *		Send WAL statistics to the collector
+ * Calculate how much WAL usage counters are increased and send
+ * WAL statistics to the collector.
+ *
+ * Must be called by processes that generate WAL.
  * ----------
  */
 void
-pgstat_send_wal(void)
+pgstat_report_wal(void)
 {
-	/* We assume this initializes to zeroes */
-	static const PgStat_MsgWal all_zeroes;
-
 	WalUsage	walusage;
 
 	/*
@@ -4692,6 +4692,33 @@ pgstat_send_wal(void)
 	WalStats.m_wal_fpi = walusage.wal_fpi;
 	WalStats.m_wal_bytes = walusage.wal_bytes;
 
+	/*
+	 * Send WAL stats message to the collector.
+	 */
+	pgstat_send_wal(true);
+
+	/*
+	 * Save the current counters for the subsequent calculation of WAL usage.
+	 */
+	prevWalUsage = pgWalUsage;
+}
+
+/* ----------
+ * pgstat_send_wal() -
+ *
+ *	Send WAL statistics to the collector.
+ *
+ * If 'force' is not set, WAL stats message is only sent if enough time has
+ * passed since last one was sent to reach PGSTAT_STAT_INTERVAL.
+ * ----------
+ */
+void
+pgstat_send_wal(bool force)
+{
+	/* We assume this initializes to zeroes */
+	static const PgStat_MsgWal all_zeroes;
+	static TimestampTz sendTime = 0;
+
 	/*
 	 * This function can be called even if nothing at all has happened. In
 	 * this case, avoid sending a completely empty message to the stats
@@ -4700,17 +4727,25 @@ pgstat_send_wal(void)
 	if (memcmp(&WalStats, &all_zeroes, sizeof(PgStat_MsgWal)) == 0)
 		return;
 
+	if (!force)
+	{
+		TimestampTz now = GetCurrentTimestamp();
+
+		/*
+		 * Don't send a message unless it's been at least PGSTAT_STAT_INTERVAL
+		 * msec since we last sent one.
+		 */
+		if (!TimestampDifferenceExceeds(sendTime, now, PGSTAT_STAT_INTERVAL))
+			return;
+		sendTime = now;
+	}
+
 	/*
 	 * Prepare and send the message
 	 */
 	pgstat_setheader(&WalStats.m_hdr, PGSTAT_MTYPE_WAL);
 	pgstat_send(&WalStats, sizeof(WalStats));
 
-	/*
-	 * Save the current counters for the subsequent calculation of WAL usage.
-	 */
-	prevWalUsage = pgWalUsage;
-
 	/*
 	 * Clear out the statistics buffer, so it can be re-used.
 	 */
@@ -6892,6 +6927,10 @@ pgstat_recv_wal(PgStat_MsgWal *msg, int len)
 	walStats.wal_fpi += msg->m_wal_fpi;
 	walStats.wal_bytes += msg->m_wal_bytes;
 	walStats.wal_buffers_full += msg->m_wal_buffers_full;
+	walStats.wal_write += msg->m_wal_write;
+	walStats.wal_write_time += msg->m_wal_write_time;
+	walStats.wal_sync += msg->m_wal_sync;
+	walStats.wal_sync_time += msg->m_wal_sync_time;
 }
 
 /* ----------
diff --git a/src/backend/postmaster/walwriter.c b/src/backend/postmaster/walwriter.c
index 4f1a8e356b..132df29aba 100644
--- a/src/backend/postmaster/walwriter.c
+++ b/src/backend/postmaster/walwriter.c
@@ -253,6 +253,9 @@ WalWriterMain(void)
 		else if (left_till_hibernate > 0)
 			left_till_hibernate--;
 
+		/* Send WAL statistics to the stats collector */
+		pgstat_send_wal(false);
+
 		/*
 		 * Sleep until we are signaled or WalWriterDelay has elapsed.  If we
 		 * haven't done anything useful for quite some time, lengthen the
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 62bff52638..c7540eec12 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1799,7 +1799,7 @@ pg_stat_get_buf_alloc(PG_FUNCTION_ARGS)
 Datum
 pg_stat_get_wal(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_WAL_COLS	5
+#define PG_STAT_GET_WAL_COLS	9
 	TupleDesc	tupdesc;
 	Datum		values[PG_STAT_GET_WAL_COLS];
 	bool		nulls[PG_STAT_GET_WAL_COLS];
@@ -1820,7 +1820,15 @@ pg_stat_get_wal(PG_FUNCTION_ARGS)
 					   NUMERICOID, -1, 0);
 	TupleDescInitEntry(tupdesc, (AttrNumber) 4, "wal_buffers_full",
 					   INT8OID, -1, 0);
-	TupleDescInitEntry(tupdesc, (AttrNumber) 5, "stats_reset",
+	TupleDescInitEntry(tupdesc, (AttrNumber) 5, "wal_write",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) 6, "wal_sync",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) 7, "wal_write_time",
+					   FLOAT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) 8, "wal_sync_time",
+					   FLOAT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) 9, "stats_reset",
 					   TIMESTAMPTZOID, -1, 0);
 
 	BlessTupleDesc(tupdesc);
@@ -1840,7 +1848,13 @@ pg_stat_get_wal(PG_FUNCTION_ARGS)
 									Int32GetDatum(-1));
 
 	values[3] = Int64GetDatum(wal_stats->wal_buffers_full);
-	values[4] = TimestampTzGetDatum(wal_stats->stat_reset_timestamp);
+	values[4] = Int64GetDatum(wal_stats->wal_write);
+	values[5] = Int64GetDatum(wal_stats->wal_sync);
+
+	/* convert counter from microsec to millisec for display */
+	values[6] = Float8GetDatum((double) wal_stats->wal_write_time / 1000.0);
+	values[7] = Float8GetDatum((double) wal_stats->wal_sync_time / 1000.0);
+	values[8] = TimestampTzGetDatum(wal_stats->stat_reset_timestamp);
 
 	/* Returns the record as Datum */
 	PG_RETURN_DATUM(HeapTupleGetDatum(heap_form_tuple(tupdesc, values, nulls)));
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 442850e8ad..da05b5acb1 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1485,6 +1485,15 @@ static struct config_bool ConfigureNamesBool[] =
 		false,
 		NULL, NULL, NULL
 	},
+	{
+		{"track_wal_io_timing", PGC_SUSET, STATS_COLLECTOR,
+			gettext_noop("Collects timing statistics for WAL I/O activity."),
+			NULL
+		},
+		&track_wal_io_timing,
+		false,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"update_process_title", PGC_SUSET, PROCESS_TITLE,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee06528bb0..c6483fa1ff 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -586,6 +586,7 @@
 #track_activities = on
 #track_counts = on
 #track_io_timing = off
+#track_wal_io_timing = off
 #track_functions = none			# none, pl, all
 #track_activity_query_size = 1024	# (change requires restart)
 #stats_temp_directory = 'pg_stat_tmp'
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 75ec1073bd..1e53d9d4ca 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -131,6 +131,7 @@ extern int	recovery_min_apply_delay;
 extern char *PrimaryConnInfo;
 extern char *PrimarySlotName;
 extern bool wal_receiver_create_temp_slot;
+extern bool track_wal_io_timing;
 
 /* indirectly set via GUC system */
 extern TransactionId recoveryTargetXid;
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 7ba7c2ff8a..933659efc6 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5546,9 +5546,9 @@
 { oid => '1136', descr => 'statistics: information about WAL activity',
   proname => 'pg_stat_get_wal', proisstrict => 'f', provolatile => 's',
   proparallel => 'r', prorettype => 'record', proargtypes => '',
-  proallargtypes => '{int8,int8,numeric,int8,timestamptz}',
-  proargmodes => '{o,o,o,o,o}',
-  proargnames => '{wal_records,wal_fpi,wal_bytes,wal_buffers_full,stats_reset}',
+  proallargtypes => '{int8,int8,numeric,int8,int8,int8,float8,float8,timestamptz}',
+  proargmodes => '{o,o,o,o,o,o,o,o,o}',
+  proargnames => '{wal_records,wal_fpi,wal_bytes,wal_buffers_full,wal_write,wal_sync,wal_write_time,wal_sync_time,stats_reset}',
   prosrc => 'pg_stat_get_wal' },
 
 { oid => '2306', descr => 'statistics: information about SLRU caches',
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 724068cf87..7dd375a114 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -474,6 +474,12 @@ typedef struct PgStat_MsgWal
 	PgStat_Counter m_wal_fpi;
 	uint64		m_wal_bytes;
 	PgStat_Counter m_wal_buffers_full;
+	PgStat_Counter m_wal_write;
+	PgStat_Counter m_wal_sync;
+	PgStat_Counter m_wal_write_time;	/* time spend writing wal records in
+										 * micro seconds */
+	PgStat_Counter m_wal_sync_time; /* time spend syncing wal records in micro
+									 * seconds */
 } PgStat_MsgWal;
 
 /* ----------
@@ -839,6 +845,10 @@ typedef struct PgStat_WalStats
 	PgStat_Counter wal_fpi;
 	uint64		wal_bytes;
 	PgStat_Counter wal_buffers_full;
+	PgStat_Counter wal_write;
+	PgStat_Counter wal_sync;
+	PgStat_Counter wal_write_time;
+	PgStat_Counter wal_sync_time;
 	TimestampTz stat_reset_timestamp;
 } PgStat_WalStats;
 
@@ -1590,7 +1600,8 @@ extern void pgstat_twophase_postabort(TransactionId xid, uint16 info,
 
 extern void pgstat_send_archiver(const char *xlog, bool failed);
 extern void pgstat_send_bgwriter(void);
-extern void pgstat_send_wal(void);
+extern void pgstat_report_wal(void);
+extern void pgstat_send_wal(bool force);
 
 /* ----------
  * Support functions for the SQL-callable functions to
