Add data for WAL in pg_stat_io and backend statistics
authorMichael Paquier <michael@paquier.xyz>
Tue, 4 Feb 2025 07:50:00 +0000 (16:50 +0900)
committerMichael Paquier <michael@paquier.xyz>
Tue, 4 Feb 2025 07:50:00 +0000 (16:50 +0900)
This commit adds WAL IO stats to both pg_stat_io view and per-backend IO
statistics (pg_stat_get_backend_io()).  This change is possible since
f92c854cf406, as WAL IO is not counted in blocks in some code paths
where its stats data is measured (like WAL read in xlogreader.c).

IOContext gains IOCONTEXT_INIT and IOObject IOOBJECT_WAL, with the
following combinations allowed:
- IOOBJECT_WAL/IOCONTEXT_NORMAL is used to track I/O operations done on
already-created WAL segments.
- IOOBJECT_WAL/IOCONTEXT_INIT is used for tracking I/O operations done
when initializing WAL segments.

The core changes are done in pg_stat_io.c, backend statistics inherit
them.  Backend statistics and pg_stat_io are now available for the WAL
writer, the WAL receiver and the WAL summarizer processes.

I/O timing data is controlled by the GUC track_io_timing, like the
existing data of pg_stat_io for consistency.  The timings related to
IOOBJECT_WAL show up if the GUC is enabled (disabled by default).

Bump pgstats file version, due to the additions in IOObject and
IOContext, impacting the amount of data written for the fixed-numbered
IO stats kind in the pgstats file.

Author: Nazir Bilal Yavuz
Reviewed-by: Bertrand Drouvot, Nitin Jadhav, Amit Kapila, Michael
Paquier, Melanie Plageman, Bharath Rupireddy
Discussion: https://postgr.es/m/CAN55FZ3AiQ+ZMxUuXnBpd0Rrh1YhwJ5FudkHg=JU0P+-W8T4Vg@mail.gmail.com

doc/src/sgml/monitoring.sgml
src/backend/access/transam/xlog.c
src/backend/access/transam/xlogreader.c
src/backend/access/transam/xlogrecovery.c
src/backend/utils/activity/pgstat_backend.c
src/backend/utils/activity/pgstat_io.c
src/include/pgstat.h
src/test/regress/expected/stats.out
src/test/regress/sql/stats.sql

index 4e917f159aa684c079e7c83cdcf130cb02255d09..edc2470bcf985b56ac2cd381e0bb77cda014517a 100644 (file)
@@ -2581,9 +2581,10 @@ description | Waiting for a newly initialized WAL file to reach durable storage
   </para>
 
   <para>
-   Currently, I/O on relations (e.g. tables, indexes) is tracked. However,
-   relation I/O which bypasses shared buffers (e.g. when moving a table from one
-   tablespace to another) is currently not tracked.
+   Currently, I/O on relations (e.g. tables, indexes) and WAL activity are
+   tracked. However, relation I/O which bypasses shared buffers
+   (e.g. when moving a table from one tablespace to another) is currently
+   not tracked.
   </para>
 
   <table id="pg-stat-io-view" xreflabel="pg_stat_io">
@@ -2636,6 +2637,11 @@ description | Waiting for a newly initialized WAL file to reach durable storage
           <literal>temp relation</literal>: Temporary relations.
          </para>
         </listitem>
+        <listitem>
+         <para>
+          <literal>wal</literal>: Write Ahead Logs.
+         </para>
+        </listitem>
        </itemizedlist>
        </para>
       </entry>
@@ -2660,6 +2666,13 @@ description | Waiting for a newly initialized WAL file to reach durable storage
           <literal>normal</literal>.
          </para>
         </listitem>
+        <listitem>
+         <para>
+          <literal>init</literal>: I/O operations performed while creating the
+          WAL segments are tracked in <varname>context</varname>
+          <literal>init</literal>.
+         </para>
+        </listitem>
         <listitem>
          <para>
           <literal>vacuum</literal>: I/O operations performed outside of shared
index 90ade4e7d3970b25bcc4489e256d9d99d1334d2d..9c270e7d466ceec01958225af83a8698562f015f 100644 (file)
@@ -2435,16 +2435,19 @@ XLogWrite(XLogwrtRqst WriteRqst, TimeLineID tli, bool flexible)
            {
                errno = 0;
 
-               /* Measure I/O timing to write WAL data */
-               if (track_wal_io_timing)
-                   INSTR_TIME_SET_CURRENT(start);
-               else
-                   INSTR_TIME_SET_ZERO(start);
+               /*
+                * Measure I/O timing to write WAL data, for pg_stat_io and/or
+                * pg_stat_wal.
+                */
+               start = pgstat_prepare_io_time(track_io_timing || track_wal_io_timing);
 
                pgstat_report_wait_start(WAIT_EVENT_WAL_WRITE);
                written = pg_pwrite(openLogFile, from, nleft, startoffset);
                pgstat_report_wait_end();
 
+               pgstat_count_io_op_time(IOOBJECT_WAL, IOCONTEXT_NORMAL,
+                                       IOOP_WRITE, start, 1, written);
+
                /*
                 * Increment the I/O timing and the number of times WAL data
                 * were written out to disk.
@@ -3216,6 +3219,7 @@ XLogFileInitInternal(XLogSegNo logsegno, TimeLineID logtli,
    int         fd;
    int         save_errno;
    int         open_flags = O_RDWR | O_CREAT | O_EXCL | PG_BINARY;
+   instr_time  io_start;
 
    Assert(logtli != 0);
 
@@ -3259,6 +3263,9 @@ XLogFileInitInternal(XLogSegNo logsegno, TimeLineID logtli,
                (errcode_for_file_access(),
                 errmsg("could not create file \"%s\": %m", tmppath)));
 
+   /* Measure I/O timing when initializing segment */
+   io_start = pgstat_prepare_io_time(track_io_timing);
+
    pgstat_report_wait_start(WAIT_EVENT_WAL_INIT_WRITE);
    save_errno = 0;
    if (wal_init_zero)
@@ -3294,6 +3301,14 @@ XLogFileInitInternal(XLogSegNo logsegno, TimeLineID logtli,
    }
    pgstat_report_wait_end();
 
+   /*
+    * A full segment worth of data is written when using wal_init_zero. One
+    * byte is written when not using it.
+    */
+   pgstat_count_io_op_time(IOOBJECT_WAL, IOCONTEXT_INIT, IOOP_WRITE,
+                           io_start, 1,
+                           wal_init_zero ? wal_segment_size : 1);
+
    if (save_errno)
    {
        /*
@@ -3310,6 +3325,9 @@ XLogFileInitInternal(XLogSegNo logsegno, TimeLineID logtli,
                 errmsg("could not write to file \"%s\": %m", tmppath)));
    }
 
+   /* Measure I/O timing when flushing segment */
+   io_start = pgstat_prepare_io_time(track_io_timing);
+
    pgstat_report_wait_start(WAIT_EVENT_WAL_INIT_SYNC);
    if (pg_fsync(fd) != 0)
    {
@@ -3322,6 +3340,9 @@ XLogFileInitInternal(XLogSegNo logsegno, TimeLineID logtli,
    }
    pgstat_report_wait_end();
 
+   pgstat_count_io_op_time(IOOBJECT_WAL, IOCONTEXT_INIT,
+                           IOOP_FSYNC, io_start, 1, 0);
+
    if (close(fd) != 0)
        ereport(ERROR,
                (errcode_for_file_access(),
@@ -8696,11 +8717,11 @@ issue_xlog_fsync(int fd, XLogSegNo segno, TimeLineID tli)
        wal_sync_method == WAL_SYNC_METHOD_OPEN_DSYNC)
        return;
 
-   /* Measure I/O timing to sync the WAL file */
-   if (track_wal_io_timing)
-       INSTR_TIME_SET_CURRENT(start);
-   else
-       INSTR_TIME_SET_ZERO(start);
+   /*
+    * Measure I/O timing to sync the WAL file for pg_stat_io and/or
+    * pg_stat_wal.
+    */
+   start = pgstat_prepare_io_time(track_io_timing || track_wal_io_timing);
 
    pgstat_report_wait_start(WAIT_EVENT_WAL_SYNC);
    switch (wal_sync_method)
@@ -8757,6 +8778,9 @@ issue_xlog_fsync(int fd, XLogSegNo segno, TimeLineID tli)
        INSTR_TIME_ACCUM_DIFF(PendingWalStats.wal_sync_time, end, start);
    }
 
+   pgstat_count_io_op_time(IOOBJECT_WAL, IOCONTEXT_NORMAL, IOOP_FSYNC,
+                           start, 1, 0);
+
    PendingWalStats.wal_sync++;
 }
 
index 91b6a91767d6b5acd735af143561cdaebe8205a1..6cf5ef90be0fb8cf4cbc81ef4cefb7b045f93fd8 100644 (file)
@@ -35,6 +35,7 @@
 
 #ifndef FRONTEND
 #include "pgstat.h"
+#include "storage/bufmgr.h"
 #else
 #include "common/logging.h"
 #endif
@@ -1507,6 +1508,9 @@ WALRead(XLogReaderState *state,
    char       *p;
    XLogRecPtr  recptr;
    Size        nbytes;
+#ifndef FRONTEND
+   instr_time  io_start;
+#endif
 
    p = buf;
    recptr = startptr;
@@ -1552,6 +1556,9 @@ WALRead(XLogReaderState *state,
            segbytes = nbytes;
 
 #ifndef FRONTEND
+       /* Measure I/O timing when reading segment */
+       io_start = pgstat_prepare_io_time(track_io_timing);
+
        pgstat_report_wait_start(WAIT_EVENT_WAL_READ);
 #endif
 
@@ -1561,6 +1568,9 @@ WALRead(XLogReaderState *state,
 
 #ifndef FRONTEND
        pgstat_report_wait_end();
+
+       pgstat_count_io_op_time(IOOBJECT_WAL, IOCONTEXT_NORMAL, IOOP_READ,
+                               io_start, 1, readbytes);
 #endif
 
        if (readbytes <= 0)
index cf2b007806fbe408c8c3a1fe75c191165eeec36b..473de6710d73ef9a66aef04ffe92f0df885ab289 100644 (file)
@@ -60,6 +60,7 @@
 #include "utils/datetime.h"
 #include "utils/fmgrprotos.h"
 #include "utils/guc_hooks.h"
+#include "utils/pgstat_internal.h"
 #include "utils/pg_lsn.h"
 #include "utils/ps_status.h"
 #include "utils/pg_rusage.h"
@@ -3306,6 +3307,7 @@ XLogPageRead(XLogReaderState *xlogreader, XLogRecPtr targetPagePtr, int reqLen,
    uint32      targetPageOff;
    XLogSegNo   targetSegNo PG_USED_FOR_ASSERTS_ONLY;
    int         r;
+   instr_time  io_start;
 
    XLByteToSeg(targetPagePtr, targetSegNo, wal_segment_size);
    targetPageOff = XLogSegmentOffset(targetPagePtr, wal_segment_size);
@@ -3398,6 +3400,9 @@ retry:
    /* Read the requested page */
    readOff = targetPageOff;
 
+   /* Measure I/O timing when reading segment */
+   io_start = pgstat_prepare_io_time(track_io_timing);
+
    pgstat_report_wait_start(WAIT_EVENT_WAL_READ);
    r = pg_pread(readFile, readBuf, XLOG_BLCKSZ, (off_t) readOff);
    if (r != XLOG_BLCKSZ)
@@ -3406,6 +3411,10 @@ retry:
        int         save_errno = errno;
 
        pgstat_report_wait_end();
+
+       pgstat_count_io_op_time(IOOBJECT_WAL, IOCONTEXT_NORMAL, IOOP_READ,
+                               io_start, 1, r);
+
        XLogFileName(fname, curFileTLI, readSegNo, wal_segment_size);
        if (r < 0)
        {
@@ -3426,6 +3435,9 @@ retry:
    }
    pgstat_report_wait_end();
 
+   pgstat_count_io_op_time(IOOBJECT_WAL, IOCONTEXT_NORMAL, IOOP_READ,
+                           io_start, 1, r);
+
    Assert(targetSegNo == readSegNo);
    Assert(targetPageOff == readOff);
    Assert(reqLen <= readLen);
index bcf9e4b1487a044dd17de0c795676ea1778b54de..4a667e7019cc23638c08870f84b89a7d442c62b1 100644 (file)
@@ -236,9 +236,6 @@ pgstat_tracks_backend_bktype(BackendType bktype)
        case B_DEAD_END_BACKEND:
        case B_ARCHIVER:
        case B_LOGGER:
-       case B_WAL_RECEIVER:
-       case B_WAL_WRITER:
-       case B_WAL_SUMMARIZER:
        case B_BG_WRITER:
        case B_CHECKPOINTER:
        case B_STARTUP:
@@ -249,7 +246,10 @@ pgstat_tracks_backend_bktype(BackendType bktype)
        case B_BG_WORKER:
        case B_STANDALONE_BACKEND:
        case B_SLOTSYNC_WORKER:
+       case B_WAL_RECEIVER:
        case B_WAL_SENDER:
+       case B_WAL_SUMMARIZER:
+       case B_WAL_WRITER:
            return true;
    }
 
index 7aee6a396f2c86deaf96f96d9c533ad06a1b92aa..28a431084b8997d025b4a3c3843bbff53f637838 100644 (file)
@@ -127,21 +127,24 @@ pgstat_count_io_op_time(IOObject io_object, IOContext io_context, IOOp io_op,
        INSTR_TIME_SET_CURRENT(io_time);
        INSTR_TIME_SUBTRACT(io_time, start_time);
 
-       if (io_op == IOOP_WRITE || io_op == IOOP_EXTEND)
+       if (io_object != IOOBJECT_WAL)
        {
-           pgstat_count_buffer_write_time(INSTR_TIME_GET_MICROSEC(io_time));
-           if (io_object == IOOBJECT_RELATION)
-               INSTR_TIME_ADD(pgBufferUsage.shared_blk_write_time, io_time);
-           else if (io_object == IOOBJECT_TEMP_RELATION)
-               INSTR_TIME_ADD(pgBufferUsage.local_blk_write_time, io_time);
-       }
-       else if (io_op == IOOP_READ)
-       {
-           pgstat_count_buffer_read_time(INSTR_TIME_GET_MICROSEC(io_time));
-           if (io_object == IOOBJECT_RELATION)
-               INSTR_TIME_ADD(pgBufferUsage.shared_blk_read_time, io_time);
-           else if (io_object == IOOBJECT_TEMP_RELATION)
-               INSTR_TIME_ADD(pgBufferUsage.local_blk_read_time, io_time);
+           if (io_op == IOOP_WRITE || io_op == IOOP_EXTEND)
+           {
+               pgstat_count_buffer_write_time(INSTR_TIME_GET_MICROSEC(io_time));
+               if (io_object == IOOBJECT_RELATION)
+                   INSTR_TIME_ADD(pgBufferUsage.shared_blk_write_time, io_time);
+               else if (io_object == IOOBJECT_TEMP_RELATION)
+                   INSTR_TIME_ADD(pgBufferUsage.local_blk_write_time, io_time);
+           }
+           else if (io_op == IOOP_READ)
+           {
+               pgstat_count_buffer_read_time(INSTR_TIME_GET_MICROSEC(io_time));
+               if (io_object == IOOBJECT_RELATION)
+                   INSTR_TIME_ADD(pgBufferUsage.shared_blk_read_time, io_time);
+               else if (io_object == IOOBJECT_TEMP_RELATION)
+                   INSTR_TIME_ADD(pgBufferUsage.local_blk_read_time, io_time);
+           }
        }
 
        INSTR_TIME_ADD(PendingIOStats.pending_times[io_object][io_context][io_op],
@@ -249,6 +252,8 @@ pgstat_get_io_context_name(IOContext io_context)
            return "bulkread";
        case IOCONTEXT_BULKWRITE:
            return "bulkwrite";
+       case IOCONTEXT_INIT:
+           return "init";
        case IOCONTEXT_NORMAL:
            return "normal";
        case IOCONTEXT_VACUUM:
@@ -268,6 +273,8 @@ pgstat_get_io_object_name(IOObject io_object)
            return "relation";
        case IOOBJECT_TEMP_RELATION:
            return "temp relation";
+       case IOOBJECT_WAL:
+           return "wal";
    }
 
    elog(ERROR, "unrecognized IOObject value: %d", io_object);
@@ -340,8 +347,6 @@ pgstat_io_snapshot_cb(void)
 * - Syslogger because it is not connected to shared memory
 * - Archiver because most relevant archiving IO is delegated to a
 *   specialized command or module
-* - WAL Receiver, WAL Writer, and WAL Summarizer IO are not tracked in
-*   pg_stat_io for now
 *
 * Function returns true if BackendType participates in the cumulative stats
 * subsystem for IO and false if it does not.
@@ -362,9 +367,6 @@ pgstat_tracks_io_bktype(BackendType bktype)
        case B_DEAD_END_BACKEND:
        case B_ARCHIVER:
        case B_LOGGER:
-       case B_WAL_RECEIVER:
-       case B_WAL_WRITER:
-       case B_WAL_SUMMARIZER:
            return false;
 
        case B_AUTOVAC_LAUNCHER:
@@ -376,7 +378,10 @@ pgstat_tracks_io_bktype(BackendType bktype)
        case B_SLOTSYNC_WORKER:
        case B_STANDALONE_BACKEND:
        case B_STARTUP:
+       case B_WAL_RECEIVER:
        case B_WAL_SENDER:
+       case B_WAL_SUMMARIZER:
+       case B_WAL_WRITER:
            return true;
    }
 
@@ -402,6 +407,15 @@ pgstat_tracks_io_object(BackendType bktype, IOObject io_object,
    if (!pgstat_tracks_io_bktype(bktype))
        return false;
 
+   /*
+    * Currently, IO on IOOBJECT_WAL objects can only occur in the
+    * IOCONTEXT_NORMAL and IOCONTEXT_INIT IOContexts.
+    */
+   if (io_object == IOOBJECT_WAL &&
+       (io_context != IOCONTEXT_NORMAL &&
+        io_context != IOCONTEXT_INIT))
+       return false;
+
    /*
     * Currently, IO on temporary relations can only occur in the
     * IOCONTEXT_NORMAL IOContext.
@@ -468,14 +482,28 @@ pgstat_tracks_io_op(BackendType bktype, IOObject io_object,
    /*
     * Some BackendTypes will not do certain IOOps.
     */
-   if ((bktype == B_BG_WRITER || bktype == B_CHECKPOINTER) &&
+   if (bktype == B_BG_WRITER &&
        (io_op == IOOP_READ || io_op == IOOP_EVICT || io_op == IOOP_HIT))
        return false;
 
+   if (bktype == B_CHECKPOINTER &&
+       ((io_object != IOOBJECT_WAL && io_op == IOOP_READ) ||
+        (io_op == IOOP_EVICT || io_op == IOOP_HIT)))
+       return false;
+
    if ((bktype == B_AUTOVAC_LAUNCHER || bktype == B_BG_WRITER ||
         bktype == B_CHECKPOINTER) && io_op == IOOP_EXTEND)
        return false;
 
+   /*
+    * Some BackendTypes do not perform reads with IOOBJECT_WAL.
+    */
+   if (io_object == IOOBJECT_WAL && io_op == IOOP_READ &&
+       (bktype == B_WAL_RECEIVER || bktype == B_BG_WRITER ||
+        bktype == B_AUTOVAC_WORKER || bktype == B_AUTOVAC_WORKER ||
+        bktype == B_WAL_WRITER))
+       return false;
+
    /*
     * Temporary tables are not logged and thus do not require fsync'ing.
     * Writeback is not requested for temporary tables.
@@ -500,6 +528,17 @@ pgstat_tracks_io_op(BackendType bktype, IOObject io_object,
    if (!strategy_io_context && io_op == IOOP_REUSE)
        return false;
 
+   /*
+    * IOOBJECT_WAL IOObject will not do certain IOOps depending on IOContext.
+    */
+   if (io_object == IOOBJECT_WAL && io_context == IOCONTEXT_INIT &&
+       !(io_op == IOOP_WRITE || io_op == IOOP_FSYNC))
+       return false;
+
+   if (io_object == IOOBJECT_WAL && io_context == IOCONTEXT_NORMAL &&
+       !(io_op == IOOP_WRITE || io_op == IOOP_READ || io_op == IOOP_FSYNC))
+       return false;
+
    /*
     * IOOP_FSYNC IOOps done by a backend using a BufferAccessStrategy are
     * counted in the IOCONTEXT_NORMAL IOContext. See comment in
index 81ec0161c0936d0db1f9d691bf237dfaff0ae61a..dd823d3f56e37768c4079418b84ba5e633b8f09a 100644 (file)
@@ -212,7 +212,7 @@ typedef struct PgStat_TableXactStatus
  * ------------------------------------------------------------
  */
 
-#define PGSTAT_FILE_FORMAT_ID  0x01A5BCB2
+#define PGSTAT_FILE_FORMAT_ID  0x01A5BCB3
 
 typedef struct PgStat_ArchiverStats
 {
@@ -274,14 +274,16 @@ typedef enum IOObject
 {
    IOOBJECT_RELATION,
    IOOBJECT_TEMP_RELATION,
+   IOOBJECT_WAL,
 } IOObject;
 
-#define IOOBJECT_NUM_TYPES (IOOBJECT_TEMP_RELATION + 1)
+#define IOOBJECT_NUM_TYPES (IOOBJECT_WAL + 1)
 
 typedef enum IOContext
 {
    IOCONTEXT_BULKREAD,
    IOCONTEXT_BULKWRITE,
+   IOCONTEXT_INIT,
    IOCONTEXT_NORMAL,
    IOCONTEXT_VACUUM,
 } IOContext;
index a0317b7208e3e3cda48c4134132383503e24d392..9a02481ee7e82c1f880fd966e027ca9ce6b75fb7 100644 (file)
@@ -862,6 +862,33 @@ WHERE pg_stat_get_backend_pid(beid) = pg_backend_pid();
  t
 (1 row)
 
+-- Test pg_stat_io for WAL in an init context, that should do writes
+-- and syncs.
+SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
+  FROM pg_stat_io
+  WHERE context = 'init' AND object = 'wal' \gset io_sum_wal_init_
+SELECT :io_sum_wal_init_writes > 0;
+ ?column? 
+----------
+ t
+(1 row)
+
+SELECT current_setting('fsync') = 'off'
+  OR :io_sum_wal_init_fsyncs > 0;
+ ?column? 
+----------
+ t
+(1 row)
+
+-- Test pg_stat_io for WAL in a normal context, that should do reads as well.
+SELECT SUM(reads) > 0
+  FROM pg_stat_io
+  WHERE context = 'normal' AND object = 'wal';
+ ?column? 
+----------
+ t
+(1 row)
+
 -----
 -- Test that resetting stats works for reset timestamp
 -----
@@ -1256,6 +1283,7 @@ SELECT pg_stat_get_subscription_stats(NULL);
 -- - extends of relations using shared buffers
 -- - fsyncs done to ensure the durability of data dirtying shared buffers
 -- - shared buffer hits
+-- - WAL writes and fsyncs in IOContext IOCONTEXT_NORMAL
 -- There is no test for blocks evicted from shared buffers, because we cannot
 -- be sure of the state of shared buffers at the point the test is run.
 -- Create a regular table and insert some data to generate IOCONTEXT_NORMAL
@@ -1273,6 +1301,9 @@ SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
 SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
   FROM pg_stat_get_backend_io(pg_backend_pid())
   WHERE object = 'relation' \gset my_io_sum_shared_before_
+SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
+  FROM pg_stat_io
+  WHERE context = 'normal' AND object = 'wal' \gset io_sum_wal_normal_before_
 CREATE TABLE test_io_shared(a int);
 INSERT INTO test_io_shared SELECT i FROM generate_series(1,100)i;
 SELECT pg_stat_force_next_flush();
@@ -1335,6 +1366,28 @@ SELECT current_setting('fsync') = 'off'
  t
 (1 row)
 
+SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
+  FROM pg_stat_io
+  WHERE context = 'normal' AND object = 'wal' \gset io_sum_wal_normal_after_
+SELECT current_setting('synchronous_commit') = 'on';
+ ?column? 
+----------
+ t
+(1 row)
+
+SELECT :io_sum_wal_normal_after_writes > :io_sum_wal_normal_before_writes;
+ ?column? 
+----------
+ t
+(1 row)
+
+SELECT current_setting('fsync') = 'off'
+  OR :io_sum_wal_normal_after_fsyncs > :io_sum_wal_normal_before_fsyncs;
+ ?column? 
+----------
+ t
+(1 row)
+
 -- Change the tablespace so that the table is rewritten directly, then SELECT
 -- from it to cause it to be read back into shared buffers.
 SELECT sum(reads) AS io_sum_shared_before_reads
index 399c72bbcf71038cb70f112fd593b7ece5080cf9..901e7bd56e3f64d5105da845e6a99938cdc50fe6 100644 (file)
@@ -442,6 +442,20 @@ SELECT (current_schemas(true))[1] = ('pg_temp_' || beid::text) AS match
 FROM pg_stat_get_backend_idset() beid
 WHERE pg_stat_get_backend_pid(beid) = pg_backend_pid();
 
+-- Test pg_stat_io for WAL in an init context, that should do writes
+-- and syncs.
+SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
+  FROM pg_stat_io
+  WHERE context = 'init' AND object = 'wal' \gset io_sum_wal_init_
+SELECT :io_sum_wal_init_writes > 0;
+SELECT current_setting('fsync') = 'off'
+  OR :io_sum_wal_init_fsyncs > 0;
+
+-- Test pg_stat_io for WAL in a normal context, that should do reads as well.
+SELECT SUM(reads) > 0
+  FROM pg_stat_io
+  WHERE context = 'normal' AND object = 'wal';
+
 -----
 -- Test that resetting stats works for reset timestamp
 -----
@@ -602,6 +616,7 @@ SELECT pg_stat_get_subscription_stats(NULL);
 -- - extends of relations using shared buffers
 -- - fsyncs done to ensure the durability of data dirtying shared buffers
 -- - shared buffer hits
+-- - WAL writes and fsyncs in IOContext IOCONTEXT_NORMAL
 
 -- There is no test for blocks evicted from shared buffers, because we cannot
 -- be sure of the state of shared buffers at the point the test is run.
@@ -621,6 +636,9 @@ SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
 SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
   FROM pg_stat_get_backend_io(pg_backend_pid())
   WHERE object = 'relation' \gset my_io_sum_shared_before_
+SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
+  FROM pg_stat_io
+  WHERE context = 'normal' AND object = 'wal' \gset io_sum_wal_normal_before_
 CREATE TABLE test_io_shared(a int);
 INSERT INTO test_io_shared SELECT i FROM generate_series(1,100)i;
 SELECT pg_stat_force_next_flush();
@@ -649,6 +667,13 @@ SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
 SELECT :my_io_sum_shared_after_writes >= :my_io_sum_shared_before_writes;
 SELECT current_setting('fsync') = 'off'
   OR :my_io_sum_shared_after_fsyncs >= :my_io_sum_shared_before_fsyncs;
+SELECT sum(writes) AS writes, sum(fsyncs) AS fsyncs
+  FROM pg_stat_io
+  WHERE context = 'normal' AND object = 'wal' \gset io_sum_wal_normal_after_
+SELECT current_setting('synchronous_commit') = 'on';
+SELECT :io_sum_wal_normal_after_writes > :io_sum_wal_normal_before_writes;
+SELECT current_setting('fsync') = 'off'
+  OR :io_sum_wal_normal_after_fsyncs > :io_sum_wal_normal_before_fsyncs;
 
 -- Change the tablespace so that the table is rewritten directly, then SELECT
 -- from it to cause it to be read back into shared buffers.